diff --git a/Makefile b/Makefile index cc056ce026..af0bc757ab 100644 --- a/Makefile +++ b/Makefile @@ -1,6 +1,6 @@ CONCORD_BFT_DOCKER_REPO?=concordbft/ CONCORD_BFT_DOCKER_IMAGE?=concord-bft -CONCORD_BFT_DOCKER_IMAGE_VERSION?=0.41 +CONCORD_BFT_DOCKER_IMAGE_VERSION?=0.40 CONCORD_BFT_DOCKER_CONTAINER?=concord-bft CONCORD_BFT_DOCKERFILE?=Dockerfile diff --git a/bftengine/include/bftengine/DbCheckpointManager.hpp b/bftengine/include/bftengine/DbCheckpointManager.hpp index f06095b707..6013ed3ee8 100644 --- a/bftengine/include/bftengine/DbCheckpointManager.hpp +++ b/bftengine/include/bftengine/DbCheckpointManager.hpp @@ -99,7 +99,8 @@ class DbCheckpointManager { std::shared_ptr p, std::shared_ptr aggregator, const std::function& getLastBlockIdCb, - const PrepareCheckpointCallback& prepareCheckpointCb); + const PrepareCheckpointCallback& prepareCheckpointCb, + const std::function& checkpointInProcessCb); std::map getListOfDbCheckpoints() const { return dbCheckptMetadata_.dbCheckPoints_; } @@ -122,6 +123,7 @@ class DbCheckpointManager { void sendInternalCreateDbCheckpointMsg(const SeqNum& seqNum, bool noop); BlockId getLastReachableBlock() const; SeqNum getLastStableSeqNum() const; + void setCheckpointInProcess(bool) const; void setOnStableSeqNumCb_(std::function cb) { onStableSeqNumCb_ = cb; } void onStableSeqNum(SeqNum s) { if (onStableSeqNumCb_) onStableSeqNumCb_(s); @@ -165,6 +167,7 @@ class DbCheckpointManager { void cleanUp(); std::function getLastBlockIdCb_; PrepareCheckpointCallback prepareCheckpointCb_; + std::function checkpointInProcessCb_; // get total size recursively uint64_t directorySize(const _fs::path& directory, const bool& excludeHardLinks, bool recursive); // get checkpoint metadata diff --git a/bftengine/include/bftengine/ReplicaConfig.hpp b/bftengine/include/bftengine/ReplicaConfig.hpp index 8a73ba57c3..d69d7bb8b8 100644 --- a/bftengine/include/bftengine/ReplicaConfig.hpp +++ b/bftengine/include/bftengine/ReplicaConfig.hpp @@ -279,6 +279,7 @@ class ReplicaConfig : public concord::serialize::SerializableFactory p, std::shared_ptr aggregator, const std::function& getLastBlockIdCb, - const PrepareCheckpointCallback& prepareCheckpointCb) { + const PrepareCheckpointCallback& prepareCheckpointCb, + const std::function& checkpointInProcessCb) { dbClient_ = dbClient; ps_ = p; dbCheckPointDirPath_ = ReplicaConfig::instance().getdbCheckpointDirPath(); @@ -176,6 +178,7 @@ void DbCheckpointManager::initializeDbCheckpointManager(std::shared_ptr(req.request, req.request + req.requestSize), createDbChkPtMsg); if (!createDbChkPtMsg.noop) { const auto& lastStableSeqNum = DbCheckpointManager::instance().getLastStableSeqNum(); + DbCheckpointManager::instance().setCheckpointInProcess(true); if (lastStableSeqNum == static_cast(createDbChkPtMsg.seqNum)) { DbCheckpointManager::instance().createDbCheckpointAsync(createDbChkPtMsg.seqNum, timestamp, std::nullopt); } else { diff --git a/client/client_pool/test/client_pool_timer_test.cpp b/client/client_pool/test/client_pool_timer_test.cpp index 7997d0e476..b0182bdc15 100644 --- a/client/client_pool/test/client_pool_timer_test.cpp +++ b/client/client_pool/test/client_pool_timer_test.cpp @@ -25,7 +25,7 @@ using TestClient = std::shared_ptr; TEST(client_pool_timer, work_items) { uint16_t num_times_called = 0; - std::chrono::milliseconds timeout = 1ms; + std::chrono::milliseconds timeout = 10ms; auto timer = Timer(timeout, [&num_times_called](TestClient&& c) -> void { num_times_called++; }); // Wait for timeout diff --git a/install_deps.sh b/install_deps.sh index adbb2d40d7..358088cc29 100755 --- a/install_deps.sh +++ b/install_deps.sh @@ -40,7 +40,9 @@ apt-get update && apt-get ${APT_GET_FLAGS} install \ sudo \ vim \ iproute2 \ - wget + wget \ + bison \ + flex update-alternatives --install /usr/bin/clang clang /usr/lib/llvm-9/bin/clang 100 update-alternatives --install /usr/bin/clang++ clang++ /usr/lib/llvm-9/bin/clang++ 100 @@ -185,16 +187,16 @@ git clone https://github.com/relic-toolkit/relic && \ cd ${HOME} wget ${WGET_FLAGS} \ - https://github.com/facebook/rocksdb/archive/v6.8.1.tar.gz && \ - tar -xzf v6.8.1.tar.gz && \ - rm v6.8.1.tar.gz && \ - cd rocksdb-6.8.1 && \ + https://github.com/facebook/rocksdb/archive/v6.29.3.tar.gz && \ + tar -xzf v6.29.3.tar.gz && \ + rm v6.29.3.tar.gz && \ + cd rocksdb-6.29.3 && \ EXTRA_CXXFLAGS="-fno-omit-frame-pointer -g " \ EXTRA_CFLAGS="-fno-omit-frame-pointer -g " \ PORTABLE=1 make -j$(nproc) USE_RTTI=1 shared_lib && \ PORTABLE=1 make install-shared && \ cd ${HOME} && \ - rm -r rocksdb-6.8.1 + rm -r rocksdb-6.29.3 cd ${HOME} git clone https://github.com/emil-e/rapidcheck.git && \ @@ -335,15 +337,15 @@ git clone -b v0.9.7 --depth 1 https://github.com/yhirose/cpp-httplib && \ # Thrift is the protocol used by Jaeger to export metrics cd $HOME -wget ${WGET_FLAGS} https://archive.apache.org/dist/thrift/0.11.0/thrift-0.11.0.tar.gz && \ - tar xzf thrift-0.11.0.tar.gz && \ +git clone -b 0.11.0 --depth 1 https://github.com/apache/thrift.git thrift-0.11.0 && \ cd thrift-0.11.0 && \ + ./bootstrap.sh && \ ./configure CXXFLAGS='-g -O2' \ --without-python --enable-static --disable-shared \ --disable-tests --disable-tutorial --disable-coverage && \ make -j$(nproc) install && \ cd ${HOME} && \ - rm -r thrift-0.11.0 thrift-0.11.0.tar.gz + rm -r thrift-0.11.0 # TODO: Upgrade to opentelemetry-cpp # Tracing via Jaeger and Thrift protocol diff --git a/kvbc/CMakeLists.txt b/kvbc/CMakeLists.txt index 20270628c4..c32d9debf9 100644 --- a/kvbc/CMakeLists.txt +++ b/kvbc/CMakeLists.txt @@ -34,6 +34,18 @@ add_library(kvbc src/ClientImp.cpp src/resources-manager/ReplicaResources.cpp src/resources-manager/AdaptivePruningManager.cpp src/resources-manager/IntervalMappingResourceManager.cpp + + src/blockchain_misc.cpp + src/kvbc_adapter/common/state_snapshot_adapter.cpp + src/kvbc_adapter/categorization/db_checkpoint_adapter.cpp + src/kvbc_adapter/categorization/kv_blockchain_adapter.cpp + src/kvbc_adapter/categorization/app_state_adapter.cpp + src/kvbc_adapter/categorization/blocks_deleter_adapter.cpp + + src/kvbc_adapter/v4blockchain/blocks_deleter_adapter.cpp + src/kvbc_adapter/v4blockchain/app_state_adapter.cpp + + src/kvbc_adapter/replica_adapter.cpp ) if (BUILD_ROCKSDB_STORAGE) @@ -43,7 +55,14 @@ if (BUILD_ROCKSDB_STORAGE) src/categorization/blocks.cpp src/categorization/blockchain.cpp src/categorization/block_merkle_category.cpp - src/migrations/block_merkle_latest_ver_cf_migration.cpp) + src/migrations/block_merkle_latest_ver_cf_migration.cpp + + src/v4blockchain/v4_blockchain.cpp + src/v4blockchain/detail/latest_keys.cpp + src/v4blockchain/detail/categories.cpp + src/v4blockchain/detail/blocks.cpp + src/v4blockchain/detail/st_chain.cpp + src/v4blockchain/detail/blockchain.cpp) endif (BUILD_ROCKSDB_STORAGE) target_link_libraries(kvbc PUBLIC corebft util) diff --git a/kvbc/benchmark/kvbcbench/main.cpp b/kvbc/benchmark/kvbcbench/main.cpp index 08ae240aae..e72437dc70 100644 --- a/kvbc/benchmark/kvbcbench/main.cpp +++ b/kvbc/benchmark/kvbcbench/main.cpp @@ -17,7 +17,6 @@ #include #include #include -#include #include #include @@ -27,20 +26,20 @@ #include #include #include -#include #include #include "categorization/base_types.h" #include "categorization/column_families.h" #include "categorization/updates.h" #include "categorized_kvbc_msgs.cmf.hpp" -#include "categorization/kv_blockchain.h" +#include "kvbc_adapter/replica_adapter.hpp" #include "performance_handler.h" #include "rocksdb/native_client.h" #include "diagnostics.h" #include "diagnostics_server.h" #include "input.h" #include "pre_execution.h" +#include "ReplicaResources.h" using namespace std; @@ -233,8 +232,8 @@ std::shared_ptr completeRocksdbConfiguration( // Use the same block cache and table options for all column familes for now. for (auto& d : cf_descs) { - auto* cf_table_options = - reinterpret_cast<::rocksdb::BlockBasedTableOptions*>(d.options.table_factory->GetOptions()); + auto* cf_table_options = reinterpret_cast<::rocksdb::BlockBasedTableOptions*>( + d.options.table_factory->GetOptions<::rocksdb::BlockBasedTableOptions>()); cf_table_options->block_cache = table_options.block_cache; cf_table_options->filter_policy.reset(::rocksdb::NewBloomFilterPolicy(10, false)); } @@ -261,7 +260,7 @@ PreExecConfig preExecConfig(const po::variables_map& config, void addBlocks(const po::variables_map& config, std::shared_ptr& db, - categorization::KeyValueBlockchain& kvbc, + adapter::ReplicaBlockchain& kvbc, InputData& input, std::shared_ptr& add_block_recorder, std::shared_ptr& conflict_detection_recorder) { @@ -315,11 +314,11 @@ void addBlocks(const po::variables_map& config, updates.add(kCategoryMerkle, categorization::BlockMerkleUpdates(std::move(merkle_input))); updates.add(kCategoryImmutable, std::move(immutable_updates)); updates.add(kCategoryVersioned, std::move(versioned_updates)); - kvbc.addBlock(std::move(updates)); + kvbc.add(std::move(updates)); } else { auto&& merkle_input = std::move(input.block_merkle_input[i - 1]); updates.add(kCategoryMerkle, categorization::BlockMerkleUpdates(std::move(merkle_input))); - kvbc.addBlock(std::move(updates)); + kvbc.add(std::move(updates)); } } } @@ -361,13 +360,13 @@ int main(int argc, char** argv) { }; auto opts = storage::rocksdb::NativeClient::UserOptions{"kvbcbench_rocksdb_opts.ini", completeInit}; auto db = storage::rocksdb::NativeClient::newClient(config["rocksdb-path"].as(), false, opts); - auto kvbc = kvbc::categorization::KeyValueBlockchain( - db, - false, - std::map{ - {kCategoryMerkle, kvbc::categorization::CATEGORY_TYPE::block_merkle}, - {kCategoryImmutable, kvbc::categorization::CATEGORY_TYPE::immutable}, - {kCategoryVersioned, kvbc::categorization::CATEGORY_TYPE::versioned_kv}}); + auto kvbc = + kvbc::adapter::ReplicaBlockchain(db, + false, + std::map{ + {kCategoryMerkle, kvbc::categorization::CATEGORY_TYPE::block_merkle}, + {kCategoryImmutable, kvbc::categorization::CATEGORY_TYPE::immutable}, + {kCategoryVersioned, kvbc::categorization::CATEGORY_TYPE::versioned_kv}}); auto pre_exec_config = preExecConfig(config, input.block_merkle_read_keys.size(), input.ver_read_keys.size()); auto pre_exec_sim = PreExecutionSimulator(pre_exec_config, input.block_merkle_read_keys, input.ver_read_keys, kvbc); diff --git a/kvbc/benchmark/kvbcbench/pre_execution.h b/kvbc/benchmark/kvbcbench/pre_execution.h index dfc03bb4be..6609c24640 100644 --- a/kvbc/benchmark/kvbcbench/pre_execution.h +++ b/kvbc/benchmark/kvbcbench/pre_execution.h @@ -18,7 +18,7 @@ #include "assertUtils.hpp" #include "categorization/block_merkle_category.h" -#include "categorization/kv_blockchain.h" +#include "kvbc_adapter/replica_adapter.hpp" #include "input.h" namespace concord::kvbc::bench { @@ -46,7 +46,7 @@ class PreExecutionSimulator { PreExecutionSimulator(const PreExecConfig& config, const ReadKeys& merkle_read_keys, const ReadKeys& versioned_read_keys, - categorization::KeyValueBlockchain& kvbc) + adapter::ReplicaBlockchain& kvbc) : config_(config), merkle_read_keys_(merkle_read_keys), versioned_read_keys_(versioned_read_keys), kvbc_(kvbc) {} void start() { @@ -106,7 +106,7 @@ class PreExecutionSimulator { const ReadKeys& merkle_read_keys_; const ReadKeys& versioned_read_keys_; - categorization::KeyValueBlockchain& kvbc_; + adapter::ReplicaBlockchain& kvbc_; }; } // namespace concord::kvbc::bench diff --git a/kvbc/benchmark/state_snapshot_benchmarks/hash_state_benchmark.cpp b/kvbc/benchmark/state_snapshot_benchmarks/hash_state_benchmark.cpp index b6d1d28531..97731bc587 100644 --- a/kvbc/benchmark/state_snapshot_benchmarks/hash_state_benchmark.cpp +++ b/kvbc/benchmark/state_snapshot_benchmarks/hash_state_benchmark.cpp @@ -108,8 +108,8 @@ void completeRocksdbConfiguration(::rocksdb::Options& db_options, // Use the same block cache and table options for all column familes for now. for (auto& d : cf_descs) { - auto* cf_table_options = - reinterpret_cast<::rocksdb::BlockBasedTableOptions*>(d.options.table_factory->GetOptions()); + auto* cf_table_options = reinterpret_cast<::rocksdb::BlockBasedTableOptions*>( + d.options.table_factory->GetOptions<::rocksdb::BlockBasedTableOptions>()); cf_table_options->block_cache = table_options.block_cache; cf_table_options->filter_policy.reset(::rocksdb::NewBloomFilterPolicy(10, false)); } diff --git a/kvbc/include/Replica.h b/kvbc/include/Replica.h index e006da5ae8..fbb8815b1a 100644 --- a/kvbc/include/Replica.h +++ b/kvbc/include/Replica.h @@ -11,7 +11,7 @@ #include "st_reconfiguraion_sm.hpp" #include "OpenTracing.hpp" -#include "categorization/kv_blockchain.h" +#include "kvbc_adapter/replica_adapter.hpp" #include "categorization/db_categories.h" #include "communication/ICommunication.hpp" #include "communication/CommFactory.hpp" @@ -47,87 +47,90 @@ class Replica : public IReplica, public: ///////////////////////////////////////////////////////////////////////////////////////////////////////////////////// // IReplica implementation - Status start() override; - Status stop() override; - bool isRunning() const override { return (m_currentRepStatus == RepStatus::Running); } - RepStatus getReplicaStatus() const override; - const IReader &getReadOnlyStorage() const override; - BlockId addBlockToIdleReplica(categorization::Updates &&updates) override; - void set_command_handler(std::shared_ptr handler) override; + Status start() override final; + Status stop() override final; + bool isRunning() const override final { return (m_currentRepStatus == RepStatus::Running); } + RepStatus getReplicaStatus() const override final; + const IReader &getReadOnlyStorage() const override final; + BlockId addBlockToIdleReplica(categorization::Updates &&updates) override final; + void set_command_handler(std::shared_ptr handler) override final; ///////////////////////////////////////////////////////////////////////////////////////////////////////////////////// // IBlocksDeleter implementation - void deleteGenesisBlock() override; - BlockId deleteBlocksUntil(BlockId until) override; + void deleteGenesisBlock() override final; + BlockId deleteBlocksUntil(BlockId until) override final; + void deleteLastReachableBlock() override final; ///////////////////////////////////////////////////////////////////////////////////////////////////////////////////// ///////////////////////////////////////////////////////////////////////////////////////////////////////////////////// // IReader std::optional get(const std::string &category_id, const std::string &key, - BlockId block_id) const override; + BlockId block_id) const override final; - std::optional getLatest(const std::string &category_id, const std::string &key) const override; + std::optional getLatest(const std::string &category_id, + const std::string &key) const override final; void multiGet(const std::string &category_id, const std::vector &keys, const std::vector &versions, - std::vector> &values) const override; + std::vector> &values) const override final; void multiGetLatest(const std::string &category_id, const std::vector &keys, - std::vector> &values) const override; + std::vector> &values) const override final; std::optional getLatestVersion(const std::string &category_id, - const std::string &key) const override; + const std::string &key) const override final; void multiGetLatestVersion(const std::string &category_id, const std::vector &keys, - std::vector> &versions) const override; + std::vector> &versions) const override final; - std::optional getBlockUpdates(BlockId block_id) const override; + std::optional getBlockUpdates(BlockId block_id) const override final; // Get the current genesis block ID in the system. - BlockId getGenesisBlockId() const override; + BlockId getGenesisBlockId() const override final; // Get the last block ID in the system. - BlockId getLastBlockId() const override; + BlockId getLastBlockId() const override final; ///////////////////////////////////////////////////////////////////////////////////////////////////////////////////// - + // checkpoint + void checkpointInProcess(bool flag); ///////////////////////////////////////////////////////////////////////////////////////////////////////////////////// // IBlockAdder - BlockId add(categorization::Updates &&) override; + BlockId add(categorization::Updates &&) override final; ///////////////////////////////////////////////////////////////////////////////////////////////////////////////////// ///////////////////////////////////////////////////////////////////////////////////////////////////////////////////// // IAppState implementation - bool hasBlock(BlockId blockId) const override; + bool hasBlock(BlockId blockId) const override final; bool getBlock(uint64_t blockId, char *outBlock, uint32_t outBlockMaxSize, - uint32_t *outBlockActualSize) const override; + uint32_t *outBlockActualSize) const override final; std::future getBlockAsync(uint64_t blockId, char *outBlock, uint32_t outBlockMaxSize, - uint32_t *outBlockActualSize) override; - bool getPrevDigestFromBlock(uint64_t blockId, bftEngine::bcst::StateTransferDigest *) const override; + uint32_t *outBlockActualSize) override final; + bool getPrevDigestFromBlock(uint64_t blockId, bftEngine::bcst::StateTransferDigest *) const override final; void getPrevDigestFromBlock(const char *blockData, const uint32_t blockSize, - bftEngine::bcst::StateTransferDigest *outPrevBlockDigest) const override; + bftEngine::bcst::StateTransferDigest *outPrevBlockDigest) const override final; bool putBlock(const uint64_t blockId, const char *blockData, const uint32_t blockSize, - bool lastBlock = true) override; + bool lastBlock = true) override final; std::future putBlockAsync(uint64_t blockId, const char *block, const uint32_t blockSize, - bool lastblock) override; - uint64_t getLastReachableBlockNum() const override; - uint64_t getGenesisBlockNum() const override; + bool lastblock) override final; + uint64_t getLastReachableBlockNum() const override final; + uint64_t getGenesisBlockNum() const override final; // This method is used by state-transfer in order to find the latest block id in either the state-transfer chain or // the main blockchain - uint64_t getLastBlockNum() const override; - size_t postProcessUntilBlockId(uint64_t max_block_id) override; + uint64_t getLastBlockNum() const override final; + size_t postProcessUntilBlockId(uint64_t max_block_id) override final; ///////////////////////////////////////////////////////////////////////////////////////////////////////////////////// bool getBlockFromObjectStore(uint64_t blockId, @@ -135,6 +138,9 @@ class Replica : public IReplica, uint32_t outblockMaxSize, uint32_t *outBlockSize) const; bool getPrevDigestFromObjectStoreBlock(uint64_t blockId, bftEngine::bcst::StateTransferDigest *) const; + void getPrevDigestFromObjectStoreBlock(const char *blockData, + const uint32_t blockSize, + bftEngine::bcst::StateTransferDigest *outPrevBlockDigest) const; bool putBlockToObjectStore(const uint64_t blockId, const char *blockData, const uint32_t blockSize, @@ -163,21 +169,18 @@ class Replica : public IReplica, void registerStBasedReconfigurationHandler(std::shared_ptr); std::shared_ptr nativeDbClient() { - if (m_kvBlockchain) return m_kvBlockchain->db(); - return nullptr; + return storage::rocksdb::NativeClient::fromIDBClient(m_dbSet.dataDBClient); } - std::optional &kvBlockchain() { return m_kvBlockchain; } + std::optional &kvBlockchain() { return op_kvBlockchain; } - void setStateSnapshotValueConverter(const categorization::KeyValueBlockchain::Converter &c) { - m_stateSnapshotValueConverter = c; - } + void setStateSnapshotValueConverter(const Converter &c) { m_stateSnapshotValueConverter = c; } void setLastApplicationTransactionTimeCallback(const LastApplicationTransactionTimeCallback &cb) { m_lastAppTxnCallback = cb; } - ~Replica() override; + virtual ~Replica(); protected: RawBlock getBlockInternal(BlockId blockId) const; @@ -227,9 +230,10 @@ class Replica : public IReplica, RepStatus m_currentRepStatus; concord::kvbc::IStorageFactory::DatabaseSet m_dbSet; - // The categorization KeyValueBlockchain is used for a normal read-write replica. - std::optional m_kvBlockchain; - categorization::KeyValueBlockchain::Converter m_stateSnapshotValueConverter{concord::kvbc::valueFromKvbcProto}; + // The ReplicaBlockchain is used for a normal read-write replica. + std::optional op_kvBlockchain; + adapter::ReplicaBlockchain *m_kvBlockchain{nullptr}; + Converter m_stateSnapshotValueConverter{concord::kvbc::valueFromKvbcProto}; kvbc::LastApplicationTransactionTimeCallback m_lastAppTxnCallback{newestPublicEventGroupRecordTime}; // The IdbAdapter instance is used for a read-only replica. std::unique_ptr m_bcDbAdapter; @@ -248,26 +252,27 @@ class Replica : public IReplica, const std::shared_ptr secretsManager_; std::unique_ptr stReconfigurationSM_; std::shared_ptr cronTableRegistry_{std::make_shared()}; - concord::util::ThreadPool blocksIOWorkersPool_; std::unique_ptr creEngine_; std::shared_ptr creClient_; ReplicaResourceEntity replicaResources_; + concord::util::ThreadPool blocks_io_workers_pool; performance::AdaptivePruningManager AdaptivePruningManager_; - - private: struct Recorders { static constexpr uint64_t MAX_VALUE_MICROSECONDS = 2ULL * 1000ULL * 1000ULL; // 2 seconds - + const std::string component_ = "iappstate"; Recorders() { auto ®istrar = concord::diagnostics::RegistrarSingleton::getInstance(); - registrar.perf.registerComponent("iappstate", - {get_block_duration, put_block_duration, delete_batch_blocks_duration}); + if (!registrar.perf.isRegisteredComponent(component_)) { + registrar.perf.registerComponent(component_, {get_block_duration, put_block_duration}); + } } + + ~Recorders() {} + DEFINE_SHARED_RECORDER(get_block_duration, 1, MAX_VALUE_MICROSECONDS, 3, concord::diagnostics::Unit::MICROSECONDS); DEFINE_SHARED_RECORDER(put_block_duration, 1, MAX_VALUE_MICROSECONDS, 3, concord::diagnostics::Unit::MICROSECONDS); - DEFINE_SHARED_RECORDER( - delete_batch_blocks_duration, 1, MAX_VALUE_MICROSECONDS, 3, concord::diagnostics::Unit::MICROSECONDS); }; + // All these recorders need to be shared Recorders histograms_; }; // namespace concord::kvbc diff --git a/kvbc/include/block_metadata.hpp b/kvbc/include/block_metadata.hpp index 862b10ecf4..1db65e614a 100644 --- a/kvbc/include/block_metadata.hpp +++ b/kvbc/include/block_metadata.hpp @@ -44,8 +44,9 @@ class BlockMetadata : public IBlockMetadata { BlockMetadata(const IReader& storage) : IBlockMetadata(storage) { logger_ = logging::getLogger("skvbc.MetadataStorage"); } - virtual uint64_t getLastBlockSequenceNum() const override; - virtual std::string serialize(uint64_t sequence_num) const override; + uint64_t getLastBlockSequenceNum() const override; + std::string serialize(uint64_t sequence_num) const override; + static uint64_t getSequenceNum(const std::string& data); }; } // namespace kvbc diff --git a/kvbc/include/blockchain_misc.hpp b/kvbc/include/blockchain_misc.hpp new file mode 100644 index 0000000000..36899ea86d --- /dev/null +++ b/kvbc/include/blockchain_misc.hpp @@ -0,0 +1,32 @@ +// Concord +// +// Copyright (c) 2022-2023 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 subcomponent's license, as noted in the LICENSE +// file. + +#pragma once + +#include +#include + +namespace concord::kvbc { +enum BLOCKCHAIN_VERSION { CATEGORIZED_BLOCKCHAIN = 1, NATURAL_BLOCKCHAIN = 4, INVALID_BLOCKCHAIN_VERSION }; + +// Key or value converter interface. +// Allows users to convert keys or values to any format that is appropriate. +using Converter = std::function; + +namespace bcutil { +class BlockChainUtils { + public: + static std::string publicStateHashKey(); +}; +} // namespace bcutil +} // end namespace concord::kvbc diff --git a/kvbc/include/categorization/kv_blockchain.h b/kvbc/include/categorization/kv_blockchain.h index e226b1d169..ca674c9e43 100644 --- a/kvbc/include/categorization/kv_blockchain.h +++ b/kvbc/include/categorization/kv_blockchain.h @@ -14,6 +14,7 @@ #pragma once #include "updates.h" +#include "blockchain_misc.hpp" #include "rocksdb/native_client.h" #include "blocks.h" #include "blockchain.h" @@ -39,14 +40,6 @@ namespace concord::kvbc::categorization { class KeyValueBlockchain { using VersionedRawBlock = std::pair>; - public: - // Key or value converter interface. - // Allows users to convert keys or values to any format that is appropriate. - using Converter = std::function; - - // The noop converter returns the input string as is, without modifying it. - static const Converter kNoopConverter; - public: // Creates a key-value blockchain. // If `category_types` is nullopt, the persisted categories in storage will be used. @@ -143,43 +136,10 @@ class KeyValueBlockchain { // Precondition3: `block_id_at_checkpoint` <= getLastReachableBlockId() void trimBlocksFromSnapshot(BlockId block_id_at_checkpoint); - // Computes and persists the public state hash by: - // h0 = hash("") - // h1 = hash(h0 || hash(k1) || v1) - // h2 = hash(h1 || hash(k2) || v2) - // ... - // hN = hash(hN-1 || hash(kN) || vN) - // - // This method is supposed to be called on DB snapshots only and not on the actual blockchain. - // Precondition: The current KeyValueBlockchain instance points to a DB snapshot. - void computeAndPersistPublicStateHash(BlockId checkpoint_block_id, const Converter& value_converter = kNoopConverter); - - // Returns the public state keys as of the current point in the blockchain's history. - // Returns std::nullopt if no public keys have been persisted. - std::optional getPublicStateKeys() const; - - // Iterate over all public key values, calling the given function multiple times with two parameters: - // * key - // * value - void iteratePublicStateKeyValues(const std::function& f) const; - - // Iterate over public key values from the key after `after_key`, calling the given function multiple times with two - // parameters: - // * key - // * value - // - // If `after_key` is not a public key, false is returned and no iteration is done (no calls to `f`). Else, iteration - // is done and the returned value is true, even if there are 0 public keys to actually iterate. - bool iteratePublicStateKeyValues(const std::function& f, - const std::string& after_key) const; - // The key used in the default column family for persisting the current public state hash. static std::string publicStateHashKey(); private: - bool iteratePublicStateKeyValuesImpl(const std::function& f, - const std::optional& after_key) const; - BlockId addBlock(CategoryInput&& category_updates, concord::storage::rocksdb::NativeWriteBatch& write_batch); // tries to link the state transfer chain to the main blockchain @@ -318,12 +278,10 @@ class KeyValueBlockchain { const VersionedRawBlock& getLastRawBlock(KeyValueBlockchain& kvbc) { return kvbc.last_raw_block_; } }; // namespace concord::kvbc::categorization - std::string getPruningStatus(); - void setAggregator(std::shared_ptr aggregator) { aggregator_ = aggregator; delete_metrics_comp_.SetAggregator(aggregator_); - add_metrics_comp_.SetAggregator(aggregator); + add_metrics_comp_.SetAggregator(aggregator_); } friend struct KeyValueBlockchain_tester; diff --git a/kvbc/include/categorization/updates.h b/kvbc/include/categorization/updates.h index 29c66aea00..fb11ced20e 100644 --- a/kvbc/include/categorization/updates.h +++ b/kvbc/include/categorization/updates.h @@ -235,6 +235,8 @@ struct Updates { const CategoryInput& categoryUpdates() const { return category_updates_; } + CategoryInput&& categoryUpdates() { return std::move(category_updates_); } + // Appends a key-value of an `Update` type to already existing key-values for that category. // Precondition: The given `category_id` is of the same type as the passed updates. // Returns true on success or false if the given `category_id` doesn't exist. @@ -264,7 +266,6 @@ struct Updates { private: friend bool operator==(const Updates&, const Updates&); - friend class KeyValueBlockchain; CategoryInput category_updates_; }; diff --git a/kvbc/include/db_interfaces.h b/kvbc/include/db_interfaces.h index ebd1707694..b898aa657d 100644 --- a/kvbc/include/db_interfaces.h +++ b/kvbc/include/db_interfaces.h @@ -96,6 +96,10 @@ class IBlocksDeleter { // Throws on errors or if until <= genesis . virtual BlockId deleteBlocksUntil(BlockId until) = 0; + // This method should get the last block ID in the system and deletes it. + // The last block id is the latest block id. + virtual void deleteLastReachableBlock() = 0; + virtual ~IBlocksDeleter() = default; }; diff --git a/kvbc/include/kvbc_adapter/categorization/app_state_adapter.hpp b/kvbc/include/kvbc_adapter/categorization/app_state_adapter.hpp new file mode 100644 index 0000000000..1928e654fa --- /dev/null +++ b/kvbc/include/kvbc_adapter/categorization/app_state_adapter.hpp @@ -0,0 +1,87 @@ +// Concord +// +// Copyright (c) 2022-2023 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 subcomponent's license, as noted in the LICENSE +// file. + +#pragma once + +#include +#include +#include + +#include "Logger.hpp" +#include "thread_pool.hpp" +#include "blockchain_misc.hpp" +#include "kv_types.hpp" +#include "rocksdb/native_client.h" +#include "categorization/base_types.h" +#include "categorization/updates.h" +#include "categorized_kvbc_msgs.cmf.hpp" +#include "ReplicaConfig.hpp" +#include "db_interfaces.h" +#include "categorization/kv_blockchain.h" +#include "ReplicaResources.h" +#include "kvbc_adapter/replica_adapter_auxilliary_types.hpp" +#include "bcstatetransfer/SimpleBCStateTransfer.hpp" + +using concord::storage::rocksdb::NativeClient; + +namespace concord::kvbc::adapter::categorization { + +class AppStateAdapter : public bftEngine::bcst::IAppState { + public: + virtual ~AppStateAdapter() { kvbc_ = nullptr; } + explicit AppStateAdapter(std::shared_ptr &kvbc); + + ///////////////////////////////////////////////////////////////////////////////////////////////////////////////////// + // IAppState implementation + bool hasBlock(BlockId blockId) const override final { return kvbc_->hasBlock(blockId); } + bool getBlock(uint64_t blockId, + char *outBlock, + uint32_t outBlockMaxSize, + uint32_t *outBlockActualSize) const override final; + std::future getBlockAsync(uint64_t blockId, + char *outBlock, + uint32_t outBlockMaxSize, + uint32_t *outBlockActualSize) override final { + // This function should never be called + ConcordAssert(false); + return std::async([]() { return false; }); + } + bool getPrevDigestFromBlock(uint64_t blockId, bftEngine::bcst::StateTransferDigest *) const override final; + void getPrevDigestFromBlock(const char *blockData, + const uint32_t blockSize, + bftEngine::bcst::StateTransferDigest *outPrevBlockDigest) const override final; + bool putBlock(const uint64_t blockId, + const char *blockData, + const uint32_t blockSize, + bool lastBlock = true) override final; + std::future putBlockAsync(uint64_t blockId, + const char *block, + const uint32_t blockSize, + bool lastblock) override final { + // This functions should not be called + ConcordAssert(false); + return std::async([]() { return false; }); + } + uint64_t getLastReachableBlockNum() const override final { return kvbc_->getLastReachableBlockId(); } + uint64_t getGenesisBlockNum() const override final { return kvbc_->getGenesisBlockId(); } + // This method is used by state-transfer in order to find the latest block id in either the state-transfer chain or + // the main blockchain + uint64_t getLastBlockNum() const override final; + size_t postProcessUntilBlockId(uint64_t max_block_id) override final; + ///////////////////////////////////////////////////////////////////////////////////////////////////////////////////// + + private: + concord::kvbc::categorization::KeyValueBlockchain *kvbc_{nullptr}; + logging::Logger logger_; +}; +} // namespace concord::kvbc::adapter::categorization diff --git a/kvbc/include/kvbc_adapter/categorization/blocks_deleter_adapter.hpp b/kvbc/include/kvbc_adapter/categorization/blocks_deleter_adapter.hpp new file mode 100644 index 0000000000..b0ae21323f --- /dev/null +++ b/kvbc/include/kvbc_adapter/categorization/blocks_deleter_adapter.hpp @@ -0,0 +1,74 @@ +// Concord +// +// Copyright (c) 2022-2023 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 subcomponent's license, as noted in the LICENSE +// file. + +#pragma once + +#include +#include +#include + +#include "Logger.hpp" +#include "thread_pool.hpp" +#include "blockchain_misc.hpp" +#include "kv_types.hpp" +#include "rocksdb/native_client.h" +#include "categorization/base_types.h" +#include "categorization/updates.h" +#include "categorized_kvbc_msgs.cmf.hpp" +#include "ReplicaConfig.hpp" +#include "db_interfaces.h" +#include "categorization/kv_blockchain.h" +#include "ReplicaResources.h" +#include "kvbc_adapter/replica_adapter_auxilliary_types.hpp" + +using concord::storage::rocksdb::NativeClient; + +namespace concord::kvbc::adapter::categorization { + +class BlocksDeleterAdapter : public IBlocksDeleter { + public: + virtual ~BlocksDeleterAdapter() { kvbc_ = nullptr; } + explicit BlocksDeleterAdapter(std::shared_ptr &kvbc, + const std::optional &aux_types = std::nullopt); + + ///////////////////////////////////////////////////////////////////////////////////////////////////////////////////// + // IBlocksDeleter implementation + void deleteGenesisBlock() override final; + BlockId deleteBlocksUntil(BlockId until) override final; + void deleteLastReachableBlock() override final { return kvbc_->deleteLastReachableBlock(); } + ///////////////////////////////////////////////////////////////////////////////////////////////////////////////////// + + private: + concord::kvbc::categorization::KeyValueBlockchain *kvbc_{nullptr}; + std::shared_ptr replica_resources_; + + struct Recorders { + static constexpr uint64_t MAX_VALUE_MICROSECONDS = 2ULL * 1000ULL * 1000ULL; // 2 seconds + const std::string component_ = "iblockdeleter"; + + Recorders() { + auto ®istrar = concord::diagnostics::RegistrarSingleton::getInstance(); + if (!registrar.perf.isRegisteredComponent(component_)) { + registrar.perf.registerComponent(component_, {delete_batch_blocks_duration}); + } + } + + ~Recorders() {} + + DEFINE_SHARED_RECORDER( + delete_batch_blocks_duration, 1, MAX_VALUE_MICROSECONDS, 3, concord::diagnostics::Unit::MICROSECONDS); + }; + Recorders histograms_; +}; + +} // namespace concord::kvbc::adapter::categorization \ No newline at end of file diff --git a/kvbc/include/kvbc_adapter/categorization/db_checkpoint_adapter.hpp b/kvbc/include/kvbc_adapter/categorization/db_checkpoint_adapter.hpp new file mode 100644 index 0000000000..e725855bf2 --- /dev/null +++ b/kvbc/include/kvbc_adapter/categorization/db_checkpoint_adapter.hpp @@ -0,0 +1,43 @@ +// Concord +// +// Copyright (c) 2022-2023 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 subcomponent's license, as noted in the LICENSE +// file. + +#pragma once + +#include +#include + +#include "categorization/base_types.h" +#include "categorized_kvbc_msgs.cmf.hpp" +#include "state_snapshot_interface.hpp" +#include "categorization/kv_blockchain.h" + +namespace concord::kvbc::adapter::categorization { +class DbCheckpointImpl : public IDBCheckpoint { + public: + explicit DbCheckpointImpl(std::shared_ptr& kvbc); + + /////////////////////////////////IDBCheckpoint//////////////////////////////////////////////////////////////////////// + void trimBlocksFromCheckpoint(BlockId block_id_at_checkpoint) override final { + return kvbc_->trimBlocksFromSnapshot(block_id_at_checkpoint); + } + + void checkpointInProcess(bool) override final {} + + ////////////////////////////////////////////////////////////////////////////////////////////////////////////////////// + + virtual ~DbCheckpointImpl() { kvbc_ = nullptr; } + + private: + concord::kvbc::categorization::KeyValueBlockchain* kvbc_{nullptr}; +}; +} // namespace concord::kvbc::adapter::categorization diff --git a/kvbc/include/kvbc_adapter/categorization/kv_blockchain_adapter.hpp b/kvbc/include/kvbc_adapter/categorization/kv_blockchain_adapter.hpp new file mode 100644 index 0000000000..11742b76ed --- /dev/null +++ b/kvbc/include/kvbc_adapter/categorization/kv_blockchain_adapter.hpp @@ -0,0 +1,102 @@ +// Concord +// +// Copyright (c) 2022-2023 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 subcomponent's license, as noted in the LICENSE +// file. + +#pragma once + +#include +#include +#include + +#include "Logger.hpp" +#include "blockchain_misc.hpp" +#include "kv_types.hpp" +#include "rocksdb/native_client.h" +#include "categorization/base_types.h" +#include "categorization/updates.h" +#include "categorized_kvbc_msgs.cmf.hpp" +#include "ReplicaConfig.hpp" +#include "db_interfaces.h" +#include "categorization/kv_blockchain.h" +#include "ReplicaResources.h" +#include "kvbc_adapter/replica_adapter_auxilliary_types.hpp" + +using concord::storage::rocksdb::NativeClient; + +namespace concord::kvbc::adapter::categorization { + +class KeyValueBlockchain : public IReader, public IBlockAdder { + public: + virtual ~KeyValueBlockchain() { kvbc_ = nullptr; } + explicit KeyValueBlockchain(std::shared_ptr &kvbc); + + ///////////////////////////////////////////////////////////////////////////////////////////////////////////////////// + // IReader + std::optional get(const std::string &category_id, + const std::string &key, + BlockId block_id) const override final { + return kvbc_->get(category_id, key, block_id); + } + + std::optional getLatest(const std::string &category_id, + const std::string &key) const override final { + return kvbc_->getLatest(category_id, key); + } + + void multiGet(const std::string &category_id, + const std::vector &keys, + const std::vector &versions, + std::vector> &values) const override final { + return kvbc_->multiGet(category_id, keys, versions, values); + } + + void multiGetLatest(const std::string &category_id, + const std::vector &keys, + std::vector> &values) const override final { + return kvbc_->multiGetLatest(category_id, keys, values); + } + + std::optional getLatestVersion( + const std::string &category_id, const std::string &key) const override final { + return kvbc_->getLatestVersion(category_id, key); + } + + void multiGetLatestVersion( + const std::string &category_id, + const std::vector &keys, + std::vector> &versions) const override final { + return kvbc_->multiGetLatestVersion(category_id, keys, versions); + } + + std::optional getBlockUpdates(BlockId block_id) const override final { + return kvbc_->getBlockUpdates(block_id); + } + + // Get the current genesis block ID in the system. + BlockId getGenesisBlockId() const override final { return kvbc_->getGenesisBlockId(); } + + // Get the last block ID in the system. + BlockId getLastBlockId() const override final { return kvbc_->getLastReachableBlockId(); } + ///////////////////////////////////////////////////////////////////////////////////////////////////////////////////// + + ///////////////////////////////////////////////////////////////////////////////////////////////////////////////////// + // IBlockAdder + BlockId add(concord::kvbc::categorization::Updates &&updates) override final { + return kvbc_->addBlock(std::move(updates)); + } + ///////////////////////////////////////////////////////////////////////////////////////////////////////////////////// + + private: + concord::kvbc::categorization::KeyValueBlockchain *kvbc_{nullptr}; +}; + +} // namespace concord::kvbc::adapter::categorization diff --git a/kvbc/include/kvbc_adapter/common/state_snapshot_adapter.hpp b/kvbc/include/kvbc_adapter/common/state_snapshot_adapter.hpp new file mode 100644 index 0000000000..aba1c55813 --- /dev/null +++ b/kvbc/include/kvbc_adapter/common/state_snapshot_adapter.hpp @@ -0,0 +1,82 @@ +// Concord +// +// Copyright (c) 2022-2023 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 subcomponent's license, as noted in the LICENSE +// file. + +#pragma once + +#include +#include +#include + +#include "db_interfaces.h" +#include "categorization/base_types.h" +#include "categorized_kvbc_msgs.cmf.hpp" +#include "state_snapshot_interface.hpp" +#include "categorization/kv_blockchain.h" +#include "rocksdb/native_client.h" +#include "ReplicaConfig.hpp" + +namespace concord::kvbc::adapter::common::statesnapshot { +class KVBCStateSnapshot : public concord::kvbc::IKVBCStateSnapshot { + public: + explicit KVBCStateSnapshot(const concord::kvbc::IReader* reader, + const std::shared_ptr& native_client) + : reader_{reader}, native_client_(native_client) { + ConcordAssertNE(reader_, nullptr); + } + + ////////////////////////////IKVBCStateSnapshot//////////////////////////////////////////////////////////////////////// + // Computes and persists the public state hash by: + // h0 = hash("") + // h1 = hash(h0 || hash(k1) || v1) + // h2 = hash(h1 || hash(k2) || v2) + // ... + // hN = hash(hN-1 || hash(kN) || vN) + // + // This method is supposed to be called on DB snapshots only and not on the actual blockchain. + // Precondition: The current KeyValueBlockchain instance points to a DB snapshot. + void computeAndPersistPublicStateHash( + BlockId checkpoint_block_id, + const Converter& value_converter = [](std::string&& s) -> std::string { return std::move(s); }) override final; + + // Returns the public state keys as of the current point in the blockchain's history. + // Returns std::nullopt if no public keys have been persisted. + std::optional getPublicStateKeys() const override final; + + // Iterate over all public key values, calling the given function multiple times with two parameters: + // * key + // * value + void iteratePublicStateKeyValues(const std::function& f) const override final; + + // Iterate over public key values from the key after `after_key`, calling the given function multiple times with two + // parameters: + // * key + // * value + // + // If `after_key` is not a public key, false is returned and no iteration is done (no calls to `f`). Else, iteration + // is done and the returned value is true, even if there are 0 public keys to actually iterate. + bool iteratePublicStateKeyValues(const std::function& f, + const std::string& after_key) const override final; + + ////////////////////////////////////////////////////////////////////////////////////////////////////////////////////// + + virtual ~KVBCStateSnapshot() { reader_ = nullptr; } + + private: + bool iteratePublicStateKeyValuesImpl(const std::function& f, + const std::optional& after_key) const; + + private: + const concord::kvbc::IReader* reader_{nullptr}; + std::shared_ptr native_client_; +}; +} // namespace concord::kvbc::adapter::common::statesnapshot diff --git a/kvbc/include/categorization/categorized_reader.h b/kvbc/include/kvbc_adapter/idempotent_reader.h similarity index 86% rename from kvbc/include/categorization/categorized_reader.h rename to kvbc/include/kvbc_adapter/idempotent_reader.h index 11369616bb..900178bc7e 100644 --- a/kvbc/include/categorization/categorized_reader.h +++ b/kvbc/include/kvbc_adapter/idempotent_reader.h @@ -15,21 +15,23 @@ #include "assertUtils.hpp" #include "db_interfaces.h" -#include "kv_blockchain.h" +#include "replica_adapter.hpp" #include -namespace concord::kvbc::categorization { +namespace concord::kvbc::adapter { // A utility that adapts a KeyValueBlockchain instance to an IReader. -class CategorizedReader : public IReader { +class IdempotentReader : public IReader { public: - // Constructs a CategorizedReader from a non-null KeyValueBlockchain pointer. + // Constructs a IdempotentReader from a non-null KeyValueBlockchain pointer. // Precondition: kvbc != nullptr - CategorizedReader(const std::shared_ptr &kvbc) : kvbc_{kvbc} { + IdempotentReader(const std::shared_ptr &kvbc) : kvbc_{kvbc} { ConcordAssertNE(kvbc, nullptr); } + virtual ~IdempotentReader() = default; + public: std::optional get(const std::string &category_id, const std::string &key, @@ -72,10 +74,10 @@ class CategorizedReader : public IReader { BlockId getGenesisBlockId() const override { return kvbc_->getGenesisBlockId(); } - BlockId getLastBlockId() const override { return kvbc_->getLastReachableBlockId(); } + BlockId getLastBlockId() const override { return kvbc_->getLastBlockId(); } private: - const std::shared_ptr kvbc_; + const std::shared_ptr kvbc_; }; -} // namespace concord::kvbc::categorization +} // namespace concord::kvbc::adapter diff --git a/kvbc/include/kvbc_adapter/replica_adapter.hpp b/kvbc/include/kvbc_adapter/replica_adapter.hpp new file mode 100644 index 0000000000..e25cf07a9f --- /dev/null +++ b/kvbc/include/kvbc_adapter/replica_adapter.hpp @@ -0,0 +1,227 @@ +// Concord +// +// Copyright (c) 2022-2023 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 subcomponent's license, as noted in the LICENSE +// file. + +#pragma once + +#include +#include +#include +#include + +#include "assertUtils.hpp" + +#include "kv_types.hpp" +#include "rocksdb/native_client.h" +#include "blockchain_misc.hpp" +#include "ReplicaConfig.hpp" +#include "db_interfaces.h" +#include "bcstatetransfer/SimpleBCStateTransfer.hpp" +#include "state_snapshot_interface.hpp" +#include "ISystemResourceEntity.hpp" +#include "replica_adapter_auxilliary_types.hpp" +#include "categorization/kv_blockchain.h" +#include "v4blockchain/v4_blockchain.h" +#include "v4blockchain/detail/detail.h" + +namespace concord::kvbc::adapter { +class ReplicaBlockchain : public IBlocksDeleter, + public IReader, + public IBlockAdder, + public bftEngine::bcst::IAppState, + public IKVBCStateSnapshot, + public IDBCheckpoint { + public: + virtual ~ReplicaBlockchain(); + explicit ReplicaBlockchain( + const std::shared_ptr &native_client, + bool link_st_chain, + const std::optional> &category_types = std::nullopt, + const std::optional &aux_types = std::nullopt); + + ///////////////////////////////////////////////////////////////////////////////////////////////////////////////////// + // IBlocksDeleter implementation + void deleteGenesisBlock() override final { return deleter_->deleteGenesisBlock(); } + BlockId deleteBlocksUntil(BlockId until) override final { return deleter_->deleteBlocksUntil(until); } + void deleteLastReachableBlock() override final { return deleter_->deleteLastReachableBlock(); } + ///////////////////////////////////////////////////////////////////////////////////////////////////////////////////// + + ///////////////////////////////////////////////////////////////////////////////////////////////////////////////////// + // IReader + std::optional get(const std::string &category_id, + const std::string &key, + BlockId block_id) const override final { + return reader_->get(category_id, key, block_id); + } + + std::optional getLatest(const std::string &category_id, + const std::string &key) const override final { + return reader_->getLatest(category_id, key); + } + + void multiGet(const std::string &category_id, + const std::vector &keys, + const std::vector &versions, + std::vector> &values) const override final { + return reader_->multiGet(category_id, keys, versions, values); + } + + void multiGetLatest(const std::string &category_id, + const std::vector &keys, + std::vector> &values) const override final { + auto scoped = v4blockchain::detail::ScopedDuration{"multiGetLatest"}; + return reader_->multiGetLatest(category_id, keys, values); + } + + std::optional getLatestVersion(const std::string &category_id, + const std::string &key) const override final { + return reader_->getLatestVersion(category_id, key); + } + + void multiGetLatestVersion(const std::string &category_id, + const std::vector &keys, + std::vector> &versions) const override final { + auto scoped = v4blockchain::detail::ScopedDuration{"multiGetLatestVersion"}; + return reader_->multiGetLatestVersion(category_id, keys, versions); + } + + std::optional getBlockUpdates(BlockId block_id) const override final { + return reader_->getBlockUpdates(block_id); + } + + // Get the current genesis block ID in the system. + BlockId getGenesisBlockId() const override final { return reader_->getGenesisBlockId(); } + + // Get the last block ID in the system. + BlockId getLastBlockId() const override final { return reader_->getLastBlockId(); } + ///////////////////////////////////////////////////////////////////////////////////////////////////////////////////// + + ///////////////////////////////////////////////////////////////////////////////////////////////////////////////////// + // IBlockAdder + BlockId add(categorization::Updates &&updates) override final { + auto scoped = v4blockchain::detail::ScopedDuration{"Add block"}; + return adder_->add(std::move(updates)); + } + ///////////////////////////////////////////////////////////////////////////////////////////////////////////////////// + + ///////////////////////////////////////////////////////////////////////////////////////////////////////////////////// + // IAppState implementation + bool hasBlock(BlockId blockId) const override final { return app_state_->hasBlock(blockId); } + bool getBlock(uint64_t blockId, + char *outBlock, + uint32_t outBlockMaxSize, + uint32_t *outBlockActualSize) const override final { + return app_state_->getBlock(blockId, outBlock, outBlockMaxSize, outBlockActualSize); + } + + std::future getBlockAsync(uint64_t blockId, + char *outBlock, + uint32_t outBlockMaxSize, + uint32_t *outBlockActualSize) override final { + ConcordAssert(false); + return std::async([]() { return false; }); + } + + bool getPrevDigestFromBlock(uint64_t blockId, bftEngine::bcst::StateTransferDigest *st_digest) const override final { + return app_state_->getPrevDigestFromBlock(blockId, st_digest); + } + + void getPrevDigestFromBlock(const char *blockData, + const uint32_t blockSize, + bftEngine::bcst::StateTransferDigest *outPrevBlockDigest) const override final { + return app_state_->getPrevDigestFromBlock(blockData, blockSize, outPrevBlockDigest); + } + + bool putBlock(const uint64_t blockId, + const char *blockData, + const uint32_t blockSize, + bool lastBlock = true) override final { + return app_state_->putBlock(blockId, blockData, blockSize, lastBlock); + } + + std::future putBlockAsync(uint64_t blockId, + const char *block, + const uint32_t blockSize, + bool lastBlock) override final { + // This functions should not be called + ConcordAssert(false); + return std::async([]() { return false; }); + } + + uint64_t getLastReachableBlockNum() const override final { return app_state_->getLastReachableBlockNum(); } + uint64_t getGenesisBlockNum() const override final { return app_state_->getGenesisBlockNum(); } + // This method is used by state-transfer in order to find the latest block id in either the state-transfer chain or + // the main blockchain + uint64_t getLastBlockNum() const override final { return app_state_->getLastBlockNum(); } + size_t postProcessUntilBlockId(uint64_t max_block_id) override final { + return app_state_->postProcessUntilBlockId(max_block_id); + } + ////////////////////////////////////////////////////////////////////////////////////////////////////////////////////// + + ////////////////////////////////////IKVBCStateSnapshot//////////////////////////////////////////////////////////////// + void computeAndPersistPublicStateHash( + BlockId checkpoint_block_id, + const Converter &value_converter = [](std::string &&s) -> std::string { return std::move(s); }) override final { + state_snapshot_->computeAndPersistPublicStateHash(checkpoint_block_id, value_converter); + } + + std::optional getPublicStateKeys() const override final { + return state_snapshot_->getPublicStateKeys(); + } + + void iteratePublicStateKeyValues(const std::function &f) const override final { + state_snapshot_->iteratePublicStateKeyValues(f); + } + + bool iteratePublicStateKeyValues(const std::function &f, + const std::string &after_key) const override final { + return state_snapshot_->iteratePublicStateKeyValues(f, after_key); + } + + ////////////////////////////////////////////////////////////////////////////////////////////////////////////////////// + + ////////////////////////////////////IDBCheckpoint///////////////////////////////////////////////////////////////////// + void trimBlocksFromCheckpoint(BlockId block_id_at_checkpoint) override final { + return db_chkpt_->trimBlocksFromCheckpoint(block_id_at_checkpoint); + } + + void checkpointInProcess(bool flag) override final { db_chkpt_->checkpointInProcess(flag); } + ////////////////////////////////////////////////////////////////////////////////////////////////////////////////////// + + private: + void switch_to_rawptr(); + + private: + logging::Logger logger_; + //////////Common Interfaces ///////////////// + + ////////////////Set A of Unique Ptrs for better memory management //////////////////////////////////////////////////// + std::unique_ptr up_deleter_; + std::unique_ptr up_reader_; + std::unique_ptr up_adder_; + std::unique_ptr up_app_state_; + std::unique_ptr up_state_snapshot_; + std::unique_ptr up_db_chkpt_; + + ////////////////Set A of Ptrs for better performance //////////////////////////////////////////////////// + IBlocksDeleter *deleter_{nullptr}; + IReader *reader_{nullptr}; + IBlockAdder *adder_{nullptr}; + bftEngine::bcst::IAppState *app_state_{nullptr}; + IKVBCStateSnapshot *state_snapshot_{nullptr}; + IDBCheckpoint *db_chkpt_{nullptr}; + + //////////////Blockchain Abstractions ////////////////// + std::shared_ptr kvbc_{nullptr}; + std::shared_ptr v4_kvbc_{nullptr}; +}; +} // namespace concord::kvbc::adapter diff --git a/kvbc/include/kvbc_adapter/replica_adapter_auxilliary_types.hpp b/kvbc/include/kvbc_adapter/replica_adapter_auxilliary_types.hpp new file mode 100644 index 0000000000..a9740d320f --- /dev/null +++ b/kvbc/include/kvbc_adapter/replica_adapter_auxilliary_types.hpp @@ -0,0 +1,32 @@ +// Concord +// +// Copyright (c) 2022-2023 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 subcomponent's license, as noted in the LICENSE +// file. + +#pragma once +#include +#include +#include + +#include "Metrics.hpp" +#include "ISystemResourceEntity.hpp" + +namespace concord::kvbc::adapter::aux { + +struct AdapterAuxTypes { + explicit AdapterAuxTypes(std::shared_ptr aggregator, + concord::performance::ISystemResourceEntity& resource_entity) + : aggregator_(aggregator), resource_entity_(resource_entity) {} + std::shared_ptr aggregator_; + concord::performance::ISystemResourceEntity& resource_entity_; +}; + +} // namespace concord::kvbc::adapter::aux \ No newline at end of file diff --git a/kvbc/include/kvbc_adapter/v4blockchain/app_state_adapter.hpp b/kvbc/include/kvbc_adapter/v4blockchain/app_state_adapter.hpp new file mode 100644 index 0000000000..fbf855d46b --- /dev/null +++ b/kvbc/include/kvbc_adapter/v4blockchain/app_state_adapter.hpp @@ -0,0 +1,79 @@ +// Concord +// +// Copyright (c) 2022-2023 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 subcomponent's license, as noted in the LICENSE +// file. + +#pragma once + +#include + +#include "bcstatetransfer/SimpleBCStateTransfer.hpp" +#include "v4blockchain/v4_blockchain.h" +#include "kvbc_adapter/replica_adapter_auxilliary_types.hpp" + +namespace concord::kvbc::adapter::v4blockchain { + +class AppStateAdapter : public bftEngine::bcst::IAppState { + public: + virtual ~AppStateAdapter() { kvbc_ = nullptr; } + explicit AppStateAdapter(std::shared_ptr &kvbc) + : kvbc_{kvbc.get()} {} + + ///////////////////////////////////////////////////////////////////////////////////////////////////////////////////// + virtual bool hasBlock(uint64_t id) const override { return kvbc_->hasBlock(id); } + virtual bool getBlock(uint64_t blockId, + char *outBlock, + uint32_t outBlockMaxSize, + uint32_t *outBlockActualSize) const override; + + virtual std::future getBlockAsync(uint64_t blockId, + char *outBlock, + uint32_t outBlockMaxSize, + uint32_t *outBlockActualSize) override { + // This function is implemented in the replica.cpp + ConcordAssert(false); + return std::async([]() { return false; }); + } + virtual bool getPrevDigestFromBlock(uint64_t blockId, + bftEngine::bcst::StateTransferDigest *outPrevBlockDigest) const override; + + // Extracts a digest out of in-memory block (raw block). + virtual void getPrevDigestFromBlock(const char *blockData, + const uint32_t blockSize, + bftEngine::bcst::StateTransferDigest *outPrevBlockDigest) const override; + + virtual bool putBlock(const uint64_t blockId, + const char *blockData, + const uint32_t blockSize, + bool lastBlock) override; + + virtual std::future putBlockAsync(uint64_t blockId, + const char *block, + const uint32_t blockSize, + bool lastBlock) override { + // This function is implemented in replica.cpp + ConcordAssert(false); + return std::async([]() { return false; }); + } + + virtual uint64_t getLastReachableBlockNum() const override { return kvbc_->getLastReachableBlockId(); } + + virtual uint64_t getGenesisBlockNum() const override { return kvbc_->getGenesisBlockId(); } + + virtual uint64_t getLastBlockNum() const override; + + virtual size_t postProcessUntilBlockId(uint64_t max_block_id) override; + + private: + concord::kvbc::v4blockchain::KeyValueBlockchain *kvbc_{nullptr}; +}; + +} // namespace concord::kvbc::adapter::v4blockchain diff --git a/kvbc/include/kvbc_adapter/v4blockchain/blocks_adder_adapter.hpp b/kvbc/include/kvbc_adapter/v4blockchain/blocks_adder_adapter.hpp new file mode 100644 index 0000000000..ed7dd5d1ea --- /dev/null +++ b/kvbc/include/kvbc_adapter/v4blockchain/blocks_adder_adapter.hpp @@ -0,0 +1,43 @@ +// Concord +// +// Copyright (c) 2022-2023 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 subcomponent's license, as noted in the LICENSE +// file. + +#pragma once + +#include + +#include "db_interfaces.h" +#include "v4blockchain/v4_blockchain.h" + +using concord::storage::rocksdb::NativeClient; + +namespace concord::kvbc::adapter::v4blockchain { + +class BlocksAdderAdapter : public IBlockAdder { + public: + virtual ~BlocksAdderAdapter() { kvbc_ = nullptr; } + explicit BlocksAdderAdapter(std::shared_ptr &kvbc, + const std::optional &aux_types = std::nullopt) + : kvbc_{kvbc.get()} {} + + ///////////////////////////////////////////////////////////////////////////////////////////////////////////////////// + // IBlockAdder + BlockId add(concord::kvbc::categorization::Updates &&updates) override final { + return kvbc_->add(std::move(updates)); + } + ///////////////////////////////////////////////////////////////////////////////////////////////////////////////////// + + private: + concord::kvbc::v4blockchain::KeyValueBlockchain *kvbc_{nullptr}; +}; + +} // namespace concord::kvbc::adapter::v4blockchain diff --git a/kvbc/include/kvbc_adapter/v4blockchain/blocks_db_checkpoint_adapter.hpp b/kvbc/include/kvbc_adapter/v4blockchain/blocks_db_checkpoint_adapter.hpp new file mode 100644 index 0000000000..5fe492e986 --- /dev/null +++ b/kvbc/include/kvbc_adapter/v4blockchain/blocks_db_checkpoint_adapter.hpp @@ -0,0 +1,45 @@ +// Concord +// +// Copyright (c) 2022-2023 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 subcomponent's license, as noted in the LICENSE +// file. + +#pragma once + +#include +#include + +#include "categorization/base_types.h" +#include "categorized_kvbc_msgs.cmf.hpp" +#include "state_snapshot_interface.hpp" +#include "v4blockchain/v4_blockchain.h" + +namespace concord::kvbc::adapter::v4blockchain { +class BlocksDbCheckpointAdapter : public IDBCheckpoint { + public: + explicit BlocksDbCheckpointAdapter(std::shared_ptr& v4_kvbc) + : v4_kvbc_(v4_kvbc.get()) {} + + /////////////////////////////////IDBCheckpoint//////////////////////////////////////////////////////////////////////// + void trimBlocksFromCheckpoint(BlockId block_id_at_checkpoint) override final { + return v4_kvbc_->trimBlocksFromSnapshot(block_id_at_checkpoint); + } + + // Do nothing, its a NOOP for categorized blockchain + void checkpointInProcess(bool flag) override final { v4_kvbc_->checkpointInProcess(flag); } + + ////////////////////////////////////////////////////////////////////////////////////////////////////////////////////// + + virtual ~BlocksDbCheckpointAdapter() { v4_kvbc_ = nullptr; } + + private: + concord::kvbc::v4blockchain::KeyValueBlockchain* v4_kvbc_{nullptr}; +}; +} // namespace concord::kvbc::adapter::v4blockchain diff --git a/kvbc/include/kvbc_adapter/v4blockchain/blocks_deleter_adapter.hpp b/kvbc/include/kvbc_adapter/v4blockchain/blocks_deleter_adapter.hpp new file mode 100644 index 0000000000..5e7b2bd426 --- /dev/null +++ b/kvbc/include/kvbc_adapter/v4blockchain/blocks_deleter_adapter.hpp @@ -0,0 +1,65 @@ +// Concord +// +// Copyright (c) 2022-2023 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 subcomponent's license, as noted in the LICENSE +// file. + +#pragma once + +#include + +#include "db_interfaces.h" +#include "v4blockchain/v4_blockchain.h" +#include "kvbc_adapter/replica_adapter_auxilliary_types.hpp" +#include "resources-manager/ISystemResourceEntity.hpp" +#include "performance_handler.h" +#include "diagnostics.h" + +using concord::storage::rocksdb::NativeClient; + +namespace concord::kvbc::adapter::v4blockchain { + +class BlocksDeleterAdapter : public IBlocksDeleter { + public: + virtual ~BlocksDeleterAdapter() { kvbc_ = nullptr; } + explicit BlocksDeleterAdapter(std::shared_ptr &kvbc, + const std::optional &aux_types = std::nullopt); + + ///////////////////////////////////////////////////////////////////////////////////////////////////////////////////// + // IBlocksDeleter implementation + void deleteGenesisBlock() override final { kvbc_->deleteGenesisBlock(); } + void deleteLastReachableBlock() override final { return kvbc_->deleteLastReachableBlock(); } + BlockId deleteBlocksUntil(BlockId until) override final; + ///////////////////////////////////////////////////////////////////////////////////////////////////////////////////// + + private: + concord::kvbc::v4blockchain::KeyValueBlockchain *kvbc_{nullptr}; + std::shared_ptr replica_resources_; + + struct Recorders { + static constexpr uint64_t MAX_VALUE_MICROSECONDS = 2ULL * 1000ULL * 1000ULL; // 2 seconds + const std::string component_ = "iblockdeleter"; + + Recorders() { + auto ®istrar = concord::diagnostics::RegistrarSingleton::getInstance(); + if (!registrar.perf.isRegisteredComponent(component_)) { + registrar.perf.registerComponent(component_, {delete_batch_blocks_duration}); + } + } + + ~Recorders() {} + + DEFINE_SHARED_RECORDER( + delete_batch_blocks_duration, 1, MAX_VALUE_MICROSECONDS, 3, concord::diagnostics::Unit::MICROSECONDS); + }; + Recorders histograms_; +}; + +} // namespace concord::kvbc::adapter::v4blockchain diff --git a/kvbc/include/kvbc_adapter/v4blockchain/blocks_reader_adapter.hpp b/kvbc/include/kvbc_adapter/v4blockchain/blocks_reader_adapter.hpp new file mode 100644 index 0000000000..6c2012220d --- /dev/null +++ b/kvbc/include/kvbc_adapter/v4blockchain/blocks_reader_adapter.hpp @@ -0,0 +1,81 @@ +// Concord +// +// Copyright (c) 2022-2023 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 subcomponent's license, as noted in the LICENSE +// file. + +#pragma once + +#include + +#include "db_interfaces.h" +#include "v4blockchain/v4_blockchain.h" + +namespace concord::kvbc::adapter::v4blockchain { +class BlocksReaderAdapter : public IReader { + public: + virtual ~BlocksReaderAdapter() { kvbc_ = nullptr; } + explicit BlocksReaderAdapter(std::shared_ptr &kvbc) + : kvbc_{kvbc.get()} {} + + ///////////////////////////////////////////////////////////////////////////////////////////////////////////////////// + // IReader + std::optional get(const std::string &category_id, + const std::string &key, + BlockId block_id) const override final { + return kvbc_->get(category_id, key, block_id); + } + + std::optional getLatest(const std::string &category_id, + const std::string &key) const override final { + return kvbc_->getLatest(category_id, key); + } + + void multiGet(const std::string &category_id, + const std::vector &keys, + const std::vector &versions, + std::vector> &values) const override final { + return kvbc_->multiGet(category_id, keys, versions, values); + } + + void multiGetLatest(const std::string &category_id, + const std::vector &keys, + std::vector> &values) const override final { + return kvbc_->multiGetLatest(category_id, keys, values); + } + + std::optional getLatestVersion( + const std::string &category_id, const std::string &key) const override final { + return kvbc_->getLatestVersion(category_id, key); + } + + void multiGetLatestVersion( + const std::string &category_id, + const std::vector &keys, + std::vector> &versions) const override final { + return kvbc_->multiGetLatestVersion(category_id, keys, versions); + } + + std::optional getBlockUpdates(BlockId block_id) const override final { + return kvbc_->getBlockUpdates(block_id); + } + + // Get the current genesis block ID in the system. + BlockId getGenesisBlockId() const override final { return kvbc_->getGenesisBlockId(); } + + // Get the last block ID in the system. + BlockId getLastBlockId() const override final { return kvbc_->getLastReachableBlockId(); } + ///////////////////////////////////////////////////////////////////////////////////////////////////////////////////// + + private: + concord::kvbc::v4blockchain::KeyValueBlockchain *kvbc_{nullptr}; +}; + +} // namespace concord::kvbc::adapter::v4blockchain \ No newline at end of file diff --git a/kvbc/include/metadata_block_id.h b/kvbc/include/metadata_block_id.h index 16e739c12f..a12bf4dc3e 100644 --- a/kvbc/include/metadata_block_id.h +++ b/kvbc/include/metadata_block_id.h @@ -13,7 +13,7 @@ #pragma once #include "assertUtils.hpp" -#include "categorization/kv_blockchain.h" +#include "kvbc_adapter/replica_adapter.hpp" #include "endianness.hpp" #include "kv_types.hpp" #include "PersistentStorage.hpp" @@ -25,9 +25,9 @@ namespace concord::kvbc { // Persist the last KVBC block ID in big-endian in metadata's user data field. template -void persistLastBlockIdInMetadata(const categorization::KeyValueBlockchain &blockchain, +void persistLastBlockIdInMetadata(const adapter::ReplicaBlockchain &blockchain, const std::shared_ptr &metadata) { - const auto user_data = concordUtils::toBigEndianArrayBuffer(blockchain.getLastReachableBlockId()); + const auto user_data = concordUtils::toBigEndianArrayBuffer(blockchain.getLastBlockId()); if constexpr (in_transaction) { metadata->setUserDataInTransaction(user_data.data(), user_data.size()); } else { diff --git a/kvbc/include/reconfiguration_kvbc_handler.hpp b/kvbc/include/reconfiguration_kvbc_handler.hpp index d7426e8022..0c22099e72 100644 --- a/kvbc/include/reconfiguration_kvbc_handler.hpp +++ b/kvbc/include/reconfiguration_kvbc_handler.hpp @@ -15,12 +15,12 @@ #include "ReplicaConfig.hpp" #include "reconfiguration/ireconfiguration.hpp" #include "db_interfaces.h" +#include "blockchain_misc.hpp" #include "hex_tools.h" #include "block_metadata.hpp" #include "kvbc_key_types.hpp" #include "SigManager.hpp" #include "reconfiguration/reconfiguration_handler.hpp" -#include "categorization/kv_blockchain.h" #include "kvbc_app_filter/value_from_kvbc_proto.h" #include "AdaptivePruningManager.hpp" #include "IntervalMappingResourceManager.hpp" @@ -56,7 +56,7 @@ class StateSnapshotReconfigurationHandler : public ReconfigurationBlockTools, public: StateSnapshotReconfigurationHandler(kvbc::IBlockAdder& block_adder, kvbc::IReader& ro_storage, - const categorization::KeyValueBlockchain::Converter& state_value_converter, + const Converter& state_value_converter, const kvbc::LastApplicationTransactionTimeCallback& last_app_txn_time_cb_) : ReconfigurationBlockTools{block_adder, ro_storage}, state_value_converter_{state_value_converter}, @@ -97,7 +97,7 @@ class StateSnapshotReconfigurationHandler : public ReconfigurationBlockTools, private: // Allows users to convert state values to any format that is appropriate. // The default converter extracts the value from the ValueWithTrids protobuf type. - categorization::KeyValueBlockchain::Converter state_value_converter_{valueFromKvbcProto}; + Converter state_value_converter_{valueFromKvbcProto}; // Return the time of the last application-level transaction stored in the blockchain. // The result must be a string that can be parsed via google::protobuf::util::TimeUtil::FromString(). diff --git a/kvbc/include/replica_state_sync.h b/kvbc/include/replica_state_sync.h index 1f08963e76..4912d85355 100644 --- a/kvbc/include/replica_state_sync.h +++ b/kvbc/include/replica_state_sync.h @@ -18,7 +18,7 @@ #include "storage/db_types.h" #include "Logger.hpp" #include "db_adapter_interface.h" -#include "categorization/kv_blockchain.h" +#include "kvbc_adapter/replica_adapter.hpp" #include "PersistentStorage.hpp" #include @@ -33,7 +33,7 @@ class ReplicaStateSync { // Synchronizes replica state and returns a number of deleted blocks. virtual uint64_t execute(logging::Logger& logger, - categorization::KeyValueBlockchain& blockchain, + adapter::ReplicaBlockchain& blockchain, const std::shared_ptr& metadata, uint64_t lastExecutedSeqNum, uint32_t maxNumOfBlocksToDelete) = 0; diff --git a/kvbc/include/replica_state_sync_imp.hpp b/kvbc/include/replica_state_sync_imp.hpp index 6a31fb87d7..ed36da29b8 100644 --- a/kvbc/include/replica_state_sync_imp.hpp +++ b/kvbc/include/replica_state_sync_imp.hpp @@ -26,18 +26,18 @@ class ReplicaStateSyncImp : public ReplicaStateSync { ~ReplicaStateSyncImp() override = default; uint64_t execute(logging::Logger& logger, - categorization::KeyValueBlockchain& blockchain, + adapter::ReplicaBlockchain& blockchain, const std::shared_ptr& metadata, uint64_t lastExecutedSeqNum, uint32_t maxNumOfBlocksToDelete) override; uint64_t executeBasedOnBftSeqNum(logging::Logger& logger, - categorization::KeyValueBlockchain& blockchain, + adapter::ReplicaBlockchain& blockchain, uint64_t lastExecutedSeqNum, uint32_t maxNumOfBlocksToDelete); uint64_t executeBasedOnBlockId(logging::Logger& logger, - categorization::KeyValueBlockchain& blockchain, + adapter::ReplicaBlockchain& blockchain, const std::shared_ptr& metadata, uint32_t maxNumOfBlocksToDelete); diff --git a/kvbc/include/state_snapshot_interface.hpp b/kvbc/include/state_snapshot_interface.hpp new file mode 100644 index 0000000000..09c6f6ed7e --- /dev/null +++ b/kvbc/include/state_snapshot_interface.hpp @@ -0,0 +1,76 @@ +// Concord +// +// Copyright (c) 2022-2023 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 subcomponent's license, as noted in the LICENSE +// file. + +#pragma once +#include +#include +#include + +#include "kv_types.hpp" +#include "categorized_kvbc_msgs.cmf.hpp" +#include "blockchain_misc.hpp" + +namespace concord::kvbc { + +class IDBCheckpoint { + public: + // Trims the DB snapshot such that its last reachable block is equal to `block_id_at_checkpoint`. + // This method is supposed to be called on DB snapshots only and not on the actual blockchain. + // Precondition1: The current KeyValueBlockchain instance points to a DB snapshot. + // Precondition2: `block_id_at_checkpoint` >= INITIAL_GENESIS_BLOCK_ID + // Precondition3: `block_id_at_checkpoint` <= getLastReachableBlockId() + virtual void trimBlocksFromCheckpoint(BlockId block_id_at_checkpoint) = 0; + + // Notify storage the start and end of the checkpoint process + virtual void checkpointInProcess(bool) = 0; + + virtual ~IDBCheckpoint() = default; +}; + +// State snapshot support. +class IKVBCStateSnapshot { + public: + // Computes and persists the public state hash by: + // h0 = hash("") + // h1 = hash(h0 || hash(k1) || v1) + // h2 = hash(h1 || hash(k2) || v2) + // ... + // hN = hash(hN-1 || hash(kN) || vN) + // + // This method is supposed to be called on DB snapshots only and not on the actual blockchain. + // Precondition: The current KeyValueBlockchain instance points to a DB snapshot. + virtual void computeAndPersistPublicStateHash(BlockId checkpoint_block_id, const Converter& value_converter) = 0; + + // Returns the public state keys as of the current point in the blockchain's history. + // Returns std::nullopt if no public keys have been persisted. + virtual std::optional getPublicStateKeys() const = 0; + + // Iterate over all public key values, calling the given function multiple times with two parameters: + // * key + // * value + virtual void iteratePublicStateKeyValues(const std::function& f) const = 0; + + // Iterate over public key values from the key after `after_key`, calling the given function multiple times with two + // parameters: + // * key + // * value + // + // If `after_key` is not a public key, false is returned and no iteration is done (no calls to `f`). Else, iteration + // is done and the returned value is true, even if there are 0 public keys to actually iterate. + virtual bool iteratePublicStateKeyValues(const std::function& f, + const std::string& after_key) const = 0; + + virtual ~IKVBCStateSnapshot() = default; +}; + +} // namespace concord::kvbc \ No newline at end of file diff --git a/kvbc/include/v4blockchain/detail/blockchain.h b/kvbc/include/v4blockchain/detail/blockchain.h new file mode 100644 index 0000000000..147f15d9e8 --- /dev/null +++ b/kvbc/include/v4blockchain/detail/blockchain.h @@ -0,0 +1,118 @@ +// Concord +// +// Copyright (c) 2022 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 subcomponent's license, as noted in the LICENSE +// file. + +#pragma once + +// #include "blocks.h" +#include +#include +#include +#include +#include "rocksdb/native_client.h" +#include +#include "kv_types.hpp" +#include "v4blockchain/detail/blocks.h" +#include "categorization/updates.h" +#include "endianness.hpp" +#include "kv_types.hpp" +#include "thread_pool.hpp" +#include "v4blockchain/detail/column_families.h" + +namespace concord::kvbc::v4blockchain::detail { +/* + This class composes the blockchain out of detail::Block. + It knows to : + - add block to storage. + - read block from storage. + - maintain the state of the genesis and last reachable blocks. +*/ +class Blockchain { + public: + static constexpr auto MAX_BLOCK_ID = std::numeric_limits::max(); + static constexpr auto INVALID_BLOCK_ID = BlockId{0}; + + // creates the blockchain column family if it does not exists and loads the last and genesis block ids. + Blockchain(const std::shared_ptr& native_client); + ///////////////////ADD//////////////////////////////////////// + // construct a new block from the input updates and links it to the previous block by storing the last block digest. + BlockId addBlock(const concord::kvbc::categorization::Updates&, storage::rocksdb::NativeWriteBatch&); + //////////////////DELETE////////////////////////////////////// + // Delete up to until not including until if until is within last reachable block, + // else delete up to last reachable block and not including last reachable block. + // Do nothing of last reachable block is same as the genesis block. + BlockId deleteBlocksUntil(BlockId until); + void deleteGenesisBlock(); + void deleteLastReachableBlock(storage::rocksdb::NativeWriteBatch&); + ///////////////////State Transfer///////////////////////////////// + bool hasBlock(BlockId) const; + /////////////////////////////////////////////////////////////// + // Loads from storage the last and first block ids respectivly. + std::optional loadLastReachableBlockId(); + std::optional loadGenesisBlockId(); + void setLastReachable(BlockId id) { last_reachable_block_id_ = id; } + void setBlockId(BlockId id); + BlockId getLastReachable() const { return last_reachable_block_id_; } + BlockId getGenesisBlockId() const { return genesis_block_id_; } + + // Returns the buffer that represents the block + std::optional getBlockData(concord::kvbc::BlockId id) const; + std::optional getBlockUpdates(BlockId id) const; + + concord::util::digest::BlockDigest getBlockParentDigest(concord::kvbc::BlockId id) const; + + // Returns the actual values from blockchain DB for each of the block ids. + // This function expects unique blocks in block_ids. + // values is a result argument which will contain all the values for each block + // Order of blocks in block_ids is different from the order of blocks in the values + // block_ids.size() >= values.size() after the execution of this function. + void multiGetBlockData(const std::vector& block_ids, + std::unordered_map>& values) const; + + // Returns the actual values from blockchain DB in the form of update operations + // for each of the block ids. + // This block id in block_ids can be non-unique. + // values is a result argument which will contain all the values for each block + // Order of blocks in block_ids is different from the order of blocks in the values + // block_ids.size() >= values.size() after the execution of this function. + void multiGetBlockUpdates(std::vector block_ids, + std::unordered_map>& values) const; + + concord::util::digest::BlockDigest calculateBlockDigest(concord::kvbc::BlockId id) const; + + // Generates a key (big endian string representation) from the block id. + static std::string generateKey(BlockId id) { return concordUtils::toBigEndianStringBuffer(id); } + + // Non copyable and moveable + Blockchain(const Blockchain&) = delete; + Blockchain(Blockchain&&) = delete; + Blockchain& operator=(const Blockchain&) = delete; + Blockchain& operator=(Blockchain&&) = delete; + + // stats for tests + uint64_t from_future{}; + uint64_t from_storage{}; + + private: + void deleteBlock(BlockId id, storage::rocksdb::NativeWriteBatch& wb) { + wb.del(v4blockchain::detail::BLOCKS_CF, generateKey(id)); + } + + private: + std::atomic last_reachable_block_id_{INVALID_BLOCK_ID}; + std::atomic genesis_block_id_{INVALID_BLOCK_ID}; + std::shared_ptr native_client_; + util::ThreadPool thread_pool_{1}; + std::optional> future_digest_; +}; + +} // namespace concord::kvbc::v4blockchain::detail diff --git a/kvbc/include/v4blockchain/detail/blocks.h b/kvbc/include/v4blockchain/detail/blocks.h new file mode 100644 index 0000000000..1486ec1405 --- /dev/null +++ b/kvbc/include/v4blockchain/detail/blocks.h @@ -0,0 +1,111 @@ +// Concord +// +// Copyright (c) 2022 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 subcomponent's license, as noted in the LICENSE +// file. + +#pragma once + +// #include "blocks.h" +#include "kv_types.hpp" +#include "Digest.hpp" +#include "bcstatetransfer/SimpleBCStateTransfer.hpp" +#include +#include "categorized_kvbc_msgs.cmf.hpp" +#include "assertUtils.hpp" +#include "categorization/updates.h" +#include "v4blockchain/detail/detail.h" + +namespace concord::kvbc::v4blockchain::detail { +/* +Block is the unit that composes the block chain. +It recursively links the state by including the digest of the previous block. +In this implementation, the block will contains all data i.e. the set of key-values. +its layout is as follows +|VERSION|DIGEST|KEY-VALUES| +it uses a buffer, and fast access due to the known locations. +It is stored as is in DB and is a passable unit for the use of state-transfer. +*/ +class Block { + public: + static constexpr block_version BLOCK_VERSION = block_version::V1; + static constexpr uint64_t HEADER_SIZE = sizeof(version_type) + sizeof(concord::util::digest::BlockDigest); + // Pre-reserve buffer size + Block(uint64_t reserve_size) : buffer_(HEADER_SIZE, 0) { + buffer_.reserve(reserve_size); + addVersion(); + } + Block() : Block(HEADER_SIZE) {} + + Block(const std::string& buffer) : buffer_(buffer.size()) { + std::copy(buffer.cbegin(), buffer.cend(), buffer_.begin()); + } + + Block(const Block&) = default; + Block(Block&&) = default; + Block& operator=(const Block&) & = default; + Block& operator=(Block&&) & = default; + + const block_version& getVersion() const { + ConcordAssert(isValid_); + ConcordAssert(buffer_.size() >= sizeof(version_type)); + return *reinterpret_cast(buffer_.data()); + } + + const concord::util::digest::BlockDigest& parentDigest() const { + ConcordAssert(isValid_); + ConcordAssert(buffer_.size() >= HEADER_SIZE); + return *reinterpret_cast(buffer_.data() + sizeof(version_type)); + } + + void addDigest(const concord::util::digest::BlockDigest& digest) { + ConcordAssert(isValid_); + ConcordAssert(buffer_.size() >= HEADER_SIZE); + std::copy(digest.cbegin(), digest.cend(), buffer_.data() + sizeof(version_type)); + } + + concord::util::digest::BlockDigest calculateDigest(concord::kvbc::BlockId id) const { + ConcordAssert(isValid_); + return calculateDigest(id, reinterpret_cast(buffer_.data()), buffer_.size()); + } + + void addUpdates(const concord::kvbc::categorization::Updates& category_updates); + + concord::kvbc::categorization::Updates getUpdates() const; + + const std::vector& getBuffer() const { + ConcordAssert(isValid_); + return buffer_; + } + + std::vector&& moveBuffer() { + ConcordAssert(isValid_); + isValid_ = false; + return std::move(buffer_); + } + + static concord::util::digest::BlockDigest calculateDigest(concord::kvbc::BlockId id, + const char* buffer, + uint64_t size) { + return bftEngine::bcst::computeBlockDigest(id, buffer, size); + } + + private: + void addVersion() { + ConcordAssert(isValid_); + ConcordAssert(buffer_.size() >= sizeof(block_version)); + *((block_version*)buffer_.data()) = BLOCK_VERSION; + } + + std::vector buffer_; + bool isValid_{true}; +}; + +} // namespace concord::kvbc::v4blockchain::detail diff --git a/kvbc/include/v4blockchain/detail/categories.h b/kvbc/include/v4blockchain/detail/categories.h new file mode 100644 index 0000000000..3ee34bbdd0 --- /dev/null +++ b/kvbc/include/v4blockchain/detail/categories.h @@ -0,0 +1,60 @@ +// Concord +// +// Copyright (c) 2022 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 subcomponent's license, as noted in the LICENSE +// file. + +#pragma once +#include +#include +#include +#include "rocksdb/native_client.h" +#include +#include "categorization/base_types.h" + +namespace concord::kvbc::v4blockchain::detail { +/* + To support the notion of categories, introduced in categorized storage. + A category is mapped to a char that is used as the prefix of the key in the latest CF. +*/ +class Categories { + public: + // Magic number, from this version new categories won't be defined, therefore it's sufficient. + static constexpr int8_t MAX_NUM_CATEGORIES = 100; + // It's more convinient that the prefix is printable. + static constexpr char PREFIX_START = 0x21; + Categories(const std::shared_ptr&, + const std::optional>&); + + Categories() = delete; + + // Throws if the category does not exist + const std::string& categoryPrefix(const std::string& cat_id) const { return category_to_prefix_.at(cat_id); } + concord::kvbc::categorization::CATEGORY_TYPE categoryType(const std::string& cat_id) const { + return category_types_.at(cat_id); + } + + const std::unordered_map& prefixMap() const { return category_to_prefix_; } + + private: + void loadCategories(); + void initNewBlockchainCategories( + const std::optional>&); + void initExistingBlockchainCategories( + const std::optional>&); + void addNewCategory(const std::string&, concord::kvbc::categorization::CATEGORY_TYPE); + + private: + std::shared_ptr native_client_; + std::unordered_map category_to_prefix_; + std::unordered_map category_types_; +}; + +} // namespace concord::kvbc::v4blockchain::detail diff --git a/kvbc/include/v4blockchain/detail/column_families.h b/kvbc/include/v4blockchain/detail/column_families.h new file mode 100644 index 0000000000..0128d2f32f --- /dev/null +++ b/kvbc/include/v4blockchain/detail/column_families.h @@ -0,0 +1,26 @@ +// Concord +// +// Copyright (c) 2020-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 subcomponent's license, as noted in the LICENSE +// file. + +#pragma once + +#include + +namespace concord::kvbc::v4blockchain::detail { + +// Blockchain +inline const auto BLOCKS_CF = std::string{"v4_blocks"}; +inline const auto ST_CHAIN_CF = std::string{"v4_st_chain"}; +inline const auto LATEST_KEYS_CF = std::string{"v4_latest_keys"}; +inline const auto CATEGORIES_CF = std::string{"v4_categories"}; + +} // namespace concord::kvbc::v4blockchain::detail diff --git a/kvbc/include/v4blockchain/detail/detail.h b/kvbc/include/v4blockchain/detail/detail.h new file mode 100644 index 0000000000..f7352ebe5b --- /dev/null +++ b/kvbc/include/v4blockchain/detail/detail.h @@ -0,0 +1,31 @@ +// Concord +// +// Copyright (c) 2022 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 subcomponent's license, as noted in the LICENSE +// file. + +#pragma once +#include + +namespace concord::kvbc::v4blockchain::detail { +using version_type = uint16_t; +enum class block_version : version_type { V1 = 0x1 }; +struct ScopedDuration { + ScopedDuration(const char* msg) : msg_(msg) {} + ~ScopedDuration() { + auto jobDuration = + std::chrono::duration_cast(std::chrono::steady_clock::now() - start).count(); + LOG_INFO(V4_BLOCK_LOG, msg_ << " duration [" << jobDuration << "] micro"); + } + const char* msg_; + const std::chrono::time_point start = std::chrono::steady_clock::now(); +}; + +} // namespace concord::kvbc::v4blockchain::detail diff --git a/kvbc/include/v4blockchain/detail/latest_keys.h b/kvbc/include/v4blockchain/detail/latest_keys.h new file mode 100644 index 0000000000..603f73cb7b --- /dev/null +++ b/kvbc/include/v4blockchain/detail/latest_keys.h @@ -0,0 +1,160 @@ +// Concord +// +// Copyright (c) 2022 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 subcomponent's license, as noted in the LICENSE +// file. + +#pragma once + +#include "rocksdb/native_client.h" +#include +#include +#include "categorization/updates.h" +#include "v4blockchain/detail/categories.h" +#include "rocksdb/time_stamp_comparator.h" +#include +#include "endianness.hpp" + +namespace concord::kvbc::v4blockchain::detail { + +/* +The latest keys are the state of the blockchain. +It's implemented as a column family, where all the keys of a block that is being added are added to it. +It uses the RocksDb timestamp API to mark the version of the key. +A key can be: +- newly added - in this case it has a single version i.e. the block that it was added. +- updated - a key may have several versions (accessible) until we mark its history as save to delete. +- deleted - trying to access with the deletion version or higher will return nullopt. + +For backward compatibility with the previous categorized implementation, a key belongs to a category. +the category is represented by a prefix, the following properties of categories will be honored by this +implementation as well: +- version category : stale on update i.e. a keys is prunable although it's the latest version when its block is deleted. +- immutable - updating an immutable key is an error. +*/ +class LatestKeys { + public: + // Array is used for making it "sliceable" + using Flags = std::array; + // E.L need to be used with compaction filter + static constexpr Flags STALE_ON_UPDATE = {0x1}; + static constexpr size_t FLAGS_SIZE = STALE_ON_UPDATE.size(); + LatestKeys(const std::shared_ptr&, + const std::optional>&, + std::function&& f); + void addBlockKeys(const concord::kvbc::categorization::Updates&, BlockId, storage::rocksdb::NativeWriteBatch&); + + void handleCategoryUpdates(const std::string& block_version, + const std::string& category_id, + const concord::kvbc::categorization::BlockMerkleInput&, + concord::storage::rocksdb::NativeWriteBatch&); + void handleCategoryUpdates(const std::string& block_version, + const std::string& category_id, + const concord::kvbc::categorization::VersionedInput&, + concord::storage::rocksdb::NativeWriteBatch&); + void handleCategoryUpdates(const std::string& block_version, + const std::string& category_id, + const concord::kvbc::categorization::ImmutableInput&, + concord::storage::rocksdb::NativeWriteBatch&); + + // Delete the last added block keys + void revertLastBlockKeys(const concord::kvbc::categorization::Updates&, BlockId, storage::rocksdb::NativeWriteBatch&); + + void revertCategoryKeys(const std::string& block_version, + const std::string& prev_block_version, + const std::string& category_id, + const categorization::BlockMerkleInput& updates, + concord::storage::rocksdb::NativeWriteBatch& write_batch); + void revertCategoryKeys(const std::string& block_version, + const std::string& prev_block_version, + const std::string& category_id, + const categorization::VersionedInput& updates, + concord::storage::rocksdb::NativeWriteBatch& write_batch); + void revertCategoryKeys(const std::string& block_version, + const std::string& prev_block_version, + const std::string& category_id, + const categorization::ImmutableInput& updates, + concord::storage::rocksdb::NativeWriteBatch& write_batch); + + template + void revertCategoryKeysImp(const std::string& block_version, + const std::string& prev_block_version, + const std::string& category_id, + const UPDATES& updates, + concord::storage::rocksdb::NativeWriteBatch& write_batch); + template + void revertDeletedKeysImp(const std::string& block_version, + const std::string& prev_block_version, + const std::string& category_id, + const DELETES& deletes, + concord::storage::rocksdb::NativeWriteBatch& write_batch); + + const std::string& getCategoryPrefix(const std::string& category) const { + return category_mapping_.categoryPrefix(category); + } + // Mark a version that is safe to delete on compaction. + void trimHistoryUntil(BlockId block_id); + + static ::rocksdb::Slice getFlagsSlice(const ::rocksdb::Slice& val) { + ConcordAssertGT(val.size(), FLAGS_SIZE); + return ::rocksdb::Slice(val.data() + val.size() - FLAGS_SIZE, FLAGS_SIZE); + } + // check the key flags posfix for stale on update + static bool isStaleOnUpdate(const ::rocksdb::Slice& val) { + auto flags_sl = getFlagsSlice(val); + auto stale_flag = concord::storage::rocksdb::detail::toSlice(STALE_ON_UPDATE); + return flags_sl == stale_flag; + } + + ::rocksdb::CompactionFilter* getCompFilter() { return &comp_filter_; } + + // get the value and return deserialized value if needed. + std::optional getValue(BlockId latest_block_id, + const std::string& category_id, + const std::string& latest_version, + const std::string& key) const; + + // return multiple values, supposed to be more efficient. + void multiGetValue(BlockId latest_block_id, + const std::string& category_id, + const std::string& latest_version, + const std::vector& keys, + std::vector>& values) const; + + // returns the latest block id nearest to the last block id or latest version. + std::optional getLatestVersion(const std::string& category_id, + const std::string& latest_version, + const std::string& key) const; + // returns multiple latest block ids which which are nearest to the last block id or latest version. + void multiGetLatestVersion(const std::string& category_id, + const std::string& latest_version, + const std::vector& keys, + std::vector>& versions) const; + + private: + // This filter is used to delete stale on update keys if their version is smaller than the genesis block + // It's being called by RocksDB on compaction + struct LKCompactionFilter : ::rocksdb::CompactionFilter { + LKCompactionFilter(std::function f) : genesis_id(f) {} + const char* Name() const override { return "LatestKeysCompactionFilter"; } + bool Filter(int /*level*/, + const ::rocksdb::Slice& key, + const ::rocksdb::Slice& val, + std::string* /*new_value*/, + bool* /*value_changed*/) const override; + std::function genesis_id; + }; + + std::shared_ptr native_client_; + v4blockchain::detail::Categories category_mapping_; + LKCompactionFilter comp_filter_; +}; + +} // namespace concord::kvbc::v4blockchain::detail diff --git a/kvbc/include/v4blockchain/detail/st_chain.h b/kvbc/include/v4blockchain/detail/st_chain.h new file mode 100644 index 0000000000..739e93ed64 --- /dev/null +++ b/kvbc/include/v4blockchain/detail/st_chain.h @@ -0,0 +1,50 @@ +// Concord +// +// Copyright (c) 2022 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 subcomponent's license, as noted in the LICENSE +// file. + +#pragma once + +#include +#include "rocksdb/native_client.h" +#include "kv_types.hpp" +#include "v4blockchain/detail/blocks.h" +#include "v4blockchain/detail/blockchain.h" + +namespace concord::kvbc::v4blockchain::detail { + +class StChain { + public: + StChain(const std::shared_ptr&); + ////// Blocks operations///////////////////////////////// + bool hasBlock(kvbc::BlockId) const; + void addBlock(const kvbc::BlockId, const char* block, const uint32_t blockSize); + void deleteBlock(const kvbc::BlockId id, storage::rocksdb::NativeWriteBatch& wb); + std::optional getBlock(kvbc::BlockId) const; + // Returns the buffer that represents the block + std::optional getBlockData(concord::kvbc::BlockId) const; + concord::util::digest::BlockDigest getBlockParentDigest(concord::kvbc::BlockId id) const; + ///////// ST last block ID + void resetChain() { last_block_id_ = 0; } + void updateLastIdAfterDeletion(const kvbc::BlockId); + // reads the last block id from storage. + void loadLastBlockId(); + kvbc::BlockId getLastBlockId() const { return last_block_id_; } + // If last block id was deleted, laod from storage the new last. + void updateLastIdIfBigger(const kvbc::BlockId); + + private: + // if last_block_id_ is 0 it means no ST chain + std::atomic last_block_id_; + std::shared_ptr native_client_; +}; + +} // namespace concord::kvbc::v4blockchain::detail diff --git a/kvbc/include/v4blockchain/v4_blockchain.h b/kvbc/include/v4blockchain/v4_blockchain.h new file mode 100644 index 0000000000..6e7c5bab8f --- /dev/null +++ b/kvbc/include/v4blockchain/v4_blockchain.h @@ -0,0 +1,154 @@ +// Concord +// +// Copyright (c) 2020-2022 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 subcomponent's license, as noted in the LICENSE +// file. + +#pragma once + +#include "categorization/updates.h" +#include "rocksdb/native_client.h" +#include "v4blockchain/detail/st_chain.h" +#include "v4blockchain/detail/latest_keys.h" +#include "v4blockchain/detail/blockchain.h" +#include +#include + +namespace concord::kvbc::v4blockchain { +/* +This class is the entrypoint to storage. +It dispatches all calls to the relevant targets (blockchain,latest keys,state transfer) and glues the flows. +*/ +class KeyValueBlockchain { + public: + KeyValueBlockchain(const std::shared_ptr &, + bool link_st_chain, + const std::optional> + &category_types = std::nullopt); + + /////////////////////// Add Block /////////////////////// + BlockId add(categorization::Updates &&); + BlockId add(const categorization::Updates &, storage::rocksdb::NativeWriteBatch &); + ////////////////////// DELETE ////////////////////////// + BlockId deleteBlocksUntil(BlockId until); + void deleteGenesisBlock(); + void deleteLastReachableBlock(); + ///////////////////// State Transfer//////////////////// + // Returns true if a block exists in the blockchain or state-transfer chain + bool hasBlock(BlockId) const; + // if the block exists, returns the content of the block i.e. raw block + // the block origin can be the blockchain or the state-transfer chain + std::optional getBlockData(const BlockId &) const; + // Insert the block buffer to the ST chain, if last block is true, it links the ST chain + // To the blockchain. + void addBlockToSTChain(const BlockId &, const char *block, const uint32_t blockSize, bool lastBlock); + // Adds a range of blocks from the ST chain to the blockchain, + // The rangs starts from the blockchain last_reachable +1 + size_t linkUntilBlockId(BlockId until_block_id); + // Adds consecutive blocks from the ST chain to the blockchain until ST chain is empty or a gap is found. + void linkSTChain(); + // Atomic delete block from the ST chain and add to the blockchain. + void writeSTLinkTransaction(const BlockId, const categorization::Updates &); + // Each block contains the genesis block at the time of that block insertion. + // On State-transfer, we read this key and prune up to this block. + void pruneOnSTLink(const categorization::Updates &); + // Gets the digest from block, the digest represents the digest of the previous block i.e. parent digest + concord::util::digest::BlockDigest parentDigest(BlockId block_id) const; + std::optional getLastStatetransferBlockId() const; + + //////////////////Garbage collection for Keys that use TimeStamp API/////////////////////////// + // Using the RocksDB timestamp API, means that older user versions are not being deleted + // On compaction unless they are mark as safe to delete. + // If we get a new sequnce number it means that the previous sequence nunber was committed and it's + // safe to trim up to the last block that was added during that sn. + // An exception is when db checkpoint is being taken where no trimming is allowed. + uint64_t markHistoryForGarbageCollectionIfNeeded(const categorization::Updates &updates); + void checkpointInProcess(bool flag) { checkpointInProcess_ = flag; } + uint64_t getBlockSequenceNumber(const categorization::Updates &updates) const; + std::optional getLastBlockSequenceNumber() { return last_block_sn_; } + void setLastBlockSequenceNumber(uint64_t sn) { last_block_sn_ = sn; } + // Stats for testing + uint64_t gc_counter{}; + + // In v4 storage in contrast to the categorized storage, pruning does not impact the state i.e. the digest + // Of the blocks, in order to restrict deviation in the tail we add the genesis at the time the block is added, + // as part of the block. + // On state transfer completion this value can be used for pruning. + void addGenesisBlockKey(categorization::Updates &updates) const; + + const v4blockchain::detail::Blockchain &getBlockchain() const { return block_chain_; }; + const v4blockchain::detail::StChain &getStChain() const { return state_transfer_chain_; }; + const v4blockchain::detail::LatestKeys &getLatestKeys() const { return latest_keys_; }; + ///////////////////////////////////////////////////////////////////////////////////////////////////////////////////// + // IReader + std::optional get(const std::string &category_id, + const std::string &key, + BlockId block_id) const; + + std::optional getLatest(const std::string &category_id, const std::string &key) const; + + void multiGet(const std::string &category_id, + const std::vector &keys, + const std::vector &versions, + std::vector> &values) const; + + void multiGetLatest(const std::string &category_id, + const std::vector &keys, + std::vector> &values) const; + + std::optional getLatestVersion(const std::string &category_id, + const std::string &key) const; + + void multiGetLatestVersion(const std::string &category_id, + const std::vector &keys, + std::vector> &versions) const; + + std::optional getBlockUpdates(BlockId block_id) const { + return block_chain_.getBlockUpdates(block_id); + } + + // Get the current genesis block ID in the system. + BlockId getGenesisBlockId() const { return block_chain_.getGenesisBlockId(); } + + // Get the last block ID in the system. + BlockId getLastReachableBlockId() const { return block_chain_.getLastReachable(); } + + ///////////////////////////////////////////////////////////////////////////////////////////////////////////////////// + + // Trims the DB snapshot such that its last reachable block is equal to `block_id_at_checkpoint`. + // This will trim in latest keys and blocks column families + // This method is supposed to be called on DB snapshots only and not on the actual blockchain. + // Precondition1: The current KeyValueBlockchain instance points to a DB snapshot. + // Precondition2: `block_id_at_checkpoint` >= INITIAL_GENESIS_BLOCK_ID + // Precondition3: `block_id_at_checkpoint` <= getLastReachableBlockId() + void trimBlocksFromSnapshot(BlockId block_id_at_checkpoint); + + private: // Member functons + std::optional getValueFromUpdate(BlockId block_id, + const std::string &key, + const categorization::BlockMerkleInput &category_input) const; + std::optional getValueFromUpdate(BlockId block_id, + const std::string &key, + const categorization::VersionedInput &category_input) const; + std::optional getValueFromUpdate(BlockId block_id, + const std::string &key, + const categorization::ImmutableInput &category_input) const; + + private: // Data members + std::shared_ptr native_client_; + v4blockchain::detail::Blockchain block_chain_; + v4blockchain::detail::StChain state_transfer_chain_; + v4blockchain::detail::LatestKeys latest_keys_; + // flag to mark whether a checkpoint is being taken. + std::atomic_bool checkpointInProcess_{false}; + std::optional last_block_sn_; +}; + +} // namespace concord::kvbc::v4blockchain diff --git a/kvbc/src/Replica.cpp b/kvbc/src/Replica.cpp index c2ba5f2ade..49dbda8d68 100644 --- a/kvbc/src/Replica.cpp +++ b/kvbc/src/Replica.cpp @@ -2,36 +2,29 @@ // // KV Blockchain replica implementation. -#include -#include #include -#include #include "Replica.h" #include -#include -#include -#include -#include #include #include #include #include "assertUtils.hpp" #include "endianness.hpp" +#include "json_output.hpp" #include "communication/CommDefs.hpp" #include "kv_types.hpp" -#include "hex_tools.h" #include "replica_state_sync.h" #include "sliver.hpp" #include "metadata_block_id.h" #include "bftengine/DbMetadataStorage.hpp" #include "rocksdb/native_client.h" +#include "categorization/blocks.h" #include "pruning_handler.hpp" #include "IRequestHandler.hpp" #include "RequestHandler.h" #include "reconfiguration_kvbc_handler.hpp" #include "st_reconfiguraion_sm.hpp" #include "bftengine/ControlHandler.hpp" -#include "throughput.hpp" #include "bftengine/EpochManager.hpp" #include "bftengine/ReconfigurationCmd.hpp" #include "client/reconfiguration/st_based_reconfiguration_client.hpp" @@ -127,7 +120,7 @@ class KvbcRequestHandler : public bftEngine::RequestHandler { static std::shared_ptr create( const std::shared_ptr &user_req_handler, const std::shared_ptr &cron_table_registry, - categorization::KeyValueBlockchain &blockchain, + adapter::ReplicaBlockchain &blockchain, std::shared_ptr aggregator_, ISystemResourceEntity &resourceEntity) { return std::shared_ptr{ @@ -149,7 +142,7 @@ class KvbcRequestHandler : public bftEngine::RequestHandler { private: KvbcRequestHandler(const std::shared_ptr &user_req_handler, const std::shared_ptr &cron_table_registry, - categorization::KeyValueBlockchain &blockchain, + adapter::ReplicaBlockchain &blockchain, std::shared_ptr aggregator_, ISystemResourceEntity &resourceEntity) : bftEngine::RequestHandler(resourceEntity, aggregator_), blockchain_{blockchain} { @@ -164,7 +157,7 @@ class KvbcRequestHandler : public bftEngine::RequestHandler { private: std::shared_ptr persistent_storage_; - categorization::KeyValueBlockchain &blockchain_; + adapter::ReplicaBlockchain &blockchain_; }; void Replica::registerReconfigurationHandlers(std::shared_ptr requestHandler) { requestHandler->setReconfigurationHandler(std::make_shared( @@ -303,7 +296,7 @@ void Replica::saveReconfigurationCmdToResPages(const std::string &key) { } void Replica::createReplicaAndSyncState() { - ConcordAssert(m_kvBlockchain.has_value()); + ConcordAssertNE(m_kvBlockchain, nullptr); auto requestHandler = KvbcRequestHandler::create(m_cmdHandler, cronTableRegistry_, *m_kvBlockchain, aggregator_, replicaResources_); registerReconfigurationHandlers(requestHandler); @@ -356,10 +349,11 @@ void Replica::createReplicaAndSyncState() { auto db = storage::rocksdb::NativeClient::newClient( path, read_only, storage::rocksdb::NativeClient::DefaultOptions{}); const auto link_st_chain = false; - auto kvbc = categorization::KeyValueBlockchain{db, link_st_chain}; - kvbc.trimBlocksFromSnapshot(block_id_at_checkpoint); + auto kvbc = adapter::ReplicaBlockchain{db, link_st_chain}; + kvbc.trimBlocksFromCheckpoint(block_id_at_checkpoint); kvbc.computeAndPersistPublicStateHash(block_id_at_checkpoint, value_converter); - }); + }, + [this](bool flag) { checkpointInProcess(flag); }); } /** @@ -381,26 +375,12 @@ BlockId Replica::addBlockToIdleReplica(categorization::Updates &&updates) { throw std::logic_error{"addBlockToIdleReplica() called on a non-idle replica"}; } - return m_kvBlockchain->addBlock(std::move(updates)); + return m_kvBlockchain->add(std::move(updates)); } -void Replica::deleteGenesisBlock() { - const auto genesisBlock = m_kvBlockchain->getGenesisBlockId(); - if (genesisBlock == 0) { - throw std::logic_error{"Cannot delete the genesis block from an empty blockchain"}; - } - m_kvBlockchain->deleteBlock(genesisBlock); -} +void Replica::deleteGenesisBlock() { return m_kvBlockchain->deleteGenesisBlock(); } BlockId Replica::deleteBlocksUntil(BlockId until) { - ISystemResourceEntity::scopedDurMeasurment mes(replicaResources_, ISystemResourceEntity::type::pruning_utilization); - const auto genesisBlock = m_kvBlockchain->getGenesisBlockId(); - if (genesisBlock == 0) { - throw std::logic_error{"Cannot delete a block range from an empty blockchain"}; - } else if (until <= genesisBlock) { - throw std::invalid_argument{"Invalid 'until' value passed to deleteBlocksUntil()"}; - } - // Inform State Transfer about pruning. We must do it in this thread context for persistency considerations, and in // this layer to lower the chance for bugs (there are multiple callers to this function), in which pruning is not // notified to ST. In that case, ST state will be corrupted. @@ -408,24 +388,15 @@ BlockId Replica::deleteBlocksUntil(BlockId until) { // We assume until > 0, see check above m_stateTransfer->reportLastAgreedPrunableBlockId(until - 1); } - - const auto lastReachableBlock = m_kvBlockchain->getLastReachableBlockId(); - const auto lastDeletedBlock = std::min(lastReachableBlock, until - 1); - const auto start = std::chrono::steady_clock::now(); - for (auto i = genesisBlock; i <= lastDeletedBlock; ++i) { - ISystemResourceEntity::scopedDurMeasurment mes(replicaResources_, - ISystemResourceEntity::type::pruning_avg_time_micro); - ConcordAssert(m_kvBlockchain->deleteBlock(i)); - } - auto jobDuration = - std::chrono::duration_cast(std::chrono::steady_clock::now() - start).count(); - histograms_.delete_batch_blocks_duration->recordAtomic(jobDuration); - return lastDeletedBlock; + ISystemResourceEntity::scopedDurMeasurment mes(replicaResources_, ISystemResourceEntity::type::pruning_utilization); + return m_kvBlockchain->deleteBlocksUntil(until); } +void Replica::deleteLastReachableBlock() { return m_kvBlockchain->deleteLastReachableBlock(); } + BlockId Replica::add(categorization::Updates &&updates) { replicaResources_.addMeasurement({ISystemResourceEntity::type::add_blocks_accumulated, 1, 0, 0}); - return m_kvBlockchain->addBlock(std::move(updates)); + return m_kvBlockchain->add(std::move(updates)); } std::optional Replica::get(const std::string &category_id, @@ -475,9 +446,11 @@ BlockId Replica::getLastBlockId() const { if (replicaConfig_.isReadOnly) { return m_bcDbAdapter->getLastReachableBlockId(); } - return m_kvBlockchain->getLastReachableBlockId(); + return m_kvBlockchain->getLastBlockId(); } +void Replica::checkpointInProcess(bool flag) { m_kvBlockchain->checkpointInProcess(flag); } + void Replica::set_command_handler(std::shared_ptr handler) { m_cmdHandler = handler; } Replica::Replica(ICommunication *comm, @@ -497,8 +470,8 @@ Replica::Replica(ICommunication *comm, aggregator_(aggregator), pm_{pm}, secretsManager_{secretsManager}, - blocksIOWorkersPool_((replicaConfig.numWorkerThreadsForBlockIO > 0) ? replicaConfig.numWorkerThreadsForBlockIO - : std::thread::hardware_concurrency()), + blocks_io_workers_pool((replicaConfig.numWorkerThreadsForBlockIO > 0) ? replicaConfig.numWorkerThreadsForBlockIO + : std::thread::hardware_concurrency()), AdaptivePruningManager_{ concord::performance::IntervalMappingResourceManager::createIntervalMappingResourceManager( replicaResources_, @@ -586,17 +559,35 @@ Replica::Replica(ICommunication *comm, throw std::invalid_argument{msg}; } } - m_kvBlockchain.emplace( - storage::rocksdb::NativeClient::fromIDBClient(m_dbSet.dataDBClient), linkStChain, kvbc_categories); - m_kvBlockchain->setAggregator(aggregator); - + op_kvBlockchain.emplace(storage::rocksdb::NativeClient::fromIDBClient(m_dbSet.dataDBClient), + linkStChain, + kvbc_categories, + concord::kvbc::adapter::aux::AdapterAuxTypes(this->aggregator_, this->replicaResources_)); + m_kvBlockchain = &(op_kvBlockchain.value()); + LOG_INFO(logger, "ARC06: Replica::REPLICA" << KVLOG(this, m_kvBlockchain)); auto ®istrar = concord::diagnostics::RegistrarSingleton::getInstance(); - concord::diagnostics::StatusHandler handler( - "pruning", "Pruning Status", [this]() { return m_kvBlockchain->getPruningStatus(); }); + concord::diagnostics::StatusHandler handler("pruning", "Pruning Status", [this]() { + std::ostringstream oss; + std::unordered_map result; + result.insert( + concordUtils::toPair("versionedNumOfDeletedKeys", + aggregator_->GetCounter("kv_blockchain_deletes", "numOfVersionedKeysDeleted").Get())); + result.insert( + concordUtils::toPair("immutableNumOfDeletedKeys", + aggregator_->GetCounter("kv_blockchain_deletes", "numOfImmutableKeysDeleted").Get())); + result.insert(concordUtils::toPair( + "merkleNumOfDeletedKeys", aggregator_->GetCounter("kv_blockchain_deletes", "numOfMerkleKeysDeleted").Get())); + result.insert(concordUtils::toPair("getGenesisBlockId()", getGenesisBlockId())); + result.insert(concordUtils::toPair("getLastReachableBlockId()", getLastBlockId())); + result.insert(concordUtils::toPair("isPruningInProgress", + bftEngine::ControlStateManager::instance().getPruningProcessStatus())); + oss << concordUtils::kContainerToJson(result); + return oss.str(); + }); registrar.status.registerHandler(handler); } - m_dbSet.dataDBClient->setAggregator(aggregator); - m_dbSet.metadataDBClient->setAggregator(aggregator); + m_dbSet.dataDBClient->setAggregator(aggregator_); + m_dbSet.metadataDBClient->setAggregator(aggregator_); auto stKeyManipulator = std::shared_ptr{storageFactory->newSTKeyManipulator()}; m_stateTransfer = bftEngine::bcst::create(stConfig, this, m_metadataDBClient, stKeyManipulator, aggregator_); if (!replicaConfig.isReadOnly) { @@ -620,6 +611,9 @@ Replica::~Replica() { } } if (creEngine_) creEngine_->stop(); + if (m_kvBlockchain) { + m_kvBlockchain = nullptr; + } } /* @@ -630,26 +624,7 @@ bool Replica::putBlock(const uint64_t blockId, const char *blockData, const uint if (replicaConfig_.isReadOnly) { return putBlockToObjectStore(blockId, blockData, blockSize, lastBlock); } - - auto view = std::string_view{blockData, blockSize}; - const auto rawBlock = categorization::RawBlock::deserialize(view); - if (m_kvBlockchain->hasBlock(blockId)) { - const auto existingRawBlock = m_kvBlockchain->getRawBlock(blockId); - if (rawBlock != existingRawBlock) { - LOG_ERROR(logger, - "found existing (and different) block ID[" << blockId << "] when receiving from state transfer"); - - // TODO consider assert? - m_kvBlockchain->deleteBlock(blockId); - throw std::runtime_error( - __PRETTY_FUNCTION__ + - std::string("found existing (and different) block when receiving state transfer, block ID: ") + - std::to_string(blockId)); - } - } else { - m_kvBlockchain->addRawBlock(rawBlock, blockId, lastBlock); - } - return true; + return m_kvBlockchain->putBlock(blockId, blockData, blockSize, lastBlock); } std::future Replica::putBlockAsync(uint64_t blockId, @@ -659,7 +634,7 @@ std::future Replica::putBlockAsync(uint64_t blockId, static uint64_t callCounter = 0; static constexpr size_t snapshotThresh = 1000; - auto future = blocksIOWorkersPool_.async( + auto future = blocks_io_workers_pool.async( [this](uint64_t blockId, const char *block, const uint32_t blockSize, bool lastBlock) { auto start = std::chrono::steady_clock::now(); bool result = false; @@ -701,7 +676,7 @@ uint64_t Replica::getLastReachableBlockNum() const { if (replicaConfig_.isReadOnly) { return m_bcDbAdapter->getLastReachableBlockId(); } - return m_kvBlockchain->getLastReachableBlockId(); + return m_kvBlockchain->getLastReachableBlockNum(); } uint64_t Replica::getGenesisBlockNum() const { return getGenesisBlockId(); } @@ -710,11 +685,7 @@ uint64_t Replica::getLastBlockNum() const { if (replicaConfig_.isReadOnly) { return m_bcDbAdapter->getLatestBlockId(); } - const auto last = m_kvBlockchain->getLastStatetransferBlockId(); - if (last) { - return *last; - } - return m_kvBlockchain->getLastReachableBlockId(); + return m_kvBlockchain->getLastBlockNum(); } size_t Replica::postProcessUntilBlockId(uint64_t max_block_id) { @@ -722,34 +693,7 @@ size_t Replica::postProcessUntilBlockId(uint64_t max_block_id) { // read only replica do not post process return 0; } - const BlockId last_reachable_block = m_kvBlockchain->getLastReachableBlockId(); - BlockId last_st_block_id = 0; - if (auto last_st_block_id_opt = m_kvBlockchain->getLastStatetransferBlockId()) { - last_st_block_id = last_st_block_id_opt.value(); - } - if ((max_block_id == last_reachable_block) && (last_st_block_id == 0)) { - LOG_INFO(CAT_BLOCK_LOG, - "Consensus blockchain is fully linked, no proc-processing is required!" - << KVLOG(max_block_id, last_reachable_block)); - return 0; - } - if ((max_block_id < last_reachable_block) || (max_block_id > last_st_block_id)) { - auto msg = std::stringstream{}; - msg << "Cannot post-process:" << KVLOG(max_block_id, last_reachable_block, last_st_block_id) << std::endl; - throw std::invalid_argument{msg.str()}; - } - - try { - return m_kvBlockchain->linkUntilBlockId(last_reachable_block + 1, max_block_id); - } catch (const std::exception &e) { - LOG_FATAL( - CAT_BLOCK_LOG, - "Aborting due to failure to link," << KVLOG(last_reachable_block, max_block_id) << ", reason: " << e.what()); - std::terminate(); - } catch (...) { - LOG_FATAL(CAT_BLOCK_LOG, "Aborting due to failure to link," << KVLOG(last_reachable_block, max_block_id)); - std::terminate(); - } + return m_kvBlockchain->postProcessUntilBlockId(max_block_id); } RawBlock Replica::getBlockInternal(BlockId blockId) const { return m_bcDbAdapter->getRawBlock(blockId); } @@ -762,19 +706,7 @@ bool Replica::getBlock(uint64_t blockId, char *outBlock, uint32_t outBlockMaxSiz if (replicaConfig_.isReadOnly) { return getBlockFromObjectStore(blockId, outBlock, outBlockMaxSize, outBlockActualSize); } - const auto rawBlock = m_kvBlockchain->getRawBlock(blockId); - if (!rawBlock) { - throw NotFoundException{"Raw block not found: " + std::to_string(blockId)}; - } - const auto &ser = categorization::RawBlock::serialize(*rawBlock); - if (ser.size() > outBlockMaxSize) { - LOG_ERROR(logger, KVLOG(ser.size(), outBlockMaxSize)); - throw std::runtime_error("not enough space to copy block!"); - } - *outBlockActualSize = ser.size(); - LOG_DEBUG(logger, KVLOG(blockId, *outBlockActualSize)); - std::memcpy(outBlock, ser.data(), *outBlockActualSize); - return true; + return m_kvBlockchain->getBlock(blockId, outBlock, outBlockMaxSize, outBlockActualSize); } std::future Replica::getBlockAsync(uint64_t blockId, @@ -783,8 +715,7 @@ std::future Replica::getBlockAsync(uint64_t blockId, uint32_t *outBlockActualSize) { static uint64_t callCounter = 0; static constexpr size_t snapshotThresh = 1000; - - auto future = blocksIOWorkersPool_.async( + auto future = blocks_io_workers_pool.async( [this](uint64_t blockId, char *outBlock, uint32_t outBlockMaxSize, uint32_t *outBlockActualSize) { bool result = false; auto start = std::chrono::steady_clock::now(); @@ -849,29 +780,16 @@ bool Replica::getPrevDigestFromBlock(BlockId blockId, StateTransferDigest *outPr if (replicaConfig_.isReadOnly) { return getPrevDigestFromObjectStoreBlock(blockId, outPrevBlockDigest); } - ConcordAssert(blockId > 0); - const auto parent_digest = m_kvBlockchain->parentDigest(blockId); - - if (!parent_digest.has_value()) { - LOG_WARN(logger, "parent digest not found," << KVLOG(blockId)); - return false; - } - static_assert(parent_digest->size() == DIGEST_SIZE); - static_assert(sizeof(StateTransferDigest) == DIGEST_SIZE); - std::memcpy(outPrevBlockDigest, parent_digest->data(), DIGEST_SIZE); - return true; + return m_kvBlockchain->getPrevDigestFromBlock(blockId, outPrevBlockDigest); } void Replica::getPrevDigestFromBlock(const char *blockData, const uint32_t blockSize, StateTransferDigest *outPrevBlockDigest) const { - ConcordAssertGT(blockSize, 0); - auto view = std::string_view{blockData, blockSize}; - const auto rawBlock = categorization::RawBlock::deserialize(view); - - static_assert(rawBlock.data.parent_digest.size() == DIGEST_SIZE); - static_assert(sizeof(StateTransferDigest) == DIGEST_SIZE); - std::memcpy(outPrevBlockDigest, rawBlock.data.parent_digest.data(), DIGEST_SIZE); + if (replicaConfig_.isReadOnly) { + return getPrevDigestFromObjectStoreBlock(blockData, blockSize, outPrevBlockDigest); + } + return m_kvBlockchain->getPrevDigestFromBlock(blockData, blockSize, outPrevBlockDigest); } bool Replica::getPrevDigestFromObjectStoreBlock(uint64_t blockId, @@ -891,6 +809,18 @@ bool Replica::getPrevDigestFromObjectStoreBlock(uint64_t blockId, } } +void Replica::getPrevDigestFromObjectStoreBlock(const char *blockData, + const uint32_t blockSize, + StateTransferDigest *outPrevBlockDigest) const { + ConcordAssertGT(blockSize, 0); + auto view = std::string_view{blockData, blockSize}; + const auto rawBlock = categorization::RawBlock::deserialize(view); + + static_assert(rawBlock.data.parent_digest.size() == DIGEST_SIZE); + static_assert(sizeof(StateTransferDigest) == DIGEST_SIZE); + std::memcpy(outPrevBlockDigest, rawBlock.data.parent_digest.data(), DIGEST_SIZE); +} + void Replica::registerStBasedReconfigurationHandler( std::shared_ptr handler) { // api for higher level application to register the handler diff --git a/kvbc/src/block_metadata.cpp b/kvbc/src/block_metadata.cpp index b481c95eb9..1d7a3eaceb 100644 --- a/kvbc/src/block_metadata.cpp +++ b/kvbc/src/block_metadata.cpp @@ -20,9 +20,7 @@ uint64_t BlockMetadata::getLastBlockSequenceNum() const { IBlockMetadata::kBlockMetadataKeyStr); auto sequenceNum = uint64_t{0}; if (value) { - const auto& data = std::get(*value).data; - ConcordAssertEQ(data.size(), sizeof(uint64_t)); - sequenceNum = concordUtils::fromBigEndianBuffer(data.data()); + return getSequenceNum(std::get(*value).data); } else { LOG_WARN(logger_, "Unable to get last block sequence number"); } @@ -31,5 +29,10 @@ uint64_t BlockMetadata::getLastBlockSequenceNum() const { return sequenceNum; } +uint64_t BlockMetadata::getSequenceNum(const std::string& data) { + ConcordAssertEQ(data.size(), sizeof(uint64_t)); + return concordUtils::fromBigEndianBuffer(data.data()); +} + } // namespace kvbc } // namespace concord diff --git a/kvbc/src/blockchain_misc.cpp b/kvbc/src/blockchain_misc.cpp new file mode 100644 index 0000000000..de9e2100ed --- /dev/null +++ b/kvbc/src/blockchain_misc.cpp @@ -0,0 +1,23 @@ +// Concord +// +// Copyright (c) 2022-2023 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 subcomponent's license, as noted in the LICENSE +// file. + +#include "blockchain_misc.hpp" +#include "storage/merkle_tree_key_manipulator.h" + +namespace concord::kvbc::bcutil { + +static const auto kPublicStateHashKey = concord::storage::v2MerkleTree::detail::serialize( + concord::storage::v2MerkleTree::detail::EBFTSubtype::PublicStateHashAtDbCheckpoint); + +std::string BlockChainUtils::publicStateHashKey() { return kPublicStateHashKey; } + +} // end of namespace concord::kvbc::bcutil diff --git a/kvbc/src/categorization/kv_blockchain.cpp b/kvbc/src/categorization/kv_blockchain.cpp index c14c79db1a..38b14d576b 100644 --- a/kvbc/src/categorization/kv_blockchain.cpp +++ b/kvbc/src/categorization/kv_blockchain.cpp @@ -14,14 +14,12 @@ #include "categorization/kv_blockchain.h" #include "bcstatetransfer/SimpleBCStateTransfer.hpp" #include "bftengine/ControlStateManager.hpp" -#include "json_output.hpp" #include "diagnostics.h" #include "performance_handler.h" #include "kvbc_key_types.hpp" #include "categorization/db_categories.h" #include "endianness.hpp" #include "migrations/block_merkle_latest_ver_cf_migration.h" -#include "storage/merkle_tree_key_manipulator.h" #include "categorization/details.h" #include "ReplicaConfig.hpp" #include "throughput.hpp" @@ -33,17 +31,12 @@ namespace concord::kvbc::categorization { using ::bftEngine::bcst::computeBlockDigest; -using concordUtils::toPair; template void nullopts(std::vector>& vec, std::size_t count) { vec.resize(count, std::nullopt); } -const KeyValueBlockchain::Converter KeyValueBlockchain::kNoopConverter = [](std::string&& v) -> std::string { - return std::move(v); -}; - KeyValueBlockchain::Recorders KeyValueBlockchain::histograms_; KeyValueBlockchain::KeyValueBlockchain(const std::shared_ptr& native_client, @@ -188,7 +181,7 @@ BlockId KeyValueBlockchain::addBlock(Updates&& updates) { // Use new client batch and column families auto write_batch = native_client_->getBatch(); addGenesisBlockKey(updates); - auto block_id = addBlock(std::move(updates.category_updates_), write_batch); + auto block_id = addBlock(updates.categoryUpdates(), write_batch); native_client_->write(std::move(write_batch)); block_chain_.setAddedBlockId(block_id); return block_id; @@ -402,96 +395,6 @@ void KeyValueBlockchain::trimBlocksFromSnapshot(BlockId block_id_at_checkpoint) } } -static const auto kPublicStateHashKey = concord::storage::v2MerkleTree::detail::serialize( - concord::storage::v2MerkleTree::detail::EBFTSubtype::PublicStateHashAtDbCheckpoint); - -std::string KeyValueBlockchain::publicStateHashKey() { return kPublicStateHashKey; } - -std::optional KeyValueBlockchain::getPublicStateKeys() const { - const auto opt_val = getLatest(kConcordInternalCategoryId, keyTypes::state_public_key_set); - if (!opt_val) { - return std::nullopt; - } - auto public_state = PublicStateKeys{}; - const auto val = std::get_if(&opt_val.value()); - ConcordAssertNE(val, nullptr); - detail::deserialize(val->data, public_state); - return std::make_optional(std::move(public_state)); -} - -void KeyValueBlockchain::iteratePublicStateKeyValues(const std::function& f) const { - const auto ret = iteratePublicStateKeyValuesImpl(f, std::nullopt); - ConcordAssert(ret); -} - -bool KeyValueBlockchain::iteratePublicStateKeyValues(const std::function& f, - const std::string& after_key) const { - return iteratePublicStateKeyValuesImpl(f, after_key); -} - -bool KeyValueBlockchain::iteratePublicStateKeyValuesImpl(const std::function& f, - const std::optional& after_key) const { - const auto public_state = getPublicStateKeys(); - if (!public_state) { - return true; - } - - auto idx = 0ull; - if (after_key) { - auto it = std::lower_bound(public_state->keys.cbegin(), public_state->keys.cend(), *after_key); - if (it == public_state->keys.cend() || *it != *after_key) { - return false; - } - // Start from the key after `after_key`. - idx = std::distance(public_state->keys.cbegin(), it) + 1; - } - - const auto batch_size = bftEngine::ReplicaConfig::instance().stateIterationMultiGetBatchSize; - auto keys_batch = std::vector{}; - keys_batch.reserve(batch_size); - auto opt_values = std::vector>{}; - opt_values.reserve(batch_size); - while (idx < public_state->keys.size()) { - keys_batch.clear(); - opt_values.clear(); - while (keys_batch.size() < batch_size) { - if (idx == public_state->keys.size()) { - break; - } - keys_batch.push_back(public_state->keys[idx]); - ++idx; - } - multiGetLatest(kExecutionProvableCategory, keys_batch, opt_values); - ConcordAssertEQ(keys_batch.size(), opt_values.size()); - for (auto i = 0ull; i < keys_batch.size(); ++i) { - auto& opt_value = opt_values[i]; - ConcordAssert(opt_value.has_value()); - auto value = std::get_if(&opt_value.value()); - ConcordAssertNE(value, nullptr); - f(std::move(keys_batch[i]), std::move(value->data)); - } - } - return true; -} - -static const auto kInitialHash = detail::hash(std::string{}); - -void KeyValueBlockchain::computeAndPersistPublicStateHash(BlockId checkpoint_block_id, - const Converter& value_converter) { - auto hash = kInitialHash; - iteratePublicStateKeyValues([&](std::string&& key, std::string&& value) { - value = value_converter(std::move(value)); - auto hasher = Hasher{}; - hasher.init(); - hasher.update(hash.data(), hash.size()); - const auto key_hash = detail::hash(key); - hasher.update(key_hash.data(), key_hash.size()); - hasher.update(value.data(), value.size()); - hash = hasher.finish(); - }); - native_client_->put(kPublicStateHashKey, detail::serialize(StateHash{checkpoint_block_id, hash})); -} - /////////////////////// Delete block /////////////////////// bool KeyValueBlockchain::deleteBlock(const BlockId& block_id) { diagnostics::TimeRecorder scoped_timer(*histograms_.deleteBlock); @@ -761,7 +664,6 @@ void KeyValueBlockchain::deleteLastReachableBlock(BlockId block_id, } // Updates per category - void KeyValueBlockchain::insertCategoryMapping(const std::string& cat_id, const CATEGORY_TYPE type) { // check if we know this category type already ConcordAssertEQ(category_types_.count(cat_id), 0); @@ -984,22 +886,4 @@ void KeyValueBlockchain::writeSTLinkTransaction(const BlockId block_id, RawBlock block_chain_.setAddedBlockId(new_block_id); } -std::string KeyValueBlockchain::getPruningStatus() { - std::ostringstream oss; - std::unordered_map result; - - result.insert(toPair("versionedNumOfDeletedKeys", - aggregator_->GetCounter("kv_blockchain_deletes", "numOfVersionedKeysDeleted").Get())); - result.insert(toPair("immutableNumOfDeletedKeys", - aggregator_->GetCounter("kv_blockchain_deletes", "numOfImmutableKeysDeleted").Get())); - result.insert(toPair("merkleNumOfDeletedKeys", - aggregator_->GetCounter("kv_blockchain_deletes", "numOfMerkleKeysDeleted").Get())); - result.insert(toPair("getGenesisBlockId()", getGenesisBlockId())); - result.insert(toPair("getLastReachableBlockId()", getLastReachableBlockId())); - result.insert(toPair("isPruningInProgress", bftEngine::ControlStateManager::instance().getPruningProcessStatus())); - - oss << concordUtils::kContainerToJson(result); - return oss.str(); -} - } // namespace concord::kvbc::categorization diff --git a/kvbc/src/kvbc_adapter/categorization/app_state_adapter.cpp b/kvbc/src/kvbc_adapter/categorization/app_state_adapter.cpp new file mode 100644 index 0000000000..4ffa7ab545 --- /dev/null +++ b/kvbc/src/kvbc_adapter/categorization/app_state_adapter.cpp @@ -0,0 +1,133 @@ +// Concord +// +// Copyright (c) 2022 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 subcomponent's license, as noted in the LICENSE +// file. + +#include "kvbc_adapter/categorization/app_state_adapter.hpp" +#include "assertUtils.hpp" + +namespace concord::kvbc::adapter::categorization { + +AppStateAdapter::AppStateAdapter(std::shared_ptr &kvbc) + : kvbc_{kvbc.get()}, logger_(logging::getLogger("skvbc.replica.appstateadapter")) { + ConcordAssertNE(kvbc_, nullptr); +} + +///////////////////////////////////////////////////////////////////////////////////////////////////////////////////// +// IAppState implementation +bool AppStateAdapter::getBlock(uint64_t blockId, + char *outBlock, + uint32_t outBlockMaxSize, + uint32_t *outBlockActualSize) const { + const auto rawBlock = kvbc_->getRawBlock(blockId); + if (!rawBlock) { + throw NotFoundException{"Raw block not found: " + std::to_string(blockId)}; + } + const auto &ser = concord::kvbc::categorization::RawBlock::serialize(*rawBlock); + if (ser.size() > outBlockMaxSize) { + LOG_ERROR(logger_, KVLOG(ser.size(), outBlockMaxSize)); + throw std::runtime_error("not enough space to copy block!"); + } + *outBlockActualSize = ser.size(); + std::memcpy(outBlock, ser.data(), *outBlockActualSize); + return true; +} +bool AppStateAdapter::getPrevDigestFromBlock(uint64_t blockId, + bftEngine::bcst::StateTransferDigest *outPrevBlockDigest) const { + ConcordAssert(blockId > 0); + const auto parent_digest = kvbc_->parentDigest(blockId); + + if (!parent_digest.has_value()) { + LOG_WARN(logger_, "parent digest not found," << KVLOG(blockId)); + return false; + } + static_assert(parent_digest->size() == DIGEST_SIZE); + static_assert(sizeof(bftEngine::bcst::StateTransferDigest) == DIGEST_SIZE); + std::memcpy(outPrevBlockDigest, parent_digest->data(), DIGEST_SIZE); + return true; +} +void AppStateAdapter::getPrevDigestFromBlock(const char *blockData, + const uint32_t blockSize, + bftEngine::bcst::StateTransferDigest *outPrevBlockDigest) const { + ConcordAssertGT(blockSize, 0); + auto view = std::string_view{blockData, blockSize}; + const auto rawBlock = concord::kvbc::categorization::RawBlock::deserialize(view); + + static_assert(rawBlock.data.parent_digest.size() == DIGEST_SIZE); + static_assert(sizeof(bftEngine::bcst::StateTransferDigest) == DIGEST_SIZE); + std::memcpy(outPrevBlockDigest, rawBlock.data.parent_digest.data(), DIGEST_SIZE); +} +bool AppStateAdapter::putBlock(const uint64_t blockId, + const char *blockData, + const uint32_t blockSize, + bool lastBlock) { + auto view = std::string_view{blockData, blockSize}; + const auto rawBlock = concord::kvbc::categorization::RawBlock::deserialize(view); + if (kvbc_->hasBlock(blockId)) { + const auto existingRawBlock = kvbc_->getRawBlock(blockId); + if (rawBlock != existingRawBlock) { + LOG_ERROR(logger_, + "found existing (and different) block ID[" << blockId << "] when receiving from state transfer"); + + // TODO consider assert? + kvbc_->deleteBlock(blockId); + throw std::runtime_error( + __PRETTY_FUNCTION__ + + std::string("found existing (and different) block when receiving state transfer, block ID: ") + + std::to_string(blockId)); + } + } else { + kvbc_->addRawBlock(rawBlock, blockId, lastBlock); + } + return true; +} +// This method is used by state-transfer in order to find the latest block id in either the state-transfer chain or +// the main blockchain +uint64_t AppStateAdapter::getLastBlockNum() const { + const auto last = kvbc_->getLastStatetransferBlockId(); + if (last) { + return *last; + } + return kvbc_->getLastReachableBlockId(); +} +size_t AppStateAdapter::postProcessUntilBlockId(uint64_t max_block_id) { + const BlockId last_reachable_block = kvbc_->getLastReachableBlockId(); + BlockId last_st_block_id = 0; + if (auto last_st_block_id_opt = kvbc_->getLastStatetransferBlockId()) { + last_st_block_id = last_st_block_id_opt.value(); + } + if ((max_block_id == last_reachable_block) && (last_st_block_id == 0)) { + LOG_INFO(CAT_BLOCK_LOG, + "Consensus blockchain is fully linked, no proc-processing is required!" + << KVLOG(max_block_id, last_reachable_block)); + return 0; + } + if ((max_block_id < last_reachable_block) || (max_block_id > last_st_block_id)) { + auto msg = std::stringstream{}; + msg << "Cannot post-process:" << KVLOG(max_block_id, last_reachable_block, last_st_block_id) << std::endl; + throw std::invalid_argument{msg.str()}; + } + + try { + return kvbc_->linkUntilBlockId(last_reachable_block + 1, max_block_id); + } catch (const std::exception &e) { + LOG_FATAL( + CAT_BLOCK_LOG, + "Aborting due to failure to link," << KVLOG(last_reachable_block, max_block_id) << ", reason: " << e.what()); + std::terminate(); + } catch (...) { + LOG_FATAL(CAT_BLOCK_LOG, "Aborting due to failure to link," << KVLOG(last_reachable_block, max_block_id)); + std::terminate(); + } +} +///////////////////////////////////////////////////////////////////////////////////////////////////////////////////// + +} // End of namespace concord::kvbc::adapter::categorization diff --git a/kvbc/src/kvbc_adapter/categorization/blocks_deleter_adapter.cpp b/kvbc/src/kvbc_adapter/categorization/blocks_deleter_adapter.cpp new file mode 100644 index 0000000000..b7b7ae6bb5 --- /dev/null +++ b/kvbc/src/kvbc_adapter/categorization/blocks_deleter_adapter.cpp @@ -0,0 +1,67 @@ +// Concord +// +// Copyright (c) 2022 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 subcomponent's license, as noted in the LICENSE +// file. + +#include "kvbc_adapter/categorization/blocks_deleter_adapter.hpp" +#include "assertUtils.hpp" +#include "ReplicaResources.h" + +using concord::performance::ISystemResourceEntity; + +namespace concord::kvbc::adapter::categorization { + +BlocksDeleterAdapter::BlocksDeleterAdapter(std::shared_ptr &kvbc, + const std::optional &aux_types) + : kvbc_{kvbc.get()} { + if (aux_types.has_value()) { + replica_resources_.reset(&(aux_types->resource_entity_)); + } else { + replica_resources_ = std::make_shared(); + } + ConcordAssertNE(kvbc_, nullptr); + ConcordAssertEQ(!replica_resources_, false); +} + +///////////////////////////////////////////////////////////////////////////////////////////////////////////////////// +// IBlocksDeleter implementation +void BlocksDeleterAdapter::deleteGenesisBlock() { + const auto genesisBlock = kvbc_->getGenesisBlockId(); + if (genesisBlock == 0) { + throw std::logic_error{"Cannot delete the genesis block from an empty blockchain"}; + } + kvbc_->deleteBlock(genesisBlock); +} +BlockId BlocksDeleterAdapter::deleteBlocksUntil(BlockId until) { + const auto genesisBlock = kvbc_->getGenesisBlockId(); + if (genesisBlock == 0) { + throw std::logic_error{"Cannot delete a block range from an empty blockchain"}; + } else if (until <= genesisBlock) { + throw std::invalid_argument{"Invalid 'until' value passed to deleteBlocksUntil()"}; + } + + const auto lastReachableBlock = kvbc_->getLastReachableBlockId(); + const auto lastDeletedBlock = std::min(lastReachableBlock, until - 1); + const auto start = std::chrono::steady_clock::now(); + for (auto i = genesisBlock; i <= lastDeletedBlock; ++i) { + ISystemResourceEntity::scopedDurMeasurment mes(*replica_resources_, + ISystemResourceEntity::type::pruning_avg_time_micro); + ConcordAssert(kvbc_->deleteBlock(i)); + } + auto jobDuration = + std::chrono::duration_cast(std::chrono::steady_clock::now() - start).count(); + histograms_.delete_batch_blocks_duration->recordAtomic(jobDuration); + return lastDeletedBlock; +} + +///////////////////////////////////////////////////////////////////////////////////////////////////////////////////// + +} // namespace concord::kvbc::adapter::categorization diff --git a/kvbc/src/kvbc_adapter/categorization/db_checkpoint_adapter.cpp b/kvbc/src/kvbc_adapter/categorization/db_checkpoint_adapter.cpp new file mode 100644 index 0000000000..e439150244 --- /dev/null +++ b/kvbc/src/kvbc_adapter/categorization/db_checkpoint_adapter.cpp @@ -0,0 +1,24 @@ +// Concord +// +// Copyright (c) 2022 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 subcomponent's license, as noted in the LICENSE +// file. + +#include "assertUtils.hpp" +#include "kvbc_adapter/categorization/db_checkpoint_adapter.hpp" + +namespace concord::kvbc::adapter::categorization { + +DbCheckpointImpl::DbCheckpointImpl(std::shared_ptr& kvbc) + : kvbc_(kvbc.get()) { + ConcordAssertNE(kvbc_, nullptr); +} + +} // namespace concord::kvbc::adapter::categorization diff --git a/kvbc/src/kvbc_adapter/categorization/kv_blockchain_adapter.cpp b/kvbc/src/kvbc_adapter/categorization/kv_blockchain_adapter.cpp new file mode 100644 index 0000000000..393f964da7 --- /dev/null +++ b/kvbc/src/kvbc_adapter/categorization/kv_blockchain_adapter.cpp @@ -0,0 +1,24 @@ +// Concord +// +// Copyright (c) 2022 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 subcomponent's license, as noted in the LICENSE +// file. + +#include "kvbc_adapter/categorization/kv_blockchain_adapter.hpp" +#include "assertUtils.hpp" + +namespace concord::kvbc::adapter::categorization { + +KeyValueBlockchain::KeyValueBlockchain(std::shared_ptr &kvbc) + : kvbc_{kvbc.get()} { + ConcordAssertNE(kvbc_, nullptr); +} + +} // namespace concord::kvbc::adapter::categorization \ No newline at end of file diff --git a/kvbc/src/kvbc_adapter/common/state_snapshot_adapter.cpp b/kvbc/src/kvbc_adapter/common/state_snapshot_adapter.cpp new file mode 100644 index 0000000000..f64ffe25af --- /dev/null +++ b/kvbc/src/kvbc_adapter/common/state_snapshot_adapter.cpp @@ -0,0 +1,111 @@ +// Concord +// +// Copyright (c) 2022-2023 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 subcomponent's license, as noted in the LICENSE +// file. + +#include +#include "categorization/details.h" +#include "categorization/db_categories.h" +#include "kvbc_key_types.hpp" +#include "kvbc_adapter/common/state_snapshot_adapter.hpp" + +namespace concord::kvbc::adapter::common::statesnapshot { +////////////////////////////IKVBCStateSnapshot//////////////////////////////////////////////////////////////////////// +void KVBCStateSnapshot::computeAndPersistPublicStateHash(BlockId checkpoint_block_id, + const Converter& value_converter) { + auto hash = concord::kvbc::categorization::detail::hash(std::string{}); + iteratePublicStateKeyValues([&](std::string&& key, std::string&& value) { + value = value_converter(std::move(value)); + auto hasher = concord::kvbc::categorization::Hasher{}; + hasher.init(); + hasher.update(hash.data(), hash.size()); + const auto key_hash = concord::kvbc::categorization::detail::hash(key); + hasher.update(key_hash.data(), key_hash.size()); + hasher.update(value.data(), value.size()); + hash = hasher.finish(); + }); + native_client_->put(concord::kvbc::bcutil::BlockChainUtils::publicStateHashKey(), + concord::kvbc::categorization::detail::serialize( + concord::kvbc::categorization::StateHash{checkpoint_block_id, hash})); +} + +std::optional KVBCStateSnapshot::getPublicStateKeys() const { + const auto opt_val = reader_->getLatest(concord::kvbc::categorization::kConcordInternalCategoryId, + concord::kvbc::keyTypes::state_public_key_set); + if (!opt_val) { + return std::nullopt; + } + auto public_state = concord::kvbc::categorization::PublicStateKeys{}; + const auto val = std::get_if(&opt_val.value()); + ConcordAssertNE(val, nullptr); + concord::kvbc::categorization::detail::deserialize(val->data, public_state); + return std::make_optional(std::move(public_state)); +} + +void KVBCStateSnapshot::iteratePublicStateKeyValues(const std::function& f) const { + const auto ret = iteratePublicStateKeyValuesImpl(f, std::nullopt); + ConcordAssert(ret); +} + +bool KVBCStateSnapshot::iteratePublicStateKeyValues(const std::function& f, + const std::string& after_key) const { + return iteratePublicStateKeyValuesImpl(f, after_key); + ; +} + +////////////////////////////////////////////////////////////////////////////////////////////////////////////////////// + +bool KVBCStateSnapshot::iteratePublicStateKeyValuesImpl(const std::function& f, + const std::optional& after_key) const { + const auto public_state = getPublicStateKeys(); + if (!public_state) { + return true; + } + + auto idx = 0ull; + if (after_key) { + auto it = std::lower_bound(public_state->keys.cbegin(), public_state->keys.cend(), *after_key); + if (it == public_state->keys.cend() || *it != *after_key) { + return false; + } + // Start from the key after `after_key`. + idx = std::distance(public_state->keys.cbegin(), it) + 1; + } + + const auto batch_size = bftEngine::ReplicaConfig::instance().stateIterationMultiGetBatchSize; + auto keys_batch = std::vector{}; + keys_batch.reserve(batch_size); + auto opt_values = std::vector>{}; + opt_values.reserve(batch_size); + while (idx < public_state->keys.size()) { + keys_batch.clear(); + opt_values.clear(); + while (keys_batch.size() < batch_size) { + if (idx == public_state->keys.size()) { + break; + } + keys_batch.push_back(public_state->keys[idx]); + ++idx; + } + reader_->multiGetLatest(concord::kvbc::categorization::kExecutionProvableCategory, keys_batch, opt_values); + ConcordAssertEQ(keys_batch.size(), opt_values.size()); + for (auto i = 0ull; i < keys_batch.size(); ++i) { + auto& opt_value = opt_values[i]; + ConcordAssert(opt_value.has_value()); + auto value = std::get_if(&opt_value.value()); + ConcordAssertNE(value, nullptr); + f(std::move(keys_batch[i]), std::move(value->data)); + } + } + return true; +} + +} // namespace concord::kvbc::adapter::common::statesnapshot diff --git a/kvbc/src/kvbc_adapter/replica_adapter.cpp b/kvbc/src/kvbc_adapter/replica_adapter.cpp new file mode 100644 index 0000000000..63ec0ea058 --- /dev/null +++ b/kvbc/src/kvbc_adapter/replica_adapter.cpp @@ -0,0 +1,87 @@ +// Concord +// +// Copyright (c) 2022 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 subcomponent's license, as noted in the LICENSE +// file. + +#include "kvbc_adapter/replica_adapter.hpp" +#include "kvbc_adapter/common/state_snapshot_adapter.hpp" +#include "kvbc_adapter/categorization/blocks_deleter_adapter.hpp" +#include "kvbc_adapter/categorization/kv_blockchain_adapter.hpp" +#include "kvbc_adapter/categorization/app_state_adapter.hpp" +#include "kvbc_adapter/categorization/db_checkpoint_adapter.hpp" +#include "kvbc_adapter/v4blockchain/blocks_deleter_adapter.hpp" +#include "kvbc_adapter/v4blockchain/blocks_adder_adapter.hpp" +#include "kvbc_adapter/v4blockchain/blocks_reader_adapter.hpp" +#include "kvbc_adapter/v4blockchain/app_state_adapter.hpp" +#include "kvbc_adapter/v4blockchain/blocks_db_checkpoint_adapter.hpp" + +namespace concord::kvbc::adapter { +ReplicaBlockchain::~ReplicaBlockchain() { + deleter_ = nullptr; + reader_ = nullptr; + adder_ = nullptr; + app_state_ = nullptr; + state_snapshot_ = nullptr; + db_chkpt_ = nullptr; +} + +void ReplicaBlockchain::switch_to_rawptr() { + deleter_ = up_deleter_.get(); + reader_ = up_reader_.get(); + adder_ = up_adder_.get(); + app_state_ = up_app_state_.get(); + state_snapshot_ = up_state_snapshot_.get(); + db_chkpt_ = up_db_chkpt_.get(); +} + +ReplicaBlockchain::ReplicaBlockchain( + const std::shared_ptr &native_client, + bool link_st_chain, + const std::optional> &category_types, + const std::optional &aux_types) + : logger_(logging::getLogger("skvbc.replica.adapter")) { + if (bftEngine::ReplicaConfig::instance().kvBlockchainVersion == BLOCKCHAIN_VERSION::CATEGORIZED_BLOCKCHAIN) { + LOG_INFO(CAT_BLOCK_LOG, "Instantiating categorized type blockchain"); + kvbc_ = std::make_shared( + native_client, link_st_chain, category_types); + if (aux_types.has_value()) { + kvbc_->setAggregator(aux_types->aggregator_); + } + up_deleter_ = std::make_unique(kvbc_, aux_types); + up_reader_ = std::make_unique(kvbc_); + up_adder_ = std::make_unique(kvbc_); + up_app_state_ = std::make_unique(kvbc_); + up_state_snapshot_ = std::make_unique( + up_reader_.get(), native_client); + up_db_chkpt_ = std::make_unique(kvbc_); + } else if (bftEngine::ReplicaConfig::instance().kvBlockchainVersion == BLOCKCHAIN_VERSION::NATURAL_BLOCKCHAIN) { + LOG_INFO(V4_BLOCK_LOG, "Instantiating v4 type blockchain"); + v4_kvbc_ = + std::make_shared(native_client, link_st_chain, category_types); + up_deleter_ = std::make_unique(v4_kvbc_, aux_types); + up_reader_ = std::make_unique(v4_kvbc_); + up_adder_ = std::make_unique(v4_kvbc_); + up_app_state_ = std::make_unique(v4_kvbc_); + up_state_snapshot_ = std::make_unique( + up_reader_.get(), native_client); + up_db_chkpt_ = std::make_unique(v4_kvbc_); + } + + switch_to_rawptr(); + ConcordAssertNE(deleter_, nullptr); + ConcordAssertNE(reader_, nullptr); + ConcordAssertNE(adder_, nullptr); + ConcordAssertNE(app_state_, nullptr); + ConcordAssertNE(state_snapshot_, nullptr); + ConcordAssertNE(db_chkpt_, nullptr); +} + +} // namespace concord::kvbc::adapter diff --git a/kvbc/src/kvbc_adapter/v4blockchain/app_state_adapter.cpp b/kvbc/src/kvbc_adapter/v4blockchain/app_state_adapter.cpp new file mode 100644 index 0000000000..e119634d38 --- /dev/null +++ b/kvbc/src/kvbc_adapter/v4blockchain/app_state_adapter.cpp @@ -0,0 +1,113 @@ +// Concord +// +// Copyright (c) 2022-2023 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 subcomponent's license, as noted in the LICENSE +// file. + +#include "db_adapter_interface.h" +#include "kvbc_adapter/v4blockchain/app_state_adapter.hpp" +#include "v4blockchain/detail/blocks.h" +#include "v4blockchain/detail/detail.h" + +using namespace concord::kvbc; + +namespace concord::kvbc::adapter::v4blockchain { + +bool AppStateAdapter::getBlock(uint64_t blockId, + char *outBlock, + uint32_t outBlockMaxSize, + uint32_t *outBlockActualSize) const { + auto blockData = kvbc_->getBlockData(blockId); + if (!blockData) { + throw kvbc::NotFoundException{"block not found: " + std::to_string(blockId)}; + } + if (blockData->size() > outBlockMaxSize) { + LOG_ERROR(V4_BLOCK_LOG, KVLOG(blockData->size(), outBlockMaxSize)); + throw std::runtime_error("not enough space to copy block!"); + } + *outBlockActualSize = blockData->size(); + std::memcpy(outBlock, blockData->c_str(), *outBlockActualSize); + return true; +} + +bool AppStateAdapter::getPrevDigestFromBlock(uint64_t blockId, + bftEngine::bcst::StateTransferDigest *outPrevBlockDigest) const { + ConcordAssert(blockId > 0); + const auto parent_digest = kvbc_->parentDigest(blockId); + static_assert(parent_digest.size() == DIGEST_SIZE); + static_assert(sizeof(bftEngine::bcst::StateTransferDigest) == DIGEST_SIZE); + std::memcpy(outPrevBlockDigest, parent_digest.data(), DIGEST_SIZE); + return true; +} + +void AppStateAdapter::getPrevDigestFromBlock(const char *blockData, + const uint32_t blockSize, + bftEngine::bcst::StateTransferDigest *outPrevBlockDigest) const { + ConcordAssertGE(blockSize, ::v4blockchain::detail::Block::HEADER_SIZE); + const auto &digest = *reinterpret_cast( + blockData + sizeof(::v4blockchain::detail::version_type)); + + std::memcpy(outPrevBlockDigest, digest.data(), DIGEST_SIZE); +} + +bool AppStateAdapter::putBlock(const uint64_t blockId, + const char *blockData, + const uint32_t blockSize, + bool lastBlock) { + const auto lastReachable = kvbc_->getLastReachableBlockId(); + if (blockId <= lastReachable) { + const auto msg = "Cannot add an existing block ID " + std::to_string(blockId); + throw std::invalid_argument{msg}; + } + + kvbc_->addBlockToSTChain(blockId, blockData, blockSize, lastBlock); + return true; +} + +uint64_t AppStateAdapter::getLastBlockNum() const { + const auto last = kvbc_->getLastStatetransferBlockId(); + if (last) { + return *last; + } + return kvbc_->getLastReachableBlockId(); +} + +size_t AppStateAdapter::postProcessUntilBlockId(uint64_t max_block_id) { + const BlockId last_reachable_block = kvbc_->getLastReachableBlockId(); + BlockId last_st_block_id = 0; + if (auto last_st_block_id_opt = kvbc_->getLastStatetransferBlockId()) { + last_st_block_id = last_st_block_id_opt.value(); + } + if ((max_block_id == last_reachable_block) && (last_st_block_id == 0)) { + LOG_INFO(CAT_BLOCK_LOG, + "Consensus blockchain is fully linked, no proc-processing is required!" + << KVLOG(max_block_id, last_reachable_block)); + return 0; + } + if ((max_block_id < last_reachable_block) || (max_block_id > last_st_block_id)) { + auto msg = std::stringstream{}; + msg << "Cannot post-process:" << KVLOG(max_block_id, last_reachable_block, last_st_block_id) << std::endl; + throw std::invalid_argument{msg.str()}; + } + + try { + return kvbc_->linkUntilBlockId(max_block_id); + } catch (const std::exception &e) { + LOG_FATAL( + CAT_BLOCK_LOG, + "Aborting due to failure to link," << KVLOG(last_reachable_block, max_block_id) << ", reason: " << e.what()); + std::terminate(); + } catch (...) { + LOG_FATAL(CAT_BLOCK_LOG, "Aborting due to failure to link," << KVLOG(last_reachable_block, max_block_id)); + std::terminate(); + } +} + +} // namespace concord::kvbc::adapter::v4blockchain diff --git a/kvbc/src/kvbc_adapter/v4blockchain/blocks_deleter_adapter.cpp b/kvbc/src/kvbc_adapter/v4blockchain/blocks_deleter_adapter.cpp new file mode 100644 index 0000000000..bb532f62d5 --- /dev/null +++ b/kvbc/src/kvbc_adapter/v4blockchain/blocks_deleter_adapter.cpp @@ -0,0 +1,49 @@ +// Concord +// +// Copyright (c) 2022 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 subcomponent's license, as noted in the LICENSE +// file. + +#include "kvbc_adapter/v4blockchain/blocks_deleter_adapter.hpp" +#include "assertUtils.hpp" +#include "ReplicaResources.h" + +using concord::performance::ISystemResourceEntity; + +namespace concord::kvbc::adapter::v4blockchain { + +BlocksDeleterAdapter::BlocksDeleterAdapter(std::shared_ptr &kvbc, + const std::optional &aux_types) + : kvbc_{kvbc.get()} { + if (aux_types.has_value()) { + replica_resources_.reset(&(aux_types->resource_entity_)); + } else { + replica_resources_ = std::make_shared(); + } + ConcordAssertNE(kvbc_, nullptr); + ConcordAssertEQ(!replica_resources_, false); +} + +BlockId BlocksDeleterAdapter::deleteBlocksUntil(BlockId until) { + const auto start = std::chrono::steady_clock::now(); + ISystemResourceEntity::scopedDurMeasurment mes(*replica_resources_, + ISystemResourceEntity::type::pruning_avg_time_micro); + auto upTo = kvbc_->deleteBlocksUntil(until); + + auto jobDuration = + std::chrono::duration_cast(std::chrono::steady_clock::now() - start).count(); + histograms_.delete_batch_blocks_duration->recordAtomic(jobDuration); + + return upTo; +} + +///////////////////////////////////////////////////////////////////////////////////////////////////////////////////// + +} // namespace concord::kvbc::adapter::v4blockchain diff --git a/kvbc/src/merkle_tree_storage_factory.cpp b/kvbc/src/merkle_tree_storage_factory.cpp index 37683c839c..40e47d6ef9 100644 --- a/kvbc/src/merkle_tree_storage_factory.cpp +++ b/kvbc/src/merkle_tree_storage_factory.cpp @@ -17,11 +17,12 @@ #include "storage/merkle_tree_key_manipulator.h" #include "rocksdb/client.h" #include "rocksdb/native_client.h" +#include "v4blockchain/detail/column_families.h" #include #include #include - +#include "rocksdb/time_stamp_comparator.h" #include namespace concord::kvbc::v2MerkleTree { @@ -35,15 +36,18 @@ std::shared_ptr completeRocksDBConfiguration( auto table_options = ::rocksdb::BlockBasedTableOptions{}; table_options.block_cache = ::rocksdb::NewLRUCache(rocksdbBlockCacheBytes); table_options.filter_policy.reset(::rocksdb::NewBloomFilterPolicy(10, false)); - if (rocksdbRowCacheBytes) db_options.row_cache = ::rocksdb::NewLRUCache(rocksdbRowCacheBytes); + db_options.row_cache.reset(); db_options.table_factory.reset(NewBlockBasedTableFactory(table_options)); // Use the same block cache and table options for all column familes for now. for (auto& d : cf_descs) { - auto* cf_table_options = - reinterpret_cast<::rocksdb::BlockBasedTableOptions*>(d.options.table_factory->GetOptions()); + auto* cf_table_options = reinterpret_cast<::rocksdb::BlockBasedTableOptions*>( + d.options.table_factory->GetOptions<::rocksdb::BlockBasedTableOptions>()); cf_table_options->block_cache = table_options.block_cache; cf_table_options->filter_policy.reset(::rocksdb::NewBloomFilterPolicy(10, false)); + if (d.name == v4blockchain::detail::LATEST_KEYS_CF) { + d.options.comparator = concord::storage::rocksdb::getLexicographic64TsComparator(); + } } return db_options.statistics; } diff --git a/kvbc/src/reconfiguration_kvbc_handler.cpp b/kvbc/src/reconfiguration_kvbc_handler.cpp index 7b1046af42..83ef0129d3 100644 --- a/kvbc/src/reconfiguration_kvbc_handler.cpp +++ b/kvbc/src/reconfiguration_kvbc_handler.cpp @@ -21,12 +21,12 @@ #include "kvbc_app_filter/kvbc_key_types.h" #include "concord.cmf.hpp" #include "secrets_manager_plain.h" -#include "communication/StateControl.hpp" #include "rocksdb/native_client.h" -#include "categorization/categorized_reader.h" +#include "kvbc_adapter/idempotent_reader.h" #include "categorization/db_categories.h" #include "categorization/details.h" #include "categorized_kvbc_msgs.cmf.hpp" +#include "ReplicaResources.h" #include #include #include @@ -36,11 +36,19 @@ namespace concord::kvbc::reconfiguration { using bftEngine::impl::DbCheckpointManager; using bftEngine::impl::SigManager; using concord::kvbc::KvbAppFilter; -using concord::kvbc::categorization::CategorizedReader; -using concord::kvbc::categorization::KeyValueBlockchain; +using concord::kvbc::adapter::IdempotentReader; using concord::messages::SnapshotResponseStatus; using concord::storage::rocksdb::NativeClient; +std::string ToHexad(const std::string& s) { + std::ostringstream ret; + + for (std::string::size_type i = 0; i < s.length(); ++i) + ret << std::hex << std::setfill('0') << std::setw(2) << std::uppercase << (int)s[i]; + + return ret.str(); +} + kvbc::BlockId ReconfigurationBlockTools::persistReconfigurationBlock( const std::vector& data, const uint64_t bft_seq_num, @@ -196,8 +204,8 @@ bool StateSnapshotReconfigurationHandler::handle(const concord::messages::StateS read_only, NativeClient::DefaultOptions{}); const auto link_st_chain = false; - const auto kvbc = std::make_shared(db, link_st_chain); - const auto reader = CategorizedReader{kvbc}; + const auto idempotent_kvbc = std::make_shared(db, link_st_chain); + const auto reader = IdempotentReader{idempotent_kvbc}; const auto filter = KvbAppFilter{&reader, ""}; if (bftEngine::ReplicaConfig::instance().enableEventGroups) { // TODO: We currently only support new participants and, therefore, the event group ID will always be the last @@ -208,7 +216,7 @@ bool StateSnapshotReconfigurationHandler::handle(const concord::messages::StateS resp.data->blockchain_height = reader.getLastBlockId(); resp.data->blockchain_height_type = messages::BlockchainHeightType::BlockId; } - const auto public_state = kvbc->getPublicStateKeys(); + const auto public_state = idempotent_kvbc->getPublicStateKeys(); if (!public_state) { resp.data->key_value_count_estimate = 0; } else { @@ -295,7 +303,7 @@ bool StateSnapshotReconfigurationHandler::handle(const concord::messages::Signed const auto read_only = true; try { auto db = NativeClient::newClient(snapshot_path, read_only, NativeClient::DefaultOptions{}); - const auto ser_hash = db->get(KeyValueBlockchain::publicStateHashKey()); + const auto ser_hash = db->get(concord::kvbc::bcutil::BlockChainUtils::publicStateHashKey()); if (!ser_hash) { LOG_ERROR(getLogger(), "SignedPublicStateHashRequest: missing public state hash for snapshot ID = " @@ -361,7 +369,7 @@ bool StateSnapshotReconfigurationHandler::handle(const concord::messages::StateS try { auto db = NativeClient::newClient(snapshot_path, read_only, NativeClient::DefaultOptions{}); const auto link_st_chain = false; - const auto kvbc = KeyValueBlockchain{db, link_st_chain}; + const auto kvbc = adapter::ReplicaBlockchain{db, link_st_chain}; const auto public_state = kvbc.getPublicStateKeys(); auto values = std::vector>{}; kvbc.multiGetLatest(categorization::kExecutionProvableCategory, req.keys, values); @@ -433,6 +441,10 @@ concord::messages::ClientStateReply KvbcClientReconfigurationHandler::buildRepli concord::messages::deserialize(data_buf, cmd); creply.response = cmd; } + auto key = std::string{kvbc::keyTypes::reconfiguration_epoch_key} + command_type + std::to_string(clientid); + LOG_INFO(getLogger(), + "Getting epoch, cat " << concord::kvbc::categorization::kConcordReconfigurationCategoryId << " key " + << key << " hex " << ToHexad(key) << " from block " << arg.block_id); auto epoch_data = ro_storage_.get( concord::kvbc::categorization::kConcordReconfigurationCategoryId, std::string{kvbc::keyTypes::reconfiguration_epoch_key} + command_type + std::to_string(clientid), @@ -1237,9 +1249,9 @@ bool InternalPostKvReconfigurationHandler::handle(const concord::messages::Clien if (!bftEngine::ReplicaConfig::instance().saveClinetKeyFile) return true; // Now that keys have exchanged, lets persist the new key in the file system uint32_t group_id = 0; - for (const auto& [id, cgr] : bftEngine::ReplicaConfig::instance().clientGroups) { + for (const auto& [gid, cgr] : bftEngine::ReplicaConfig::instance().clientGroups) { if (std::find(cgr.begin(), cgr.end(), sender_id) != cgr.end()) { - group_id = id; + group_id = gid; break; } } diff --git a/kvbc/src/replica_state_sync_imp.cpp b/kvbc/src/replica_state_sync_imp.cpp index bdd5c431ac..270b9631fc 100644 --- a/kvbc/src/replica_state_sync_imp.cpp +++ b/kvbc/src/replica_state_sync_imp.cpp @@ -10,7 +10,6 @@ // notices and license terms. Your use of these subcomponents is subject to the // terms and conditions of the subcomponent's license, as noted in the LICENSE // file. -// #include "assertUtils.hpp" #include "replica_state_sync_imp.hpp" @@ -28,7 +27,7 @@ namespace concord::kvbc { ReplicaStateSyncImp::ReplicaStateSyncImp(IBlockMetadata* blockMetadata) : blockMetadata_(blockMetadata) {} uint64_t ReplicaStateSyncImp::execute(logging::Logger& logger, - categorization::KeyValueBlockchain& blockchain, + adapter::ReplicaBlockchain& blockchain, const std::shared_ptr& metadata, uint64_t lastExecutedSeqNum, uint32_t maxNumOfBlocksToDelete) { @@ -58,7 +57,7 @@ uint64_t ReplicaStateSyncImp::execute(logging::Logger& logger, } uint64_t ReplicaStateSyncImp::executeBasedOnBftSeqNum(logging::Logger& logger, - categorization::KeyValueBlockchain& blockchain, + adapter::ReplicaBlockchain& blockchain, uint64_t lastExecutedSeqNum, uint32_t maxNumOfBlocksToDelete) { if (!lastExecutedSeqNum) { @@ -67,7 +66,7 @@ uint64_t ReplicaStateSyncImp::executeBasedOnBftSeqNum(logging::Logger& logger, } uint64_t removedBlocksNum = 0; const auto genesisBlockId = blockchain.getGenesisBlockId(); - BlockId lastReachableBlockId = blockchain.getLastReachableBlockId(); + BlockId lastReachableBlockId = blockchain.getLastBlockId(); uint64_t lastBlockSeqNum = 0; while (lastReachableBlockId && genesisBlockId <= lastReachableBlockId) { // Get execution sequence number stored in the current last block. @@ -88,17 +87,17 @@ uint64_t ReplicaStateSyncImp::executeBasedOnBftSeqNum(logging::Logger& logger, throw std::runtime_error(__PRETTY_FUNCTION__ + error); } blockchain.deleteLastReachableBlock(); - lastReachableBlockId = blockchain.getLastReachableBlockId(); + lastReachableBlockId = blockchain.getLastBlockId(); ++removedBlocksNum; } LOG_INFO(logger, "Inconsistent blockchain block deleted " - << KVLOG(removedBlocksNum, lastExecutedSeqNum, lastBlockSeqNum, blockchain.getLastReachableBlockId())); + << KVLOG(removedBlocksNum, lastExecutedSeqNum, lastBlockSeqNum, blockchain.getLastBlockId())); return removedBlocksNum; } uint64_t ReplicaStateSyncImp::executeBasedOnBlockId(logging::Logger& logger, - categorization::KeyValueBlockchain& blockchain, + adapter::ReplicaBlockchain& blockchain, const std::shared_ptr& metadata, uint32_t maxNumOfBlocksToDelete) { if (0 == maxNumOfBlocksToDelete) { @@ -110,7 +109,7 @@ uint64_t ReplicaStateSyncImp::executeBasedOnBlockId(logging::Logger& logger, const auto lastMtdBlockId = getLastBlockIdFromMetadata(metadata); ConcordAssert(lastMtdBlockId.has_value()); const auto genesisBlockId = blockchain.getGenesisBlockId(); - auto lastReachableKvbcBlockId = blockchain.getLastReachableBlockId(); + auto lastReachableKvbcBlockId = blockchain.getLastBlockId(); auto deletedBlocks = uint64_t{0}; // Even though the KVBC implementation at the time of writing cannot delete the only block left (the genesis block in // the loop below can be the only one left), we still write the code here so that it attempts to delete, even though @@ -118,7 +117,7 @@ uint64_t ReplicaStateSyncImp::executeBasedOnBlockId(logging::Logger& logger, // stopping the system and await manual intervention as the next startup would lead to the same situation. while (lastReachableKvbcBlockId > lastMtdBlockId && genesisBlockId <= lastReachableKvbcBlockId) { blockchain.deleteLastReachableBlock(); - lastReachableKvbcBlockId = blockchain.getLastReachableBlockId(); + lastReachableKvbcBlockId = blockchain.getLastBlockId(); ++deletedBlocks; if (deletedBlocks == maxNumOfBlocksToDelete) { diff --git a/kvbc/src/resources-manager/IntervalMappingResourceManager.cpp b/kvbc/src/resources-manager/IntervalMappingResourceManager.cpp index e4baae805c..6a068b7e27 100644 --- a/kvbc/src/resources-manager/IntervalMappingResourceManager.cpp +++ b/kvbc/src/resources-manager/IntervalMappingResourceManager.cpp @@ -16,7 +16,7 @@ using namespace concord::performance; // IntervalMappingResourceManager::IntervalMappingResourceManager(ISystemResourceEntity &replicaResources, // std::vector> &&intervalMapping) -// : replicaResources_(replicaResources), intervalMapping_(std::move(intervalMapping)) { +// : replica_resources_(replicaResources), intervalMapping_(std::move(intervalMapping)) { // std::ostringstream intervals; // for(const auto& p:intervalMapping_){ // intervals << "{" << p.first << "," << p.second << "},"; diff --git a/kvbc/src/v4blockchain/detail/blockchain.cpp b/kvbc/src/v4blockchain/detail/blockchain.cpp new file mode 100644 index 0000000000..a2cef82db9 --- /dev/null +++ b/kvbc/src/v4blockchain/detail/blockchain.cpp @@ -0,0 +1,212 @@ +// Concord +// +// Copyright (c) 2022 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 subcomponent's license, as noted in the LICENSE +// file. + +#include "v4blockchain/detail/blockchain.h" +#include "Logger.hpp" +#include + +namespace concord::kvbc::v4blockchain::detail { + +Blockchain::Blockchain(const std::shared_ptr& native_client) + : native_client_{native_client} { + if (native_client->createColumnFamilyIfNotExisting(v4blockchain::detail::BLOCKS_CF)) { + LOG_INFO(V4_BLOCK_LOG, "Created [" << v4blockchain::detail::BLOCKS_CF << "] column family for the main blockchain"); + } + auto last_reachable_block_id = loadLastReachableBlockId(); + if (last_reachable_block_id) { + last_reachable_block_id_ = last_reachable_block_id.value(); + LOG_INFO(V4_BLOCK_LOG, "Last reachable block was loaded from storage " << last_reachable_block_id_); + } + auto genesis_blockId = loadGenesisBlockId(); + if (genesis_blockId) { + genesis_block_id_ = genesis_blockId.value(); + LOG_INFO(CAT_BLOCK_LOG, "Genesis block was loaded from storage " << genesis_block_id_); + } +} + +/* +1 - define the id of the new block +2 - calculate the digest of the previous block +3 - create the block, add the updates and digest to it. +4 - put it in the write batch. +5 - save the block buffer for the next block digest calculation. +Note - the increment for the last_reachable_block_id_ is done after the write batch was written to storage. +*/ +BlockId Blockchain::addBlock(const concord::kvbc::categorization::Updates& category_updates, + storage::rocksdb::NativeWriteBatch& wb) { + BlockId id = last_reachable_block_id_ + 1; + // If future from the previous add exist get its value + concord::util::digest::BlockDigest digest; + if (future_digest_) { + ++from_future; + digest = future_digest_->get(); + } else { + ++from_storage; + digest = calculateBlockDigest(last_reachable_block_id_); + } + auto blockKey = generateKey(id); + v4blockchain::detail::Block block; + block.addUpdates(category_updates); + block.addDigest(digest); + wb.put(v4blockchain::detail::BLOCKS_CF, blockKey, block.getBuffer()); + future_digest_ = thread_pool_.async( + [](BlockId id, v4blockchain::detail::Block&& block) { return block.calculateDigest(id); }, id, std::move(block)); + return id; +} + +// Delete up to until not including until, +// returns the last block id that was deleted. +BlockId Blockchain::deleteBlocksUntil(BlockId until) { + ConcordAssertGT(genesis_block_id_, INVALID_BLOCK_ID); + ConcordAssertLT(genesis_block_id_, until); + // We have a single block on the chain + if (last_reachable_block_id_ == genesis_block_id_) { + LOG_WARN(V4_BLOCK_LOG, "Deleting the last block in the blockchain is not supported " << last_reachable_block_id_); + return genesis_block_id_ - 1; + } + // We don't want to erase all the blockchain + const auto last_deleted_block = std::min(last_reachable_block_id_.load() - 1, until - 1); + auto write_batch = native_client_->getBatch(); + for (uint64_t i = genesis_block_id_; i <= last_deleted_block; ++i) { + deleteBlock(i, write_batch); + } + native_client_->write(std::move(write_batch)); + auto blocks_deleted = (last_deleted_block - genesis_block_id_) + 1; + genesis_block_id_ = last_deleted_block + 1; + + LOG_INFO(V4_BLOCK_LOG, "Deleted " << blocks_deleted << " blocks, new genesis is " << genesis_block_id_); + return last_deleted_block; +} + +void Blockchain::deleteGenesisBlock() { + ConcordAssertGT(genesis_block_id_, INVALID_BLOCK_ID); + ConcordAssertLT(genesis_block_id_, last_reachable_block_id_); + auto write_batch = native_client_->getBatch(); + deleteBlock(genesis_block_id_, write_batch); + native_client_->write(std::move(write_batch)); + ++genesis_block_id_; + LOG_INFO(V4_BLOCK_LOG, "Deleted genesis, new genesis is " << genesis_block_id_); +} + +void Blockchain::deleteLastReachableBlock(storage::rocksdb::NativeWriteBatch& write_batch) { + ConcordAssertGT(last_reachable_block_id_, INVALID_BLOCK_ID); + ConcordAssertLT(genesis_block_id_, last_reachable_block_id_); + deleteBlock(last_reachable_block_id_, write_batch); +} + +concord::util::digest::BlockDigest Blockchain::calculateBlockDigest(concord::kvbc::BlockId id) const { + if (id < concord::kvbc::INITIAL_GENESIS_BLOCK_ID) { + concord::util::digest::BlockDigest empty_digest; + empty_digest.fill(0); + return empty_digest; + } + auto block_str = getBlockData(id); + ConcordAssert(block_str.has_value()); + return v4blockchain::detail::Block::calculateDigest(id, block_str->c_str(), block_str->size()); +} + +concord::util::digest::BlockDigest Blockchain::getBlockParentDigest(concord::kvbc::BlockId id) const { + auto block_str = getBlockData(id); + ConcordAssert(block_str.has_value()); + return v4blockchain::detail::Block{*block_str}.parentDigest(); +} + +std::optional Blockchain::getBlockData(concord::kvbc::BlockId id) const { + auto blockKey = generateKey(id); + return native_client_->get(v4blockchain::detail::BLOCKS_CF, blockKey); +} + +std::optional Blockchain::getBlockUpdates(BlockId id) const { + auto block_buffer = getBlockData(id); + if (!block_buffer) return std::nullopt; + auto block = v4blockchain::detail::Block(*block_buffer); + return block.getUpdates(); +} + +void Blockchain::multiGetBlockData(const std::vector& block_ids, + std::unordered_map>& values) const { + values.clear(); + std::vector block_keys(block_ids.size()); + std::transform(block_ids.cbegin(), block_ids.cend(), block_keys.begin(), [](BlockId bid) { + return Blockchain::generateKey(bid); + }); + std::vector<::rocksdb::PinnableSlice> slices(block_ids.size()); + std::vector<::rocksdb::Status> statuses(block_ids.size()); + native_client_->multiGet(v4blockchain::detail::BLOCKS_CF, block_keys, slices, statuses); + for (auto i = 0ull; i < slices.size(); ++i) { + const auto& status = statuses[i]; + const auto& slice = slices[i]; + const auto& block_id = block_ids[i]; + if (status.ok()) { + if (!values.try_emplace(block_id, slice.ToString()).second) { + throw std::logic_error{std::string("Duplicate block ids should not be sent: ") + block_keys[i]}; + } + } else if (status.IsNotFound()) { + if (!values.try_emplace(block_id, std::nullopt).second) { + throw std::logic_error{std::string("Duplicate block ids should not be sent: ") + block_keys[i]}; + } + } else { + // Should never happen. + throw std::runtime_error{"BLOCK_CF multiGet() failure: " + status.ToString()}; + } + } +} + +void Blockchain::multiGetBlockUpdates( + std::vector block_ids, std::unordered_map>& values) const { + auto uqid = std::unique(block_ids.begin(), block_ids.end()); + block_ids.resize(std::distance(block_ids.begin(), uqid)); + std::unordered_map> updates; + multiGetBlockData(block_ids, updates); + ConcordAssertEQ(block_ids.size(), updates.size()); + for (const auto& block_buffer : updates) { + if (block_buffer.second) { + auto block = v4blockchain::detail::Block(*(block_buffer.second)); + values.emplace(block_buffer.first, block.getUpdates()); + } else { + values.emplace(block_buffer.first, std::nullopt); + } + } +} + +// get the closest key to MAX_BLOCK_ID +std::optional Blockchain::loadLastReachableBlockId() { + auto itr = native_client_->getIterator(v4blockchain::detail::BLOCKS_CF); + itr.seekAtMost(generateKey(MAX_BLOCK_ID)); + if (!itr) { + return std::optional{}; + } + return concordUtils::fromBigEndianBuffer(itr.keyView().data()); +} + +// get the closest key to INITIAL_GENESIS_BLOCK_ID +std::optional Blockchain::loadGenesisBlockId() { + auto itr = native_client_->getIterator(detail::BLOCKS_CF); + itr.seekAtLeast(generateKey(concord::kvbc::INITIAL_GENESIS_BLOCK_ID)); + if (!itr) { + return std::optional{}; + } + return concordUtils::fromBigEndianBuffer(itr.keyView().data()); +} + +void Blockchain::setBlockId(BlockId id) { + setLastReachable(id); + if (genesis_block_id_ == INVALID_BLOCK_ID) genesis_block_id_ = id; +} + +bool Blockchain::hasBlock(BlockId block_id) const { + if (block_id > last_reachable_block_id_) return false; + return native_client_->getSlice(v4blockchain::detail::BLOCKS_CF, generateKey(block_id)).has_value(); +} + +} // namespace concord::kvbc::v4blockchain::detail diff --git a/kvbc/src/v4blockchain/detail/blocks.cpp b/kvbc/src/v4blockchain/detail/blocks.cpp new file mode 100644 index 0000000000..f24125d8e8 --- /dev/null +++ b/kvbc/src/v4blockchain/detail/blocks.cpp @@ -0,0 +1,38 @@ +// Concord +// +// Copyright (c) 2022 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 subcomponent's license, as noted in the LICENSE +// file. + +#include "v4blockchain/detail/blocks.h" + +namespace concord::kvbc::v4blockchain::detail { + +void Block::addUpdates(const concord::kvbc::categorization::Updates& category_updates) { + ConcordAssert(isValid_); + static thread_local std::vector updates_buffer; + concord::kvbc::categorization::serialize(updates_buffer, category_updates.categoryUpdates()); + buffer_.resize(HEADER_SIZE + updates_buffer.size()); + auto dist = std::distance(buffer_.begin(), buffer_.begin() + HEADER_SIZE); + std::copy(updates_buffer.begin(), updates_buffer.end(), buffer_.begin() + dist); + updates_buffer.clear(); +} + +categorization::Updates Block::getUpdates() const { + ConcordAssert(isValid_); + ConcordAssert(buffer_.size() > HEADER_SIZE); + concord::kvbc::categorization::CategoryInput category_updates; + const uint8_t* start = buffer_.data() + HEADER_SIZE; + const uint8_t* end = buffer_.data() + buffer_.size(); + concord::kvbc::categorization::deserialize(start, end, category_updates); + return concord::kvbc::categorization::Updates(std::move(category_updates)); +} + +} // namespace concord::kvbc::v4blockchain::detail diff --git a/kvbc/src/v4blockchain/detail/categories.cpp b/kvbc/src/v4blockchain/detail/categories.cpp new file mode 100644 index 0000000000..5ac1ddb72b --- /dev/null +++ b/kvbc/src/v4blockchain/detail/categories.cpp @@ -0,0 +1,138 @@ +// Concord +// +// Copyright (c) 2022 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 subcomponent's license, as noted in the LICENSE +// file. + +#include "v4blockchain/detail/categories.h" +#include "v4blockchain/detail/column_families.h" +#include "Logger.hpp" + +using namespace concord::kvbc; +namespace concord::kvbc::v4blockchain::detail { + +Categories::Categories(const std::shared_ptr& client, + const std::optional>& category_types) + : native_client_(client) { + if (native_client_->createColumnFamilyIfNotExisting(v4blockchain::detail::CATEGORIES_CF)) { + LOG_INFO(V4_BLOCK_LOG, "Created [" << v4blockchain::detail::CATEGORIES_CF << "]"); + } + loadCategories(); + if (category_to_prefix_.empty()) { + initNewBlockchainCategories(category_types); + } else { + initExistingBlockchainCategories(category_types); + } +} + +void Categories::loadCategories() { + auto itr = native_client_->getIterator(detail::CATEGORIES_CF); + itr.first(); + while (itr) { + if (itr.valueView().size() != 2) { + LOG_FATAL(V4_BLOCK_LOG, "Category type value of [" << itr.key() << "] is invalid (bigger than one)."); + ConcordAssertEQ(itr.valueView().size(), 1); + } + auto cat_type = static_cast(itr.valueView()[0]); + auto cat_prefix = std::string(1, static_cast(itr.valueView()[1])); + switch (cat_type) { + case categorization::CATEGORY_TYPE::block_merkle: + category_types_[itr.key()] = categorization::CATEGORY_TYPE::block_merkle; + category_to_prefix_[itr.key()] = cat_prefix; + LOG_INFO(V4_BLOCK_LOG, + "Created category [" << itr.key() << "] as type BlockMerkleCategory with prefix " << cat_prefix); + break; + case categorization::CATEGORY_TYPE::immutable: + category_types_[itr.key()] = categorization::CATEGORY_TYPE::immutable; + category_to_prefix_[itr.key()] = cat_prefix; + LOG_INFO(V4_BLOCK_LOG, + "Created category [" << itr.key() << "] as type ImmutableKeyValueCategory with prefix " << cat_prefix); + break; + case categorization::CATEGORY_TYPE::versioned_kv: + category_types_[itr.key()] = categorization::CATEGORY_TYPE::versioned_kv; + category_to_prefix_[itr.key()] = cat_prefix; + LOG_INFO(V4_BLOCK_LOG, + "Created category [" << itr.key() << "] as type VersionedKeyValueCategory with prefix " << cat_prefix); + break; + default: + ConcordAssert(false); + break; + } + itr.next(); + } +} + +void Categories::initNewBlockchainCategories( + const std::optional>& category_types) { + if (!category_types) { + const auto msg = "Category types needed when constructing a KeyValueBlockchain for a new blockchain"; + LOG_ERROR(V4_BLOCK_LOG, msg); + throw std::invalid_argument{msg}; + } + // Add all categories passed by the user. + for (const auto& [category_id, type] : *category_types) { + addNewCategory(category_id, type); + } +} + +void Categories::addNewCategory(const std::string& cat_id, categorization::CATEGORY_TYPE type) { + // cache the type in memory and store it in DB + auto inserted = category_types_.try_emplace(cat_id, type).second; + if (!inserted) { + LOG_FATAL(V4_BLOCK_LOG, "Category [" << cat_id << "] already exists in type map"); + ConcordAssert(false); + } + + auto prefix = std::string(1, static_cast(PREFIX_START + category_types_.size())); + for (const auto& [k, v] : category_to_prefix_) { + (void)k; // unused + ConcordAssertNE(prefix, v); + } + inserted = category_to_prefix_.try_emplace(cat_id, prefix).second; + if (!inserted) { + LOG_FATAL(V4_BLOCK_LOG, "Category [" << cat_id << "] already exists in prefix map"); + ConcordAssert(false); + } + ConcordAssertLE(category_types_.size(), MAX_NUM_CATEGORIES); + std::string value = {static_cast(type), static_cast(PREFIX_START + category_types_.size())}; + native_client_->put(detail::CATEGORIES_CF, cat_id, value); +} + +void Categories::initExistingBlockchainCategories( + const std::optional>& category_types) { + if (!category_types) { + return; + } + + // Make sure the user passed all existing categories on disk with their correct types. + for (const auto& [category_id, type] : category_types_) { + auto it = category_types->find(category_id); + if (it == category_types->cend()) { + const auto msg = + "Category ID [" + category_id + "] exists on disk, but is missing from the given category types parameter"; + LOG_ERROR(V4_BLOCK_LOG, msg); + throw std::invalid_argument{msg}; + } else if (it->second != type) { + const auto msg = "Category ID [" + category_id + "] parameter with type [" + categoryStringType(it->second) + + "] differs from type on disk [" + categoryStringType(type) + "]"; + LOG_ERROR(V4_BLOCK_LOG, msg); + throw std::invalid_argument{msg}; + } + } + + // If the user passed a new category, add it. + for (const auto& [category_id, type] : *category_types) { + if (category_types_.find(category_id) == category_types_.cend()) { + addNewCategory(category_id, type); + } + } +} + +} // namespace concord::kvbc::v4blockchain::detail diff --git a/kvbc/src/v4blockchain/detail/latest_keys.cpp b/kvbc/src/v4blockchain/detail/latest_keys.cpp new file mode 100644 index 0000000000..94d581bcc6 --- /dev/null +++ b/kvbc/src/v4blockchain/detail/latest_keys.cpp @@ -0,0 +1,325 @@ +// Concord +// +// Copyright (c) 2022 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 subcomponent's license, as noted in the LICENSE +// file. + +#include "endianness.hpp" +#include "v4blockchain/detail/latest_keys.h" +#include "v4blockchain/detail/column_families.h" +#include "Logger.hpp" +#include "rocksdb/time_stamp_comparator.h" +#include "v4blockchain/detail/blockchain.h" +#include "rocksdb/details.h" + +using namespace concord::kvbc; +namespace concord::kvbc::v4blockchain::detail { + +template +auto getSliceArray(const Sliceable&... sls) { + return std::array<::rocksdb::Slice, sizeof...(sls)>{sls...}; +} + +std::string ToHexad(const std::string& s) { + std::ostringstream ret; + + for (std::string::size_type i = 0; i < s.length(); ++i) + ret << std::hex << std::setfill('0') << std::setw(2) << std::uppercase << (int)s[i]; + + return ret.str(); +} + +LatestKeys::LatestKeys(const std::shared_ptr& native_client, + const std::optional>& categories, + std::function&& f) + : native_client_{native_client}, category_mapping_(native_client, categories), comp_filter_(std::move(f)) { + if (native_client_->createColumnFamilyIfNotExisting(v4blockchain::detail::LATEST_KEYS_CF, + concord::storage::rocksdb::getLexicographic64TsComparator(), + getCompFilter())) { + LOG_INFO(V4_BLOCK_LOG, + "Created [" << v4blockchain::detail::LATEST_KEYS_CF << "] column family for the latest keys"); + } +} + +void LatestKeys::addBlockKeys(const concord::kvbc::categorization::Updates& updates, + BlockId block_id, + storage::rocksdb::NativeWriteBatch& write_batch) { + LOG_DEBUG(V4_BLOCK_LOG, "Adding keys of block [" << block_id << "] to the latest CF"); + auto block_key = v4blockchain::detail::Blockchain::generateKey(block_id); + ConcordAssertEQ(block_key.size(), concord::storage::rocksdb::TIME_STAMP_SIZE); + for (const auto& [category_id, updates] : updates.categoryUpdates().kv) { + std::visit([category_id = category_id, &write_batch, &block_key, this]( + const auto& updates) { handleCategoryUpdates(block_key, category_id, updates, write_batch); }, + updates); + } +} + +void LatestKeys::handleCategoryUpdates(const std::string& block_version, + const std::string& category_id, + const categorization::BlockMerkleInput& updates, + concord::storage::rocksdb::NativeWriteBatch& write_batch) { + const auto& prefix = category_mapping_.categoryPrefix(category_id); + // add keys + Flags flags = {0}; + auto sl_flags = concord::storage::rocksdb::detail::toSlice(flags); + for (const auto& [k, v] : updates.kv) { + write_batch.put( + v4blockchain::detail::LATEST_KEYS_CF, getSliceArray(prefix, k, block_version), getSliceArray(v, sl_flags)); + } + for (const auto& k : updates.deletes) { + write_batch.del(v4blockchain::detail::LATEST_KEYS_CF, getSliceArray(prefix, k, block_version)); + } +} + +void LatestKeys::handleCategoryUpdates(const std::string& block_version, + const std::string& category_id, + const categorization::VersionedInput& updates, + concord::storage::rocksdb::NativeWriteBatch& write_batch) { + const auto& prefix = category_mapping_.categoryPrefix(category_id); + // add keys + Flags flags = {0}; + for (const auto& [k, v] : updates.kv) { + if (v.stale_on_update) { + flags = STALE_ON_UPDATE; + } + auto sl_flags = concord::storage::rocksdb::detail::toSlice(flags); + LOG_DEBUG(V4_BLOCK_LOG, + "adding key " << std::hash{}(k) << " version " + << concordUtils::fromBigEndianBuffer(block_version.data()) << " category " + << category_id << " prefix " << prefix << " key hex " << ToHexad(k) << " key " << k); + + write_batch.put( + v4blockchain::detail::LATEST_KEYS_CF, getSliceArray(prefix, k, block_version), getSliceArray(v.data, sl_flags)); + } + for (const auto& k : updates.deletes) { + write_batch.del(v4blockchain::detail::LATEST_KEYS_CF, getSliceArray(prefix, k, block_version)); + } +} + +void LatestKeys::handleCategoryUpdates(const std::string& block_version, + const std::string& category_id, + const categorization::ImmutableInput& updates, + concord::storage::rocksdb::NativeWriteBatch& write_batch) { + static thread_local std::string out_ts; + static thread_local std::string get_key; + const auto& prefix = category_mapping_.categoryPrefix(category_id); + // add keys + auto sl_flags = concord::storage::rocksdb::detail::toSlice(STALE_ON_UPDATE); + for (const auto& [k, v] : updates.kv) { + get_key.clear(); + get_key.append(prefix); + get_key.append(k); + // check if key exists - immutable does not allow to update + // auto opt_val = native_client_->get(v4blockchain::detail::LATEST_KEYS_CF, get_key, block_version, &out_ts); + // if (opt_val) { + // throw std::runtime_error("Trying to update immutable key: " + k); + // } + write_batch.put( + v4blockchain::detail::LATEST_KEYS_CF, getSliceArray(prefix, k, block_version), getSliceArray(v.data, sl_flags)); + } +} + +/* +Iterate over updates, for each key: +1 - read its previous version by calling get with version id - 1. +2 - if the previous version does not exist, mark the key for delete. +3 - if the previous version exists, update the value with old value and id as version. +*/ +void LatestKeys::revertLastBlockKeys(const concord::kvbc::categorization::Updates& updates, + BlockId block_id, + storage::rocksdb::NativeWriteBatch& write_batch) { + ConcordAssertGT(block_id, 0); + LOG_DEBUG(V4_BLOCK_LOG, "Reverting keys of block [" << block_id << "] "); + auto block_key = v4blockchain::detail::Blockchain::generateKey(block_id); + auto prev_block_key = v4blockchain::detail::Blockchain::generateKey(block_id - 1); + ConcordAssertEQ(block_key.size(), concord::storage::rocksdb::TIME_STAMP_SIZE); + for (const auto& [category_id, updates] : updates.categoryUpdates().kv) { + std::visit( + [category_id = category_id, &write_batch, &block_key, &prev_block_key, this](const auto& updates) { + revertCategoryKeys(block_key, prev_block_key, category_id, updates, write_batch); + }, + updates); + } +} + +void LatestKeys::revertCategoryKeys(const std::string& block_version, + const std::string& prev_block_version, + const std::string& category_id, + const categorization::BlockMerkleInput& updates, + concord::storage::rocksdb::NativeWriteBatch& write_batch) { + revertCategoryKeysImp(block_version, prev_block_version, category_id, updates.kv, write_batch); + revertDeletedKeysImp(block_version, prev_block_version, category_id, updates.deletes, write_batch); +} + +void LatestKeys::revertCategoryKeys(const std::string& block_version, + const std::string& prev_block_version, + const std::string& category_id, + const categorization::VersionedInput& updates, + concord::storage::rocksdb::NativeWriteBatch& write_batch) { + revertCategoryKeysImp(block_version, prev_block_version, category_id, updates.kv, write_batch); + revertDeletedKeysImp(block_version, prev_block_version, category_id, updates.deletes, write_batch); +} + +void LatestKeys::revertCategoryKeys(const std::string& block_version, + const std::string& prev_block_version, + const std::string& category_id, + const categorization::ImmutableInput& updates, + concord::storage::rocksdb::NativeWriteBatch& write_batch) { + revertCategoryKeysImp(block_version, prev_block_version, category_id, updates.kv, write_batch); +} + +template +void LatestKeys::revertCategoryKeysImp(const std::string& block_version, + const std::string& prev_block_version, + const std::string& category_id, + const UPDATES& updates, + concord::storage::rocksdb::NativeWriteBatch& write_batch) { + std::string get_key; + std::string out_ts; + const auto& prefix = category_mapping_.categoryPrefix(category_id); + for (const auto& [k, _] : updates) { + (void)_; // unsued + get_key.append(prefix); + get_key.append(k); + // check if key exists for the previous version + auto opt_val = native_client_->get(v4blockchain::detail::LATEST_KEYS_CF, get_key, prev_block_version, &out_ts); + // if no previous version, delete it. + if (!opt_val) { + write_batch.del(v4blockchain::detail::LATEST_KEYS_CF, getSliceArray(prefix, k, block_version)); + continue; + } + // add the previous value with the current block as its version. + write_batch.put( + v4blockchain::detail::LATEST_KEYS_CF, getSliceArray(prefix, k, block_version), getSliceArray(*opt_val)); + } +} + +template +void LatestKeys::revertDeletedKeysImp(const std::string& block_version, + const std::string& prev_block_version, + const std::string& category_id, + const DELETES& deletes, + concord::storage::rocksdb::NativeWriteBatch& write_batch) { + static thread_local std::string get_key; + static thread_local std::string out_ts; + const auto& prefix = category_mapping_.categoryPrefix(category_id); + for (const auto& k : deletes) { + get_key.clear(); + get_key.append(prefix); + get_key.append(k); + // check if key exists for the previous version + auto opt_val = native_client_->get(v4blockchain::detail::LATEST_KEYS_CF, get_key, prev_block_version, &out_ts); + // A key was deleted in the last block, but we can't find the previous version for revert. + // I think it's not a scenario for assertion or excepion as the input is from external source i.e. execution engine. + if (!opt_val) { + LOG_WARN(V4_BLOCK_LOG, "Couldn't find previous version for deleted key " << get_key); + continue; + } + // add the previous value with the current block as its version. + write_batch.put( + v4blockchain::detail::LATEST_KEYS_CF, getSliceArray(prefix, k, block_version), getSliceArray(*opt_val)); + } +} + +void LatestKeys::trimHistoryUntil(BlockId block_id) { + auto block_key = v4blockchain::detail::Blockchain::generateKey(block_id); + auto& raw_db = native_client_->rawDB(); + auto status = raw_db.IncreaseFullHistoryTsLow( + native_client_->columnFamilyHandle(v4blockchain::detail::LATEST_KEYS_CF), block_key); + if (!status.ok()) { + LOG_ERROR(V4_BLOCK_LOG, "Failed trimming history due to " << status.ToString()); + throw std::runtime_error(status.ToString()); + } +} + +std::optional LatestKeys::getValue(BlockId latest_block_id, + const std::string& category_id, + const std::string& latest_version, + const std::string& key) const { + std::string get_key; + std::string out_ts; + const auto& prefix = category_mapping_.categoryPrefix(category_id); + get_key.append(prefix); + get_key.append(key); + auto opt_val = native_client_->get(v4blockchain::detail::LATEST_KEYS_CF, get_key, latest_version, &out_ts); + if (!opt_val) { + return std::nullopt; + } + auto actual_version = concordUtils::fromBigEndianBuffer(out_ts.data()); + const size_t total_val_size = opt_val->size(); + + switch (category_mapping_.categoryType(category_id)) { + case concord::kvbc::categorization::CATEGORY_TYPE::block_merkle: + return categorization::MerkleValue{{actual_version, opt_val->substr(0, total_val_size - FLAGS_SIZE)}}; + case concord::kvbc::categorization::CATEGORY_TYPE::immutable: { + return categorization::ImmutableValue{{actual_version, opt_val->substr(0, total_val_size - FLAGS_SIZE)}}; + } + case concord::kvbc::categorization::CATEGORY_TYPE::versioned_kv: { + return categorization::VersionedValue{{actual_version, opt_val->substr(0, total_val_size - FLAGS_SIZE)}}; + } + default: + ConcordAssert(false); + } +} + +void LatestKeys::multiGetValue(BlockId latest_block_id, + const std::string& category_id, + const std::string& latest_version, + const std::vector& keys, + std::vector>& values) const { + values.clear(); + values.reserve(keys.size()); + for (const auto& key : keys) { + values.push_back(getValue(latest_block_id, category_id, latest_version, key)); + } +} + +std::optional LatestKeys::getLatestVersion(const std::string& category_id, + const std::string& latest_version, + const std::string& key) const { + std::string get_key; + std::string out_ts; + const auto& prefix = category_mapping_.categoryPrefix(category_id); + get_key.append(prefix); + get_key.append(key); + auto opt_val = native_client_->get(v4blockchain::detail::LATEST_KEYS_CF, get_key, latest_version, &out_ts); + if (!opt_val) { + return std::nullopt; + } + BlockId version = concordUtils::fromBigEndianBuffer(out_ts.data()); + return categorization::TaggedVersion{false, version}; +} + +void LatestKeys::multiGetLatestVersion(const std::string& category_id, + const std::string& latest_version, + const std::vector& keys, + std::vector>& versions) const { + versions.clear(); + versions.reserve(keys.size()); + for (const auto& key : keys) { + versions.push_back(getLatestVersion(category_id, latest_version, key)); + } +} + +bool LatestKeys::LKCompactionFilter::Filter(int /*level*/, + const ::rocksdb::Slice& key, + const ::rocksdb::Slice& val, + std::string* /*new_value*/, + bool* /*value_changed*/) const { + if (!LatestKeys::isStaleOnUpdate(val)) return false; + auto genesis = genesis_id(); + auto ts_slice = storage::rocksdb::ExtractTimestampFromUserKey(key, concord::storage::rocksdb::TIME_STAMP_SIZE); + auto key_version = concordUtils::fromBigEndianBuffer(ts_slice.data()); + if (key_version >= genesis) return false; + LOG_INFO(V4_BLOCK_LOG, "Filtering key with version " << key_version << " genesis is " << genesis); + return true; +} + +} // namespace concord::kvbc::v4blockchain::detail diff --git a/kvbc/src/v4blockchain/detail/st_chain.cpp b/kvbc/src/v4blockchain/detail/st_chain.cpp new file mode 100644 index 0000000000..e1eab4b037 --- /dev/null +++ b/kvbc/src/v4blockchain/detail/st_chain.cpp @@ -0,0 +1,99 @@ +// Concord +// +// Copyright (c) 2020 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 subcomponent's license, as noted in the LICENSE +// file. + +#include "v4blockchain/detail/st_chain.h" +#include "v4blockchain/detail/column_families.h" +#include "v4blockchain/detail/blockchain.h" +#include "Logger.hpp" + +namespace concord::kvbc::v4blockchain::detail { + +StChain::StChain(const std::shared_ptr& native_client) + : native_client_{native_client} { + if (native_client_->createColumnFamilyIfNotExisting(v4blockchain::detail::ST_CHAIN_CF)) { + LOG_INFO(V4_BLOCK_LOG, + "Created [" << v4blockchain::detail::ST_CHAIN_CF << "] column family for the state transfer blockchain"); + } + loadLastBlockId(); + if (last_block_id_ > 0) { + LOG_INFO(CAT_BLOCK_LOG, "State transfer last block id: " << last_block_id_); + } +} + +/////////// BLOCKS///////////////////////// +bool StChain::hasBlock(BlockId block_id) const { + if (last_block_id_ < block_id) return false; + return native_client_ + ->getSlice(v4blockchain::detail::ST_CHAIN_CF, v4blockchain::detail::Blockchain::generateKey(block_id)) + .has_value(); +} + +void StChain::addBlock(const BlockId id, const char* block, const uint32_t blockSize) { + auto write_batch = native_client_->getBatch(); + write_batch.put(v4blockchain::detail::ST_CHAIN_CF, + v4blockchain::detail::Blockchain::generateKey(id), + ::rocksdb::Slice(block, blockSize)); + native_client_->write(std::move(write_batch)); + updateLastIdIfBigger(id); +} + +void StChain::deleteBlock(const BlockId id, storage::rocksdb::NativeWriteBatch& wb) { + wb.del(v4blockchain::detail::ST_CHAIN_CF, v4blockchain::detail::Blockchain::generateKey(id)); +} +/////////////// ST LAST BLOCK ID //////////// +void StChain::loadLastBlockId() { + auto itr = native_client_->getIterator(v4blockchain::detail::ST_CHAIN_CF); + auto max_db_key = v4blockchain::detail::Blockchain::generateKey(v4blockchain::detail::Blockchain::MAX_BLOCK_ID); + itr.seekAtMost(max_db_key); + if (!itr) { + last_block_id_ = 0; + return; + } + last_block_id_ = concordUtils::fromBigEndianBuffer(itr.keyView().data()); +} + +void StChain::updateLastIdAfterDeletion(const BlockId id) { + if (last_block_id_ == 0 || last_block_id_ != id) { + return; + } + last_block_id_ = 0; + loadLastBlockId(); + return; +} + +void StChain::updateLastIdIfBigger(const BlockId id) { + if (last_block_id_ >= id) { + return; + } + last_block_id_ = id; +} + +std::optional StChain::getBlock(kvbc::BlockId id) const { + auto key = v4blockchain::detail::Blockchain::generateKey(id); + auto opt_block_str = native_client_->get(v4blockchain::detail::ST_CHAIN_CF, key); + if (!opt_block_str) return std::nullopt; + return v4blockchain::detail::Block(*opt_block_str); +} + +concord::util::digest::BlockDigest StChain::getBlockParentDigest(concord::kvbc::BlockId id) const { + auto block = getBlock(id); + ConcordAssert(block.has_value()); + return block->parentDigest(); +} + +std::optional StChain::getBlockData(concord::kvbc::BlockId id) const { + auto key = v4blockchain::detail::Blockchain::generateKey(id); + return native_client_->get(v4blockchain::detail::ST_CHAIN_CF, key); +} + +} // namespace concord::kvbc::v4blockchain::detail diff --git a/kvbc/src/v4blockchain/v4_blockchain.cpp b/kvbc/src/v4blockchain/v4_blockchain.cpp new file mode 100644 index 0000000000..e565376034 --- /dev/null +++ b/kvbc/src/v4blockchain/v4_blockchain.cpp @@ -0,0 +1,476 @@ +// Concord +// +// Copyright (c) 2022 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 subcomponent's license, as noted in the LICENSE +// file. + +#include + +#include "v4blockchain/v4_blockchain.h" +#include "v4blockchain/detail/detail.h" +#include "categorization/base_types.h" +#include "categorization/db_categories.h" +#include "kvbc_key_types.hpp" +#include "block_metadata.hpp" +#include "throughput.hpp" + +namespace concord::kvbc::v4blockchain { + +using namespace concord::kvbc; + +std::string ToHexad(const std::string &s) { + std::ostringstream ret; + + for (std::string::size_type i = 0; i < s.length(); ++i) + ret << std::hex << std::setfill('0') << std::setw(2) << std::uppercase << (int)s[i]; + + return ret.str(); +} + +KeyValueBlockchain::KeyValueBlockchain( + const std::shared_ptr &native_client, + bool link_st_chain, + const std::optional> &category_types) + : native_client_{native_client}, + block_chain_{native_client_}, + state_transfer_chain_{native_client_}, + latest_keys_{native_client_, category_types, [&]() { return block_chain_.getGenesisBlockId(); }} { + if (!link_st_chain || state_transfer_chain_.getLastBlockId() == 0) return; + // Make sure that if linkSTChainFrom() has been interrupted (e.g. a crash or an abnormal shutdown), all DBAdapter + // methods will return the correct values. For example, if state transfer had completed and linkSTChainFrom() was + // interrupted, getLatestBlockId() should be equal to getLastReachableBlockId() on the next startup. Another example + // is getValue() that returns keys from the blockchain only and ignores keys in the temporary state + // transfer chain. + LOG_INFO(V4_BLOCK_LOG, "Try to link ST temporary chain, this might take some time..."); + auto old_last_reachable_block_id = getLastReachableBlockId(); + linkSTChain(); + auto new_last_reachable_block_id = getLastReachableBlockId(); + LOG_INFO(V4_BLOCK_LOG, + "Done linking ST temporary chain:" << KVLOG(old_last_reachable_block_id, new_last_reachable_block_id)); +} + +//////////////////////////// ADDER//////////////////////////////////////////// +/* + 1 - check if we can perform GC on latest CF + 2 - add the block to the blocks CF + 3 - add the keys to the latest CF + 4 - atomic write to storage + 5 - increment last reachable block. +*/ +BlockId KeyValueBlockchain::add(categorization::Updates &&updates) { + // Should be performed before we add the block with the current Updates. + auto sequence_number = markHistoryForGarbageCollectionIfNeeded(updates); + auto write_batch = native_client_->getBatch(); + // addGenesisBlockKey(updates); + auto block_id = add(updates, write_batch); + + native_client_->write(std::move(write_batch)); + block_chain_.setBlockId(block_id); + if (sequence_number > 0) setLastBlockSequenceNumber(sequence_number); + return block_id; +} + +BlockId KeyValueBlockchain::add(const categorization::Updates &updates, + storage::rocksdb::NativeWriteBatch &write_batch) { + BlockId block_id{}; + { block_id = block_chain_.addBlock(updates, write_batch); } + { latest_keys_.addBlockKeys(updates, block_id, write_batch); } + return block_id; +} + +//////////////////////////// DELETER//////////////////////////////////////////// + +BlockId KeyValueBlockchain::deleteBlocksUntil(BlockId until) { + auto scoped = v4blockchain::detail::ScopedDuration{"deleteBlocksUntil"}; + return block_chain_.deleteBlocksUntil(until); +} + +void KeyValueBlockchain::deleteGenesisBlock() { + auto scoped = v4blockchain::detail::ScopedDuration{"deleteGenesisBlock"}; + block_chain_.deleteGenesisBlock(); +} + +void KeyValueBlockchain::deleteLastReachableBlock() { + // validate conditions + auto last_reachable_id = block_chain_.getLastReachable(); + auto genesis_id = block_chain_.getGenesisBlockId(); + ConcordAssertLT(genesis_id, last_reachable_id); + // get block updates + auto write_batch = native_client_->getBatch(); + auto updates = block_chain_.getBlockUpdates(last_reachable_id); + ConcordAssert(updates.has_value()); + // revert from latest keys + latest_keys_.revertLastBlockKeys(*updates, last_reachable_id, write_batch); + // delete from blockchain + block_chain_.deleteLastReachableBlock(write_batch); + // atomically commit changes + native_client_->write(std::move(write_batch)); + block_chain_.setBlockId(--last_reachable_id); + LOG_INFO(V4_BLOCK_LOG, "Deleted last reachable, new value is " << last_reachable_id); +} + +/* + If the bft sequence number for this updates is bigger than the last, it means that the last sequence number + was commited, and it's safe to mark the version of the last block that corresponds to that sequence number as safe + for rocksdb garbage collection. + https://github.com/facebook/rocksdb/wiki/User-defined-Timestamp-(Experimental)#compaction-and-garbage-collection-gc + - if last_block_sn_ is nullopt we're probably on first start. + - we don't want to trim when checkpoint is in process. +*/ +uint64_t KeyValueBlockchain::markHistoryForGarbageCollectionIfNeeded(const categorization::Updates &updates) { + if (checkpointInProcess_) return 0; + auto sequence_number = getBlockSequenceNumber(updates); + // Optional not set yet or same sn + if (!last_block_sn_ || *last_block_sn_ == sequence_number) return sequence_number; + // This is abnormal + if (sequence_number < *last_block_sn_) { + LOG_WARN( + V4_BLOCK_LOG, + "Sequence number for trim history " << sequence_number << " is lower than previous value " << *last_block_sn_); + return 0; + } + ++gc_counter; + auto block_id = block_chain_.getLastReachable(); + latest_keys_.trimHistoryUntil(block_id); + if (sequence_number % 100 == 0) { + LOG_INFO(V4_BLOCK_LOG, "History was marked for trim up to block " << block_id); + } + return sequence_number; +} + +uint64_t KeyValueBlockchain::getBlockSequenceNumber(const categorization::Updates &updates) const { + static std::string key = std::string(1, IBlockMetadata::kBlockMetadataKey); + const auto &input = updates.categoryUpdates(); + if (input.kv.count(categorization::kConcordInternalCategoryId) == 0) { + return 0; + } + const auto &ver_input = + std::get(input.kv.at(categorization::kConcordInternalCategoryId)); + if (ver_input.kv.count(key) == 0) { + return 0; + } + return BlockMetadata::getSequenceNum(ver_input.kv.at(key).data); +} + +void KeyValueBlockchain::addGenesisBlockKey(categorization::Updates &updates) const { + const auto stale_on_update = true; + updates.addCategoryIfNotExisting(categorization::kConcordInternalCategoryId); + updates.appendKeyValue( + categorization::kConcordInternalCategoryId, + std::string{keyTypes::genesis_block_key}, + categorization::VersionedUpdates::ValueType{ + concordUtils::toBigEndianStringBuffer(block_chain_.getGenesisBlockId()), stale_on_update}); +} + +///////////// STATE TRANSFER//////////////////////// +bool KeyValueBlockchain::hasBlock(BlockId block_id) const { + const auto last_reachable_block = block_chain_.getLastReachable(); + if (block_id > last_reachable_block) { + return state_transfer_chain_.hasBlock(block_id); + } + return block_chain_.hasBlock(block_id); +} + +std::optional KeyValueBlockchain::getBlockData(const BlockId &block_id) const { + const auto last_reachable_block = getLastReachableBlockId(); + // Try to take it from the ST chain + if (block_id > last_reachable_block) { + return state_transfer_chain_.getBlockData(block_id); + } + // Try from the blockchain itself + return block_chain_.getBlockData(block_id); +} + +std::optional KeyValueBlockchain::getLastStatetransferBlockId() const { + if (state_transfer_chain_.getLastBlockId() == 0) return std::nullopt; + return state_transfer_chain_.getLastBlockId(); +} + +concord::util::digest::BlockDigest KeyValueBlockchain::parentDigest(BlockId block_id) const { + const auto last_reachable_block = getLastReachableBlockId(); + if (block_id > last_reachable_block) { + return state_transfer_chain_.getBlockParentDigest(block_id); + } + if (block_id < getGenesisBlockId()) { + LOG_ERROR(V4_BLOCK_LOG, + "Trying to get digest from block " << block_id << " while genesis is " << getGenesisBlockId()); + concord::util::digest::BlockDigest empty_digest; + empty_digest.fill(0); + return empty_digest; + } + return block_chain_.getBlockParentDigest(block_id); +} + +void KeyValueBlockchain::addBlockToSTChain(const BlockId &block_id, + const char *block, + const uint32_t block_size, + bool last_block) { + const auto last_reachable_block = getLastReachableBlockId(); + if (block_id <= last_reachable_block) { + const auto msg = "Cannot add an existing block ID " + std::to_string(block_id); + throw std::invalid_argument{msg}; + } + + if (state_transfer_chain_.hasBlock(block_id)) { + auto existing_block = state_transfer_chain_.getBlockData(block_id); + ConcordAssert(existing_block.has_value()); + auto view = std::string_view{block, block_size}; + if (view != *existing_block) { + LOG_ERROR(V4_BLOCK_LOG, + "found existing (and different) block ID[" << block_id << "] when receiving from state transfer"); + + // E.L I think it's dangerous to delete the block and there is no value in doing so + // kvbc_->deleteBlock(blockId); + throw std::runtime_error( + __PRETTY_FUNCTION__ + + std::string("found existing (and different) block when receiving state transfer, block ID: ") + + std::to_string(block_id)); + } + return; + } + + state_transfer_chain_.addBlock(block_id, block, block_size); + if (last_block) { + try { + linkSTChain(); + } catch (const std::exception &e) { + LOG_FATAL(V4_BLOCK_LOG, + "Aborting due to failure to link chains after block has been added, reason: " << e.what()); + std::terminate(); + } catch (...) { + LOG_FATAL(V4_BLOCK_LOG, "Aborting due to failure to link chains after block has been added"); + std::terminate(); + } + } +} + +void KeyValueBlockchain::linkSTChain() { + BlockId block_id = getLastReachableBlockId() + 1; + const auto last_block_id = state_transfer_chain_.getLastBlockId(); + if (last_block_id == 0) return; + + for (auto i = block_id; i <= last_block_id; ++i) { + auto block = state_transfer_chain_.getBlock(i); + if (!block) { + return; + } + auto updates = block->getUpdates(); + writeSTLinkTransaction(i, updates); + } + // Linking has fully completed and we should not have any more ST temporary blocks left. Therefore, make sure we don't + // have any value for the latest ST temporary block ID cache. + state_transfer_chain_.resetChain(); +} + +void KeyValueBlockchain::pruneOnSTLink(const categorization::Updates &updates) { + auto cat_it = updates.categoryUpdates().kv.find(categorization::kConcordInternalCategoryId); + if (cat_it == updates.categoryUpdates().kv.cend()) { + return; + } + const auto &internal_kvs = std::get(cat_it->second).kv; + auto key_it = internal_kvs.find(keyTypes::genesis_block_key); + if (key_it != internal_kvs.cend()) { + const auto block_genesis_id = concordUtils::fromBigEndianBuffer(key_it->second.data.data()); + while (getGenesisBlockId() >= INITIAL_GENESIS_BLOCK_ID && getGenesisBlockId() < getLastReachableBlockId() && + block_genesis_id > getGenesisBlockId()) { + deleteGenesisBlock(); + } + } +} + +// Atomic delete from state transfer and add to blockchain +void KeyValueBlockchain::writeSTLinkTransaction(const BlockId block_id, const categorization::Updates &updates) { + auto sequence_number = markHistoryForGarbageCollectionIfNeeded(updates); + auto write_batch = native_client_->getBatch(); + state_transfer_chain_.deleteBlock(block_id, write_batch); + auto new_block_id = add(updates, write_batch); + native_client_->write(std::move(write_batch)); + block_chain_.setBlockId(new_block_id); + pruneOnSTLink(updates); + if (sequence_number > 0) setLastBlockSequenceNumber(sequence_number); +} + +size_t KeyValueBlockchain::linkUntilBlockId(BlockId until_block_id) { + const auto from_block_id = getLastReachableBlockId() + 1; + ConcordAssertLE(from_block_id, until_block_id); + + static constexpr uint64_t report_thresh{1000}; + static uint64_t report_counter{}; + + concord::util::DurationTracker link_duration("link_duration", true); + BlockId last_added = 0; + for (auto i = from_block_id; i <= until_block_id; ++i) { + auto block = state_transfer_chain_.getBlock(i); + + if (!block) break; + last_added = i; + // First prune and then link the block to the chain. Rationale is that this will preserve the same order of block + // deletes relative to block adds on source and destination replicas. + auto updates = block->getUpdates(); + writeSTLinkTransaction(i, updates); + if ((++report_counter % report_thresh) == 0) { + auto elapsed_time_ms = link_duration.totalDuration(); + uint64_t blocks_linked_per_sec{}; + uint64_t blocks_left_to_link{}; + uint64_t estimated_time_left_sec{}; + if (elapsed_time_ms > 0) { + blocks_linked_per_sec = (((i - from_block_id + 1) * 1000) / (elapsed_time_ms)); + blocks_left_to_link = until_block_id - i; + estimated_time_left_sec = blocks_left_to_link / blocks_linked_per_sec; + } + LOG_INFO(CAT_BLOCK_LOG, + "Last block ID connected: " << i << "," + << KVLOG(from_block_id, + until_block_id, + elapsed_time_ms, + blocks_linked_per_sec, + blocks_left_to_link, + estimated_time_left_sec)); + } + } + if (last_added == state_transfer_chain_.getLastBlockId()) { + LOG_INFO(V4_BLOCK_LOG, "Added all blocks in st chain, until block " << last_added << " resetting chain"); + state_transfer_chain_.resetChain(); + } + return (last_added - from_block_id) + 1; +} + +std::optional KeyValueBlockchain::getValueFromUpdate( + BlockId block_id, const std::string &key, const categorization::BlockMerkleInput &category_input) const { + const auto valit = category_input.kv.find(key); + if (valit == category_input.kv.cend()) { + return std::nullopt; + } + return categorization::MerkleValue{{block_id, valit->second}}; +} +std::optional KeyValueBlockchain::getValueFromUpdate( + BlockId block_id, const std::string &key, const categorization::VersionedInput &category_input) const { + const auto valit = category_input.kv.find(key); + if (valit == category_input.kv.cend()) { + return std::nullopt; + } + return categorization::VersionedValue{{block_id, (valit->second).data}}; +} +std::optional KeyValueBlockchain::getValueFromUpdate( + BlockId block_id, const std::string &key, const categorization::ImmutableInput &category_input) const { + const auto valit = category_input.kv.find(key); + if (valit == category_input.kv.cend()) { + return std::nullopt; + } + return categorization::ImmutableValue{{block_id, (valit->second).data}}; +} + +std::optional KeyValueBlockchain::get(const std::string &category_id, + const std::string &key, + BlockId block_id) const { + LOG_DEBUG(V4_BLOCK_LOG, + "Reading key " << std::hash{}(key) << " block_id " << block_id << " category " << category_id + << " hex key " << ToHexad(key) << " key " << key); + + auto updates_in_block = block_chain_.getBlockUpdates(block_id); + if (!updates_in_block) { + LOG_DEBUG(V4_BLOCK_LOG, + "Failed getting updates key " << std::hash{}(key) << " block_id " << block_id << " category " + << category_id << " hex key " << ToHexad(key) << " key " << key); + return std::nullopt; + } + const auto &kv_updates = updates_in_block->categoryUpdates(category_id); + if (!kv_updates) { + LOG_DEBUG(V4_BLOCK_LOG, + "Failed getting categories updates key " << std::hash{}(key) << " block_id " << block_id + << " category " << category_id << " hex key " << ToHexad(key) + << " key " << key); + return std::nullopt; + } + std::optional ret; + std::visit( + [this, block_id, &key, &ret](const auto &specific_cat_updates) { + ret = this->getValueFromUpdate(block_id, key, specific_cat_updates); + }, + (*kv_updates).get()); + return ret; +} + +std::optional KeyValueBlockchain::getLatest(const std::string &category_id, + const std::string &key) const { + BlockId latest_block_id = block_chain_.getLastReachable(); + return latest_keys_.getValue(latest_block_id, category_id, detail::Blockchain::generateKey(latest_block_id), key); +} + +void KeyValueBlockchain::multiGet(const std::string &category_id, + const std::vector &keys, + const std::vector &versions, + std::vector> &values) const { + ConcordAssertEQ(keys.size(), versions.size()); + ConcordAssertEQ(keys.size(), versions.size()); + values.clear(); + values.reserve(keys.size()); + std::unordered_map> unique_block_updates; + block_chain_.multiGetBlockUpdates(versions, unique_block_updates); + for (size_t i = 0; i < keys.size(); ++i) { + const auto block_id = versions[i]; + const auto &key = keys[i]; + auto updates_in_block_it = unique_block_updates.find(block_id); + bool value_added_is_good = false; + if (updates_in_block_it != unique_block_updates.end()) { + if (updates_in_block_it->second) { + const auto &kv_updates = updates_in_block_it->second->categoryUpdates(category_id); + if (kv_updates) { + std::optional val; + std::visit( + [this, block_id, &key, &val](const auto &specific_cat_updates) { + val = getValueFromUpdate(block_id, key, specific_cat_updates); + }, + (*kv_updates).get()); + values.push_back(val); + value_added_is_good = true; + } + } + } + if (!value_added_is_good) { + values.push_back(std::nullopt); + } + } +} + +void KeyValueBlockchain::multiGetLatest(const std::string &category_id, + const std::vector &keys, + std::vector> &values) const { + BlockId latest_block_id = block_chain_.getLastReachable(); + return latest_keys_.multiGetValue( + latest_block_id, category_id, detail::Blockchain::generateKey(latest_block_id), keys, values); +} + +std::optional KeyValueBlockchain::getLatestVersion(const std::string &category_id, + const std::string &key) const { + return latest_keys_.getLatestVersion( + category_id, detail::Blockchain::generateKey(block_chain_.getLastReachable()), key); +} + +void KeyValueBlockchain::multiGetLatestVersion( + const std::string &category_id, + const std::vector &keys, + std::vector> &versions) const { + return latest_keys_.multiGetLatestVersion( + category_id, detail::Blockchain::generateKey(block_chain_.getLastReachable()), keys, versions); +} + +void KeyValueBlockchain::trimBlocksFromSnapshot(BlockId block_id_at_checkpoint) { + ConcordAssertNE(block_id_at_checkpoint, detail::Blockchain::INVALID_BLOCK_ID); + ConcordAssertLE(block_id_at_checkpoint, getLastReachableBlockId()); + while (block_id_at_checkpoint < getLastReachableBlockId()) { + LOG_INFO(V4_BLOCK_LOG, + "Deleting last reachable block = " << getLastReachableBlockId() + << ", DB checkpoint = " << native_client_->path()); + deleteLastReachableBlock(); + } +} + +} // namespace concord::kvbc::v4blockchain diff --git a/kvbc/test/CMakeLists.txt b/kvbc/test/CMakeLists.txt index c0a7a459aa..79e1c91995 100644 --- a/kvbc/test/CMakeLists.txt +++ b/kvbc/test/CMakeLists.txt @@ -220,6 +220,104 @@ if (BUILD_ROCKSDB_STORAGE) kvbc stdc++fs ) + + add_executable(v4_blockchain_handler_unit_test + v4blockchain/v4_blockchain_test.cpp ) + add_test(v4_blockchain_handler_unit_test v4_blockchain_handler_unit_test) + target_link_libraries(v4_blockchain_handler_unit_test PUBLIC + GTest::Main + GTest::GTest + util + corebft + kvbc + stdc++fs + ) + + add_executable(v4_blocks_unit_test + v4blockchain/blocks_test.cpp ) + add_test(v4_blocks_unit_test v4_blocks_unit_test) + target_link_libraries(v4_blocks_unit_test PUBLIC + GTest::Main + GTest::GTest + util + corebft + kvbc + stdc++fs + ) + + add_executable(v4_st_chain_unit_test + v4blockchain/st_chain_test.cpp ) + add_test(v4_st_chain_unit_test v4_st_chain_unit_test) + target_link_libraries(v4_st_chain_unit_test PUBLIC + GTest::Main + GTest::GTest + util + corebft + kvbc + stdc++fs + ) + + add_executable(v4_categories_unit_test + v4blockchain/categories_test.cpp ) + add_test(v4_categories_unit_test v4_categories_unit_test) + target_link_libraries(v4_categories_unit_test PUBLIC + GTest::Main + GTest::GTest + util + corebft + kvbc + stdc++fs + ) + + add_executable(v4_latest_keys_unit_test + v4blockchain/v4_latest_keys_test.cpp ) + add_test(v4_latest_keys_unit_test v4_latest_keys_unit_test) + target_link_libraries(v4_latest_keys_unit_test PUBLIC + GTest::Main + GTest::GTest + util + corebft + kvbc + stdc++fs + ) + + add_executable(v4_blockchain_unit_test + v4blockchain/blockchain_test.cpp ) + add_test(v4_blockchain_unit_test v4_blockchain_unit_test) + target_link_libraries(v4_blockchain_unit_test PUBLIC + GTest::Main + GTest::GTest + util + corebft + kvbc + stdc++fs + ) + + add_executable(app_state_adapter_test + kvbc_adapter/v4blockchain/app_state_adapter_test.cpp ) + add_test(app_state_adapter_test app_state_adapter_test) + target_link_libraries(app_state_adapter_test PUBLIC + GTest::Main + GTest::GTest + util + corebft + kvbc + stdc++fs + ) + + add_executable(common_state_snapshot_adapter_test + kvbc_adapter/common/state_snapshot_adapter_test.cpp ) + add_test(common_state_snapshot_adapter_test common_state_snapshot_adapter_test) + target_link_libraries(common_state_snapshot_adapter_test PUBLIC + GTest::Main + GTest::GTest + util + corebft + kvbc + stdc++fs + ) + + add_executable(pruning_test pruning_test.cpp) add_test(pruning_test pruning_test) target_link_libraries(pruning_test diff --git a/kvbc/test/categorization/blocks_test.cpp b/kvbc/test/categorization/blocks_test.cpp index 427193d623..5e8bc6c76c 100644 --- a/kvbc/test/categorization/blocks_test.cpp +++ b/kvbc/test/categorization/blocks_test.cpp @@ -20,7 +20,6 @@ #include "categorization/updates.h" #include "categorization/kv_blockchain.h" #include "categorization/db_categories.h" -#include #include #include #include diff --git a/kvbc/test/categorization/kv_blockchain_test.cpp b/kvbc/test/categorization/kv_blockchain_test.cpp index fb70d1de53..3fe516adab 100644 --- a/kvbc/test/categorization/kv_blockchain_test.cpp +++ b/kvbc/test/categorization/kv_blockchain_test.cpp @@ -13,6 +13,7 @@ #include "gtest/gtest.h" #include "gmock/gmock.h" + #include "categorization/column_families.h" #include "categorization/updates.h" #include "categorization/kv_blockchain.h" @@ -49,42 +50,6 @@ class categorized_kvbc : public Test { cleanup(); } - void addPublicState(KeyValueBlockchain& kvbc) { - auto updates = Updates{}; - auto merkle = BlockMerkleUpdates{}; - merkle.addUpdate("a", "va"); - merkle.addUpdate("b", "vb"); - merkle.addUpdate("c", "vc"); - merkle.addUpdate("d", "vd"); - auto versioned = VersionedUpdates{}; - const auto public_state = PublicStateKeys{std::vector{"a", "b", "c", "d"}}; - const auto ser_public_state = detail::serialize(public_state); - versioned.addUpdate(std::string{keyTypes::state_public_key_set}, - std::string{ser_public_state.cbegin(), ser_public_state.cend()}); - updates.add(kExecutionProvableCategory, std::move(merkle)); - updates.add(kConcordInternalCategoryId, std::move(versioned)); - ASSERT_EQ(kvbc.addBlock(std::move(updates)), 1); - } - - // Public state hash computed via https://emn178.github.io/online-tools/sha3_256.html - // - // h0 = hash("") = a7ffc6f8bf1ed76651c14756a061d662f580ff4de43b49fa82d80a4b80f8434a - // h1 = hash(h0 || hash("a") || "va") = c407fb7c52596d6d0fa0a013798dd72489dcc00607033089a8a09c73fba8bd7c - // h2 = hash(h1 || hash("b") || "vb") = 1d911b3b893221358c22f2470fac0864ef145b55eca983d4b17bdc30bf0bda2b - // h3 = hash(h2 || hash("c") || "vc") = c6314bdd9c82183d2e4e5cb8869826e1a3ace6a86a7b62ebe5ac77b732d3c792 - // h4 = hash(h3 || hash("d") || "vd") = fd4c5ea03da18deaf10365fdf001c216055aaaa796b0a98e4db7c756a426ae81 - void assertPublicStateHash() { - const auto state_hash_val = db->get(KeyValueBlockchain::publicStateHashKey()); - ASSERT_TRUE(state_hash_val.has_value()); - auto state_hash = StateHash{}; - detail::deserialize(*state_hash_val, state_hash); - ASSERT_EQ(state_hash.block_id, 1); - // Expect h4. - ASSERT_THAT(state_hash.hash, ContainerEq(Hash{0xfd, 0x4c, 0x5e, 0xa0, 0x3d, 0xa1, 0x8d, 0xea, 0xf1, 0x03, 0x65, - 0xfd, 0xf0, 0x01, 0xc2, 0x16, 0x05, 0x5a, 0xaa, 0xa7, 0x96, 0xb0, - 0xa9, 0x8e, 0x4d, 0xb7, 0xc7, 0x56, 0xa4, 0x26, 0xae, 0x81})); - } - protected: std::shared_ptr db; }; @@ -2447,150 +2412,6 @@ TEST_F(categorized_kvbc, trim_blocks_from_snapshot_called_with_bigger_block_id) ASSERT_DEATH(kvbc.trimBlocksFromSnapshot(4), ""); } -TEST_F(categorized_kvbc, compute_and_persist_hash_with_no_public_state) { - const auto link_st_chain = true; - auto kvbc = KeyValueBlockchain{ - db, - link_st_chain, - std::map{{kExecutionProvableCategory, CATEGORY_TYPE::block_merkle}, - {kConcordInternalCategoryId, CATEGORY_TYPE::versioned_kv}}}; - ASSERT_EQ(kvbc.addBlock(Updates{}), 1); - kvbc.computeAndPersistPublicStateHash(1); - const auto state_hash_val = db->get(KeyValueBlockchain::publicStateHashKey()); - ASSERT_TRUE(state_hash_val.has_value()); - auto state_hash = StateHash{}; - detail::deserialize(*state_hash_val, state_hash); - ASSERT_EQ(state_hash.block_id, 1); - // Expect the empty SHA3-256 hash. - ASSERT_THAT(state_hash.hash, ContainerEq(Hash{0xa7, 0xff, 0xc6, 0xf8, 0xbf, 0x1e, 0xd7, 0x66, 0x51, 0xc1, 0x47, - 0x56, 0xa0, 0x61, 0xd6, 0x62, 0xf5, 0x80, 0xff, 0x4d, 0xe4, 0x3b, - 0x49, 0xfa, 0x82, 0xd8, 0x0a, 0x4b, 0x80, 0xf8, 0x43, 0x4a})); -} - -TEST_F(categorized_kvbc, compute_and_persist_hash_batch_size_equals_key_count) { - const auto link_st_chain = true; - auto kvbc = KeyValueBlockchain{ - db, - link_st_chain, - std::map{{kExecutionProvableCategory, CATEGORY_TYPE::block_merkle}, - {kConcordInternalCategoryId, CATEGORY_TYPE::versioned_kv}}}; - addPublicState(kvbc); - bftEngine::ReplicaConfig::instance().stateIterationMultiGetBatchSize = 4; - kvbc.computeAndPersistPublicStateHash(1); - assertPublicStateHash(); -} - -TEST_F(categorized_kvbc, compute_and_persist_hash_batch_size_bigger_than_key_count_uneven) { - const auto link_st_chain = true; - auto kvbc = KeyValueBlockchain{ - db, - link_st_chain, - std::map{{kExecutionProvableCategory, CATEGORY_TYPE::block_merkle}, - {kConcordInternalCategoryId, CATEGORY_TYPE::versioned_kv}}}; - addPublicState(kvbc); - bftEngine::ReplicaConfig::instance().stateIterationMultiGetBatchSize = 5; - kvbc.computeAndPersistPublicStateHash(1); - assertPublicStateHash(); -} - -TEST_F(categorized_kvbc, compute_and_persist_hash_batch_size_bigger_than_key_count_even) { - const auto link_st_chain = true; - auto kvbc = KeyValueBlockchain{ - db, - link_st_chain, - std::map{{kExecutionProvableCategory, CATEGORY_TYPE::block_merkle}, - {kConcordInternalCategoryId, CATEGORY_TYPE::versioned_kv}}}; - addPublicState(kvbc); - bftEngine::ReplicaConfig::instance().stateIterationMultiGetBatchSize = 6; - kvbc.computeAndPersistPublicStateHash(1); - assertPublicStateHash(); -} - -TEST_F(categorized_kvbc, compute_and_persist_hash_batch_size_less_than_key_count_uneven) { - const auto link_st_chain = true; - auto kvbc = KeyValueBlockchain{ - db, - link_st_chain, - std::map{{kExecutionProvableCategory, CATEGORY_TYPE::block_merkle}, - {kConcordInternalCategoryId, CATEGORY_TYPE::versioned_kv}}}; - addPublicState(kvbc); - bftEngine::ReplicaConfig::instance().stateIterationMultiGetBatchSize = 3; - kvbc.computeAndPersistPublicStateHash(1); - assertPublicStateHash(); -} - -TEST_F(categorized_kvbc, compute_and_persist_hash_batch_size_less_than_key_count_even) { - const auto link_st_chain = true; - auto kvbc = KeyValueBlockchain{ - db, - link_st_chain, - std::map{{kExecutionProvableCategory, CATEGORY_TYPE::block_merkle}, - {kConcordInternalCategoryId, CATEGORY_TYPE::versioned_kv}}}; - addPublicState(kvbc); - bftEngine::ReplicaConfig::instance().stateIterationMultiGetBatchSize = 2; - kvbc.computeAndPersistPublicStateHash(1); - assertPublicStateHash(); -} - -TEST_F(categorized_kvbc, iterate_partial_public_state) { - const auto link_st_chain = true; - auto kvbc = KeyValueBlockchain{ - db, - link_st_chain, - std::map{{kExecutionProvableCategory, CATEGORY_TYPE::block_merkle}, - {kConcordInternalCategoryId, CATEGORY_TYPE::versioned_kv}}}; - addPublicState(kvbc); - auto iterated_key_values = std::vector>{}; - ASSERT_TRUE(kvbc.iteratePublicStateKeyValues( - [&](std::string&& key, std::string&& value) { iterated_key_values.push_back(std::make_pair(key, value)); }, "b")); - ASSERT_THAT(iterated_key_values, - ContainerEq(std::vector>{{"c", "vc"}, {"d", "vd"}})); -} - -TEST_F(categorized_kvbc, iterate_public_state_after_first_key) { - const auto link_st_chain = true; - auto kvbc = KeyValueBlockchain{ - db, - link_st_chain, - std::map{{kExecutionProvableCategory, CATEGORY_TYPE::block_merkle}, - {kConcordInternalCategoryId, CATEGORY_TYPE::versioned_kv}}}; - addPublicState(kvbc); - auto iterated_key_values = std::vector>{}; - ASSERT_TRUE(kvbc.iteratePublicStateKeyValues( - [&](std::string&& key, std::string&& value) { iterated_key_values.push_back(std::make_pair(key, value)); }, "a")); - ASSERT_THAT(iterated_key_values, - ContainerEq(std::vector>{{"b", "vb"}, {"c", "vc"}, {"d", "vd"}})); -} - -TEST_F(categorized_kvbc, iterate_public_state_after_last_key) { - const auto link_st_chain = true; - auto kvbc = KeyValueBlockchain{ - db, - link_st_chain, - std::map{{kExecutionProvableCategory, CATEGORY_TYPE::block_merkle}, - {kConcordInternalCategoryId, CATEGORY_TYPE::versioned_kv}}}; - addPublicState(kvbc); - auto iterated_key_values = std::vector>{}; - ASSERT_TRUE(kvbc.iteratePublicStateKeyValues( - [&](std::string&& key, std::string&& value) { iterated_key_values.push_back(std::make_pair(key, value)); }, "d")); - ASSERT_TRUE(iterated_key_values.empty()); -} - -TEST_F(categorized_kvbc, iterate_public_state_key_not_found) { - const auto link_st_chain = true; - auto kvbc = KeyValueBlockchain{ - db, - link_st_chain, - std::map{{kExecutionProvableCategory, CATEGORY_TYPE::block_merkle}, - {kConcordInternalCategoryId, CATEGORY_TYPE::versioned_kv}}}; - addPublicState(kvbc); - auto iterated_key_values = std::vector>{}; - ASSERT_FALSE(kvbc.iteratePublicStateKeyValues( - [&](std::string&& key, std::string&& value) { iterated_key_values.push_back(std::make_pair(key, value)); }, - "NON-EXISTENT")); - ASSERT_TRUE(iterated_key_values.empty()); -} - } // end namespace int main(int argc, char** argv) { diff --git a/kvbc/test/kvbc_adapter/common/state_snapshot_adapter_test.cpp b/kvbc/test/kvbc_adapter/common/state_snapshot_adapter_test.cpp new file mode 100644 index 0000000000..59ea320e71 --- /dev/null +++ b/kvbc/test/kvbc_adapter/common/state_snapshot_adapter_test.cpp @@ -0,0 +1,647 @@ +// Concord +// +// Copyright (c) 2020 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 subcomponent's license, as noted in the LICENSE +// file. + +#include "gtest/gtest.h" +#include "gmock/gmock.h" +#include "kvbc_adapter/replica_adapter.hpp" +#include "categorization/updates.h" +#include "categorization/db_categories.h" +#include "kvbc_key_types.hpp" +#include +#include +#include +#include +#include +#include "storage/test/storage_test_common.h" +#include "kvbc_adapter/common/state_snapshot_adapter.hpp" + +using concord::storage::rocksdb::NativeClient; +using namespace ::testing; + +namespace { +class common_kvbc : public Test { + protected: + void SetUp() override { + destroyDb(); + db = TestRocksDb::createNative(); + } + + void TearDown() override { destroyDb(); } + + void destroyDb() { + db.reset(); + ASSERT_EQ(0, db.use_count()); + cleanup(); + } + + void addPublicState(concord::kvbc::adapter::ReplicaBlockchain& kvbc) { + auto updates = concord::kvbc::categorization::Updates{}; + auto merkle = concord::kvbc::categorization::BlockMerkleUpdates{}; + merkle.addUpdate("a", "va"); + merkle.addUpdate("b", "vb"); + merkle.addUpdate("c", "vc"); + merkle.addUpdate("d", "vd"); + auto versioned = concord::kvbc::categorization::VersionedUpdates{}; + const auto public_state = + concord::kvbc::categorization::PublicStateKeys{std::vector{"a", "b", "c", "d"}}; + const auto ser_public_state = concord::kvbc::categorization::detail::serialize(public_state); + versioned.addUpdate(std::string{concord::kvbc::keyTypes::state_public_key_set}, + std::string{ser_public_state.cbegin(), ser_public_state.cend()}); + updates.add(concord::kvbc::categorization::kExecutionProvableCategory, std::move(merkle)); + updates.add(concord::kvbc::categorization::kConcordInternalCategoryId, std::move(versioned)); + ASSERT_EQ(kvbc.add(std::move(updates)), 1); + } + + // Public state hash computed via https://emn178.github.io/online-tools/sha3_256.html + // + // h0 = hash("") = a7ffc6f8bf1ed76651c14756a061d662f580ff4de43b49fa82d80a4b80f8434a + // h1 = hash(h0 || hash("a") || "va") = c407fb7c52596d6d0fa0a013798dd72489dcc00607033089a8a09c73fba8bd7c + // h2 = hash(h1 || hash("b") || "vb") = 1d911b3b893221358c22f2470fac0864ef145b55eca983d4b17bdc30bf0bda2b + // h3 = hash(h2 || hash("c") || "vc") = c6314bdd9c82183d2e4e5cb8869826e1a3ace6a86a7b62ebe5ac77b732d3c792 + // h4 = hash(h3 || hash("d") || "vd") = fd4c5ea03da18deaf10365fdf001c216055aaaa796b0a98e4db7c756a426ae81 + void assertPublicStateHash() { + const auto state_hash_val = db->get(concord::kvbc::bcutil::BlockChainUtils::publicStateHashKey()); + ASSERT_TRUE(state_hash_val.has_value()); + auto state_hash = concord::kvbc::categorization::StateHash{}; + concord::kvbc::categorization::detail::deserialize(*state_hash_val, state_hash); + ASSERT_EQ(state_hash.block_id, 1); + // Expect h4. + ASSERT_THAT(state_hash.hash, + ContainerEq(concord::kvbc::categorization::Hash{ + 0xfd, 0x4c, 0x5e, 0xa0, 0x3d, 0xa1, 0x8d, 0xea, 0xf1, 0x03, 0x65, 0xfd, 0xf0, 0x01, 0xc2, 0x16, + 0x05, 0x5a, 0xaa, 0xa7, 0x96, 0xb0, 0xa9, 0x8e, 0x4d, 0xb7, 0xc7, 0x56, 0xa4, 0x26, 0xae, 0x81})); + } + + std::optional getBlockchainVersion(int32_t ver) { + switch (ver) { + case 1: + return concord::kvbc::BLOCKCHAIN_VERSION::CATEGORIZED_BLOCKCHAIN; + case 4: + return concord::kvbc::BLOCKCHAIN_VERSION::NATURAL_BLOCKCHAIN; + default: + break; + } + return std::nullopt; + } + + protected: + std::shared_ptr db; +}; + +TEST_F(common_kvbc, compute_and_persist_hash_with_no_public_state) { + bool version_is_set = false; + std::map cat_map; + for (int32_t ver = 0; ver <= static_cast(concord::kvbc::BLOCKCHAIN_VERSION::INVALID_BLOCKCHAIN_VERSION); + ++ver) { + auto blockchain_version = getBlockchainVersion(ver); + if (!blockchain_version) { + continue; + } + switch (*blockchain_version) { + case concord::kvbc::BLOCKCHAIN_VERSION::CATEGORIZED_BLOCKCHAIN: + if (!version_is_set) { + bftEngine::ReplicaConfig::instance().kvBlockchainVersion = static_cast(ver); + version_is_set = true; + cat_map.emplace(concord::kvbc::categorization::kExecutionProvableCategory, + concord::kvbc::categorization::CATEGORY_TYPE::block_merkle); + cat_map.emplace(concord::kvbc::categorization::kConcordInternalCategoryId, + concord::kvbc::categorization::CATEGORY_TYPE::versioned_kv); + } + case concord::kvbc::BLOCKCHAIN_VERSION::NATURAL_BLOCKCHAIN: + if (!version_is_set) { + bftEngine::ReplicaConfig::instance().kvBlockchainVersion = static_cast(ver); + version_is_set = true; + cat_map.emplace("merkle", concord::kvbc::categorization::CATEGORY_TYPE::block_merkle); + cat_map.emplace("versioned", concord::kvbc::categorization::CATEGORY_TYPE::versioned_kv); + cat_map.emplace(concord::kvbc::categorization::kConcordInternalCategoryId, + concord::kvbc::categorization::CATEGORY_TYPE::versioned_kv); + } + { + const auto link_st_chain = false; + auto kvbc = concord::kvbc::adapter::ReplicaBlockchain{db, link_st_chain, cat_map}; + ASSERT_EQ(kvbc.add(concord::kvbc::categorization::Updates{}), 1); + kvbc.computeAndPersistPublicStateHash(1); + const auto state_hash_val = db->get(concord::kvbc::bcutil::BlockChainUtils::publicStateHashKey()); + ASSERT_TRUE(state_hash_val.has_value()); + auto state_hash = concord::kvbc::categorization::StateHash{}; + concord::kvbc::categorization::detail::deserialize(*state_hash_val, state_hash); + ASSERT_EQ(state_hash.block_id, 1); + // Expect the empty SHA3-256 hash. + ASSERT_THAT( + state_hash.hash, + ContainerEq(concord::kvbc::categorization::Hash{ + 0xa7, 0xff, 0xc6, 0xf8, 0xbf, 0x1e, 0xd7, 0x66, 0x51, 0xc1, 0x47, 0x56, 0xa0, 0x61, 0xd6, 0x62, + 0xf5, 0x80, 0xff, 0x4d, 0xe4, 0x3b, 0x49, 0xfa, 0x82, 0xd8, 0x0a, 0x4b, 0x80, 0xf8, 0x43, 0x4a})); + } + version_is_set = false; + break; + case concord::kvbc::BLOCKCHAIN_VERSION::INVALID_BLOCKCHAIN_VERSION: + version_is_set = false; + break; + } + } +} + +TEST_F(common_kvbc, compute_and_persist_hash_batch_size_equals_key_count) { + bool version_is_set = false; + std::map cat_map; + for (int32_t ver = 0; ver <= static_cast(concord::kvbc::BLOCKCHAIN_VERSION::INVALID_BLOCKCHAIN_VERSION); + ++ver) { + auto blockchain_version = getBlockchainVersion(ver); + if (!blockchain_version) { + continue; + } + switch (*blockchain_version) { + case concord::kvbc::BLOCKCHAIN_VERSION::CATEGORIZED_BLOCKCHAIN: + if (!version_is_set) { + bftEngine::ReplicaConfig::instance().kvBlockchainVersion = static_cast(ver); + version_is_set = true; + cat_map.emplace(concord::kvbc::categorization::kExecutionProvableCategory, + concord::kvbc::categorization::CATEGORY_TYPE::block_merkle); + cat_map.emplace(concord::kvbc::categorization::kConcordInternalCategoryId, + concord::kvbc::categorization::CATEGORY_TYPE::versioned_kv); + } + case concord::kvbc::BLOCKCHAIN_VERSION::NATURAL_BLOCKCHAIN: + if (!version_is_set) { + bftEngine::ReplicaConfig::instance().kvBlockchainVersion = static_cast(ver); + version_is_set = true; + cat_map.emplace("merkle", concord::kvbc::categorization::CATEGORY_TYPE::block_merkle); + cat_map.emplace("versioned", concord::kvbc::categorization::CATEGORY_TYPE::versioned_kv); + cat_map.emplace(concord::kvbc::categorization::kConcordInternalCategoryId, + concord::kvbc::categorization::CATEGORY_TYPE::versioned_kv); + } + { + const auto link_st_chain = true; + auto kvbc = concord::kvbc::adapter::ReplicaBlockchain{ + db, + link_st_chain, + std::map{ + {concord::kvbc::categorization::kExecutionProvableCategory, + concord::kvbc::categorization::CATEGORY_TYPE::block_merkle}, + {concord::kvbc::categorization::kConcordInternalCategoryId, + concord::kvbc::categorization::CATEGORY_TYPE::versioned_kv}}}; + addPublicState(kvbc); + bftEngine::ReplicaConfig::instance().stateIterationMultiGetBatchSize = 4; + kvbc.computeAndPersistPublicStateHash(1); + assertPublicStateHash(); + } + version_is_set = false; + break; + case concord::kvbc::BLOCKCHAIN_VERSION::INVALID_BLOCKCHAIN_VERSION: + version_is_set = false; + break; + } + } +} + +TEST_F(common_kvbc, compute_and_persist_hash_batch_size_bigger_than_key_count_uneven) { + bool version_is_set = false; + std::map cat_map; + for (int32_t ver = 0; ver <= static_cast(concord::kvbc::BLOCKCHAIN_VERSION::INVALID_BLOCKCHAIN_VERSION); + ++ver) { + auto blockchain_version = getBlockchainVersion(ver); + if (!blockchain_version) { + continue; + } + switch (*blockchain_version) { + case concord::kvbc::BLOCKCHAIN_VERSION::CATEGORIZED_BLOCKCHAIN: + if (!version_is_set) { + bftEngine::ReplicaConfig::instance().kvBlockchainVersion = static_cast(ver); + version_is_set = true; + cat_map.emplace(concord::kvbc::categorization::kExecutionProvableCategory, + concord::kvbc::categorization::CATEGORY_TYPE::block_merkle); + cat_map.emplace(concord::kvbc::categorization::kConcordInternalCategoryId, + concord::kvbc::categorization::CATEGORY_TYPE::versioned_kv); + } + case concord::kvbc::BLOCKCHAIN_VERSION::NATURAL_BLOCKCHAIN: + if (!version_is_set) { + bftEngine::ReplicaConfig::instance().kvBlockchainVersion = static_cast(ver); + version_is_set = true; + cat_map.emplace("merkle", concord::kvbc::categorization::CATEGORY_TYPE::block_merkle); + cat_map.emplace("versioned", concord::kvbc::categorization::CATEGORY_TYPE::versioned_kv); + cat_map.emplace(concord::kvbc::categorization::kConcordInternalCategoryId, + concord::kvbc::categorization::CATEGORY_TYPE::versioned_kv); + } + { + const auto link_st_chain = true; + auto kvbc = concord::kvbc::adapter::ReplicaBlockchain{ + db, + link_st_chain, + std::map{ + {concord::kvbc::categorization::kExecutionProvableCategory, + concord::kvbc::categorization::CATEGORY_TYPE::block_merkle}, + {concord::kvbc::categorization::kConcordInternalCategoryId, + concord::kvbc::categorization::CATEGORY_TYPE::versioned_kv}}}; + addPublicState(kvbc); + bftEngine::ReplicaConfig::instance().stateIterationMultiGetBatchSize = 5; + kvbc.computeAndPersistPublicStateHash(1); + assertPublicStateHash(); + } + version_is_set = false; + break; + case concord::kvbc::BLOCKCHAIN_VERSION::INVALID_BLOCKCHAIN_VERSION: + version_is_set = false; + break; + } + } +} + +TEST_F(common_kvbc, compute_and_persist_hash_batch_size_bigger_than_key_count_even) { + bool version_is_set = false; + std::map cat_map; + for (int32_t ver = 0; ver <= static_cast(concord::kvbc::BLOCKCHAIN_VERSION::INVALID_BLOCKCHAIN_VERSION); + ++ver) { + auto blockchain_version = getBlockchainVersion(ver); + if (!blockchain_version) { + continue; + } + switch (*blockchain_version) { + case concord::kvbc::BLOCKCHAIN_VERSION::CATEGORIZED_BLOCKCHAIN: + if (!version_is_set) { + bftEngine::ReplicaConfig::instance().kvBlockchainVersion = static_cast(ver); + version_is_set = true; + cat_map.emplace(concord::kvbc::categorization::kExecutionProvableCategory, + concord::kvbc::categorization::CATEGORY_TYPE::block_merkle); + cat_map.emplace(concord::kvbc::categorization::kConcordInternalCategoryId, + concord::kvbc::categorization::CATEGORY_TYPE::versioned_kv); + } + case concord::kvbc::BLOCKCHAIN_VERSION::NATURAL_BLOCKCHAIN: + if (!version_is_set) { + bftEngine::ReplicaConfig::instance().kvBlockchainVersion = static_cast(ver); + version_is_set = true; + cat_map.emplace("merkle", concord::kvbc::categorization::CATEGORY_TYPE::block_merkle); + cat_map.emplace("versioned", concord::kvbc::categorization::CATEGORY_TYPE::versioned_kv); + cat_map.emplace(concord::kvbc::categorization::kConcordInternalCategoryId, + concord::kvbc::categorization::CATEGORY_TYPE::versioned_kv); + } + { + const auto link_st_chain = true; + auto kvbc = concord::kvbc::adapter::ReplicaBlockchain{ + db, + link_st_chain, + std::map{ + {concord::kvbc::categorization::kExecutionProvableCategory, + concord::kvbc::categorization::CATEGORY_TYPE::block_merkle}, + {concord::kvbc::categorization::kConcordInternalCategoryId, + concord::kvbc::categorization::CATEGORY_TYPE::versioned_kv}}}; + addPublicState(kvbc); + bftEngine::ReplicaConfig::instance().stateIterationMultiGetBatchSize = 6; + kvbc.computeAndPersistPublicStateHash(1); + assertPublicStateHash(); + } + version_is_set = false; + break; + case concord::kvbc::BLOCKCHAIN_VERSION::INVALID_BLOCKCHAIN_VERSION: + version_is_set = false; + break; + } + } +} + +TEST_F(common_kvbc, compute_and_persist_hash_batch_size_less_than_key_count_uneven) { + bool version_is_set = false; + std::map cat_map; + for (int32_t ver = 0; ver <= static_cast(concord::kvbc::BLOCKCHAIN_VERSION::INVALID_BLOCKCHAIN_VERSION); + ++ver) { + auto blockchain_version = getBlockchainVersion(ver); + if (!blockchain_version) { + continue; + } + switch (*blockchain_version) { + case concord::kvbc::BLOCKCHAIN_VERSION::CATEGORIZED_BLOCKCHAIN: + if (!version_is_set) { + bftEngine::ReplicaConfig::instance().kvBlockchainVersion = static_cast(ver); + version_is_set = true; + cat_map.emplace(concord::kvbc::categorization::kExecutionProvableCategory, + concord::kvbc::categorization::CATEGORY_TYPE::block_merkle); + cat_map.emplace(concord::kvbc::categorization::kConcordInternalCategoryId, + concord::kvbc::categorization::CATEGORY_TYPE::versioned_kv); + } + case concord::kvbc::BLOCKCHAIN_VERSION::NATURAL_BLOCKCHAIN: + if (!version_is_set) { + bftEngine::ReplicaConfig::instance().kvBlockchainVersion = static_cast(ver); + version_is_set = true; + cat_map.emplace("merkle", concord::kvbc::categorization::CATEGORY_TYPE::block_merkle); + cat_map.emplace("versioned", concord::kvbc::categorization::CATEGORY_TYPE::versioned_kv); + cat_map.emplace(concord::kvbc::categorization::kConcordInternalCategoryId, + concord::kvbc::categorization::CATEGORY_TYPE::versioned_kv); + } + { + const auto link_st_chain = true; + auto kvbc = concord::kvbc::adapter::ReplicaBlockchain{ + db, + link_st_chain, + std::map{ + {concord::kvbc::categorization::kExecutionProvableCategory, + concord::kvbc::categorization::CATEGORY_TYPE::block_merkle}, + {concord::kvbc::categorization::kConcordInternalCategoryId, + concord::kvbc::categorization::CATEGORY_TYPE::versioned_kv}}}; + addPublicState(kvbc); + bftEngine::ReplicaConfig::instance().stateIterationMultiGetBatchSize = 3; + kvbc.computeAndPersistPublicStateHash(1); + assertPublicStateHash(); + } + version_is_set = false; + break; + case concord::kvbc::BLOCKCHAIN_VERSION::INVALID_BLOCKCHAIN_VERSION: + version_is_set = false; + break; + } + } +} + +TEST_F(common_kvbc, compute_and_persist_hash_batch_size_less_than_key_count_even) { + bool version_is_set = false; + std::map cat_map; + for (int32_t ver = 0; ver <= static_cast(concord::kvbc::BLOCKCHAIN_VERSION::INVALID_BLOCKCHAIN_VERSION); + ++ver) { + auto blockchain_version = getBlockchainVersion(ver); + if (!blockchain_version) { + continue; + } + switch (*blockchain_version) { + case concord::kvbc::BLOCKCHAIN_VERSION::CATEGORIZED_BLOCKCHAIN: + if (!version_is_set) { + bftEngine::ReplicaConfig::instance().kvBlockchainVersion = static_cast(ver); + version_is_set = true; + cat_map.emplace(concord::kvbc::categorization::kExecutionProvableCategory, + concord::kvbc::categorization::CATEGORY_TYPE::block_merkle); + cat_map.emplace(concord::kvbc::categorization::kConcordInternalCategoryId, + concord::kvbc::categorization::CATEGORY_TYPE::versioned_kv); + } + case concord::kvbc::BLOCKCHAIN_VERSION::NATURAL_BLOCKCHAIN: + if (!version_is_set) { + bftEngine::ReplicaConfig::instance().kvBlockchainVersion = static_cast(ver); + version_is_set = true; + cat_map.emplace("merkle", concord::kvbc::categorization::CATEGORY_TYPE::block_merkle); + cat_map.emplace("versioned", concord::kvbc::categorization::CATEGORY_TYPE::versioned_kv); + cat_map.emplace(concord::kvbc::categorization::kConcordInternalCategoryId, + concord::kvbc::categorization::CATEGORY_TYPE::versioned_kv); + } + { + const auto link_st_chain = true; + auto kvbc = concord::kvbc::adapter::ReplicaBlockchain{ + db, + link_st_chain, + std::map{ + {concord::kvbc::categorization::kExecutionProvableCategory, + concord::kvbc::categorization::CATEGORY_TYPE::block_merkle}, + {concord::kvbc::categorization::kConcordInternalCategoryId, + concord::kvbc::categorization::CATEGORY_TYPE::versioned_kv}}}; + addPublicState(kvbc); + bftEngine::ReplicaConfig::instance().stateIterationMultiGetBatchSize = 2; + kvbc.computeAndPersistPublicStateHash(1); + assertPublicStateHash(); + } + version_is_set = false; + break; + case concord::kvbc::BLOCKCHAIN_VERSION::INVALID_BLOCKCHAIN_VERSION: + version_is_set = false; + break; + } + } +} + +TEST_F(common_kvbc, iterate_partial_public_state) { + bool version_is_set = false; + std::map cat_map; + for (int32_t ver = 0; ver <= static_cast(concord::kvbc::BLOCKCHAIN_VERSION::INVALID_BLOCKCHAIN_VERSION); + ++ver) { + auto blockchain_version = getBlockchainVersion(ver); + if (!blockchain_version) { + continue; + } + switch (*blockchain_version) { + case concord::kvbc::BLOCKCHAIN_VERSION::CATEGORIZED_BLOCKCHAIN: + if (!version_is_set) { + bftEngine::ReplicaConfig::instance().kvBlockchainVersion = static_cast(ver); + version_is_set = true; + cat_map.emplace(concord::kvbc::categorization::kExecutionProvableCategory, + concord::kvbc::categorization::CATEGORY_TYPE::block_merkle); + cat_map.emplace(concord::kvbc::categorization::kConcordInternalCategoryId, + concord::kvbc::categorization::CATEGORY_TYPE::versioned_kv); + } + case concord::kvbc::BLOCKCHAIN_VERSION::NATURAL_BLOCKCHAIN: + if (!version_is_set) { + bftEngine::ReplicaConfig::instance().kvBlockchainVersion = static_cast(ver); + version_is_set = true; + cat_map.emplace("merkle", concord::kvbc::categorization::CATEGORY_TYPE::block_merkle); + cat_map.emplace("versioned", concord::kvbc::categorization::CATEGORY_TYPE::versioned_kv); + cat_map.emplace(concord::kvbc::categorization::kConcordInternalCategoryId, + concord::kvbc::categorization::CATEGORY_TYPE::versioned_kv); + } + { + const auto link_st_chain = true; + auto kvbc = concord::kvbc::adapter::ReplicaBlockchain{ + db, + link_st_chain, + std::map{ + {concord::kvbc::categorization::kExecutionProvableCategory, + concord::kvbc::categorization::CATEGORY_TYPE::block_merkle}, + {concord::kvbc::categorization::kConcordInternalCategoryId, + concord::kvbc::categorization::CATEGORY_TYPE::versioned_kv}}}; + addPublicState(kvbc); + auto iterated_key_values = std::vector>{}; + ASSERT_TRUE(kvbc.iteratePublicStateKeyValues( + [&](std::string&& key, std::string&& value) { + iterated_key_values.push_back(std::make_pair(key, value)); + }, + "b")); + ASSERT_THAT(iterated_key_values, + ContainerEq(std::vector>{{"c", "vc"}, {"d", "vd"}})); + } + version_is_set = false; + break; + case concord::kvbc::BLOCKCHAIN_VERSION::INVALID_BLOCKCHAIN_VERSION: + version_is_set = false; + break; + } + } +} + +TEST_F(common_kvbc, iterate_public_state_after_first_key) { + bool version_is_set = false; + std::map cat_map; + for (int32_t ver = 0; ver <= static_cast(concord::kvbc::BLOCKCHAIN_VERSION::INVALID_BLOCKCHAIN_VERSION); + ++ver) { + auto blockchain_version = getBlockchainVersion(ver); + if (!blockchain_version) { + continue; + } + switch (*blockchain_version) { + case concord::kvbc::BLOCKCHAIN_VERSION::CATEGORIZED_BLOCKCHAIN: + if (!version_is_set) { + bftEngine::ReplicaConfig::instance().kvBlockchainVersion = static_cast(ver); + version_is_set = true; + cat_map.emplace(concord::kvbc::categorization::kExecutionProvableCategory, + concord::kvbc::categorization::CATEGORY_TYPE::block_merkle); + cat_map.emplace(concord::kvbc::categorization::kConcordInternalCategoryId, + concord::kvbc::categorization::CATEGORY_TYPE::versioned_kv); + } + case concord::kvbc::BLOCKCHAIN_VERSION::NATURAL_BLOCKCHAIN: + if (!version_is_set) { + bftEngine::ReplicaConfig::instance().kvBlockchainVersion = static_cast(ver); + version_is_set = true; + cat_map.emplace("merkle", concord::kvbc::categorization::CATEGORY_TYPE::block_merkle); + cat_map.emplace("versioned", concord::kvbc::categorization::CATEGORY_TYPE::versioned_kv); + cat_map.emplace(concord::kvbc::categorization::kConcordInternalCategoryId, + concord::kvbc::categorization::CATEGORY_TYPE::versioned_kv); + } + { + const auto link_st_chain = true; + auto kvbc = concord::kvbc::adapter::ReplicaBlockchain{ + db, + link_st_chain, + std::map{ + {concord::kvbc::categorization::kExecutionProvableCategory, + concord::kvbc::categorization::CATEGORY_TYPE::block_merkle}, + {concord::kvbc::categorization::kConcordInternalCategoryId, + concord::kvbc::categorization::CATEGORY_TYPE::versioned_kv}}}; + addPublicState(kvbc); + auto iterated_key_values = std::vector>{}; + ASSERT_TRUE(kvbc.iteratePublicStateKeyValues( + [&](std::string&& key, std::string&& value) { + iterated_key_values.push_back(std::make_pair(key, value)); + }, + "a")); + ASSERT_THAT( + iterated_key_values, + ContainerEq(std::vector>{{"b", "vb"}, {"c", "vc"}, {"d", "vd"}})); + } + version_is_set = false; + break; + case concord::kvbc::BLOCKCHAIN_VERSION::INVALID_BLOCKCHAIN_VERSION: + version_is_set = false; + break; + } + } +} + +TEST_F(common_kvbc, iterate_public_state_after_last_key) { + bool version_is_set = false; + std::map cat_map; + for (int32_t ver = 0; ver <= static_cast(concord::kvbc::BLOCKCHAIN_VERSION::INVALID_BLOCKCHAIN_VERSION); + ++ver) { + auto blockchain_version = getBlockchainVersion(ver); + if (!blockchain_version) { + continue; + } + switch (*blockchain_version) { + case concord::kvbc::BLOCKCHAIN_VERSION::CATEGORIZED_BLOCKCHAIN: + if (!version_is_set) { + bftEngine::ReplicaConfig::instance().kvBlockchainVersion = static_cast(ver); + version_is_set = true; + cat_map.emplace(concord::kvbc::categorization::kExecutionProvableCategory, + concord::kvbc::categorization::CATEGORY_TYPE::block_merkle); + cat_map.emplace(concord::kvbc::categorization::kConcordInternalCategoryId, + concord::kvbc::categorization::CATEGORY_TYPE::versioned_kv); + } + case concord::kvbc::BLOCKCHAIN_VERSION::NATURAL_BLOCKCHAIN: + if (!version_is_set) { + bftEngine::ReplicaConfig::instance().kvBlockchainVersion = static_cast(ver); + version_is_set = true; + cat_map.emplace("merkle", concord::kvbc::categorization::CATEGORY_TYPE::block_merkle); + cat_map.emplace("versioned", concord::kvbc::categorization::CATEGORY_TYPE::versioned_kv); + cat_map.emplace(concord::kvbc::categorization::kConcordInternalCategoryId, + concord::kvbc::categorization::CATEGORY_TYPE::versioned_kv); + } + { + const auto link_st_chain = true; + auto kvbc = concord::kvbc::adapter::ReplicaBlockchain{ + db, + link_st_chain, + std::map{ + {concord::kvbc::categorization::kExecutionProvableCategory, + concord::kvbc::categorization::CATEGORY_TYPE::block_merkle}, + {concord::kvbc::categorization::kConcordInternalCategoryId, + concord::kvbc::categorization::CATEGORY_TYPE::versioned_kv}}}; + addPublicState(kvbc); + auto iterated_key_values = std::vector>{}; + ASSERT_TRUE(kvbc.iteratePublicStateKeyValues( + [&](std::string&& key, std::string&& value) { + iterated_key_values.push_back(std::make_pair(key, value)); + }, + "d")); + ASSERT_TRUE(iterated_key_values.empty()); + } + version_is_set = false; + break; + case concord::kvbc::BLOCKCHAIN_VERSION::INVALID_BLOCKCHAIN_VERSION: + version_is_set = false; + break; + } + } +} + +TEST_F(common_kvbc, iterate_public_state_key_not_found) { + bool version_is_set = false; + std::map cat_map; + for (int32_t ver = 0; ver <= static_cast(concord::kvbc::BLOCKCHAIN_VERSION::INVALID_BLOCKCHAIN_VERSION); + ++ver) { + auto blockchain_version = getBlockchainVersion(ver); + if (!blockchain_version) { + continue; + } + switch (*blockchain_version) { + case concord::kvbc::BLOCKCHAIN_VERSION::CATEGORIZED_BLOCKCHAIN: + if (!version_is_set) { + bftEngine::ReplicaConfig::instance().kvBlockchainVersion = static_cast(ver); + version_is_set = true; + cat_map.emplace(concord::kvbc::categorization::kExecutionProvableCategory, + concord::kvbc::categorization::CATEGORY_TYPE::block_merkle); + cat_map.emplace(concord::kvbc::categorization::kConcordInternalCategoryId, + concord::kvbc::categorization::CATEGORY_TYPE::versioned_kv); + } + case concord::kvbc::BLOCKCHAIN_VERSION::NATURAL_BLOCKCHAIN: + if (!version_is_set) { + bftEngine::ReplicaConfig::instance().kvBlockchainVersion = static_cast(ver); + version_is_set = true; + cat_map.emplace("merkle", concord::kvbc::categorization::CATEGORY_TYPE::block_merkle); + cat_map.emplace("versioned", concord::kvbc::categorization::CATEGORY_TYPE::versioned_kv); + cat_map.emplace(concord::kvbc::categorization::kConcordInternalCategoryId, + concord::kvbc::categorization::CATEGORY_TYPE::versioned_kv); + } + { + const auto link_st_chain = true; + auto kvbc = concord::kvbc::adapter::ReplicaBlockchain{ + db, + link_st_chain, + std::map{ + {concord::kvbc::categorization::kExecutionProvableCategory, + concord::kvbc::categorization::CATEGORY_TYPE::block_merkle}, + {concord::kvbc::categorization::kConcordInternalCategoryId, + concord::kvbc::categorization::CATEGORY_TYPE::versioned_kv}}}; + addPublicState(kvbc); + auto iterated_key_values = std::vector>{}; + ASSERT_FALSE(kvbc.iteratePublicStateKeyValues( + [&](std::string&& key, std::string&& value) { + iterated_key_values.push_back(std::make_pair(key, value)); + }, + "NON-EXISTENT")); + ASSERT_TRUE(iterated_key_values.empty()); + } + version_is_set = false; + break; + case concord::kvbc::BLOCKCHAIN_VERSION::INVALID_BLOCKCHAIN_VERSION: + version_is_set = false; + break; + } + } +} + +} // namespace + +int main(int argc, char** argv) { + InitGoogleTest(&argc, argv); + return RUN_ALL_TESTS(); +} \ No newline at end of file diff --git a/kvbc/test/kvbc_adapter/v4blockchain/app_state_adapter_test.cpp b/kvbc/test/kvbc_adapter/v4blockchain/app_state_adapter_test.cpp new file mode 100644 index 0000000000..24e624f08a --- /dev/null +++ b/kvbc/test/kvbc_adapter/v4blockchain/app_state_adapter_test.cpp @@ -0,0 +1,116 @@ +// Concord +// +// Copyright (c) 2020 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 subcomponent's license, as noted in the LICENSE +// file. + +#include "gtest/gtest.h" +#include "gmock/gmock.h" +#include "v4blockchain/v4_blockchain.h" +#include +#include +#include +#include +#include +#include "storage/test/storage_test_common.h" +#include "endianness.hpp" +#include "v4blockchain/detail/st_chain.h" +#include "kvbc_adapter/v4blockchain/app_state_adapter.hpp" +#include "v4blockchain/v4_blockchain.h" +#include +#include "db_adapter_interface.h" + +using concord::storage::rocksdb::NativeClient; +using namespace concord::kvbc; +using namespace ::testing; +using namespace concord; + +namespace { + +class v4_kvbc : public Test { + protected: + void SetUp() override { + destroyDb(); + db = TestRocksDb::createNative(); + categories = std::map{ + {"merkle", categorization::CATEGORY_TYPE::block_merkle}, + {"versioned", categorization::CATEGORY_TYPE::versioned_kv}, + {"versioned_2", categorization::CATEGORY_TYPE::versioned_kv}, + {"immutable", categorization::CATEGORY_TYPE::immutable}}; + bc = std::make_shared(db, true, categories); + } + + void TearDown() override { destroyDb(); } + + void destroyDb() { + db.reset(); + ASSERT_EQ(0, db.use_count()); + cleanup(); + } + + protected: + std::map categories; + std::shared_ptr db; + std::shared_ptr bc; +}; + +TEST_F(v4_kvbc, put_get_blocks) { + auto state_adapter = concord::kvbc::adapter::v4blockchain::AppStateAdapter{bc}; + char buffer[1000]; + uint32_t out = 0; + ASSERT_EQ(state_adapter.getLastBlockNum(), 0); + // Add block to blockchain and validate get + ASSERT_FALSE(state_adapter.hasBlock(1)); + categorization::Updates updates; + categorization::BlockMerkleUpdates merkle_updates; + merkle_updates.addUpdate("merkle_key1", "merkle_value1"); + merkle_updates.addUpdate("merkle_key2", "merkle_value2"); + updates.add("merkle", std::move(merkle_updates)); + + categorization::VersionedUpdates ver_updates; + ver_updates.calculateRootHash(true); + ver_updates.addUpdate("ver_key1", "ver_val1"); + ver_updates.addUpdate("ver_key2", categorization::VersionedUpdates::Value{"ver_val2", true}); + updates.add("versioned", std::move(ver_updates)); + auto updates_copy = updates; + ASSERT_EQ(bc->add(std::move(updates)), (BlockId)1); + ASSERT_TRUE(state_adapter.hasBlock(1)); + ASSERT_EQ(state_adapter.getLastBlockNum(), 1); + + ASSERT_TRUE(state_adapter.getBlock(1, buffer, 1000, &out)); + auto strblock1 = std::string(buffer, out); + auto block1 = ::v4blockchain::detail::Block{strblock1}; + auto getUpdates = block1.getUpdates(); + ASSERT_EQ(getUpdates, updates_copy); + + // Add block to ST chain + auto block = std::string("block"); + auto blockDiff = std::string("blockDiff"); + state_adapter.putBlock(8, block.c_str(), block.size(), false); + ASSERT_TRUE(state_adapter.hasBlock(8)); + ASSERT_EQ(state_adapter.getLastBlockNum(), 8); + + // try to add existing st block with different content + ASSERT_THROW(state_adapter.putBlock(8, blockDiff.c_str(), blockDiff.size(), false), std::runtime_error); + // not found + ASSERT_THROW(state_adapter.getBlock(2, buffer, 1000, &out), kvbc::NotFoundException); + // buffer too small + ASSERT_THROW(state_adapter.getBlock(8, buffer, 1, &out), std::runtime_error); + ASSERT_TRUE(state_adapter.getBlock(8, buffer, 1000, &out)); + auto getBlock = std::string(buffer, out); + ASSERT_EQ(getBlock, block); +} + +} // namespace + +int main(int argc, char** argv) { + InitGoogleTest(&argc, argv); + return RUN_ALL_TESTS(); +} diff --git a/kvbc/test/pruning_test.cpp b/kvbc/test/pruning_test.cpp index c7129d2746..9edd3de7db 100644 --- a/kvbc/test/pruning_test.cpp +++ b/kvbc/test/pruning_test.cpp @@ -12,7 +12,6 @@ #include "gtest/gtest.h" #include -#include #include "NullStateTransfer.hpp" #include "categorization/kv_blockchain.h" #include "categorization/updates.h" @@ -37,6 +36,7 @@ using concord::kvbc::BlockId; using namespace concord::kvbc; using namespace concord::kvbc::categorization; using namespace concord::kvbc::pruning; + namespace { const NodeIdType replica_0 = 0; const NodeIdType replica_1 = 1; @@ -441,10 +441,12 @@ class TestStorage : public IReader, public IBlockAdder, public IBlocksDeleter { return lastDeletedBlock; } + void deleteLastReachableBlock() override { return bc_.deleteLastReachableBlock(); } + void setGenesisBlockId(BlockId bid) { mockGenesisBlockId = bid; } private: - KeyValueBlockchain bc_; + concord::kvbc::categorization::KeyValueBlockchain bc_; std::optional mockGenesisBlockId = {}; }; diff --git a/kvbc/test/replica_state_sync_test.cpp b/kvbc/test/replica_state_sync_test.cpp index ada8b92c6b..deb2f523ba 100644 --- a/kvbc/test/replica_state_sync_test.cpp +++ b/kvbc/test/replica_state_sync_test.cpp @@ -15,7 +15,7 @@ #include "DbMetadataStorage.hpp" #include "Logger.hpp" #include "block_metadata.hpp" -#include "categorization/kv_blockchain.h" +#include "kvbc_adapter/replica_adapter.hpp" #include "db_interfaces.h" #include "metadata_block_id.h" #include "PersistentStorageImp.hpp" @@ -23,6 +23,7 @@ #include "rocksdb/native_client.h" #include "storage/merkle_tree_key_manipulator.h" #include "storage/test/storage_test_common.h" +#include "ReplicaResources.h" #include #include @@ -39,7 +40,7 @@ using concord::kvbc::IReader; using concord::kvbc::categorization::kConcordInternalCategoryId; using concord::kvbc::ReplicaStateSyncImp; using concord::kvbc::categorization::CATEGORY_TYPE; -using concord::kvbc::categorization::KeyValueBlockchain; +using concord::kvbc::adapter::ReplicaBlockchain; using concord::kvbc::categorization::TaggedVersion; using concord::kvbc::categorization::Updates; using concord::kvbc::categorization::Value; @@ -130,10 +131,10 @@ class replica_state_sync_test : public Test, public IReader { auto ver_updates = VersionedUpdates{}; ver_updates.addUpdate(std::string{BlockMetadata::kBlockMetadataKeyStr}, block_metadata_.serialize(seq_number)); updates.add(kConcordInternalCategoryId, std::move(ver_updates)); - blockchain_->addBlock(std::move(updates)); + blockchain_->add(std::move(updates)); } - void addBlockWithoutBftSeqNum() { blockchain_->addBlock(Updates{}); } + void addBlockWithoutBftSeqNum() { blockchain_->add(Updates{}); } void persistLastBlockIdInMetadata() { constexpr auto in_transaction = false; @@ -151,7 +152,7 @@ class replica_state_sync_test : public Test, public IReader { // Test case specific members that are reset on SetUp() and TearDown(). std::shared_ptr db_; - std::optional blockchain_; + std::optional blockchain_; std::shared_ptr metadata_; }; @@ -175,7 +176,7 @@ TEST_F(replica_state_sync_on_bft_seq_num_test, non_empty_blockchain_and_0_bft_se replica_state_sync_.executeBasedOnBftSeqNum( logger_, *blockchain_, last_executed_bft_seq_num, kMaxNumOfBlocksToDelete)); ASSERT_EQ(1, blockchain_->getGenesisBlockId()); - ASSERT_EQ(2, blockchain_->getLastReachableBlockId()); + ASSERT_EQ(2, blockchain_->getLastBlockId()); } TEST_F(replica_state_sync_on_bft_seq_num_test, empty_blockchain_and_non_0_bft_seq_num) { @@ -189,7 +190,7 @@ TEST_F(replica_state_sync_on_bft_seq_num_test, bft_seq_num_equal_to_block_seq_nu addBlockWithBftSeqNum(1); addBlockWithBftSeqNum(2); ASSERT_EQ(1, blockchain_->getGenesisBlockId()); - ASSERT_EQ(2, blockchain_->getLastReachableBlockId()); + ASSERT_EQ(2, blockchain_->getLastBlockId()); const auto last_executed_bft_seq_num = 2; ASSERT_EQ(0, @@ -197,28 +198,28 @@ TEST_F(replica_state_sync_on_bft_seq_num_test, bft_seq_num_equal_to_block_seq_nu logger_, *blockchain_, last_executed_bft_seq_num, kMaxNumOfBlocksToDelete)); ASSERT_EQ(1, blockchain_->getGenesisBlockId()); - ASSERT_EQ(2, blockchain_->getLastReachableBlockId()); + ASSERT_EQ(2, blockchain_->getLastBlockId()); } TEST_F(replica_state_sync_on_bft_seq_num_test, bft_seq_num_bigger_than_block_seq_num) { addBlockWithBftSeqNum(1); addBlockWithBftSeqNum(2); ASSERT_EQ(1, blockchain_->getGenesisBlockId()); - ASSERT_EQ(2, blockchain_->getLastReachableBlockId()); + ASSERT_EQ(2, blockchain_->getLastBlockId()); const auto last_executed_bft_seq_num = 42; ASSERT_EQ(0, replica_state_sync_.executeBasedOnBftSeqNum( logger_, *blockchain_, last_executed_bft_seq_num, kMaxNumOfBlocksToDelete)); ASSERT_EQ(1, blockchain_->getGenesisBlockId()); - ASSERT_EQ(2, blockchain_->getLastReachableBlockId()); + ASSERT_EQ(2, blockchain_->getLastBlockId()); } TEST_F(replica_state_sync_on_bft_seq_num_test, bft_seq_num_less_than_block_seq_num) { addBlockWithBftSeqNum(1); addBlockWithBftSeqNum(2); ASSERT_EQ(1, blockchain_->getGenesisBlockId()); - ASSERT_EQ(2, blockchain_->getLastReachableBlockId()); + ASSERT_EQ(2, blockchain_->getLastBlockId()); const auto last_executed_bft_seq_num = 1; ASSERT_EQ(1, @@ -226,36 +227,36 @@ TEST_F(replica_state_sync_on_bft_seq_num_test, bft_seq_num_less_than_block_seq_n logger_, *blockchain_, last_executed_bft_seq_num, kMaxNumOfBlocksToDelete)); ASSERT_EQ(1, blockchain_->getGenesisBlockId()); - ASSERT_EQ(1, blockchain_->getLastReachableBlockId()); + ASSERT_EQ(1, blockchain_->getLastBlockId()); } TEST_F(replica_state_sync_on_bft_seq_num_test, cannot_delete_only_block_left) { addBlockWithBftSeqNum(2); // block 1 ASSERT_EQ(1, blockchain_->getGenesisBlockId()); - ASSERT_EQ(1, blockchain_->getLastReachableBlockId()); + ASSERT_EQ(1, blockchain_->getLastBlockId()); const auto last_executed_bft_seq_num = 1; ASSERT_THROW(replica_state_sync_.executeBasedOnBftSeqNum( logger_, *blockchain_, last_executed_bft_seq_num, kMaxNumOfBlocksToDelete), std::exception); ASSERT_EQ(1, blockchain_->getGenesisBlockId()); - ASSERT_EQ(1, blockchain_->getLastReachableBlockId()); + ASSERT_EQ(1, blockchain_->getLastBlockId()); } TEST_F(replica_state_sync_on_bft_seq_num_test, cannot_delete_only_block_left_with_pruned_block) { addBlockWithBftSeqNum(2); // block 1 addBlockWithBftSeqNum(3); // block 2 // Prune block 1. - blockchain_->deleteBlock(1); + blockchain_->deleteBlocksUntil(2); ASSERT_EQ(2, blockchain_->getGenesisBlockId()); - ASSERT_EQ(2, blockchain_->getLastReachableBlockId()); + ASSERT_EQ(2, blockchain_->getLastBlockId()); const auto last_executed_bft_seq_num = 2; ASSERT_THROW(replica_state_sync_.executeBasedOnBftSeqNum( logger_, *blockchain_, last_executed_bft_seq_num, kMaxNumOfBlocksToDelete), std::exception); ASSERT_EQ(2, blockchain_->getGenesisBlockId()); - ASSERT_EQ(2, blockchain_->getLastReachableBlockId()); + ASSERT_EQ(2, blockchain_->getLastBlockId()); } TEST_F(replica_state_sync_on_bft_seq_num_test, bft_too_many_inconsistent_blocks_detected) { @@ -264,7 +265,7 @@ TEST_F(replica_state_sync_on_bft_seq_num_test, bft_too_many_inconsistent_blocks_ addBlockWithBftSeqNum(3); addBlockWithBftSeqNum(4); ASSERT_EQ(1, blockchain_->getGenesisBlockId()); - ASSERT_EQ(4, blockchain_->getLastReachableBlockId()); + ASSERT_EQ(4, blockchain_->getLastBlockId()); const auto last_executed_bft_seq_num = 2; ASSERT_THROW(replica_state_sync_.executeBasedOnBftSeqNum(logger_, *blockchain_, last_executed_bft_seq_num, 1), @@ -272,7 +273,7 @@ TEST_F(replica_state_sync_on_bft_seq_num_test, bft_too_many_inconsistent_blocks_ // Only one block is expected to be deleted. ASSERT_EQ(1, blockchain_->getGenesisBlockId()); - ASSERT_EQ(3, blockchain_->getLastReachableBlockId()); + ASSERT_EQ(3, blockchain_->getLastBlockId()); } TEST_F(replica_state_sync_on_block_id_test, missing_block_id_in_metadata) { @@ -290,7 +291,7 @@ TEST_F(replica_state_sync_on_block_id_test, zero_max_blocks_to_delete) { persistLastBlockIdInMetadata(); addBlockWithoutBftSeqNum(); // Force out of sync by adding a second block without persisting in metadata. ASSERT_EQ(0, replica_state_sync_.executeBasedOnBlockId(logger_, *blockchain_, metadata_, 0)); - ASSERT_EQ(2, blockchain_->getLastReachableBlockId()); + ASSERT_EQ(2, blockchain_->getLastBlockId()); } TEST_F(replica_state_sync_on_block_id_test, metadata_block_id_bigger_than_last_block_id) { @@ -300,7 +301,7 @@ TEST_F(replica_state_sync_on_block_id_test, metadata_block_id_bigger_than_last_b persistLastBlockIdInMetadata(); blockchain_->deleteLastReachableBlock(); ASSERT_EQ(0, replica_state_sync_.executeBasedOnBlockId(logger_, *blockchain_, metadata_, kMaxNumOfBlocksToDelete)); - ASSERT_EQ(1, blockchain_->getLastReachableBlockId()); + ASSERT_EQ(1, blockchain_->getLastBlockId()); } TEST_F(replica_state_sync_on_block_id_test, max_num_of_blocksto_delete_is_honoured) { @@ -314,7 +315,7 @@ TEST_F(replica_state_sync_on_block_id_test, max_num_of_blocksto_delete_is_honour ASSERT_THROW(replica_state_sync_.executeBasedOnBlockId(logger_, *blockchain_, metadata_, kMaxNumOfBlocksToDelete), std::runtime_error); // 2 blocks added prior to metadata persistence and 1 that is not deleted as it exceeeds `kMaxNumOfBlocksToDelete`. - ASSERT_EQ(2 + 1, blockchain_->getLastReachableBlockId()); + ASSERT_EQ(2 + 1, blockchain_->getLastBlockId()); } TEST_F(replica_state_sync_on_block_id_test, in_sync) { @@ -322,7 +323,7 @@ TEST_F(replica_state_sync_on_block_id_test, in_sync) { addBlockWithoutBftSeqNum(); persistLastBlockIdInMetadata(); ASSERT_EQ(0, replica_state_sync_.executeBasedOnBlockId(logger_, *blockchain_, metadata_, kMaxNumOfBlocksToDelete)); - ASSERT_EQ(2, blockchain_->getLastReachableBlockId()); + ASSERT_EQ(2, blockchain_->getLastBlockId()); } TEST_F(replica_state_sync_on_block_id_test, out_of_sync) { @@ -332,21 +333,21 @@ TEST_F(replica_state_sync_on_block_id_test, out_of_sync) { addBlockWithoutBftSeqNum(); addBlockWithoutBftSeqNum(); addBlockWithoutBftSeqNum(); - ASSERT_EQ(5, blockchain_->getLastReachableBlockId()); + ASSERT_EQ(5, blockchain_->getLastBlockId()); ASSERT_EQ(3, replica_state_sync_.executeBasedOnBlockId(logger_, *blockchain_, metadata_, kMaxNumOfBlocksToDelete)); - ASSERT_EQ(2, blockchain_->getLastReachableBlockId()); + ASSERT_EQ(2, blockchain_->getLastBlockId()); } TEST_F(replica_state_sync_on_block_id_test, cannot_delete_only_block_left) { persistLastBlockIdInMetadata(); addBlockWithoutBftSeqNum(); ASSERT_EQ(1, blockchain_->getGenesisBlockId()); - ASSERT_EQ(1, blockchain_->getLastReachableBlockId()); + ASSERT_EQ(1, blockchain_->getLastBlockId()); ASSERT_THROW(replica_state_sync_.executeBasedOnBlockId(logger_, *blockchain_, metadata_, kMaxNumOfBlocksToDelete), std::exception); ASSERT_EQ(1, blockchain_->getGenesisBlockId()); - ASSERT_EQ(1, blockchain_->getLastReachableBlockId()); + ASSERT_EQ(1, blockchain_->getLastBlockId()); } TEST_F(replica_state_sync_on_block_id_test, cannot_delete_only_block_left_with_pruned_block) { @@ -356,13 +357,13 @@ TEST_F(replica_state_sync_on_block_id_test, cannot_delete_only_block_left_with_p addBlockWithoutBftSeqNum(); addBlockWithoutBftSeqNum(); - blockchain_->deleteBlock(1); + blockchain_->deleteBlocksUntil(2); // Expect a throw from KVBC when trying to delete the only block (genesis, block ID = 2) in the system. ASSERT_THROW(replica_state_sync_.executeBasedOnBlockId(logger_, *blockchain_, metadata_, kMaxNumOfBlocksToDelete), std::exception); ASSERT_EQ(2, blockchain_->getGenesisBlockId()); - ASSERT_EQ(2, blockchain_->getLastReachableBlockId()); + ASSERT_EQ(2, blockchain_->getLastBlockId()); } TEST_F(replica_state_sync_test, out_of_sync_on_upgade) { @@ -375,7 +376,7 @@ TEST_F(replica_state_sync_test, out_of_sync_on_upgade) { ASSERT_EQ(1, replica_state_sync_.execute( logger_, *blockchain_, metadata_, last_executed_bft_seq_num, kMaxNumOfBlocksToDelete)); - ASSERT_EQ(1, blockchain_->getLastReachableBlockId()); + ASSERT_EQ(1, blockchain_->getLastBlockId()); // Last block ID is persisted by execute() in metadata. auto last_mtd_block_id = getLastBlockIdFromMetadata(metadata_); @@ -386,7 +387,7 @@ TEST_F(replica_state_sync_test, out_of_sync_on_upgade) { ASSERT_EQ(0, replica_state_sync_.execute( logger_, *blockchain_, metadata_, last_executed_bft_seq_num, kMaxNumOfBlocksToDelete)); - ASSERT_EQ(1, blockchain_->getLastReachableBlockId()); + ASSERT_EQ(1, blockchain_->getLastBlockId()); // Last block ID remains the same. last_mtd_block_id = getLastBlockIdFromMetadata(metadata_); @@ -403,11 +404,11 @@ TEST_F(replica_state_sync_test, metadata_not_persisted_on_first_block_after_soft ASSERT_EQ(0, replica_state_sync_.executeBasedOnBftSeqNum( logger_, *blockchain_, last_executed_bft_seq_num, kMaxNumOfBlocksToDelete)); - ASSERT_EQ(2, blockchain_->getLastReachableBlockId()); + ASSERT_EQ(2, blockchain_->getLastBlockId()); // Add a block after the upgrade from the new software version, without persisting metadata. addBlockWithoutBftSeqNum(); - ASSERT_EQ(3, blockchain_->getLastReachableBlockId()); + ASSERT_EQ(3, blockchain_->getLastBlockId()); // Calling execute() will persist the last block ID in metadata. execute() assumes that the first block after upgrade // is successfuly added to both KVBC and metadata by the replica. If that is not the case as in this test, we don't @@ -416,7 +417,7 @@ TEST_F(replica_state_sync_test, metadata_not_persisted_on_first_block_after_soft ASSERT_EQ(0, replica_state_sync_.execute( logger_, *blockchain_, metadata_, last_executed_bft_seq_num + 1, kMaxNumOfBlocksToDelete)); - ASSERT_EQ(3, blockchain_->getLastReachableBlockId()); + ASSERT_EQ(3, blockchain_->getLastBlockId()); // Last block ID is persisted by execute() in metadata. const auto last_mtd_block_id = getLastBlockIdFromMetadata(metadata_); @@ -435,7 +436,7 @@ TEST_F(replica_state_sync_test, out_of_sync_after_software_upgrade) { ASSERT_EQ(1, replica_state_sync_.execute( logger_, *blockchain_, metadata_, last_executed_bft_seq_num, kMaxNumOfBlocksToDelete)); - ASSERT_EQ(3, blockchain_->getLastReachableBlockId()); + ASSERT_EQ(3, blockchain_->getLastBlockId()); } TEST_F(replica_state_sync_test, in_sync_after_software_upgrade) { @@ -449,7 +450,7 @@ TEST_F(replica_state_sync_test, in_sync_after_software_upgrade) { ASSERT_EQ(0, replica_state_sync_.execute( logger_, *blockchain_, metadata_, last_executed_bft_seq_num, kMaxNumOfBlocksToDelete)); - ASSERT_EQ(4, blockchain_->getLastReachableBlockId()); + ASSERT_EQ(4, blockchain_->getLastBlockId()); } } // namespace diff --git a/kvbc/test/sparse_merkle_storage/kv_blockchain_db_editor_test.cpp b/kvbc/test/sparse_merkle_storage/kv_blockchain_db_editor_test.cpp index 3dbca063e4..7f16e5a269 100644 --- a/kvbc/test/sparse_merkle_storage/kv_blockchain_db_editor_test.cpp +++ b/kvbc/test/sparse_merkle_storage/kv_blockchain_db_editor_test.cpp @@ -30,7 +30,7 @@ class DbEditorTests : public DbEditorTestsBase { public: void CreateBlockchain(std::size_t db_id, BlockId blocks, std::optional mismatch_at = std::nullopt) override { auto db = TestRocksDb::create(db_id); - auto adapter = KeyValueBlockchain{ + auto adapter = concord::kvbc::categorization::KeyValueBlockchain{ concord::storage::rocksdb::NativeClient::fromIDBClient(db), true, std::map{ @@ -97,7 +97,7 @@ class DbEditorTests : public DbEditorTestsBase { void DeleteBlocksUntil(std::size_t db_id, BlockId until_block_id) override { auto db = TestRocksDb::createNative(db_id); - auto adapter = KeyValueBlockchain{db, true}; + auto adapter = concord::kvbc::categorization::KeyValueBlockchain{db, true}; for (auto i = 1ull; i < until_block_id; ++i) { adapter.deleteBlock(i); diff --git a/kvbc/test/v4blockchain/blockchain_test.cpp b/kvbc/test/v4blockchain/blockchain_test.cpp new file mode 100644 index 0000000000..78383409a6 --- /dev/null +++ b/kvbc/test/v4blockchain/blockchain_test.cpp @@ -0,0 +1,631 @@ +// Concord +// +// Copyright (c) 2020 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 subcomponent's license, as noted in the LICENSE +// file. + +#include "gtest/gtest.h" +#include "gmock/gmock.h" +#include "v4blockchain/detail/blockchain.h" +#include "storage/test/storage_test_common.h" +#include "v4blockchain/detail/column_families.h" + +using concord::storage::rocksdb::NativeClient; +using namespace concord::kvbc; +using namespace ::testing; + +namespace { + +class v4_blockchain : public Test { + protected: + void SetUp() override { + destroyDb(); + db = TestRocksDb::createNative(); + } + + void TearDown() override { destroyDb(); } + + void destroyDb() { + db.reset(); + ASSERT_EQ(0, db.use_count()); + cleanup(); + } + + protected: + std::shared_ptr db; +}; + +TEST_F(v4_blockchain, creation) { + concord::util::digest::BlockDigest empty_digest; + for (auto& d : empty_digest) { + d = 0; + } + + auto wb = db->getBatch(); + + auto blockchain = v4blockchain::detail::Blockchain{db}; + { + auto block = blockchain.getBlockData(420); + ASSERT_FALSE(block.has_value()); + } + + auto versioned_cat = std::string("versioned"); + auto key = std::string("key"); + auto val = std::string("val"); + auto updates = categorization::Updates{}; + auto ver_updates = categorization::VersionedUpdates{}; + ver_updates.addUpdate("key", "val"); + updates.add(versioned_cat, std::move(ver_updates)); + + auto id = blockchain.addBlock(updates, wb); + ASSERT_EQ(id, 1); + ASSERT_EQ(blockchain.from_storage, 1); + db->write(std::move(wb)); + + auto blockstr = blockchain.getBlockData(id); + ASSERT_TRUE(blockstr.has_value()); + + auto block = v4blockchain::detail::Block(*blockstr); + const auto& emdig = block.parentDigest(); + ASSERT_EQ(emdig, empty_digest); + + const auto& v = block.getVersion(); + ASSERT_EQ(v, v4blockchain::detail::Block::BLOCK_VERSION); + + auto reconstruct_updates = block.getUpdates(); + auto input = reconstruct_updates.categoryUpdates(); + ASSERT_EQ(input.kv.count(versioned_cat), 1); + auto reconstruct_ver_updates = std::get(input.kv[versioned_cat]); + ASSERT_EQ(reconstruct_ver_updates.kv[key].data, val); + ASSERT_EQ(reconstruct_ver_updates.kv[key].stale_on_update, false); +} + +TEST_F(v4_blockchain, calculate_empty_digest) { + concord::util::digest::BlockDigest empty_digest; + for (auto& d : empty_digest) { + d = 0; + } + + auto wb = db->getBatch(); + + auto blockchain = v4blockchain::detail::Blockchain{db}; + + { + const auto& dig = blockchain.calculateBlockDigest(0); + ASSERT_EQ(dig, empty_digest); + } +} + +TEST_F(v4_blockchain, basic_chain) { + concord::util::digest::BlockDigest empty_digest; + for (auto& d : empty_digest) { + d = 0; + } + + auto blockchain = v4blockchain::detail::Blockchain{db}; + // First block + { + auto wb = db->getBatch(); + auto versioned_cat = std::string("versioned"); + auto key = std::string("key"); + auto val = std::string("val"); + auto updates = categorization::Updates{}; + auto ver_updates = categorization::VersionedUpdates{}; + ver_updates.addUpdate("key", "val"); + updates.add(versioned_cat, std::move(ver_updates)); + + auto id = blockchain.addBlock(updates, wb); + ASSERT_EQ(id, 1); + ASSERT_EQ(blockchain.from_storage, 1); + db->write(std::move(wb)); + blockchain.setBlockId(id); + + // Validate block from storage + auto blockstr = blockchain.getBlockData(id); + ASSERT_TRUE(blockstr.has_value()); + auto block = v4blockchain::detail::Block(*blockstr); + const auto& emdig = block.parentDigest(); + ASSERT_EQ(emdig, empty_digest); + const auto& v = block.getVersion(); + ASSERT_EQ(v, v4blockchain::detail::Block::BLOCK_VERSION); + auto reconstruct_updates = block.getUpdates(); + auto input = reconstruct_updates.categoryUpdates(); + ASSERT_EQ(input.kv.count(versioned_cat), 1); + auto reconstruct_ver_updates = std::get(input.kv[versioned_cat]); + ASSERT_EQ(reconstruct_ver_updates.kv[key].data, val); + ASSERT_EQ(reconstruct_ver_updates.kv[key].stale_on_update, false); + } + + // Second + { + auto wb = db->getBatch(); + auto imm_cat = std::string("immuatables"); + auto immkey = std::string("immkey"); + auto immval = std::string("immval"); + auto updates = categorization::Updates{}; + auto imm_updates = categorization::ImmutableUpdates{}; + imm_updates.addUpdate("immkey", categorization::ImmutableUpdates::ImmutableValue{"immval", {"1", "2", "33"}}); + updates.add(imm_cat, std::move(imm_updates)); + + auto id = blockchain.addBlock(updates, wb); + ASSERT_EQ(id, 2); + ASSERT_EQ(blockchain.from_storage, 1); + ASSERT_EQ(blockchain.from_future, 1); + db->write(std::move(wb)); + blockchain.setBlockId(id); + + // Get block from storage + auto blockstr = blockchain.getBlockData(id); + ASSERT_TRUE(blockstr.has_value()); + auto block = v4blockchain::detail::Block(*blockstr); + const auto& dig = block.parentDigest(); + + // compare again digest of parent from storage + auto parent_blockstr = blockchain.getBlockData(id - 1); + ASSERT_TRUE(parent_blockstr.has_value()); + auto parent_digest = + v4blockchain::detail::Block::calculateDigest(id - 1, parent_blockstr->c_str(), parent_blockstr->size()); + ASSERT_EQ(dig, parent_digest); + + // Validate block from storage + auto reconstruct_updates = block.getUpdates(); + auto input = reconstruct_updates.categoryUpdates(); + ASSERT_EQ(input.kv.count(imm_cat), 1); + auto reconstruct_imm_updates = std::get(input.kv[imm_cat]); + ASSERT_EQ(reconstruct_imm_updates.kv[immkey].data, immval); + std::vector v = {"1", "2", "33"}; + ASSERT_EQ(reconstruct_imm_updates.kv[immkey].tags, v); + } +} + +TEST_F(v4_blockchain, adv_chain) { + concord::util::digest::BlockDigest empty_digest; + for (auto& d : empty_digest) { + d = 0; + } + + { + auto blockchain = v4blockchain::detail::Blockchain{db}; + // First block + { + auto wb = db->getBatch(); + auto versioned_cat = std::string("versioned"); + auto key = std::string("key"); + auto val = std::string("val"); + auto updates = categorization::Updates{}; + auto ver_updates = categorization::VersionedUpdates{}; + ver_updates.addUpdate("key", "val"); + updates.add(versioned_cat, std::move(ver_updates)); + + auto id = blockchain.addBlock(updates, wb); + ASSERT_EQ(id, 1); + db->write(std::move(wb)); + blockchain.setBlockId(id); + + // Validate block from storage + auto blockstr = blockchain.getBlockData(id); + ASSERT_TRUE(blockstr.has_value()); + auto block = v4blockchain::detail::Block(*blockstr); + const auto& emdig = block.parentDigest(); + ASSERT_EQ(emdig, empty_digest); + const auto& v = block.getVersion(); + ASSERT_EQ(v, v4blockchain::detail::Block::BLOCK_VERSION); + auto reconstruct_updates = block.getUpdates(); + auto input = reconstruct_updates.categoryUpdates(); + ASSERT_EQ(input.kv.count(versioned_cat), 1); + auto reconstruct_ver_updates = std::get(input.kv[versioned_cat]); + ASSERT_EQ(reconstruct_ver_updates.kv[key].data, val); + ASSERT_EQ(reconstruct_ver_updates.kv[key].stale_on_update, false); + } + + // Second + { + auto wb = db->getBatch(); + auto imm_cat = std::string("immuatables"); + auto immkey = std::string("immkey"); + auto immval = std::string("immval"); + auto updates = categorization::Updates{}; + auto imm_updates = categorization::ImmutableUpdates{}; + imm_updates.addUpdate("immkey", categorization::ImmutableUpdates::ImmutableValue{"immval", {"1", "2", "33"}}); + updates.add(imm_cat, std::move(imm_updates)); + + auto id = blockchain.addBlock(updates, wb); + ASSERT_EQ(id, 2); + ASSERT_EQ(blockchain.from_storage, 1); + ASSERT_EQ(blockchain.from_future, 1); + db->write(std::move(wb)); + blockchain.setBlockId(id); + + // Get block from storage + auto blockstr = blockchain.getBlockData(id); + ASSERT_TRUE(blockstr.has_value()); + auto block = v4blockchain::detail::Block(*blockstr); + const auto& dig = block.parentDigest(); + + // compare again digest of parent from storage + auto parent_blockstr = blockchain.getBlockData(id - 1); + ASSERT_TRUE(parent_blockstr.has_value()); + auto parent_digest = + v4blockchain::detail::Block::calculateDigest(id - 1, parent_blockstr->c_str(), parent_blockstr->size()); + ASSERT_EQ(dig, parent_digest); + + // Validate block from storage + auto reconstruct_updates = block.getUpdates(); + auto input = reconstruct_updates.categoryUpdates(); + ASSERT_EQ(input.kv.count(imm_cat), 1); + auto reconstruct_imm_updates = std::get(input.kv[imm_cat]); + ASSERT_EQ(reconstruct_imm_updates.kv[immkey].data, immval); + std::vector v = {"1", "2", "33"}; + ASSERT_EQ(reconstruct_imm_updates.kv[immkey].tags, v); + } + } + + // load blockchain from storage + { + auto blockchain = v4blockchain::detail::Blockchain{db}; + ASSERT_EQ(blockchain.getLastReachable(), 2); + auto wb = db->getBatch(); + + auto imm_cat = std::string("immuatables"); + auto immkey = std::string("immkey"); + auto immval = std::string("immval"); + auto updates = categorization::Updates{}; + auto imm_updates = categorization::ImmutableUpdates{}; + imm_updates.addUpdate("immkey", categorization::ImmutableUpdates::ImmutableValue{"immval", {"1", "2", "33"}}); + updates.add(imm_cat, std::move(imm_updates)); + + auto id = blockchain.addBlock(updates, wb); + ASSERT_EQ(id, 3); + ASSERT_EQ(blockchain.from_storage, 1); + ASSERT_EQ(blockchain.from_future, 0); + db->write(std::move(wb)); + blockchain.setBlockId(id); + } + + // load blockchain from storage + { + auto blockchain = v4blockchain::detail::Blockchain{db}; + ASSERT_EQ(blockchain.getLastReachable(), 3); + + { + auto wb = db->getBatch(); + + auto imm_cat = std::string("immuatables"); + auto immkey = std::string("immkey"); + auto immval = std::string("immval"); + auto updates = categorization::Updates{}; + auto imm_updates = categorization::ImmutableUpdates{}; + imm_updates.addUpdate("immkey", categorization::ImmutableUpdates::ImmutableValue{"immval", {"1", "2", "33"}}); + updates.add(imm_cat, std::move(imm_updates)); + + auto id = blockchain.addBlock(updates, wb); + ASSERT_EQ(id, 4); + ASSERT_EQ(blockchain.from_storage, 1); + ASSERT_EQ(blockchain.from_future, 0); + db->write(std::move(wb)); + blockchain.setBlockId(id); + } + { + auto wb = db->getBatch(); + + auto imm_cat = std::string("immuatables"); + auto immkey = std::string("immkey"); + auto immval = std::string("immval"); + auto updates = categorization::Updates{}; + auto imm_updates = categorization::ImmutableUpdates{}; + imm_updates.addUpdate("immkey", categorization::ImmutableUpdates::ImmutableValue{"immval", {"1", "2", "33"}}); + updates.add(imm_cat, std::move(imm_updates)); + + auto id = blockchain.addBlock(updates, wb); + ASSERT_EQ(id, 5); + ASSERT_EQ(blockchain.from_storage, 1); + ASSERT_EQ(blockchain.from_future, 1); + + db->write(std::move(wb)); + blockchain.setBlockId(id); + + // Get block from storage + auto blockstr = blockchain.getBlockData(5); + ASSERT_TRUE(blockstr.has_value()); + auto block = v4blockchain::detail::Block(*blockstr); + const auto& dig = block.parentDigest(); + + // compare against digest of parent from storage + auto parent_blockstr = blockchain.getBlockData(5 - 1); + ASSERT_TRUE(parent_blockstr.has_value()); + auto parent_digest = + v4blockchain::detail::Block::calculateDigest(id - 1, parent_blockstr->c_str(), parent_blockstr->size()); + ASSERT_EQ(dig, parent_digest); + } + } + + // load blockchain from storage and check all keys in multiget + { + auto blockchain = v4blockchain::detail::Blockchain{db}; + std::vector block_ids{1, 2, 3, 4, 5}; + std::unordered_map> values; + ASSERT_NO_THROW(blockchain.multiGetBlockData(block_ids, values)); + ASSERT_EQ(block_ids.size(), values.size()); + for (const auto& bid : block_ids) { + const auto valit = values.find(bid); + ASSERT_NE(valit, values.cend()); + ASSERT_TRUE((valit->second).has_value()); + } + std::unordered_map> updates; + ASSERT_NO_THROW(blockchain.multiGetBlockUpdates(block_ids, updates)); + ASSERT_EQ(block_ids.size(), updates.size()); + for (const auto& bid : block_ids) { + const auto valit = updates.find(bid); + ASSERT_NE(valit, updates.cend()); + ASSERT_TRUE((valit->second).has_value()); + } + } + + // load blockchain from storage and check all keys in multiget for duplicates + { + auto blockchain = v4blockchain::detail::Blockchain{db}; + std::vector block_ids{1, 2, 3, 3, 4, 5}; + std::unordered_map> values; + ASSERT_THROW(blockchain.multiGetBlockData(block_ids, values), std::logic_error); + std::unordered_map> updates; + blockchain.multiGetBlockUpdates(block_ids, updates); + ASSERT_GT(block_ids.size(), updates.size()); + for (const auto& bid : block_ids) { + const auto valit = updates.find(bid); + ASSERT_NE(valit, updates.cend()); + ASSERT_TRUE((valit->second).has_value()); + } + } + + // load blockchain from storage and check all keys in multiget for unknown values + { + auto blockchain = v4blockchain::detail::Blockchain{db}; + std::vector block_ids{1, 2, 3, 4, 5, 6, 7, 8}; + std::unordered_map> values; + ASSERT_NO_THROW(blockchain.multiGetBlockData(block_ids, values)); + ASSERT_EQ(block_ids.size(), values.size()); + for (const auto& bid : block_ids) { + const auto valit = values.find(bid); + ASSERT_NE(valit, values.cend()); + if (bid > blockchain.getLastReachable()) { + ASSERT_FALSE((valit->second).has_value()); + } else { + ASSERT_TRUE((valit->second).has_value()); + } + } + std::unordered_map> updates; + ASSERT_NO_THROW(blockchain.multiGetBlockUpdates(block_ids, updates)); + ASSERT_EQ(block_ids.size(), updates.size()); + for (const auto& bid : block_ids) { + const auto valit = updates.find(bid); + ASSERT_NE(valit, updates.cend()); + if (bid > blockchain.getLastReachable()) { + ASSERT_FALSE((valit->second).has_value()); + } else { + ASSERT_TRUE((valit->second).has_value()); + } + } + } + // load blockchain from storage and check all keys in multiget for duplicate unknown values + { + auto blockchain = v4blockchain::detail::Blockchain{db}; + std::vector block_ids{1, 2, 3, 4, 5, 6, 6, 7, 8}; + std::unordered_map> values; + ASSERT_THROW(blockchain.multiGetBlockData(block_ids, values), std::logic_error); + std::unordered_map> updates; + blockchain.multiGetBlockUpdates(block_ids, updates); + ASSERT_GT(block_ids.size(), updates.size()); + for (const auto& bid : block_ids) { + const auto valit = updates.find(bid); + ASSERT_NE(valit, updates.cend()); + if (bid > blockchain.getLastReachable()) { + ASSERT_FALSE((valit->second).has_value()); + } else { + ASSERT_TRUE((valit->second).has_value()); + } + } + } +} + +TEST_F(v4_blockchain, delete_until) { + auto blockchain = v4blockchain::detail::Blockchain{db}; + // Can't delete from empty blockchain + ASSERT_DEATH(blockchain.deleteBlocksUntil(1), ""); + // block 1 + { + auto wb = db->getBatch(); + auto versioned_cat = std::string("versioned"); + auto key = std::string("key"); + auto val = std::string("val"); + auto updates = categorization::Updates{}; + auto ver_updates = categorization::VersionedUpdates{}; + ver_updates.addUpdate("key", "val"); + updates.add(versioned_cat, std::move(ver_updates)); + + auto id = blockchain.addBlock(updates, wb); + ASSERT_EQ(id, 1); + db->write(std::move(wb)); + blockchain.setBlockId(id); + } + + // 2 + { + auto wb = db->getBatch(); + auto imm_cat = std::string("immuatables"); + auto immkey = std::string("immkey"); + auto immval = std::string("immval"); + auto updates = categorization::Updates{}; + auto imm_updates = categorization::ImmutableUpdates{}; + imm_updates.addUpdate("immkey", categorization::ImmutableUpdates::ImmutableValue{"immval", {"1", "2", "33"}}); + updates.add(imm_cat, std::move(imm_updates)); + + auto id = blockchain.addBlock(updates, wb); + ASSERT_EQ(id, 2); + db->write(std::move(wb)); + blockchain.setBlockId(id); + } + + // 3 + { + auto wb = db->getBatch(); + auto imm_cat = std::string("immuatables"); + auto immkey = std::string("immkey"); + auto immval = std::string("immval"); + auto updates = categorization::Updates{}; + auto imm_updates = categorization::ImmutableUpdates{}; + imm_updates.addUpdate("immkey", categorization::ImmutableUpdates::ImmutableValue{"immval", {"1", "2", "33"}}); + updates.add(imm_cat, std::move(imm_updates)); + + auto id = blockchain.addBlock(updates, wb); + ASSERT_EQ(id, 3); + db->write(std::move(wb)); + blockchain.setBlockId(id); + } + + // 4 + { + auto wb = db->getBatch(); + auto imm_cat = std::string("immuatables"); + auto immkey = std::string("immkey"); + auto immval = std::string("immval"); + auto updates = categorization::Updates{}; + auto imm_updates = categorization::ImmutableUpdates{}; + imm_updates.addUpdate("immkey", categorization::ImmutableUpdates::ImmutableValue{"immval", {"1", "2", "33"}}); + updates.add(imm_cat, std::move(imm_updates)); + + auto id = blockchain.addBlock(updates, wb); + ASSERT_EQ(id, 4); + db->write(std::move(wb)); + blockchain.setBlockId(id); + } + + { + uint64_t until = 3; + auto id = blockchain.deleteBlocksUntil(until); + ASSERT_EQ(id, until - 1); + ASSERT_EQ(until, blockchain.getGenesisBlockId()); + auto block_2 = blockchain.getBlockData(2); + ASSERT_FALSE(block_2.has_value()); + auto block_until = blockchain.getBlockData(until); + ASSERT_TRUE(block_until.has_value()); + } + + { + uint64_t until = 100; + auto id = blockchain.deleteBlocksUntil(until); + ASSERT_EQ(id, 3); + ASSERT_EQ(blockchain.getLastReachable(), blockchain.getGenesisBlockId()); + auto block_3 = blockchain.getBlockData(3); + ASSERT_FALSE(block_3.has_value()); + auto block_until = blockchain.getBlockData(4); + ASSERT_TRUE(block_until.has_value()); + } + + { + auto blockchain_local = v4blockchain::detail::Blockchain{db}; + ASSERT_EQ(blockchain_local.getLastReachable(), blockchain_local.getGenesisBlockId()); + ASSERT_EQ(blockchain_local.getLastReachable(), 4); + auto id = blockchain_local.deleteBlocksUntil(5); + // single block on the chain, no actuall deletion + ASSERT_EQ(id, 3); + // until is less than the genesis + ASSERT_DEATH(blockchain_local.deleteBlocksUntil(1), ""); + } +} + +TEST_F(v4_blockchain, delete_genesis) { + auto blockchain = v4blockchain::detail::Blockchain{db}; + // Can't delete from empty blockchain + ASSERT_DEATH(blockchain.deleteGenesisBlock(), ""); + // block 1 + { + auto wb = db->getBatch(); + auto versioned_cat = std::string("versioned"); + auto key = std::string("key"); + auto val = std::string("val"); + auto updates = categorization::Updates{}; + auto ver_updates = categorization::VersionedUpdates{}; + ver_updates.addUpdate("key", "val"); + updates.add(versioned_cat, std::move(ver_updates)); + + auto id = blockchain.addBlock(updates, wb); + ASSERT_EQ(id, 1); + db->write(std::move(wb)); + blockchain.setBlockId(id); + } + // Can't delete single block + ASSERT_DEATH(blockchain.deleteGenesisBlock(), ""); + + // 2 + { + auto wb = db->getBatch(); + auto imm_cat = std::string("immuatables"); + auto immkey = std::string("immkey"); + auto immval = std::string("immval"); + auto updates = categorization::Updates{}; + auto imm_updates = categorization::ImmutableUpdates{}; + imm_updates.addUpdate("immkey", categorization::ImmutableUpdates::ImmutableValue{"immval", {"1", "2", "33"}}); + updates.add(imm_cat, std::move(imm_updates)); + + auto id = blockchain.addBlock(updates, wb); + ASSERT_EQ(id, 2); + db->write(std::move(wb)); + blockchain.setBlockId(id); + } + + blockchain.deleteGenesisBlock(); + ASSERT_EQ(blockchain.getGenesisBlockId(), 2); + // Can't delete single block + ASSERT_DEATH(blockchain.deleteGenesisBlock(), ""); + + { + auto blockchain_local = v4blockchain::detail::Blockchain{db}; + ASSERT_EQ(blockchain_local.getGenesisBlockId(), 2); + ASSERT_DEATH(blockchain_local.deleteGenesisBlock(), ""); + } +} + +TEST_F(v4_blockchain, block_updates) { + auto blockchain = v4blockchain::detail::Blockchain{db}; + // First block + { + auto wb = db->getBatch(); + auto versioned_cat = std::string("versioned"); + auto key = std::string("key"); + auto val = std::string("val"); + auto updates = categorization::Updates{}; + auto ver_updates = categorization::VersionedUpdates{}; + ver_updates.addUpdate("key", "val"); + updates.add(versioned_cat, std::move(ver_updates)); + + auto id = blockchain.addBlock(updates, wb); + ASSERT_EQ(id, 1); + ASSERT_EQ(blockchain.from_storage, 1); + db->write(std::move(wb)); + blockchain.setBlockId(id); + + // Validate updates from storage + auto blockstr3 = blockchain.getBlockData(3); + ASSERT_FALSE(blockstr3.has_value()); + auto updates3 = blockchain.getBlockUpdates(3); + ASSERT_FALSE(updates3.has_value()); + + auto updates1 = blockchain.getBlockUpdates(id); + ASSERT_TRUE(updates1.has_value()); + ASSERT_EQ(updates1->categoryUpdates(), updates.categoryUpdates()); + } +} + +} // end namespace + +int main(int argc, char** argv) { + InitGoogleTest(&argc, argv); + return RUN_ALL_TESTS(); +} diff --git a/kvbc/test/v4blockchain/blocks_test.cpp b/kvbc/test/v4blockchain/blocks_test.cpp new file mode 100644 index 0000000000..e0087f0851 --- /dev/null +++ b/kvbc/test/v4blockchain/blocks_test.cpp @@ -0,0 +1,301 @@ +// Concord +// +// Copyright (c) 2020 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 subcomponent's license, as noted in the LICENSE +// file. + +#include "gtest/gtest.h" +#include "gmock/gmock.h" +#include "v4blockchain/detail/blocks.h" +#include +#include +#include +#include +#include +#include "storage/test/storage_test_common.h" + +using namespace concord::kvbc; +using namespace ::testing; + +namespace { + +TEST(v4_block, creation) { + { + v4blockchain::detail::Block block; + const auto& buffer = block.getBuffer(); + ASSERT_EQ(buffer.size(), v4blockchain::detail::Block::HEADER_SIZE); + const auto& v = block.getVersion(); + ASSERT_EQ(v, v4blockchain::detail::Block::BLOCK_VERSION); + } + + { + auto size = uint64_t{100}; + auto block = v4blockchain::detail::Block{size}; + const auto& buffer = block.getBuffer(); + ASSERT_EQ(buffer.size(), v4blockchain::detail::Block::HEADER_SIZE); + ASSERT_EQ(buffer.capacity(), size); + const auto& v = block.getVersion(); + ASSERT_EQ(v, v4blockchain::detail::Block::BLOCK_VERSION); + } +} + +TEST(v4_block, new_block_digest) { + // New blocks should have empty digest + concord::util::digest::BlockDigest empty_digest; + for (auto& d : empty_digest) { + d = 0; + } + + { + v4blockchain::detail::Block block; + const auto& dig = block.parentDigest(); + ASSERT_EQ(dig, empty_digest); + } + + { + auto size = uint64_t{100}; + auto block = v4blockchain::detail::Block{size}; + const auto& dig = block.parentDigest(); + ASSERT_EQ(dig, empty_digest); + } +} + +TEST(v4_block, add_digest) { + // New blocks should have empty digest + concord::util::digest::BlockDigest empty_digest; + for (auto& d : empty_digest) { + d = 0; + } + concord::util::digest::BlockDigest digest; + int i = 0; + for (auto& d : digest) { + d = i++; + } + + { + v4blockchain::detail::Block block; + block.addDigest(digest); + const auto& dig = block.parentDigest(); + ASSERT_EQ(dig, digest); + } + + { + auto size = uint64_t{100}; + auto block = v4blockchain::detail::Block{size}; + const auto& emdig = block.parentDigest(); + ASSERT_EQ(emdig, empty_digest); + block.addDigest(digest); + const auto& dig = block.parentDigest(); + ASSERT_EQ(dig, digest); + } +} + +TEST(v4_block, add_updates) { + // New blocks should have empty digest + concord::util::digest::BlockDigest empty_digest; + for (auto& d : empty_digest) { + d = 0; + } + { + auto versioned_cat = std::string("versioned"); + auto key = std::string("key"); + auto val = std::string("val"); + auto updates = categorization::Updates{}; + auto ver_updates = categorization::VersionedUpdates{}; + ver_updates.addUpdate("key", "val"); + updates.add(versioned_cat, std::move(ver_updates)); + v4blockchain::detail::Block block; + const auto& buffer = block.getBuffer(); + ASSERT_EQ(buffer.size(), v4blockchain::detail::Block::HEADER_SIZE); + block.addUpdates(updates); + + { + const auto& buffer = block.getBuffer(); + ASSERT_GT(buffer.size(), v4blockchain::detail::Block::HEADER_SIZE); + auto reconstruct_updates = block.getUpdates(); + auto input = reconstruct_updates.categoryUpdates(); + ASSERT_EQ(input.kv.count(versioned_cat), 1); + auto reconstruct_ver_updates = std::get(input.kv[versioned_cat]); + ASSERT_EQ(reconstruct_ver_updates.kv[key].data, val); + ASSERT_EQ(reconstruct_ver_updates.kv[key].stale_on_update, false); + } + } + + { + uint64_t size = 158; + v4blockchain::detail::Block block{size}; + const auto& buffer = block.getBuffer(); + ASSERT_EQ(buffer.size(), v4blockchain::detail::Block::HEADER_SIZE); + ASSERT_EQ(buffer.capacity(), size); + + { + auto imm_cat = std::string("immuatables"); + auto key = std::string("immkey"); + auto val = std::string("immval"); + auto updates = categorization::Updates{}; + auto imm_updates = categorization::ImmutableUpdates{}; + imm_updates.addUpdate("immkey", categorization::ImmutableUpdates::ImmutableValue{"immval", {"1", "2", "33"}}); + updates.add(imm_cat, std::move(imm_updates)); + const auto& buffer = block.getBuffer(); + block.addUpdates(updates); + + ASSERT_GT(buffer.size(), v4blockchain::detail::Block::HEADER_SIZE); + auto reconstruct_updates = block.getUpdates(); + auto input = reconstruct_updates.categoryUpdates(); + ASSERT_EQ(input.kv.count("versioned_cat"), 0); + ASSERT_EQ(input.kv.count(imm_cat), 1); + auto reconstruct_imm_updates = std::get(input.kv[imm_cat]); + ASSERT_EQ(reconstruct_imm_updates.kv[key].data, val); + std::vector v = {"1", "2", "33"}; + ASSERT_EQ(reconstruct_imm_updates.kv[key].tags, v); + } + } +} + +TEST(v4_block, calculate_digest) { + // New blocks should have empty digest + concord::util::digest::BlockDigest empty_digest; + for (auto& d : empty_digest) { + d = 0; + } + // Genesis block + concord::util::digest::BlockDigest genesis_digest; + { + v4blockchain::detail::Block block; + auto imm_cat = std::string("immuatables"); + auto immkey = std::string("immkey"); + auto immval = std::string("immval"); + auto updates = categorization::Updates{}; + auto imm_updates = categorization::ImmutableUpdates{}; + imm_updates.addUpdate("immkey", categorization::ImmutableUpdates::ImmutableValue{"immval", {"1", "2", "33"}}); + updates.add(imm_cat, std::move(imm_updates)); + + auto versioned_cat = std::string("versioned"); + auto verkey = std::string("verkey"); + auto verval = std::string("verval"); + auto ver_updates = categorization::VersionedUpdates{}; + ver_updates.addUpdate("verkey", "verval"); + updates.add(versioned_cat, std::move(ver_updates)); + + block.addUpdates(updates); + + auto reconstruct_updates = block.getUpdates(); + auto input = reconstruct_updates.categoryUpdates(); + ASSERT_EQ(input.kv.count(versioned_cat), 1); + ASSERT_EQ(input.kv.count(imm_cat), 1); + auto reconstruct_imm_updates = std::get(input.kv[imm_cat]); + ASSERT_EQ(reconstruct_imm_updates.kv[immkey].data, immval); + std::vector v = {"1", "2", "33"}; + ASSERT_EQ(reconstruct_imm_updates.kv[immkey].tags, v); + + auto reconstruct_ver_updates = std::get(input.kv[versioned_cat]); + ASSERT_EQ(reconstruct_ver_updates.kv[verkey].data, verval); + ASSERT_EQ(reconstruct_ver_updates.kv[verkey].stale_on_update, false); + + const auto& dig = block.parentDigest(); + ASSERT_EQ(dig, empty_digest); + + genesis_digest = block.calculateDigest(1); + } + + { + auto size = uint64_t{100}; + auto block = v4blockchain::detail::Block{size}; + const auto& emdig = block.parentDigest(); + ASSERT_EQ(emdig, empty_digest); + + auto merkle_cat = std::string("merkle_cat"); + auto merkey = std::string("merkey"); + auto merval = std::string("merval"); + auto updates = categorization::Updates{}; + auto merkle_updates = categorization::BlockMerkleUpdates{}; + + merkle_updates.addUpdate("merkey", "merval"); + merkle_updates.addDelete("merdel"); + + block.addDigest(genesis_digest); + updates.add(merkle_cat, std::move(merkle_updates)); + block.addUpdates(updates); + + auto reconstruct_updates = block.getUpdates(); + auto input = reconstruct_updates.categoryUpdates(); + auto reconstruct_mer_updates = std::get(input.kv[merkle_cat]); + ASSERT_EQ(reconstruct_mer_updates.kv[merkey], merval); + ASSERT_EQ(reconstruct_mer_updates.deletes[0], "merdel"); + ASSERT_EQ(reconstruct_mer_updates.deletes.size(), 1); + + const auto& dig = block.parentDigest(); + ASSERT_EQ(dig, genesis_digest); + ASSERT_NE(dig, empty_digest); + } +} + +TEST(v4_block, buffer_size_for_updates) { + // New blocks should have empty digest + concord::util::digest::BlockDigest empty_digest; + for (auto& d : empty_digest) { + d = 0; + } + + auto block_data = std::string("block"); + + v4blockchain::detail::Block block; + ASSERT_EQ(block.getBuffer().size(), v4blockchain::detail::Block::HEADER_SIZE); + auto imm_cat = std::string("immuatables"); + auto immkey = std::string("immkey"); + auto immval = std::string("immval"); + auto updates = categorization::Updates{}; + auto imm_updates = categorization::ImmutableUpdates{}; + imm_updates.addUpdate("immkey", categorization::ImmutableUpdates::ImmutableValue{"immval", {"1", "2", "33"}}); + updates.add(imm_cat, std::move(imm_updates)); + + auto versioned_cat = std::string("versioned"); + auto verkey = std::string("verkey"); + auto verval = std::string("verval"); + auto ver_updates = categorization::VersionedUpdates{}; + ver_updates.addUpdate("verkey", "verval"); + updates.add(versioned_cat, std::move(ver_updates)); + + block.addUpdates(updates); + + std::vector updates_buffer; + concord::kvbc::categorization::serialize(updates_buffer, updates.categoryUpdates()); + ASSERT_EQ(block.getBuffer().size(), v4blockchain::detail::Block::HEADER_SIZE + updates_buffer.size()); + + const auto& dig = block.parentDigest(); + ASSERT_EQ(dig, empty_digest); + + auto dig1 = v4blockchain::detail::Block::calculateDigest(1, block_data.c_str(), block_data.size()); + block.addDigest(dig1); + ASSERT_EQ(block.getBuffer().size(), v4blockchain::detail::Block::HEADER_SIZE + updates_buffer.size()); + + auto reconstruct_updates = block.getUpdates(); + auto input = reconstruct_updates.categoryUpdates(); + ASSERT_EQ(input.kv.count(versioned_cat), 1); + ASSERT_EQ(input.kv.count(imm_cat), 1); + auto reconstruct_imm_updates = std::get(input.kv[imm_cat]); + ASSERT_EQ(reconstruct_imm_updates.kv[immkey].data, immval); + std::vector v = {"1", "2", "33"}; + ASSERT_EQ(reconstruct_imm_updates.kv[immkey].tags, v); + + auto reconstruct_ver_updates = std::get(input.kv[versioned_cat]); + ASSERT_EQ(reconstruct_ver_updates.kv[verkey].data, verval); + ASSERT_EQ(reconstruct_ver_updates.kv[verkey].stale_on_update, false); + + const auto& dig2 = block.parentDigest(); + ASSERT_EQ(dig1, dig2); +} + +} // end namespace + +int main(int argc, char** argv) { + InitGoogleTest(&argc, argv); + return RUN_ALL_TESTS(); +} diff --git a/kvbc/test/v4blockchain/categories_test.cpp b/kvbc/test/v4blockchain/categories_test.cpp new file mode 100644 index 0000000000..f13b1a3be0 --- /dev/null +++ b/kvbc/test/v4blockchain/categories_test.cpp @@ -0,0 +1,140 @@ +// Concord +// +// Copyright (c) 2020 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 subcomponent's license, as noted in the LICENSE +// file. + +#include "gtest/gtest.h" +#include "gmock/gmock.h" +#include "v4blockchain/v4_blockchain.h" +#include +#include +#include +#include +#include +#include "storage/test/storage_test_common.h" +#include "endianness.hpp" +#include "v4blockchain/detail/column_families.h" +#include "categorization/db_categories.h" +#include "categorized_kvbc_msgs.cmf.hpp" +#include "v4blockchain/detail/categories.h" + +using concord::storage::rocksdb::NativeClient; +using namespace concord::kvbc; +using namespace ::testing; + +namespace { + +class v4_kvbc : public Test { + protected: + void SetUp() override { + destroyDb(); + db = TestRocksDb::createNative(); + categories = std::map{ + {"merkle", categorization::CATEGORY_TYPE::block_merkle}, + {"versioned", categorization::CATEGORY_TYPE::versioned_kv}, + {"versioned_2", categorization::CATEGORY_TYPE::versioned_kv}, + {"immutable", categorization::CATEGORY_TYPE::immutable}}; + } + + void TearDown() override { destroyDb(); } + + void destroyDb() { + db.reset(); + ASSERT_EQ(0, db.use_count()); + cleanup(); + } + + protected: + std::map categories; + std::shared_ptr db; +}; + +TEST_F(v4_kvbc, creation) { + v4blockchain::detail::Categories category_mapping{db, categories}; + + ASSERT_TRUE(db->hasColumnFamily(v4blockchain::detail::CATEGORIES_CF)); + + for (const auto& [k, v] : categories) { + ASSERT_EQ(category_mapping.categoryPrefix(k).size(), 1); + ASSERT_EQ(category_mapping.categoryType(k), v); + } +} + +TEST_F(v4_kvbc, load) { + std::unordered_map init; + std::unordered_map load; + { + v4blockchain::detail::Categories category_mapping{db, categories}; + + ASSERT_TRUE(db->hasColumnFamily(v4blockchain::detail::CATEGORIES_CF)); + init = category_mapping.prefixMap(); + for (const auto& [k, v] : categories) { + ASSERT_EQ(category_mapping.categoryPrefix(k).size(), 1); + ASSERT_EQ(category_mapping.categoryType(k), v); + } + } + + { + v4blockchain::detail::Categories category_mapping{db, std::nullopt}; + load = category_mapping.prefixMap(); + ASSERT_TRUE(db->hasColumnFamily(v4blockchain::detail::CATEGORIES_CF)); + for (const auto& [k, v] : categories) { + ASSERT_EQ(category_mapping.categoryPrefix(k).size(), 1); + ASSERT_EQ(category_mapping.categoryType(k), v); + } + } + ASSERT_EQ(init, load); + ASSERT_EQ(init.size(), categories.size()); +} + +TEST_F(v4_kvbc, add_cat_on_load) { + { + v4blockchain::detail::Categories category_mapping{db, categories}; + + ASSERT_TRUE(db->hasColumnFamily(v4blockchain::detail::CATEGORIES_CF)); + + for (const auto& [k, v] : categories) { + ASSERT_EQ(category_mapping.categoryPrefix(k).size(), 1); + ASSERT_EQ(category_mapping.categoryType(k), v); + } + } + + { + std::map reload = categories; + reload["imm2"] = categorization::CATEGORY_TYPE::immutable; + v4blockchain::detail::Categories category_mapping{db, reload}; + + ASSERT_TRUE(db->hasColumnFamily(v4blockchain::detail::CATEGORIES_CF)); + + for (const auto& [k, v] : reload) { + ASSERT_EQ(category_mapping.categoryPrefix(k).size(), 1); + ASSERT_EQ(category_mapping.categoryType(k), v); + } + } +} + +TEST_F(v4_kvbc, throw_if_not_exist) { + { + v4blockchain::detail::Categories category_mapping{db, categories}; + + ASSERT_TRUE(db->hasColumnFamily(v4blockchain::detail::CATEGORIES_CF)); + + ASSERT_THROW(category_mapping.categoryPrefix("imm3").size(), std::out_of_range); + ASSERT_THROW(category_mapping.categoryType("imm3"), std::out_of_range); + } +} + +} // namespace + +int main(int argc, char** argv) { + InitGoogleTest(&argc, argv); + return RUN_ALL_TESTS(); +} diff --git a/kvbc/test/v4blockchain/st_chain_test.cpp b/kvbc/test/v4blockchain/st_chain_test.cpp new file mode 100644 index 0000000000..6c2619f2d5 --- /dev/null +++ b/kvbc/test/v4blockchain/st_chain_test.cpp @@ -0,0 +1,200 @@ +// Concord +// +// Copyright (c) 2020 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 subcomponent's license, as noted in the LICENSE +// file. + +#include "gtest/gtest.h" +#include "gmock/gmock.h" +#include "v4blockchain/v4_blockchain.h" +#include +#include +#include +#include +#include +#include "storage/test/storage_test_common.h" +#include "endianness.hpp" +#include "v4blockchain/detail/st_chain.h" + +using concord::storage::rocksdb::NativeClient; +using namespace concord::kvbc; +using namespace ::testing; +using namespace concord; + +namespace { + +class v4_kvbc : public Test { + protected: + void SetUp() override { + destroyDb(); + db = TestRocksDb::createNative(); + categories = std::map{ + {"merkle", categorization::CATEGORY_TYPE::block_merkle}, + {"versioned", categorization::CATEGORY_TYPE::versioned_kv}, + {"versioned_2", categorization::CATEGORY_TYPE::versioned_kv}, + {"immutable", categorization::CATEGORY_TYPE::immutable}}; + } + + void TearDown() override { destroyDb(); } + + void destroyDb() { + db.reset(); + ASSERT_EQ(0, db.use_count()); + cleanup(); + } + + protected: + std::map categories; + std::shared_ptr db; +}; + +TEST_F(v4_kvbc, blocks) { + { + auto st_chain = v4blockchain::detail::StChain{db}; + auto id1 = kvbc::BlockId{1}; + auto block1 = std::string{"block1"}; + ASSERT_TRUE(db->hasColumnFamily(v4blockchain::detail::ST_CHAIN_CF)); + ASSERT_FALSE(st_chain.hasBlock(id1)); + { st_chain.addBlock(id1, block1.c_str(), block1.size()); } + + ASSERT_TRUE(st_chain.hasBlock(id1)); + + { + auto write_batch = db->getBatch(); + st_chain.deleteBlock(id1, write_batch); + db->write(std::move(write_batch)); + } + ASSERT_FALSE(st_chain.hasBlock(id1)); + } +} + +TEST_F(v4_kvbc, get_blocks) { + { + auto st_chain = v4blockchain::detail::StChain{db}; + auto id1 = kvbc::BlockId{1}; + auto block1_str = std::string{"block1"}; + auto block1 = v4blockchain::detail::Block(block1_str); + ASSERT_TRUE(db->hasColumnFamily(v4blockchain::detail::ST_CHAIN_CF)); + ASSERT_FALSE(st_chain.getBlock(id1).has_value()); + ASSERT_FALSE(st_chain.getBlockData(id1).has_value()); + { st_chain.addBlock(id1, block1_str.c_str(), block1_str.size()); } + auto opt_block = st_chain.getBlock(id1); + ASSERT_TRUE(opt_block.has_value()); + ASSERT_EQ(opt_block->getBuffer(), block1.getBuffer()); + auto opt_block_data = st_chain.getBlockData(id1); + ASSERT_TRUE(opt_block_data.has_value()); + ASSERT_EQ(*opt_block_data, block1_str); + + { + auto write_batch = db->getBatch(); + st_chain.deleteBlock(id1, write_batch); + db->write(std::move(write_batch)); + } + + ASSERT_FALSE(st_chain.getBlock(id1).has_value()); + ASSERT_FALSE(st_chain.getBlockData(id1).has_value()); + } +} + +TEST_F(v4_kvbc, load_block_id) { + { + auto st_chain = v4blockchain::detail::StChain{db}; + ASSERT_EQ(st_chain.getLastBlockId(), 0); + } + auto id1 = kvbc::BlockId{1}; + auto id5 = kvbc::BlockId{5}; + auto block_buff = std::string{"block1"}; + { + auto st_chain = v4blockchain::detail::StChain{db}; + auto block1 = std::string{"block1"}; + ASSERT_TRUE(db->hasColumnFamily(v4blockchain::detail::ST_CHAIN_CF)); + st_chain.addBlock(id1, block_buff.c_str(), block_buff.size()); + } + + { + auto st_chain = v4blockchain::detail::StChain{db}; + ASSERT_EQ(st_chain.getLastBlockId(), id1); + st_chain.addBlock(id5, block_buff.c_str(), block_buff.size()); + } + + { + auto st_chain = v4blockchain::detail::StChain{db}; + ASSERT_EQ(st_chain.getLastBlockId(), id5); + } +} + +TEST_F(v4_kvbc, update_block_id) { + auto id1 = kvbc::BlockId{1}; + auto id3 = kvbc::BlockId{3}; + auto id5 = kvbc::BlockId{5}; + auto id7 = kvbc::BlockId{7}; + auto block_buff = std::string{"block1"}; + { + auto st_chain = v4blockchain::detail::StChain{db}; + auto block1 = std::string{"block1"}; + ASSERT_TRUE(db->hasColumnFamily(v4blockchain::detail::ST_CHAIN_CF)); + st_chain.addBlock(id1, block_buff.c_str(), block_buff.size()); + } + // delete the single block + { + auto st_chain = v4blockchain::detail::StChain{db}; + ASSERT_EQ(st_chain.getLastBlockId(), id1); + auto write_batch = db->getBatch(); + st_chain.deleteBlock(id1, write_batch); + db->write(std::move(write_batch)); + st_chain.updateLastIdAfterDeletion(id1); + ASSERT_EQ(st_chain.getLastBlockId(), 0); + } + + // load an empty chain and add a new block. + { + auto st_chain = v4blockchain::detail::StChain{db}; + ASSERT_EQ(st_chain.getLastBlockId(), 0); + auto block1 = std::string{"block1"}; + ASSERT_TRUE(db->hasColumnFamily(v4blockchain::detail::ST_CHAIN_CF)); + st_chain.addBlock(id1, block_buff.c_str(), block_buff.size()); + } + + { + auto st_chain = v4blockchain::detail::StChain{db}; + ASSERT_EQ(st_chain.getLastBlockId(), id1); + st_chain.addBlock(id5, block_buff.c_str(), block_buff.size()); + st_chain.updateLastIdAfterDeletion(id5); + ASSERT_EQ(st_chain.getLastBlockId(), id5); + } + + { + auto st_chain = v4blockchain::detail::StChain{db}; + ASSERT_EQ(st_chain.getLastBlockId(), id5); + st_chain.updateLastIdIfBigger(id3); + ASSERT_EQ(st_chain.getLastBlockId(), id5); + st_chain.updateLastIdIfBigger(id7); + ASSERT_EQ(st_chain.getLastBlockId(), id7); + } + // load reads the actual idx from storage + { + auto st_chain = v4blockchain::detail::StChain{db}; + ASSERT_EQ(st_chain.getLastBlockId(), id5); + st_chain.updateLastIdAfterDeletion(id3); + ASSERT_EQ(st_chain.getLastBlockId(), id5); + auto write_batch = db->getBatch(); + st_chain.deleteBlock(id5, write_batch); + db->write(std::move(write_batch)); + st_chain.updateLastIdAfterDeletion(id5); + ASSERT_EQ(st_chain.getLastBlockId(), id1); + } +} + +} // namespace + +int main(int argc, char** argv) { + InitGoogleTest(&argc, argv); + return RUN_ALL_TESTS(); +} diff --git a/kvbc/test/v4blockchain/v4_blockchain_test.cpp b/kvbc/test/v4blockchain/v4_blockchain_test.cpp new file mode 100644 index 0000000000..c429d352a7 --- /dev/null +++ b/kvbc/test/v4blockchain/v4_blockchain_test.cpp @@ -0,0 +1,1913 @@ +// Concord +// +// Copyright (c) 2020 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 subcomponent's license, as noted in the LICENSE +// file. + +#include "gtest/gtest.h" +#include "gmock/gmock.h" +#include "v4blockchain/v4_blockchain.h" +#include +#include +#include +#include +#include +#include +#include +#include +#include "storage/test/storage_test_common.h" +#include "endianness.hpp" +#include "v4blockchain/detail/column_families.h" +#include "categorization/db_categories.h" +#include "block_metadata.hpp" +#include "kvbc_key_types.hpp" + +using concord::storage::rocksdb::NativeClient; +using namespace concord::kvbc; +using namespace ::testing; + +namespace { + +class v4_kvbc : public Test { + protected: + void SetUp() override { + destroyDb(); + db = TestRocksDb::createNative(); + } + + void TearDown() override { destroyDb(); } + + void destroyDb() { + db.reset(); + ASSERT_EQ(0, db.use_count()); + cleanup(); + } + void create_blocks(uint64_t num_blocks, + uint32_t& num_merkle_each, + uint32_t& num_versioned_each, + uint32_t& num_immutable_each) { + std::map cat_map{ + {"merkle", categorization::CATEGORY_TYPE::block_merkle}, + {"versioned", categorization::CATEGORY_TYPE::versioned_kv}, + {"immutable", categorization::CATEGORY_TYPE::immutable}}; + v4blockchain::KeyValueBlockchain blockchain{db, true, cat_map}; + std::mt19937 rgen; + std::uniform_int_distribution dist(10, 100); + num_merkle_each = dist(rgen); + num_versioned_each = dist(rgen); + num_immutable_each = dist(rgen); + // Keys are: + // _key__ + // Values are: + // _value__ + for (uint64_t blk = 1; blk <= num_blocks; ++blk) { + categorization::Updates updates; + + categorization::BlockMerkleUpdates merkle_updates; + for (uint32_t kid = 1; kid <= num_merkle_each; ++kid) { + std::string key = "merkle_key_" + std::to_string(blk) + "_" + std::to_string(kid); + std::string val = "merkle_value_" + std::to_string(blk) + "_" + std::to_string(kid); + merkle_updates.addUpdate(std::move(key), std::move(val)); + } + updates.add("merkle", std::move(merkle_updates)); + + categorization::VersionedUpdates ver_updates; + ver_updates.calculateRootHash(false); + for (uint32_t kid = 1; kid <= num_versioned_each; ++kid) { + std::string key = "versioned_key_" + std::to_string(blk) + "_" + std::to_string(kid); + std::string val = "versioned_value_" + std::to_string(blk) + "_" + std::to_string(kid); + ver_updates.addUpdate(std::move(key), std::move(val)); + } + updates.add("versioned", std::move(ver_updates)); + + categorization::ImmutableUpdates immutable_updates; + for (uint32_t kid = 1; kid <= num_immutable_each; ++kid) { + std::string key = "immutable_key_" + std::to_string(blk) + "_" + std::to_string(kid); + std::string val = "immutable_value_" + std::to_string(blk) + "_" + std::to_string(kid); + immutable_updates.addUpdate(std::move(key), {std::move(val), {std::to_string(blk), std::to_string(kid)}}); + } + updates.add("immutable", std::move(immutable_updates)); + ASSERT_EQ(blockchain.add(std::move(updates)), (BlockId)blk); + } + } + + void add_deletes_to_blocks( + uint64_t& tot_num_blocks, + uint32_t num_merkle_each, + uint32_t num_versioned_each, + uint32_t num_immutable_each, + std::function const& key_deletion_filter) { + std::map cat_map{ + {"merkle", categorization::CATEGORY_TYPE::block_merkle}, + {"versioned", categorization::CATEGORY_TYPE::versioned_kv}, + {"immutable", categorization::CATEGORY_TYPE::immutable}}; + v4blockchain::KeyValueBlockchain blockchain{db, true, cat_map}; + // Keys are: + // _key__ + // Values are: + // _value__ + uint64_t new_tot_num_blocks = tot_num_blocks; + for (uint64_t blk = 1; blk <= tot_num_blocks; ++blk) { + categorization::Updates updates; + + categorization::BlockMerkleUpdates merkle_updates; + bool is_delete_added = false; + for (uint32_t kid = 1; kid <= num_merkle_each; ++kid) { + std::string key = "merkle_key_" + std::to_string(blk) + "_" + std::to_string(kid); + if (key_deletion_filter(key, blk, kid, num_merkle_each)) { + merkle_updates.addDelete(std::move(key)); + is_delete_added = true; + } + } + if (is_delete_added) { + updates.add("merkle", std::move(merkle_updates)); + } + is_delete_added = false; + + categorization::VersionedUpdates ver_updates; + ver_updates.calculateRootHash(false); + for (uint32_t kid = 1; kid <= num_versioned_each; ++kid) { + std::string key = "versioned_key_" + std::to_string(blk) + "_" + std::to_string(kid); + if (key_deletion_filter(key, blk, kid, num_versioned_each)) { + ver_updates.addDelete(std::move(key)); + is_delete_added = true; + } + } + if (is_delete_added) { + updates.add("versioned", std::move(ver_updates)); + } + if (!updates.empty()) { + ASSERT_EQ(blockchain.add(std::move(updates)), (BlockId)(new_tot_num_blocks + 1)); + new_tot_num_blocks++; + } + } + tot_num_blocks = new_tot_num_blocks; + } + + protected: + std::shared_ptr db; +}; + +// Add a block which contains updates per category. +// Each category handles its updates and returs an output which goes to the block structure. +// The block structure is then inserted into the DB. +// we test that the block that is written to DB contains the expected data. +TEST_F(v4_kvbc, creation) { + v4blockchain::KeyValueBlockchain blockchain{ + db, + true, + std::map{{"merkle", categorization::CATEGORY_TYPE::block_merkle}, + {"versioned", categorization::CATEGORY_TYPE::versioned_kv}, + {"versioned_2", categorization::CATEGORY_TYPE::versioned_kv}, + {"immutable", categorization::CATEGORY_TYPE::immutable}}}; + + ASSERT_TRUE(db->hasColumnFamily(v4blockchain::detail::BLOCKS_CF)); + ASSERT_TRUE(db->hasColumnFamily(v4blockchain::detail::ST_CHAIN_CF)); + ASSERT_TRUE(db->hasColumnFamily(v4blockchain::detail::LATEST_KEYS_CF)); +} + +TEST_F(v4_kvbc, add_blocks) { + concord::util::digest::BlockDigest empty_digest; + for (auto& d : empty_digest) { + d = 0; + } + { + v4blockchain::KeyValueBlockchain blockchain{ + db, + true, + std::map{ + {"merkle", categorization::CATEGORY_TYPE::block_merkle}, + {"versioned", categorization::CATEGORY_TYPE::versioned_kv}, + {"versioned_2", categorization::CATEGORY_TYPE::versioned_kv}, + {"immutable", categorization::CATEGORY_TYPE::immutable}, + {categorization::kConcordInternalCategoryId, categorization::CATEGORY_TYPE::versioned_kv}}}; + // Add block1 + { + categorization::Updates updates; + categorization::BlockMerkleUpdates merkle_updates; + merkle_updates.addUpdate("merkle_key1", "merkle_value1"); + merkle_updates.addUpdate("merkle_key2", "merkle_value2"); + updates.add("merkle", std::move(merkle_updates)); + + categorization::VersionedUpdates ver_updates; + ver_updates.calculateRootHash(true); + ver_updates.addUpdate("ver_key1", "ver_val1"); + ver_updates.addUpdate("ver_key2", categorization::VersionedUpdates::Value{"ver_val2", true}); + updates.add("versioned", std::move(ver_updates)); + ASSERT_EQ(blockchain.add(std::move(updates)), (BlockId)1); + } + // Add block2 + { + categorization::Updates updates; + categorization::BlockMerkleUpdates merkle_updates; + merkle_updates.addUpdate("merkle_key3", "merkle_value3"); + merkle_updates.addDelete("merkle_key1"); + updates.add("merkle", std::move(merkle_updates)); + + categorization::VersionedUpdates ver_updates; + ver_updates.calculateRootHash(false); + ver_updates.addUpdate("ver_key3", "ver_val3"); + ver_updates.addDelete("ver_key2"); + updates.add("versioned", std::move(ver_updates)); + + categorization::VersionedUpdates ver_updates_2; + ver_updates_2.calculateRootHash(false); + ver_updates_2.addUpdate("ver_key4", "ver_val4"); + updates.add("versioned_2", std::move(ver_updates_2)); + + categorization::ImmutableUpdates immutable_updates; + immutable_updates.addUpdate("immutable_key2", {"immutable_val2", {"1", "2"}}); + updates.add("immutable", std::move(immutable_updates)); + ASSERT_EQ(blockchain.add(std::move(updates)), (BlockId)2); + } + } + + { + v4blockchain::KeyValueBlockchain blockchain{ + db, + true, + std::map{ + {"merkle", categorization::CATEGORY_TYPE::block_merkle}, + {"versioned", categorization::CATEGORY_TYPE::versioned_kv}, + {"versioned_2", categorization::CATEGORY_TYPE::versioned_kv}, + {"immutable", categorization::CATEGORY_TYPE::immutable}, + {categorization::kConcordInternalCategoryId, categorization::CATEGORY_TYPE::versioned_kv}}}; + + // Add block3 + { + categorization::Updates updates; + categorization::BlockMerkleUpdates merkle_updates; + merkle_updates.addUpdate("merkle_key4", "merkle_value4"); + merkle_updates.addDelete("merkle_key4"); + updates.add("merkle", std::move(merkle_updates)); + + categorization::VersionedUpdates ver_updates; + ver_updates.calculateRootHash(false); + ver_updates.addUpdate("ver_key4", "ver_val4"); + ver_updates.addDelete("ver_key4"); + updates.add("versioned", std::move(ver_updates)); + + categorization::VersionedUpdates ver_updates_2; + ver_updates_2.calculateRootHash(false); + ver_updates_2.addUpdate("ver_key5", "ver_val5"); + updates.add("versioned_2", std::move(ver_updates_2)); + + categorization::ImmutableUpdates immutable_updates; + immutable_updates.addUpdate("immutable_key20", {"immutable_val20", {"1", "2"}}); + updates.add("immutable", std::move(immutable_updates)); + ASSERT_EQ(blockchain.add(std::move(updates)), (BlockId)3); + } + + // get block 1 from DB and test it + { + auto block1_db_val = blockchain.getBlockchain().getBlockData(1); + ASSERT_TRUE(block1_db_val.has_value()); + + auto block1 = v4blockchain::detail::Block(*block1_db_val); + ASSERT_EQ(block1.parentDigest(), empty_digest); + auto updates_info = block1.getUpdates().categoryUpdates(); + // Get the merkle updates output of block1 + auto merkle_variant = updates_info.kv["merkle"]; + auto merkle_update = std::get(merkle_variant); + ASSERT_EQ(merkle_update.kv.size(), 2); + ASSERT_EQ(merkle_update.kv["merkle_key1"], "merkle_value1"); + ASSERT_EQ(merkle_update.kv["merkle_key2"], "merkle_value2"); + ASSERT_EQ(merkle_update.deletes.size(), 0); + + auto ver_variant = updates_info.kv["versioned"]; + auto ver_out1 = std::get(ver_variant); + ASSERT_EQ(ver_out1.kv.size(), 2); + ASSERT_EQ(ver_out1.kv["ver_key1"].data, "ver_val1"); + ASSERT_EQ(ver_out1.kv["ver_key1"].stale_on_update, false); + ASSERT_EQ(ver_out1.kv["ver_key2"].data, "ver_val2"); + ASSERT_EQ(ver_out1.kv["ver_key2"].stale_on_update, true); + } + } +} + +TEST_F(v4_kvbc, add_and_read_blocks) { + v4blockchain::KeyValueBlockchain blockchain{ + db, + true, + std::map{ + {"merkle", categorization::CATEGORY_TYPE::block_merkle}, + {"versioned", categorization::CATEGORY_TYPE::versioned_kv}, + {"versioned_2", categorization::CATEGORY_TYPE::versioned_kv}, + {"immutable", categorization::CATEGORY_TYPE::immutable}, + {categorization::kConcordInternalCategoryId, categorization::CATEGORY_TYPE::versioned_kv}}}; + std::string no_flags = {0}; + std::string stale_on_update_flag = {1}; + // Add block1 and read + auto imm_val1 = categorization::ImmutableValueUpdate{"immutable_val20", {"1", "2"}}; + auto ver_val = categorization::ValueWithFlags{"ver_val", true}; + { + std::string out_ts; + uint64_t block_version = 1; + auto block_version_str = v4blockchain::detail::Blockchain::generateKey(block_version); + categorization::Updates updates; + categorization::BlockMerkleUpdates merkle_updates; + merkle_updates.addUpdate("merkle_key1", "merkle_value1"); + updates.add("merkle", std::move(merkle_updates)); + + categorization::ImmutableUpdates immutable_updates; + immutable_updates.addUpdate("immutable_key20", {"immutable_val20", {"1", "2"}}); + updates.add("immutable", std::move(immutable_updates)); + + categorization::VersionedUpdates ver_updates; + ver_updates.calculateRootHash(true); + ver_updates.addUpdate("ver_key2", categorization::VersionedUpdates::Value{"ver_val", true}); + updates.add("versioned", std::move(ver_updates)); + + // Save serialization of the updates + std::vector updates_buffer; + concord::kvbc::categorization::serialize(updates_buffer, updates.categoryUpdates()); + + ASSERT_EQ(blockchain.add(std::move(updates)), (BlockId)1); + auto opt_block = blockchain.getBlockchain().getBlockData(1); + ASSERT_TRUE(opt_block.has_value()); + + std::vector updates_from_storage{(*opt_block).begin() + v4blockchain::detail::Block::HEADER_SIZE, + (*opt_block).end()}; + + ASSERT_EQ(updates_buffer, updates_from_storage); + + // Get keys from latest + // Merkle + auto val = db->get(v4blockchain::detail::LATEST_KEYS_CF, + blockchain.getLatestKeys().getCategoryPrefix("merkle") + "merkle_key1", + block_version_str, + &out_ts); + ASSERT_TRUE(val.has_value()); + ASSERT_EQ(*val, "merkle_value1" + no_flags); + ASSERT_EQ(out_ts, block_version_str); + auto iout_ts = concordUtils::fromBigEndianBuffer(out_ts.data()); + ASSERT_EQ(1, iout_ts); + + // Immutable + // without category prefix + out_ts.clear(); + val = db->get(v4blockchain::detail::LATEST_KEYS_CF, std::string("immutable_key20"), block_version_str, &out_ts); + ASSERT_FALSE(val.has_value()); + + // get key1 updated value of this timestamp + val = db->get(v4blockchain::detail::LATEST_KEYS_CF, + blockchain.getLatestKeys().getCategoryPrefix("immutable") + "immutable_key20", + block_version_str, + &out_ts); + ASSERT_TRUE(val.has_value()); + + ASSERT_EQ(*val, imm_val1.data + stale_on_update_flag); + ASSERT_EQ(out_ts, block_version_str); + iout_ts = concordUtils::fromBigEndianBuffer(out_ts.data()); + ASSERT_EQ(1, iout_ts); + out_ts.clear(); + + // Versioned + val = db->get(v4blockchain::detail::LATEST_KEYS_CF, + blockchain.getLatestKeys().getCategoryPrefix("versioned") + "ver_key2", + block_version_str, + &out_ts); + ASSERT_TRUE(val.has_value()); + + ASSERT_EQ(*val, ver_val.data + std::string(1, v4blockchain::detail::LatestKeys::STALE_ON_UPDATE[0])); + ASSERT_EQ(out_ts, block_version_str); + iout_ts = concordUtils::fromBigEndianBuffer(out_ts.data()); + ASSERT_EQ(1, iout_ts); + out_ts.clear(); + } +} + +TEST_F(v4_kvbc, trim_history_get_block_sequence_number) { + v4blockchain::KeyValueBlockchain blockchain{ + db, + true, + std::map{ + {"merkle", categorization::CATEGORY_TYPE::block_merkle}, + {"versioned", categorization::CATEGORY_TYPE::versioned_kv}, + {"versioned_2", categorization::CATEGORY_TYPE::versioned_kv}, + {"immutable", categorization::CATEGORY_TYPE::immutable}, + {categorization::kConcordInternalCategoryId, categorization::CATEGORY_TYPE::versioned_kv}}}; + { + categorization::Updates updates; + ASSERT_EQ(blockchain.getBlockSequenceNumber(updates), 0); + } + { + categorization::Updates updates; + categorization::VersionedUpdates ver_updates; + ver_updates.addUpdate("ver_key2", categorization::VersionedUpdates::Value{"ver_val", true}); + updates.add(concord::kvbc::categorization::kConcordInternalCategoryId, std::move(ver_updates)); + ASSERT_EQ(blockchain.getBlockSequenceNumber(updates), 0); + } + + { + uint64_t id = 10; + auto sid = concordUtils::toBigEndianStringBuffer(id); + categorization::Updates updates; + categorization::VersionedUpdates ver_updates; + ver_updates.addUpdate(std::string(1, concord::kvbc::IBlockMetadata::kBlockMetadataKey), + categorization::VersionedUpdates::Value{sid, true}); + updates.add(concord::kvbc::categorization::kConcordInternalCategoryId, std::move(ver_updates)); + ASSERT_EQ(blockchain.getBlockSequenceNumber(updates), id); + } +} + +TEST_F(v4_kvbc, check_if_trim_history_is_needed) { + v4blockchain::KeyValueBlockchain blockchain{ + db, + true, + std::map{ + {"merkle", categorization::CATEGORY_TYPE::block_merkle}, + {"versioned", categorization::CATEGORY_TYPE::versioned_kv}, + {"versioned_2", categorization::CATEGORY_TYPE::versioned_kv}, + {"immutable", categorization::CATEGORY_TYPE::immutable}, + {categorization::kConcordInternalCategoryId, categorization::CATEGORY_TYPE::versioned_kv}}}; + { + categorization::Updates updates; + ASSERT_EQ(blockchain.markHistoryForGarbageCollectionIfNeeded(updates), 0); + ASSERT_EQ(blockchain.gc_counter, 0); + } + { + // No meta data sn key + uint64_t id = 10; + auto sid = concordUtils::toBigEndianStringBuffer(id); + categorization::Updates updates; + categorization::VersionedUpdates ver_updates; + ver_updates.addUpdate(std::string(1, 0x1), categorization::VersionedUpdates::Value{sid, true}); + updates.add(concord::kvbc::categorization::kConcordInternalCategoryId, std::move(ver_updates)); + ASSERT_EQ(blockchain.markHistoryForGarbageCollectionIfNeeded(updates), 0); + ASSERT_EQ(blockchain.gc_counter, 0); + } + + // First real set, sn should be 10 after this + { + uint64_t id = 10; + auto sid = concordUtils::toBigEndianStringBuffer(id); + categorization::Updates updates; + categorization::VersionedUpdates ver_updates; + ver_updates.addUpdate(std::string(1, concord::kvbc::IBlockMetadata::kBlockMetadataKey), + categorization::VersionedUpdates::Value{sid, true}); + updates.add(concord::kvbc::categorization::kConcordInternalCategoryId, std::move(ver_updates)); + + ASSERT_EQ(blockchain.markHistoryForGarbageCollectionIfNeeded(updates), id); + blockchain.setLastBlockSequenceNumber(id); + ASSERT_EQ(blockchain.gc_counter, 0); + } + + // try now with lower sn + { + uint64_t id = 9; + auto sid = concordUtils::toBigEndianStringBuffer(id); + categorization::Updates updates; + categorization::VersionedUpdates ver_updates; + ver_updates.addUpdate(std::string(1, concord::kvbc::IBlockMetadata::kBlockMetadataKey), + categorization::VersionedUpdates::Value{sid, true}); + updates.add(concord::kvbc::categorization::kConcordInternalCategoryId, std::move(ver_updates)); + + ASSERT_EQ(blockchain.markHistoryForGarbageCollectionIfNeeded(updates), 0); + ASSERT_EQ(blockchain.gc_counter, 0); + } + + // sn was incremented + { + uint64_t id = 11; + auto sid = concordUtils::toBigEndianStringBuffer(id); + categorization::Updates updates; + categorization::VersionedUpdates ver_updates; + ver_updates.addUpdate(std::string(1, concord::kvbc::IBlockMetadata::kBlockMetadataKey), + categorization::VersionedUpdates::Value{sid, true}); + updates.add(concord::kvbc::categorization::kConcordInternalCategoryId, std::move(ver_updates)); + + ASSERT_EQ(blockchain.markHistoryForGarbageCollectionIfNeeded(updates), id); + blockchain.setLastBlockSequenceNumber(id); + ASSERT_EQ(blockchain.gc_counter, 1); + } + blockchain.checkpointInProcess(true); + + // sn was incremented but checkpoint in process + { + uint64_t id = 15; + auto sid = concordUtils::toBigEndianStringBuffer(id); + categorization::Updates updates; + categorization::VersionedUpdates ver_updates; + ver_updates.addUpdate(std::string(1, concord::kvbc::IBlockMetadata::kBlockMetadataKey), + categorization::VersionedUpdates::Value{sid, true}); + updates.add(concord::kvbc::categorization::kConcordInternalCategoryId, std::move(ver_updates)); + + ASSERT_EQ(blockchain.markHistoryForGarbageCollectionIfNeeded(updates), 0); + ASSERT_EQ(blockchain.gc_counter, 1); + } + + blockchain.checkpointInProcess(false); + + // sn was incremented and no checkpoint in process + { + uint64_t id = 16; + auto sid = concordUtils::toBigEndianStringBuffer(id); + categorization::Updates updates; + categorization::VersionedUpdates ver_updates; + ver_updates.addUpdate(std::string(1, concord::kvbc::IBlockMetadata::kBlockMetadataKey), + categorization::VersionedUpdates::Value{sid, true}); + updates.add(concord::kvbc::categorization::kConcordInternalCategoryId, std::move(ver_updates)); + + ASSERT_EQ(blockchain.markHistoryForGarbageCollectionIfNeeded(updates), id); + blockchain.setLastBlockSequenceNumber(id); + ASSERT_EQ(blockchain.gc_counter, 2); + } +} + +TEST_F(v4_kvbc, delete_last_reachable) { + v4blockchain::KeyValueBlockchain blockchain{ + db, + true, + std::map{ + {"merkle", categorization::CATEGORY_TYPE::block_merkle}, + {"versioned", categorization::CATEGORY_TYPE::versioned_kv}, + {"versioned_2", categorization::CATEGORY_TYPE::versioned_kv}, + {"immutable", categorization::CATEGORY_TYPE::immutable}, + {categorization::kConcordInternalCategoryId, categorization::CATEGORY_TYPE::versioned_kv}}}; + // Can't delete from empty blockchain + ASSERT_DEATH(blockchain.deleteLastReachableBlock(), ""); + auto key1 = std::string("merkle_key1"); + auto key2 = std::string("merkle_key2"); + auto val1 = std::string("merkle_value1"); + auto val2 = std::string("merkle_value2"); + auto val_updated = std::string("merkle_value_updated"); + + auto ver_key1 = std::string("ver_key1"); + auto ver_key2 = std::string("ver_key2"); + auto ver_val1 = std::string("ver_val1"); + auto ver_val2 = std::string("ver_val2"); + auto ver_val_updated = std::string("ver_value_updated"); + + // Add block 1 with 2 keys + { + categorization::Updates updates; + // Merkle + categorization::BlockMerkleUpdates merkle_updates; + merkle_updates.addUpdate(std::string("merkle_key1"), std::string("merkle_value1")); + merkle_updates.addUpdate(std::string("merkle_key2"), std::string("merkle_value2")); + updates.add("merkle", std::move(merkle_updates)); + // Versioned + categorization::VersionedUpdates ver_updates; + ver_updates.calculateRootHash(false); + ver_updates.addUpdate("ver_key1", "ver_val1"); + ver_updates.addUpdate("ver_key2", categorization::VersionedUpdates::Value{"ver_val2", true}); + updates.add("versioned", std::move(ver_updates)); + + auto id = blockchain.add(std::move(updates)); + ASSERT_EQ(id, 1); + + std::string out_ts; + auto block_id1_str = v4blockchain::detail::Blockchain::generateKey(id); + auto val = db->get(v4blockchain::detail::LATEST_KEYS_CF, + blockchain.getLatestKeys().getCategoryPrefix("merkle") + key1, + block_id1_str, + &out_ts); + ASSERT_TRUE(val.has_value()); + std::string val_db((*val).begin(), (*val).end() - v4blockchain::detail::LatestKeys::FLAGS_SIZE); + ASSERT_EQ(val_db, val1); + + ASSERT_EQ(out_ts, block_id1_str); + auto iout_ts = concordUtils::fromBigEndianBuffer(out_ts.data()); + ASSERT_EQ(id, iout_ts); + out_ts.clear(); + + // check that the value of the versioned category i.e. ValueWithFlags was read from DB succuessfully. + val = db->get(v4blockchain::detail::LATEST_KEYS_CF, + blockchain.getLatestKeys().getCategoryPrefix("versioned") + ver_key1, + block_id1_str, + &out_ts); + ASSERT_TRUE(val.has_value()); + ASSERT_FALSE(v4blockchain::detail::LatestKeys::isStaleOnUpdate(*val)); + { + std::string ver_val_db((*val).begin(), (*val).end() - v4blockchain::detail::LatestKeys::FLAGS_SIZE); + concord::kvbc::categorization::ValueWithFlags db_val_wf = concord::kvbc::categorization::ValueWithFlags{ + ver_val_db, v4blockchain::detail::LatestKeys::isStaleOnUpdate(*val)}; + auto val_wf = concord::kvbc::categorization::ValueWithFlags{"ver_val1", false}; + ASSERT_EQ(val_wf, db_val_wf); + ASSERT_EQ(out_ts, block_id1_str); + iout_ts = concordUtils::fromBigEndianBuffer(out_ts.data()); + ASSERT_EQ(id, iout_ts); + out_ts.clear(); + } + + val = db->get(v4blockchain::detail::LATEST_KEYS_CF, + blockchain.getLatestKeys().getCategoryPrefix("versioned") + ver_key2, + block_id1_str, + &out_ts); + ASSERT_TRUE(val.has_value()); + ASSERT_TRUE(v4blockchain::detail::LatestKeys::isStaleOnUpdate(*val)); + + std::string ver_val_db((*val).begin(), (*val).end() - v4blockchain::detail::LatestKeys::FLAGS_SIZE); + concord::kvbc::categorization::ValueWithFlags db_val_wf = concord::kvbc::categorization::ValueWithFlags{ + ver_val_db, v4blockchain::detail::LatestKeys::isStaleOnUpdate(*val)}; + auto val_wf = concord::kvbc::categorization::ValueWithFlags{"ver_val2", true}; + ASSERT_EQ(val_wf, db_val_wf); + ASSERT_EQ(out_ts, block_id1_str); + iout_ts = concordUtils::fromBigEndianBuffer(out_ts.data()); + ASSERT_EQ(id, iout_ts); + out_ts.clear(); + } + // Can't delete single block + ASSERT_DEATH(blockchain.deleteLastReachableBlock(), ""); + + // Add block 2 where key1 is updated and key2 is deleted. + { + categorization::Updates updates; + categorization::BlockMerkleUpdates merkle_updates; + merkle_updates.addUpdate(std::string("merkle_key1"), std::string("merkle_value_updated")); + merkle_updates.addDelete(std::string("merkle_key2")); + updates.add("merkle", std::move(merkle_updates)); + + // Versioned + categorization::VersionedUpdates ver_updates; + ver_updates.calculateRootHash(false); + ver_updates.addUpdate("ver_key1", "ver_value_updated"); + ver_updates.addDelete("ver_key2"); + updates.add("versioned", std::move(ver_updates)); + + auto id = blockchain.add(std::move(updates)); + ASSERT_EQ(id, 2); + + // Validate Merkle + std::string out_ts; + auto block_id2_str = v4blockchain::detail::Blockchain::generateKey(id); + auto val = db->get(v4blockchain::detail::LATEST_KEYS_CF, + blockchain.getLatestKeys().getCategoryPrefix("merkle") + key1, + block_id2_str, + &out_ts); + ASSERT_TRUE(val.has_value()); + std::string val_db((*val).begin(), (*val).end() - v4blockchain::detail::LatestKeys::FLAGS_SIZE); + ASSERT_EQ(val_db, val_updated); + ASSERT_EQ(out_ts, block_id2_str); + auto iout_ts = concordUtils::fromBigEndianBuffer(out_ts.data()); + ASSERT_EQ(id, iout_ts); + out_ts.clear(); + // Key2 was deleted + val = db->get(v4blockchain::detail::LATEST_KEYS_CF, + blockchain.getLatestKeys().getCategoryPrefix("merkle") + key2, + block_id2_str, + &out_ts); + ASSERT_FALSE(val.has_value()); + + // Validate versioned + val = db->get(v4blockchain::detail::LATEST_KEYS_CF, + blockchain.getLatestKeys().getCategoryPrefix("versioned") + ver_key1, + block_id2_str, + &out_ts); + ASSERT_TRUE(val.has_value()); + ASSERT_FALSE(v4blockchain::detail::LatestKeys::isStaleOnUpdate(*val)); + std::string ver_val_db((*val).begin(), (*val).end() - v4blockchain::detail::LatestKeys::FLAGS_SIZE); + concord::kvbc::categorization::ValueWithFlags db_val_wf = concord::kvbc::categorization::ValueWithFlags{ + ver_val_db, v4blockchain::detail::LatestKeys::isStaleOnUpdate(*val)}; + auto val_wf = concord::kvbc::categorization::ValueWithFlags{"ver_value_updated", false}; + + ASSERT_EQ(val_wf, db_val_wf); + ASSERT_EQ(out_ts, block_id2_str); + iout_ts = concordUtils::fromBigEndianBuffer(out_ts.data()); + ASSERT_EQ(id, iout_ts); + out_ts.clear(); + // Key2 was deleted + val = db->get(v4blockchain::detail::LATEST_KEYS_CF, + blockchain.getLatestKeys().getCategoryPrefix("versioned") + ver_key2, + block_id2_str, + &out_ts); + ASSERT_FALSE(val.has_value()); + } + + // Delete block 2 + + ASSERT_EQ(blockchain.getBlockchain().getLastReachable(), 2); + blockchain.deleteLastReachableBlock(); + ASSERT_EQ(blockchain.getBlockchain().getLastReachable(), 1); + // After deletion, lets try to read the keys using version 2 + // key1 - should get the value from block 1. + // key2 - should be recovered from deletion and get the value from block 1. + { + auto id = 2; + std::string out_ts; + auto block_id2_str = v4blockchain::detail::Blockchain::generateKey(id); + auto val = db->get(v4blockchain::detail::LATEST_KEYS_CF, + blockchain.getLatestKeys().getCategoryPrefix("merkle") + key1, + block_id2_str, + &out_ts); + ASSERT_TRUE(val.has_value()); + std::string val_db((*val).begin(), (*val).end() - v4blockchain::detail::LatestKeys::FLAGS_SIZE); + // we reverted to old value + ASSERT_EQ(val_db, val1); + + ASSERT_EQ(out_ts, block_id2_str); + auto iout_ts = concordUtils::fromBigEndianBuffer(out_ts.data()); + ASSERT_EQ(id, iout_ts); + + val = db->get(v4blockchain::detail::LATEST_KEYS_CF, + blockchain.getLatestKeys().getCategoryPrefix("merkle") + key2, + block_id2_str, + &out_ts); + ASSERT_TRUE(val.has_value()); + std::string val_db_2((*val).begin(), (*val).end() - v4blockchain::detail::LatestKeys::FLAGS_SIZE); + ASSERT_EQ(val_db_2, val2); + + // Validate versioned category + val = db->get(v4blockchain::detail::LATEST_KEYS_CF, + blockchain.getLatestKeys().getCategoryPrefix("versioned") + ver_key1, + block_id2_str, + &out_ts); + ASSERT_TRUE(val.has_value()); + ASSERT_FALSE(v4blockchain::detail::LatestKeys::isStaleOnUpdate(*val)); + { + std::string ver_val_db((*val).begin(), (*val).end() - v4blockchain::detail::LatestKeys::FLAGS_SIZE); + concord::kvbc::categorization::ValueWithFlags db_val_wf = concord::kvbc::categorization::ValueWithFlags{ + ver_val_db, v4blockchain::detail::LatestKeys::isStaleOnUpdate(*val)}; + auto val_wf = concord::kvbc::categorization::ValueWithFlags{"ver_val1", false}; + ASSERT_EQ(val_wf, db_val_wf); + ASSERT_EQ(out_ts, block_id2_str); + iout_ts = concordUtils::fromBigEndianBuffer(out_ts.data()); + ASSERT_EQ(id, iout_ts); + out_ts.clear(); + } + + val = db->get(v4blockchain::detail::LATEST_KEYS_CF, + blockchain.getLatestKeys().getCategoryPrefix("versioned") + ver_key2, + block_id2_str, + &out_ts); + ASSERT_TRUE(val.has_value()); + ASSERT_TRUE(v4blockchain::detail::LatestKeys::isStaleOnUpdate(*val)); + + std::string ver_val_db((*val).begin(), (*val).end() - v4blockchain::detail::LatestKeys::FLAGS_SIZE); + concord::kvbc::categorization::ValueWithFlags db_val_wf = concord::kvbc::categorization::ValueWithFlags{ + ver_val_db, v4blockchain::detail::LatestKeys::isStaleOnUpdate(*val)}; + auto val_wf = concord::kvbc::categorization::ValueWithFlags{"ver_val2", true}; + ASSERT_EQ(val_wf, db_val_wf); + ASSERT_EQ(out_ts, block_id2_str); + iout_ts = concordUtils::fromBigEndianBuffer(out_ts.data()); + ASSERT_EQ(id, iout_ts); + out_ts.clear(); + } + + // Let's add block 2 again and validate the key1 and key2 are updated correctly + // Add block 2 where key1 is updated and key2 is deleted. + { + categorization::Updates updates; + categorization::BlockMerkleUpdates merkle_updates; + merkle_updates.addUpdate(std::string("merkle_key1"), std::string("merkle_value_updated")); + merkle_updates.addDelete(std::string("merkle_key2")); + updates.add("merkle", std::move(merkle_updates)); + + // Versioned + categorization::VersionedUpdates ver_updates; + ver_updates.calculateRootHash(false); + ver_updates.addUpdate("ver_key1", "ver_value_updated"); + ver_updates.addDelete("ver_key2"); + updates.add("versioned", std::move(ver_updates)); + + auto id = blockchain.add(std::move(updates)); + ASSERT_EQ(id, 2); + + // Validate Merkle + std::string out_ts; + auto block_id2_str = v4blockchain::detail::Blockchain::generateKey(id); + auto val = db->get(v4blockchain::detail::LATEST_KEYS_CF, + blockchain.getLatestKeys().getCategoryPrefix("merkle") + key1, + block_id2_str, + &out_ts); + ASSERT_TRUE(val.has_value()); + std::string val_db((*val).begin(), (*val).end() - v4blockchain::detail::LatestKeys::FLAGS_SIZE); + ASSERT_EQ(val_db, val_updated); + ASSERT_EQ(out_ts, block_id2_str); + auto iout_ts = concordUtils::fromBigEndianBuffer(out_ts.data()); + ASSERT_EQ(id, iout_ts); + out_ts.clear(); + // Key2 was deleted + val = db->get(v4blockchain::detail::LATEST_KEYS_CF, + blockchain.getLatestKeys().getCategoryPrefix("merkle") + key2, + block_id2_str, + &out_ts); + ASSERT_FALSE(val.has_value()); + + // Validate versioned + val = db->get(v4blockchain::detail::LATEST_KEYS_CF, + blockchain.getLatestKeys().getCategoryPrefix("versioned") + ver_key1, + block_id2_str, + &out_ts); + ASSERT_TRUE(val.has_value()); + ASSERT_FALSE(v4blockchain::detail::LatestKeys::isStaleOnUpdate(*val)); + std::string ver_val_db((*val).begin(), (*val).end() - v4blockchain::detail::LatestKeys::FLAGS_SIZE); + concord::kvbc::categorization::ValueWithFlags db_val_wf = concord::kvbc::categorization::ValueWithFlags{ + ver_val_db, v4blockchain::detail::LatestKeys::isStaleOnUpdate(*val)}; + auto val_wf = concord::kvbc::categorization::ValueWithFlags{"ver_value_updated", false}; + + ASSERT_EQ(val_wf, db_val_wf); + ASSERT_EQ(out_ts, block_id2_str); + iout_ts = concordUtils::fromBigEndianBuffer(out_ts.data()); + ASSERT_EQ(id, iout_ts); + out_ts.clear(); + // Key2 was deleted + val = db->get(v4blockchain::detail::LATEST_KEYS_CF, + blockchain.getLatestKeys().getCategoryPrefix("versioned") + ver_key2, + block_id2_str, + &out_ts); + ASSERT_FALSE(val.has_value()); + } + ASSERT_EQ(blockchain.getBlockchain().getLastReachable(), 2); +} + +/////////////////STATE TRANSFER////////////////////// +TEST_F(v4_kvbc, has_blocks) { + v4blockchain::KeyValueBlockchain blockchain{ + db, + true, + std::map{ + {"merkle", categorization::CATEGORY_TYPE::block_merkle}, + {"versioned", categorization::CATEGORY_TYPE::versioned_kv}, + {"versioned_2", categorization::CATEGORY_TYPE::versioned_kv}, + {"immutable", categorization::CATEGORY_TYPE::immutable}, + {categorization::kConcordInternalCategoryId, categorization::CATEGORY_TYPE::versioned_kv}}}; + + // Add block1 + { + categorization::Updates updates; + categorization::BlockMerkleUpdates merkle_updates; + merkle_updates.addUpdate("merkle_key1", "merkle_value1"); + merkle_updates.addUpdate("merkle_key2", "merkle_value2"); + updates.add("merkle", std::move(merkle_updates)); + + categorization::VersionedUpdates ver_updates; + ver_updates.calculateRootHash(true); + ver_updates.addUpdate("ver_key1", "ver_val1"); + ver_updates.addUpdate("ver_key2", categorization::VersionedUpdates::Value{"ver_val2", true}); + updates.add("versioned", std::move(ver_updates)); + ASSERT_EQ(blockchain.add(std::move(updates)), (BlockId)1); + } + + ASSERT_TRUE(blockchain.hasBlock(1)); + ASSERT_TRUE(blockchain.getBlockData(1).has_value()); + ASSERT_FALSE(blockchain.hasBlock(2)); + ASSERT_FALSE(blockchain.getBlockData(2).has_value()); + + // add block the the ST chain + auto block = std::string("block"); + blockchain.addBlockToSTChain(2, block.c_str(), block.size(), false); + ASSERT_TRUE(blockchain.hasBlock(2)); + auto st_block = blockchain.getBlockData(2); + ASSERT_TRUE(st_block.has_value()); + ASSERT_EQ(*st_block, block); +} + +TEST_F(v4_kvbc, add_blocks_to_st_chain) { + v4blockchain::KeyValueBlockchain blockchain{ + db, + true, + std::map{ + {"merkle", categorization::CATEGORY_TYPE::block_merkle}, + {"versioned", categorization::CATEGORY_TYPE::versioned_kv}, + {"versioned_2", categorization::CATEGORY_TYPE::versioned_kv}, + {"immutable", categorization::CATEGORY_TYPE::immutable}, + {categorization::kConcordInternalCategoryId, categorization::CATEGORY_TYPE::versioned_kv}}}; + + std::string block_data; + // Add block1 + { + categorization::Updates updates; + categorization::BlockMerkleUpdates merkle_updates; + merkle_updates.addUpdate("merkle_key1", "merkle_value1"); + merkle_updates.addUpdate("merkle_key2", "merkle_value2"); + updates.add("merkle", std::move(merkle_updates)); + + categorization::VersionedUpdates ver_updates; + ver_updates.calculateRootHash(true); + ver_updates.addUpdate("ver_key1", "ver_val1"); + ver_updates.addUpdate("ver_key2", categorization::VersionedUpdates::Value{"ver_val2", true}); + updates.add("versioned", std::move(ver_updates)); + ASSERT_EQ(blockchain.add(std::move(updates)), (BlockId)1); + } + + // Add st block with id 3 i.e. block 2 is missing. + block_data = *(blockchain.getBlockData(1)); + blockchain.addBlockToSTChain(3, block_data.c_str(), block_data.size(), false); + blockchain.addBlockToSTChain(5, block_data.c_str(), block_data.size(), false); + ASSERT_FALSE(blockchain.hasBlock(2)); + ASSERT_TRUE(blockchain.hasBlock(3)); + ASSERT_EQ(blockchain.getLastReachableBlockId(), 1); + blockchain.linkSTChain(); + ASSERT_EQ(blockchain.getLastReachableBlockId(), 1); + // now add block 2 and link will add until block 3 + blockchain.addBlockToSTChain(2, block_data.c_str(), block_data.size(), false); + blockchain.linkSTChain(); + ASSERT_EQ(blockchain.getLastReachableBlockId(), 3); + auto block3 = blockchain.getBlockchain().getBlockData(3); + ASSERT_TRUE(block3.has_value()); +} + +TEST_F(v4_kvbc, add_altot_of_blocks_to_st_chain) { + v4blockchain::KeyValueBlockchain blockchain{ + db, + true, + std::map{ + {"merkle", categorization::CATEGORY_TYPE::block_merkle}, + {"versioned", categorization::CATEGORY_TYPE::versioned_kv}, + {"versioned_2", categorization::CATEGORY_TYPE::versioned_kv}, + {"immutable", categorization::CATEGORY_TYPE::immutable}, + {categorization::kConcordInternalCategoryId, categorization::CATEGORY_TYPE::versioned_kv}}}; + + std::string block_data; + // Add blocks + for (BlockId i = 1; i < 100; ++i) { + categorization::Updates updates; + categorization::BlockMerkleUpdates merkle_updates; + merkle_updates.addUpdate("merkle_key1", "merkle_value1"); + merkle_updates.addUpdate("merkle_key2", "merkle_value2"); + updates.add("merkle", std::move(merkle_updates)); + + categorization::VersionedUpdates ver_updates; + ver_updates.calculateRootHash(true); + ver_updates.addUpdate("ver_key1", "ver_val1"); + ver_updates.addUpdate("ver_key2", categorization::VersionedUpdates::Value{"ver_val2", true}); + updates.add("versioned", std::move(ver_updates)); + ASSERT_EQ(blockchain.add(std::move(updates)), i); + } + ASSERT_EQ(blockchain.getLastReachableBlockId(), 99); + + // Add st block with id 3 i.e. block 2 is missing. + block_data = *(blockchain.getBlockData(1)); + + // try to add with block id less than the last reachable + ASSERT_THROW(blockchain.addBlockToSTChain(3, block_data.c_str(), block_data.size(), false), std::invalid_argument); + for (BlockId i = 100; i < 1000; ++i) { + blockchain.addBlockToSTChain(i, block_data.c_str(), block_data.size(), false); + } + + ASSERT_TRUE(blockchain.hasBlock(999)); + ASSERT_EQ(blockchain.getLastReachableBlockId(), 99); + ASSERT_EQ(blockchain.getStChain().getLastBlockId(), 999); + blockchain.addBlockToSTChain(1000, block_data.c_str(), block_data.size(), true); + ASSERT_EQ(blockchain.getStChain().getLastBlockId(), 0); + ASSERT_EQ(blockchain.getLastReachableBlockId(), 1000); +} + +TEST_F(v4_kvbc, st_link_until) { + v4blockchain::KeyValueBlockchain blockchain{ + db, + true, + std::map{ + {"merkle", categorization::CATEGORY_TYPE::block_merkle}, + {"versioned", categorization::CATEGORY_TYPE::versioned_kv}, + {"versioned_2", categorization::CATEGORY_TYPE::versioned_kv}, + {"immutable", categorization::CATEGORY_TYPE::immutable}, + {categorization::kConcordInternalCategoryId, categorization::CATEGORY_TYPE::versioned_kv}}}; + + std::string block_data; + // Add blocks + for (BlockId i = 1; i < 10; ++i) { + categorization::Updates updates; + categorization::BlockMerkleUpdates merkle_updates; + merkle_updates.addUpdate("merkle_key1", "merkle_value1"); + merkle_updates.addUpdate("merkle_key2", "merkle_value2"); + updates.add("merkle", std::move(merkle_updates)); + + categorization::VersionedUpdates ver_updates; + ver_updates.calculateRootHash(true); + ver_updates.addUpdate("ver_key1", "ver_val1"); + ver_updates.addUpdate("ver_key2", categorization::VersionedUpdates::Value{"ver_val2", true}); + updates.add("versioned", std::move(ver_updates)); + ASSERT_EQ(blockchain.add(std::move(updates)), i); + } + ASSERT_EQ(blockchain.getLastReachableBlockId(), 9); + + // Add st block with id 3 i.e. block 2 is missing. + block_data = *(blockchain.getBlockData(1)); + + for (BlockId i = 10; i < 20; ++i) { + blockchain.addBlockToSTChain(i, block_data.c_str(), block_data.size(), false); + } + + ASSERT_FALSE(blockchain.hasBlock(999)); + ASSERT_TRUE(blockchain.hasBlock(15)); + ASSERT_EQ(blockchain.getStChain().getLastBlockId(), 19); + + // we can add until block 19 + ASSERT_EQ(blockchain.linkUntilBlockId(30), 10); + // Chain should be reset + ASSERT_EQ(blockchain.getStChain().getLastBlockId(), 0); + ASSERT_EQ(blockchain.getLastReachableBlockId(), 19); + + for (BlockId i = 20; i < 200; ++i) { + blockchain.addBlockToSTChain(i, block_data.c_str(), block_data.size(), false); + } + ASSERT_TRUE(blockchain.hasBlock(199)); + ASSERT_EQ(blockchain.getStChain().getLastBlockId(), 199); + // Add partial range + ASSERT_EQ(blockchain.linkUntilBlockId(30), 11); + ASSERT_EQ(blockchain.linkUntilBlockId(100), 70); + ASSERT_EQ(blockchain.linkUntilBlockId(199), 99); + // St chain should be empty + ASSERT_EQ(blockchain.getStChain().getLastBlockId(), 0); + ASSERT_EQ(blockchain.getLastReachableBlockId(), 199); +} + +TEST_F(v4_kvbc, parent_digest) { + v4blockchain::KeyValueBlockchain blockchain{ + db, + true, + std::map{ + {"merkle", categorization::CATEGORY_TYPE::block_merkle}, + {"versioned", categorization::CATEGORY_TYPE::versioned_kv}, + {"versioned_2", categorization::CATEGORY_TYPE::versioned_kv}, + {"immutable", categorization::CATEGORY_TYPE::immutable}, + {categorization::kConcordInternalCategoryId, categorization::CATEGORY_TYPE::versioned_kv}}}; + + std::string block_data; + concord::util::digest::BlockDigest empty_digest; + for (auto& d : empty_digest) { + d = 0; + } + // block1 + { + categorization::Updates updates; + categorization::BlockMerkleUpdates merkle_updates; + merkle_updates.addUpdate("merkle_key1", "merkle_value1"); + merkle_updates.addUpdate("merkle_key2", "merkle_value2"); + updates.add("merkle", std::move(merkle_updates)); + ASSERT_EQ(blockchain.add(std::move(updates)), 1); + ASSERT_EQ(blockchain.parentDigest(0), empty_digest); + // also geneis has empty parent digest + ASSERT_EQ(blockchain.parentDigest(1), empty_digest); + } + + // block2 + { + categorization::Updates updates; + categorization::VersionedUpdates ver_updates; + ver_updates.calculateRootHash(true); + ver_updates.addUpdate("ver_key1", "ver_val1"); + ver_updates.addUpdate("ver_key2", categorization::VersionedUpdates::Value{"ver_val2", true}); + updates.add("versioned", std::move(ver_updates)); + ASSERT_EQ(blockchain.add(std::move(updates)), 2); + auto block_data = blockchain.getBlockData(1); + ASSERT_TRUE(block_data.has_value()); + auto digest = v4blockchain::detail::Block::calculateDigest(1, block_data->c_str(), block_data->size()); + ASSERT_EQ(blockchain.parentDigest(2), digest); + + auto block_data2 = blockchain.getBlockData(2); + blockchain.addBlockToSTChain(8, block_data2->c_str(), block_data2->size(), false); + ASSERT_DEATH(blockchain.parentDigest(3), ""); + ASSERT_EQ(blockchain.parentDigest(8), digest); + } +} + +TEST_F(v4_kvbc, prun_on_st) { + v4blockchain::KeyValueBlockchain blockchain{ + db, + true, + std::map{ + {"merkle", categorization::CATEGORY_TYPE::block_merkle}, + {"versioned", categorization::CATEGORY_TYPE::versioned_kv}, + {"versioned_2", categorization::CATEGORY_TYPE::versioned_kv}, + {"immutable", categorization::CATEGORY_TYPE::immutable}, + {categorization::kConcordInternalCategoryId, categorization::CATEGORY_TYPE::versioned_kv}}}; + + // Add blocks + BlockId until = 100; + for (BlockId i = 1; i <= until; ++i) { + categorization::Updates updates; + categorization::BlockMerkleUpdates merkle_updates; + merkle_updates.addUpdate("merkle_key1", "merkle_value1"); + merkle_updates.addUpdate("merkle_key2", "merkle_value2"); + updates.add("merkle", std::move(merkle_updates)); + + categorization::VersionedUpdates ver_updates; + ver_updates.calculateRootHash(true); + ver_updates.addUpdate("ver_key1", "ver_val1"); + ver_updates.addUpdate("ver_key2", categorization::VersionedUpdates::Value{"ver_val2", true}); + updates.add("versioned", std::move(ver_updates)); + // Add genesis for pruning + if (i % 20 == 0) { + const auto stale_on_update = true; + updates.addCategoryIfNotExisting(categorization::kConcordInternalCategoryId); + updates.appendKeyValue( + categorization::kConcordInternalCategoryId, + std::string{concord::kvbc::keyTypes::genesis_block_key}, + categorization::VersionedUpdates::ValueType{concordUtils::toBigEndianStringBuffer(i - 10), stale_on_update}); + } + ASSERT_EQ(blockchain.add(std::move(updates)), i); + auto block_data = *(blockchain.getBlockData(i)); + blockchain.addBlockToSTChain(until + i, block_data.c_str(), block_data.size(), false); + } + ASSERT_EQ(blockchain.getLastReachableBlockId(), 100); + ASSERT_EQ(blockchain.getStChain().getLastBlockId(), 200); + + // range with the an update that contains genesis = 10; + ASSERT_EQ(blockchain.linkUntilBlockId(130), 30); + ASSERT_EQ(blockchain.getLastReachableBlockId(), 130); + ASSERT_EQ(blockchain.getGenesisBlockId(), 10); + ASSERT_EQ(blockchain.linkUntilBlockId(160), 30); + ASSERT_EQ(blockchain.getLastReachableBlockId(), 160); + ASSERT_EQ(blockchain.getGenesisBlockId(), 50); + blockchain.linkSTChain(); + ASSERT_EQ(blockchain.getStChain().getLastBlockId(), 0); + ASSERT_EQ(blockchain.getLastReachableBlockId(), 200); + ASSERT_EQ(blockchain.getGenesisBlockId(), 90); +} + +TEST_F(v4_kvbc, all_gets) { + uint64_t max_block = 100; + uint32_t num_merkle_each = 0; + uint32_t num_versioned_each = 0; + uint32_t num_immutable_each = 0; + create_blocks(max_block, num_merkle_each, num_versioned_each, num_immutable_each); + + std::map cat_map{ + {"merkle", categorization::CATEGORY_TYPE::block_merkle}, + {"versioned", categorization::CATEGORY_TYPE::versioned_kv}, + {"immutable", categorization::CATEGORY_TYPE::immutable}}; + v4blockchain::KeyValueBlockchain blockchain{db, true, cat_map}; + ASSERT_EQ(blockchain.getGenesisBlockId(), 1); + ASSERT_EQ(blockchain.getLastReachableBlockId(), max_block); + + std::vector merkle_keys; + std::vector merkle_versions; + std::vector versioned_keys; + std::vector versioned_versions; + std::vector immutable_keys; + std::vector immutable_versions; + + ///////Checking the get/////////////////////// + for (uint64_t blk = 1; blk <= max_block; ++blk) { + for (uint32_t kid = 1; kid <= num_merkle_each; ++kid) { + std::string key = "merkle_key_" + std::to_string(blk) + "_" + std::to_string(kid); + std::string val_str = "merkle_value_" + std::to_string(blk) + "_" + std::to_string(kid); + merkle_keys.push_back(key); + merkle_versions.push_back(blk); + auto val = blockchain.get("merkle", key, blk); + ASSERT_TRUE(val.has_value()); + auto merkle_val = std::get(*val); + ASSERT_EQ(merkle_val.block_id, blk); + ASSERT_EQ(merkle_val.data, val_str); + } + + for (uint32_t kid = 1; kid <= num_versioned_each; ++kid) { + std::string key = "versioned_key_" + std::to_string(blk) + "_" + std::to_string(kid); + std::string val_str = "versioned_value_" + std::to_string(blk) + "_" + std::to_string(kid); + versioned_keys.push_back(key); + versioned_versions.push_back(blk); + auto val = blockchain.get("versioned", key, blk); + ASSERT_TRUE(val.has_value()); + auto ver_val = std::get(*val); + ASSERT_EQ(ver_val.block_id, blk); + ASSERT_EQ(ver_val.data, val_str); + } + for (uint32_t kid = 1; kid <= num_immutable_each; ++kid) { + std::string key = "immutable_key_" + std::to_string(blk) + "_" + std::to_string(kid); + std::string val_str = "immutable_value_" + std::to_string(blk) + "_" + std::to_string(kid); + immutable_keys.push_back(key); + immutable_versions.push_back(blk); + auto val = blockchain.get("immutable", key, blk); + ASSERT_TRUE(val.has_value()); + auto immutable_val = std::get(*val); + ASSERT_EQ(immutable_val.block_id, blk); + ASSERT_EQ(immutable_val.data, val_str); + } + } + + ///////Checking the multiGet/////////////////////// + std::vector > values; + blockchain.multiGet("merkle", merkle_keys, merkle_versions, values); + size_t key_start = std::string("merkle_key_").size(); + size_t val_start = std::string("merkle_value_").size(); + for (size_t i = 0; i < merkle_keys.size(); ++i) { + ASSERT_TRUE(values[i].has_value()); + auto merkle_val = std::get(*(values[i])); + ASSERT_EQ(merkle_keys[i].substr(key_start), (merkle_val.data).substr(val_start)); + } + + blockchain.multiGet("versioned", versioned_keys, versioned_versions, values); + key_start = std::string("versioned_key_").size(); + val_start = std::string("versioned_value_").size(); + for (size_t i = 0; i < versioned_keys.size(); ++i) { + ASSERT_TRUE(values[i].has_value()); + auto versioned_val = std::get(*(values[i])); + ASSERT_EQ(versioned_keys[i].substr(key_start), (versioned_val.data).substr(val_start)); + } + + blockchain.multiGet("immutable", immutable_keys, immutable_versions, values); + key_start = std::string("immutable_key_").size(); + val_start = std::string("immutable_value_").size(); + for (size_t i = 0; i < immutable_keys.size(); ++i) { + ASSERT_TRUE(values[i].has_value()); + auto immutable_val = std::get(*(values[i])); + ASSERT_EQ(immutable_keys[i].substr(key_start), (immutable_val.data).substr(val_start)); + } + + merkle_keys.clear(); + merkle_versions.clear(); + versioned_keys.clear(); + versioned_versions.clear(); + immutable_keys.clear(); + immutable_versions.clear(); + + std::set deleted_keys; + uint64_t new_max_block = max_block; + add_deletes_to_blocks( + new_max_block, + num_merkle_each, + num_versioned_each, + num_immutable_each, + [&deleted_keys](const std::string& key, uint64_t blkid, uint32_t kid, uint32_t kid_max) -> bool { + std::mt19937 rgen; + std::uniform_int_distribution dist(2, std::numeric_limits::max()); + if (((blkid + kid) * dist(rgen)) % 2 == 0) { + deleted_keys.insert(key); + return true; + } + return false; + }); + ASSERT_EQ(blockchain.getGenesisBlockId(), 1); + ASSERT_EQ(blockchain.getLastReachableBlockId(), new_max_block); + // Checking again + for (uint64_t blk = 1; blk <= max_block; ++blk) { + for (uint32_t kid = 1; kid <= num_merkle_each; ++kid) { + std::string key = "merkle_key_" + std::to_string(blk) + "_" + std::to_string(kid); + std::string val_str = "merkle_value_" + std::to_string(blk) + "_" + std::to_string(kid); + merkle_keys.push_back(key); + merkle_versions.push_back(blk); + auto val = blockchain.get("merkle", key, blk); + ASSERT_TRUE(val.has_value()); + auto merkle_val = std::get(*val); + ASSERT_EQ(merkle_val.block_id, blk); + ASSERT_EQ(merkle_val.data, val_str); + } + + for (uint32_t kid = 1; kid <= num_versioned_each; ++kid) { + std::string key = "versioned_key_" + std::to_string(blk) + "_" + std::to_string(kid); + std::string val_str = "versioned_value_" + std::to_string(blk) + "_" + std::to_string(kid); + versioned_keys.push_back(key); + versioned_versions.push_back(blk); + auto val = blockchain.get("versioned", key, blk); + ASSERT_TRUE(val.has_value()); + auto ver_val = std::get(*val); + ASSERT_EQ(ver_val.block_id, blk); + ASSERT_EQ(ver_val.data, val_str); + } + for (uint32_t kid = 1; kid <= num_immutable_each; ++kid) { + std::string key = "immutable_key_" + std::to_string(blk) + "_" + std::to_string(kid); + std::string val_str = "immutable_value_" + std::to_string(blk) + "_" + std::to_string(kid); + immutable_keys.push_back(key); + immutable_versions.push_back(blk); + auto val = blockchain.get("immutable", key, blk); + ASSERT_TRUE(val.has_value()); + auto immutable_val = std::get(*val); + ASSERT_EQ(immutable_val.block_id, blk); + ASSERT_EQ(immutable_val.data, val_str); + } + } + + ///////Checking the multiGet/////////////////////// + blockchain.multiGet("merkle", merkle_keys, merkle_versions, values); + key_start = std::string("merkle_key_").size(); + val_start = std::string("merkle_value_").size(); + for (size_t i = 0; i < merkle_keys.size(); ++i) { + ASSERT_TRUE(values[i].has_value()); + auto merkle_val = std::get(*(values[i])); + ASSERT_EQ(merkle_keys[i].substr(key_start), (merkle_val.data).substr(val_start)); + } + + blockchain.multiGet("versioned", versioned_keys, versioned_versions, values); + key_start = std::string("versioned_key_").size(); + val_start = std::string("versioned_value_").size(); + for (size_t i = 0; i < versioned_keys.size(); ++i) { + ASSERT_TRUE(values[i].has_value()); + auto versioned_val = std::get(*(values[i])); + ASSERT_EQ(versioned_keys[i].substr(key_start), (versioned_val.data).substr(val_start)); + } + + blockchain.multiGet("immutable", immutable_keys, immutable_versions, values); + key_start = std::string("immutable_key_").size(); + val_start = std::string("immutable_value_").size(); + for (size_t i = 0; i < immutable_keys.size(); ++i) { + ASSERT_TRUE(values[i].has_value()); + auto immutable_val = std::get(*(values[i])); + ASSERT_EQ(immutable_keys[i].substr(key_start), (immutable_val.data).substr(val_start)); + } + + merkle_keys.clear(); + merkle_versions.clear(); + versioned_keys.clear(); + versioned_versions.clear(); + immutable_keys.clear(); + immutable_versions.clear(); + // Checking again + for (uint64_t blk = max_block + 1; blk <= new_max_block; ++blk) { + for (uint32_t kid = 1; kid <= num_merkle_each; ++kid) { + std::string key = "merkle_key_" + std::to_string(blk) + "_" + std::to_string(kid); + std::string val_str = "merkle_value_" + std::to_string(blk) + "_" + std::to_string(kid); + merkle_keys.push_back(key); + merkle_versions.push_back(blk); + auto val = blockchain.get("merkle", key, blk); + if (deleted_keys.count(key)) { + ASSERT_FALSE(val.has_value()); + } else { + ASSERT_TRUE(val.has_value()); + auto merkle_val = std::get(*val); + ASSERT_EQ(merkle_val.block_id, blk); + ASSERT_EQ(merkle_val.data, val_str); + } + } + + // This range includes the deletes. + for (uint32_t kid = 1; kid <= num_versioned_each; ++kid) { + std::string key = "versioned_key_" + std::to_string(blk) + "_" + std::to_string(kid); + std::string val_str = "versioned_value_" + std::to_string(blk) + "_" + std::to_string(kid); + versioned_keys.push_back(key); + versioned_versions.push_back(blk); + auto val = blockchain.get("versioned", key, blk); + if (deleted_keys.count(key)) { + ASSERT_FALSE(val.has_value()); + } else { + ASSERT_TRUE(val.has_value()); + auto ver_val = std::get(*val); + ASSERT_EQ(ver_val.block_id, blk); + ASSERT_EQ(ver_val.data, val_str); + } + } + for (uint32_t kid = 1; kid <= num_immutable_each; ++kid) { + std::string key = "immutable_key_" + std::to_string(blk) + "_" + std::to_string(kid); + std::string val_str = "immutable_value_" + std::to_string(blk) + "_" + std::to_string(kid); + immutable_keys.push_back(key); + immutable_versions.push_back(blk); + auto val = blockchain.get("immutable", key, blk); + if (deleted_keys.count(key)) { + ASSERT_FALSE(val.has_value()); + } else { + ASSERT_TRUE(val.has_value()); + auto immutable_val = std::get(*val); + ASSERT_EQ(immutable_val.block_id, blk); + ASSERT_EQ(immutable_val.data, val_str); + } + } + } + + ///////Checking the multiGet/////////////////////// + blockchain.multiGet("merkle", merkle_keys, merkle_versions, values); + key_start = std::string("merkle_key_").size(); + val_start = std::string("merkle_value_").size(); + for (size_t i = 0; i < merkle_keys.size(); ++i) { + if (deleted_keys.count(merkle_keys[i])) { + ASSERT_FALSE(values[i].has_value()); + } else { + ASSERT_TRUE(values[i].has_value()); + auto merkle_val = std::get(*(values[i])); + ASSERT_EQ(merkle_keys[i].substr(key_start), (merkle_val.data).substr(val_start)); + } + } + + blockchain.multiGet("versioned", versioned_keys, versioned_versions, values); + key_start = std::string("versioned_key_").size(); + val_start = std::string("versioned_value_").size(); + for (size_t i = 0; i < versioned_keys.size(); ++i) { + if (deleted_keys.count(versioned_keys[i])) { + ASSERT_FALSE(values[i].has_value()); + } else { + ASSERT_TRUE(values[i].has_value()); + auto versioned_val = std::get(*(values[i])); + ASSERT_EQ(versioned_keys[i].substr(key_start), (versioned_val.data).substr(val_start)); + } + } + + blockchain.multiGet("immutable", immutable_keys, immutable_versions, values); + key_start = std::string("immutable_key_").size(); + val_start = std::string("immutable_value_").size(); + for (size_t i = 0; i < immutable_keys.size(); ++i) { + if (deleted_keys.count(immutable_keys[i])) { + ASSERT_FALSE(values[i].has_value()); + } else { + ASSERT_TRUE(values[i].has_value()); + auto immutable_val = std::get(*(values[i])); + ASSERT_EQ(immutable_keys[i].substr(key_start), (immutable_val.data).substr(val_start)); + } + } +} + +TEST_F(v4_kvbc, all_get_latest) { + uint64_t max_block = 100; + uint32_t num_merkle_each = 0; + uint32_t num_versioned_each = 0; + uint32_t num_immutable_each = 0; + create_blocks(max_block, num_merkle_each, num_versioned_each, num_immutable_each); + + std::map cat_map{ + {"merkle", categorization::CATEGORY_TYPE::block_merkle}, + {"versioned", categorization::CATEGORY_TYPE::versioned_kv}, + {"immutable", categorization::CATEGORY_TYPE::immutable}}; + v4blockchain::KeyValueBlockchain blockchain{db, true, cat_map}; + ASSERT_EQ(blockchain.getGenesisBlockId(), 1); + ASSERT_EQ(blockchain.getLastReachableBlockId(), max_block); + + std::vector merkle_keys; + std::vector versioned_keys; + std::vector immutable_keys; + + ///////Checking the get/////////////////////// + for (uint64_t blk = 1; blk <= max_block; ++blk) { + for (uint32_t kid = 1; kid <= num_merkle_each; ++kid) { + std::string key = "merkle_key_" + std::to_string(blk) + "_" + std::to_string(kid); + std::string val_str = "merkle_value_" + std::to_string(blk) + "_" + std::to_string(kid); + merkle_keys.push_back(key); + auto val = blockchain.getLatest("merkle", key); + ASSERT_TRUE(val.has_value()); + auto merkle_val = std::get(*val); + ASSERT_EQ(merkle_val.block_id, blk); + ASSERT_EQ(merkle_val.data, val_str); + } + + for (uint32_t kid = 1; kid <= num_versioned_each; ++kid) { + std::string key = "versioned_key_" + std::to_string(blk) + "_" + std::to_string(kid); + std::string val_str = "versioned_value_" + std::to_string(blk) + "_" + std::to_string(kid); + versioned_keys.push_back(key); + auto val = blockchain.getLatest("versioned", key); + ASSERT_TRUE(val.has_value()); + auto ver_val = std::get(*val); + ASSERT_EQ(ver_val.block_id, blk); + ASSERT_EQ(ver_val.data, val_str); + } + for (uint32_t kid = 1; kid <= num_immutable_each; ++kid) { + std::string key = "immutable_key_" + std::to_string(blk) + "_" + std::to_string(kid); + std::string val_str = "immutable_value_" + std::to_string(blk) + "_" + std::to_string(kid); + immutable_keys.push_back(key); + auto val = blockchain.getLatest("immutable", key); + ASSERT_TRUE(val.has_value()); + auto immutable_val = std::get(*val); + ASSERT_EQ(immutable_val.block_id, blk); + ASSERT_EQ(immutable_val.data, val_str); + } + } + + ///////Checking the multiGet/////////////////////// + std::vector > values; + blockchain.multiGetLatest("merkle", merkle_keys, values); + size_t key_start = std::string("merkle_key_").size(); + size_t val_start = std::string("merkle_value_").size(); + for (size_t i = 0; i < merkle_keys.size(); ++i) { + ASSERT_TRUE(values[i].has_value()); + auto merkle_val = std::get(*(values[i])); + ASSERT_EQ(merkle_keys[i].substr(key_start), (merkle_val.data).substr(val_start)); + } + + blockchain.multiGetLatest("versioned", versioned_keys, values); + key_start = std::string("versioned_key_").size(); + val_start = std::string("versioned_value_").size(); + for (size_t i = 0; i < versioned_keys.size(); ++i) { + ASSERT_TRUE(values[i].has_value()); + auto versioned_val = std::get(*(values[i])); + ASSERT_EQ(versioned_keys[i].substr(key_start), (versioned_val.data).substr(val_start)); + } + + blockchain.multiGetLatest("immutable", immutable_keys, values); + key_start = std::string("immutable_key_").size(); + val_start = std::string("immutable_value_").size(); + for (size_t i = 0; i < immutable_keys.size(); ++i) { + ASSERT_TRUE(values[i].has_value()); + auto immutable_val = std::get(*(values[i])); + ASSERT_EQ(immutable_keys[i].substr(key_start), (immutable_val.data).substr(val_start)); + } + + merkle_keys.clear(); + versioned_keys.clear(); + immutable_keys.clear(); + + std::set deleted_keys; + uint64_t new_max_block = max_block; + add_deletes_to_blocks( + new_max_block, + num_merkle_each, + num_versioned_each, + num_immutable_each, + [&deleted_keys](const std::string& key, uint64_t blkid, uint32_t kid, uint32_t kid_max) -> bool { + std::mt19937 rgen; + std::uniform_int_distribution dist(2, std::numeric_limits::max()); + if (((blkid + kid) * dist(rgen)) % 2 == 0) { + deleted_keys.insert(key); + return true; + } + return false; + }); + ASSERT_EQ(blockchain.getGenesisBlockId(), 1); + ASSERT_EQ(blockchain.getLastReachableBlockId(), new_max_block); + // Checking again + for (uint64_t blk = max_block + 1; blk <= new_max_block; ++blk) { + for (uint32_t kid = 1; kid <= num_merkle_each; ++kid) { + std::string key = "merkle_key_" + std::to_string(blk) + "_" + std::to_string(kid); + std::string val_str = "merkle_value_" + std::to_string(blk) + "_" + std::to_string(kid); + merkle_keys.push_back(key); + auto val = blockchain.getLatest("merkle", key); + if (deleted_keys.count(key)) { + ASSERT_FALSE(val.has_value()); + } else { + ASSERT_TRUE(val.has_value()); + auto merkle_val = std::get(*val); + ASSERT_EQ(merkle_val.block_id, blk); + ASSERT_EQ(merkle_val.data, val_str); + } + } + + for (uint32_t kid = 1; kid <= num_versioned_each; ++kid) { + std::string key = "versioned_key_" + std::to_string(blk) + "_" + std::to_string(kid); + std::string val_str = "versioned_value_" + std::to_string(blk) + "_" + std::to_string(kid); + versioned_keys.push_back(key); + auto val = blockchain.getLatest("versioned", key); + if (deleted_keys.count(key)) { + ASSERT_FALSE(val.has_value()); + } else { + ASSERT_TRUE(val.has_value()); + auto ver_val = std::get(*val); + ASSERT_EQ(ver_val.block_id, blk); + ASSERT_EQ(ver_val.data, val_str); + } + } + for (uint32_t kid = 1; kid <= num_immutable_each; ++kid) { + std::string key = "immutable_key_" + std::to_string(blk) + "_" + std::to_string(kid); + std::string val_str = "immutable_value_" + std::to_string(blk) + "_" + std::to_string(kid); + immutable_keys.push_back(key); + auto val = blockchain.getLatest("immutable", key); + if (deleted_keys.count(key)) { + ASSERT_FALSE(val.has_value()); + } else { + ASSERT_TRUE(val.has_value()); + auto immutable_val = std::get(*val); + ASSERT_EQ(immutable_val.block_id, blk); + ASSERT_EQ(immutable_val.data, val_str); + } + } + } + + ///////Checking the multiGet/////////////////////// + blockchain.multiGetLatest("merkle", merkle_keys, values); + key_start = std::string("merkle_key_").size(); + val_start = std::string("merkle_value_").size(); + for (size_t i = 0; i < merkle_keys.size(); ++i) { + if (deleted_keys.count(merkle_keys[i])) { + ASSERT_FALSE(values[i].has_value()); + } else { + ASSERT_TRUE(values[i].has_value()); + auto merkle_val = std::get(*(values[i])); + ASSERT_EQ(merkle_keys[i].substr(key_start), (merkle_val.data).substr(val_start)); + } + } + + blockchain.multiGetLatest("versioned", versioned_keys, values); + key_start = std::string("versioned_key_").size(); + val_start = std::string("versioned_value_").size(); + for (size_t i = 0; i < versioned_keys.size(); ++i) { + if (deleted_keys.count(versioned_keys[i])) { + ASSERT_FALSE(values[i].has_value()); + } else { + ASSERT_TRUE(values[i].has_value()); + auto versioned_val = std::get(*(values[i])); + ASSERT_EQ(versioned_keys[i].substr(key_start), (versioned_val.data).substr(val_start)); + } + } + + blockchain.multiGetLatest("immutable", immutable_keys, values); + key_start = std::string("immutable_key_").size(); + val_start = std::string("immutable_value_").size(); + for (size_t i = 0; i < immutable_keys.size(); ++i) { + if (deleted_keys.count(immutable_keys[i])) { + ASSERT_FALSE(values[i].has_value()); + } else { + ASSERT_TRUE(values[i].has_value()); + auto immutable_val = std::get(*(values[i])); + ASSERT_EQ(immutable_keys[i].substr(key_start), (immutable_val.data).substr(val_start)); + } + } +} + +TEST_F(v4_kvbc, get_block_updates_test) { + uint64_t max_block = 100; + uint32_t num_merkle_each = 0; + uint32_t num_versioned_each = 0; + uint32_t num_immutable_each = 0; + create_blocks(max_block, num_merkle_each, num_versioned_each, num_immutable_each); + + std::map cat_map{ + {"merkle", categorization::CATEGORY_TYPE::block_merkle}, + {"versioned", categorization::CATEGORY_TYPE::versioned_kv}, + {"immutable", categorization::CATEGORY_TYPE::immutable}}; + v4blockchain::KeyValueBlockchain blockchain{db, true, cat_map}; + for (uint64_t blk = 1; blk <= max_block; ++blk) { + auto updates = blockchain.getBlockUpdates(blk); + ASSERT_TRUE(updates.has_value()); + auto merkle_cat_inputs = updates->categoryUpdates("merkle"); + ASSERT_TRUE(merkle_cat_inputs.has_value()); + ASSERT_EQ(std::get(merkle_cat_inputs->get()).kv.size(), num_merkle_each); + auto versioned_cat_inputs = updates->categoryUpdates("versioned"); + ASSERT_TRUE(versioned_cat_inputs.has_value()); + ASSERT_EQ(std::get(versioned_cat_inputs->get()).kv.size(), num_versioned_each); + auto immutable_cat_inputs = updates->categoryUpdates("immutable"); + ASSERT_TRUE(immutable_cat_inputs.has_value()); + ASSERT_EQ(std::get(immutable_cat_inputs->get()).kv.size(), num_immutable_each); + } + for (uint64_t blk = max_block + 1; blk <= max_block + 10; ++blk) { + auto updates = blockchain.getBlockUpdates(blk); + ASSERT_FALSE(updates.has_value()); + } + uint64_t new_max_block = max_block; + add_deletes_to_blocks(new_max_block, + num_merkle_each, + num_versioned_each, + num_immutable_each, + [](const std::string& key, uint64_t blkid, uint32_t kid, uint32_t kid_max) -> bool { + std::mt19937 rgen; + std::uniform_int_distribution dist(2, std::numeric_limits::max()); + if (((blkid + kid) * dist(rgen)) % 2 == 0) { + return true; + } + return false; + }); + for (uint64_t blk = 1; blk <= max_block; ++blk) { + auto updates = blockchain.getBlockUpdates(blk); + ASSERT_TRUE(updates.has_value()); + auto merkle_cat_inputs = updates->categoryUpdates("merkle"); + ASSERT_TRUE(merkle_cat_inputs.has_value()); + ASSERT_EQ(std::get(merkle_cat_inputs->get()).kv.size(), num_merkle_each); + auto versioned_cat_inputs = updates->categoryUpdates("versioned"); + ASSERT_TRUE(versioned_cat_inputs.has_value()); + ASSERT_EQ(std::get(versioned_cat_inputs->get()).kv.size(), num_versioned_each); + auto immutable_cat_inputs = updates->categoryUpdates("immutable"); + ASSERT_TRUE(immutable_cat_inputs.has_value()); + ASSERT_EQ(std::get(immutable_cat_inputs->get()).kv.size(), num_immutable_each); + } + for (uint64_t blk = max_block + 1; blk <= new_max_block; ++blk) { + auto updates = blockchain.getBlockUpdates(blk); + ASSERT_TRUE(updates.has_value()); + auto merkle_cat_inputs = updates->categoryUpdates("merkle"); + ASSERT_TRUE(merkle_cat_inputs.has_value()); + ASSERT_LT(std::get(merkle_cat_inputs->get()).kv.size(), num_merkle_each); + auto versioned_cat_inputs = updates->categoryUpdates("versioned"); + ASSERT_TRUE(versioned_cat_inputs.has_value()); + ASSERT_LT(std::get(versioned_cat_inputs->get()).kv.size(), num_versioned_each); + auto immutable_cat_inputs = updates->categoryUpdates("immutable"); + ASSERT_TRUE(immutable_cat_inputs.has_value()); + ASSERT_EQ(std::get(immutable_cat_inputs->get()).kv.size(), num_immutable_each); + } + for (uint64_t blk = new_max_block + 1; blk <= new_max_block + 10; ++blk) { + auto updates = blockchain.getBlockUpdates(blk); + ASSERT_FALSE(updates.has_value()); + } +} + +TEST_F(v4_kvbc, all_get_latest_versions) { + uint64_t max_block = 100; + uint32_t num_merkle_each = 0; + uint32_t num_versioned_each = 0; + uint32_t num_immutable_each = 0; + create_blocks(max_block, num_merkle_each, num_versioned_each, num_immutable_each); + + std::map cat_map{ + {"merkle", categorization::CATEGORY_TYPE::block_merkle}, + {"versioned", categorization::CATEGORY_TYPE::versioned_kv}, + {"immutable", categorization::CATEGORY_TYPE::immutable}}; + v4blockchain::KeyValueBlockchain blockchain{db, true, cat_map}; + ASSERT_EQ(blockchain.getGenesisBlockId(), 1); + ASSERT_EQ(blockchain.getLastReachableBlockId(), max_block); + + std::vector merkle_keys; + std::vector merkle_versions; + std::vector versioned_keys; + std::vector versioned_versions; + std::vector immutable_keys; + std::vector immutable_versions; + + ///////Checking the get/////////////////////// + for (uint64_t blk = 1; blk <= max_block; ++blk) { + for (uint32_t kid = 1; kid <= num_merkle_each; ++kid) { + std::string key = "merkle_key_" + std::to_string(blk) + "_" + std::to_string(kid); + merkle_keys.push_back(key); + merkle_versions.push_back(blk); + auto latest_version = blockchain.getLatestVersion("merkle", key); + ASSERT_TRUE(latest_version.has_value()); + ASSERT_EQ(latest_version->version, blk); + } + + for (uint32_t kid = 1; kid <= num_versioned_each; ++kid) { + std::string key = "versioned_key_" + std::to_string(blk) + "_" + std::to_string(kid); + versioned_keys.push_back(key); + versioned_versions.push_back(blk); + auto latest_version = blockchain.getLatestVersion("versioned", key); + ASSERT_TRUE(latest_version.has_value()); + ASSERT_EQ(latest_version->version, blk); + } + for (uint32_t kid = 1; kid <= num_immutable_each; ++kid) { + std::string key = "immutable_key_" + std::to_string(blk) + "_" + std::to_string(kid); + immutable_keys.push_back(key); + immutable_versions.push_back(blk); + auto latest_version = blockchain.getLatestVersion("immutable", key); + ASSERT_TRUE(latest_version.has_value()); + ASSERT_EQ(latest_version->version, blk); + } + } + + ///////Checking the multiGetLatestVersion/////////////////////// + std::vector > latest_versions; + blockchain.multiGetLatestVersion("merkle", merkle_keys, latest_versions); + for (size_t i = 0; i < merkle_keys.size(); ++i) { + ASSERT_TRUE(latest_versions[i].has_value()); + ASSERT_EQ(merkle_versions[i], (latest_versions[i])->version); + } + + blockchain.multiGetLatestVersion("versioned", versioned_keys, latest_versions); + for (size_t i = 0; i < versioned_keys.size(); ++i) { + ASSERT_TRUE(latest_versions[i].has_value()); + ASSERT_EQ(versioned_versions[i], (latest_versions[i])->version); + } + + blockchain.multiGetLatestVersion("immutable", immutable_keys, latest_versions); + for (size_t i = 0; i < immutable_keys.size(); ++i) { + ASSERT_TRUE(latest_versions[i].has_value()); + ASSERT_EQ(immutable_versions[i], (latest_versions[i])->version); + } + + merkle_keys.clear(); + merkle_versions.clear(); + versioned_keys.clear(); + versioned_versions.clear(); + immutable_keys.clear(); + immutable_versions.clear(); + + std::set deleted_keys; + uint64_t new_max_block = max_block; + add_deletes_to_blocks( + new_max_block, + num_merkle_each, + num_versioned_each, + num_immutable_each, + [&deleted_keys](const std::string& key, uint64_t blkid, uint32_t kid, uint32_t kid_max) -> bool { + std::mt19937 rgen; + std::uniform_int_distribution dist(2, std::numeric_limits::max()); + if (((blkid + kid) * dist(rgen)) % 2 == 0) { + deleted_keys.insert(key); + return true; + } + return false; + }); + ASSERT_EQ(blockchain.getGenesisBlockId(), 1); + ASSERT_EQ(blockchain.getLastReachableBlockId(), new_max_block); + // Checking again + for (uint64_t blk = 1; blk <= max_block; ++blk) { + for (uint32_t kid = 1; kid <= num_merkle_each; ++kid) { + std::string key = "merkle_key_" + std::to_string(blk) + "_" + std::to_string(kid); + merkle_keys.push_back(key); + merkle_versions.push_back(blk); + auto latest_version = blockchain.getLatestVersion("merkle", key); + ASSERT_TRUE(latest_version.has_value()); + ASSERT_EQ(latest_version->version, blk); + } + + for (uint32_t kid = 1; kid <= num_versioned_each; ++kid) { + std::string key = "versioned_key_" + std::to_string(blk) + "_" + std::to_string(kid); + versioned_keys.push_back(key); + versioned_versions.push_back(blk); + auto latest_version = blockchain.getLatestVersion("versioned", key); + ASSERT_TRUE(latest_version.has_value()); + ASSERT_EQ(latest_version->version, blk); + } + for (uint32_t kid = 1; kid <= num_immutable_each; ++kid) { + std::string key = "immutable_key_" + std::to_string(blk) + "_" + std::to_string(kid); + immutable_keys.push_back(key); + immutable_versions.push_back(blk); + auto latest_version = blockchain.getLatestVersion("immutable", key); + ASSERT_TRUE(latest_version.has_value()); + ASSERT_EQ(latest_version->version, blk); + } + } + + ///////Checking the multiGetLatestVersion again after deletion/////////////////////// + blockchain.multiGetLatestVersion("merkle", merkle_keys, latest_versions); + for (size_t i = 0; i < merkle_keys.size(); ++i) { + ASSERT_TRUE(latest_versions[i].has_value()); + ASSERT_EQ(merkle_versions[i], (latest_versions[i])->version); + } + + blockchain.multiGetLatestVersion("versioned", versioned_keys, latest_versions); + for (size_t i = 0; i < versioned_keys.size(); ++i) { + ASSERT_TRUE(latest_versions[i].has_value()); + ASSERT_EQ(versioned_versions[i], (latest_versions[i])->version); + } + + blockchain.multiGetLatestVersion("immutable", immutable_keys, latest_versions); + for (size_t i = 0; i < immutable_keys.size(); ++i) { + ASSERT_TRUE(latest_versions[i].has_value()); + ASSERT_EQ(immutable_versions[i], (latest_versions[i])->version); + } + + merkle_keys.clear(); + merkle_versions.clear(); + versioned_keys.clear(); + versioned_versions.clear(); + immutable_keys.clear(); + immutable_versions.clear(); + // Checking again + for (uint64_t blk = max_block + 1; blk <= new_max_block; ++blk) { + for (uint32_t kid = 1; kid <= num_merkle_each; ++kid) { + std::string key = "merkle_key_" + std::to_string(blk) + "_" + std::to_string(kid); + merkle_keys.push_back(key); + merkle_versions.push_back(blk); + auto latest_version = blockchain.getLatestVersion("merkle", key); + if (deleted_keys.count(key)) { + ASSERT_FALSE(latest_version.has_value()); + } else { + ASSERT_TRUE(latest_version.has_value()); + ASSERT_EQ(latest_version->version, blk); + } + } + + for (uint32_t kid = 1; kid <= num_versioned_each; ++kid) { + std::string key = "versioned_key_" + std::to_string(blk) + "_" + std::to_string(kid); + versioned_keys.push_back(key); + versioned_versions.push_back(blk); + auto latest_version = blockchain.getLatestVersion("versioned", key); + if (deleted_keys.count(key)) { + ASSERT_FALSE(latest_version.has_value()); + } else { + ASSERT_TRUE(latest_version.has_value()); + ASSERT_EQ(latest_version->version, blk); + } + } + for (uint32_t kid = 1; kid <= num_immutable_each; ++kid) { + std::string key = "immutable_key_" + std::to_string(blk) + "_" + std::to_string(kid); + immutable_keys.push_back(key); + immutable_versions.push_back(blk); + auto latest_version = blockchain.getLatestVersion("immutable", key); + if (deleted_keys.count(key)) { + ASSERT_FALSE(latest_version.has_value()); + } else { + ASSERT_TRUE(latest_version.has_value()); + ASSERT_EQ(latest_version->version, blk); + } + } + } + + ///////Checking the multiGet/////////////////////// + blockchain.multiGetLatestVersion("merkle", merkle_keys, latest_versions); + for (size_t i = 0; i < merkle_keys.size(); ++i) { + if (deleted_keys.count(merkle_keys[i])) { + ASSERT_FALSE(latest_versions[i].has_value()); + } else { + ASSERT_TRUE(latest_versions[i].has_value()); + ASSERT_EQ(merkle_versions[i], (latest_versions[i])->version); + } + } + + blockchain.multiGetLatestVersion("versioned", versioned_keys, latest_versions); + for (size_t i = 0; i < versioned_keys.size(); ++i) { + if (deleted_keys.count(versioned_keys[i])) { + ASSERT_FALSE(latest_versions[i].has_value()); + } else { + ASSERT_TRUE(latest_versions[i].has_value()); + ASSERT_EQ(versioned_versions[i], (latest_versions[i])->version); + } + } + + blockchain.multiGetLatestVersion("immutable", immutable_keys, latest_versions); + for (size_t i = 0; i < immutable_keys.size(); ++i) { + if (deleted_keys.count(immutable_keys[i])) { + ASSERT_FALSE(latest_versions[i].has_value()); + } else { + ASSERT_TRUE(latest_versions[i].has_value()); + ASSERT_EQ(immutable_versions[i], (latest_versions[i])->version); + } + } +} + +TEST_F(v4_kvbc, trim_blocks) { + uint64_t max_block = 100; + uint32_t num_merkle_each = 0; + uint32_t num_versioned_each = 0; + uint32_t num_immutable_each = 0; + create_blocks(max_block, num_merkle_each, num_versioned_each, num_immutable_each); + + std::map cat_map{ + {"merkle", categorization::CATEGORY_TYPE::block_merkle}, + {"versioned", categorization::CATEGORY_TYPE::versioned_kv}, + {"immutable", categorization::CATEGORY_TYPE::immutable}}; + v4blockchain::KeyValueBlockchain blockchain{db, true, cat_map}; + ASSERT_EQ(blockchain.getGenesisBlockId(), 1); + ASSERT_EQ(blockchain.getLastReachableBlockId(), max_block); + ASSERT_NO_THROW(blockchain.trimBlocksFromSnapshot(max_block / 2)); + ASSERT_EQ(blockchain.getGenesisBlockId(), 1); + ASSERT_EQ(blockchain.getLastReachableBlockId(), max_block / 2); + ASSERT_DEATH(blockchain.trimBlocksFromSnapshot(v4blockchain::detail::Blockchain::INVALID_BLOCK_ID), ""); + ASSERT_DEATH(blockchain.trimBlocksFromSnapshot(max_block + 10), ""); +} + +} // end namespace + +int main(int argc, char** argv) { + InitGoogleTest(&argc, argv); + return RUN_ALL_TESTS(); +} diff --git a/kvbc/test/v4blockchain/v4_latest_keys_test.cpp b/kvbc/test/v4blockchain/v4_latest_keys_test.cpp new file mode 100644 index 0000000000..09adaa42c1 --- /dev/null +++ b/kvbc/test/v4blockchain/v4_latest_keys_test.cpp @@ -0,0 +1,880 @@ +// Concord +// +// Copyright (c) 2020 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 subcomponent's license, as noted in the LICENSE +// file. + +#include "gtest/gtest.h" +#include "gmock/gmock.h" +#include "v4blockchain/v4_blockchain.h" +#include +#include +#include +#include +#include +#include "storage/test/storage_test_common.h" +#include "endianness.hpp" +#include "v4blockchain/detail/column_families.h" +#include "categorization/db_categories.h" +#include "categorized_kvbc_msgs.cmf.hpp" + +using concord::storage::rocksdb::NativeClient; +using namespace concord::kvbc; +using namespace ::testing; + +namespace { + +class v4_kvbc : public Test { + protected: + void SetUp() override { + destroyDb(); + db = TestRocksDb::createNative(); + categories = std::map{ + {"merkle", categorization::CATEGORY_TYPE::block_merkle}, + {"versioned", categorization::CATEGORY_TYPE::versioned_kv}, + {"versioned_2", categorization::CATEGORY_TYPE::versioned_kv}, + {"immutable", categorization::CATEGORY_TYPE::immutable}}; + } + + void TearDown() override { destroyDb(); } + + void destroyDb() { + db.reset(); + ASSERT_EQ(0, db.use_count()); + cleanup(); + } + void testGetValueAndVersion(const v4blockchain::detail::LatestKeys& latest_keys, + BlockId latest_block_id, + const std::string& category_id, + const std::string& key, + bool val_is_null_opt, + bool ver_is_null_opt, + std::vector& keys, + std::vector& val_is_null_opts, + std::vector& ver_is_null_opts) { + auto latest_version = concord::kvbc::v4blockchain::detail::Blockchain::generateKey(latest_block_id); + keys.push_back(key); + val_is_null_opts.push_back(val_is_null_opt); + ver_is_null_opts.push_back(ver_is_null_opt); + auto value = latest_keys.getValue(latest_block_id, category_id, latest_version, key); + ASSERT_EQ(value.has_value(), !val_is_null_opt); + + auto version = latest_keys.getLatestVersion(category_id, latest_version, key); + ASSERT_EQ(version.has_value(), !ver_is_null_opt); + if (!ver_is_null_opt) { + ASSERT_FALSE(version->deleted); + ASSERT_LE(version->version, latest_block_id); + } + } + + void testMultiGetValueAndVersion(const v4blockchain::detail::LatestKeys& latest_keys, + BlockId latest_block_id, + const std::string& category_id, + const std::vector& keys, + const std::vector& val_is_null_opts, + const std::vector& ver_is_null_opts) { + auto latest_version = concord::kvbc::v4blockchain::detail::Blockchain::generateKey(latest_block_id); + std::vector> values; + latest_keys.multiGetValue(latest_block_id, category_id, latest_version, keys, values); + ASSERT_EQ(keys.size(), values.size()); + for (size_t i = 0; i < keys.size(); ++i) { + ASSERT_EQ((values[i]).has_value(), !val_is_null_opts[i]); + } + std::vector> versions; + latest_keys.multiGetLatestVersion(category_id, latest_version, keys, versions); + ASSERT_EQ(keys.size(), versions.size()); + for (size_t i = 0; i < keys.size(); ++i) { + ASSERT_EQ((versions[i]).has_value(), !ver_is_null_opts[i]); + if (!ver_is_null_opts[i]) { + ASSERT_FALSE((versions[i])->deleted); + ASSERT_LE((versions[i]->version), latest_block_id); + } + } + } + + protected: + std::map categories; + std::shared_ptr db; +}; + +TEST_F(v4_kvbc, creation) { + v4blockchain::detail::LatestKeys latest_keys{db, categories, []() { return 1; }}; + + ASSERT_TRUE(db->hasColumnFamily(v4blockchain::detail::LATEST_KEYS_CF)); + for (const auto& [k, v] : categories) { + (void)v; + ASSERT_TRUE(latest_keys.getCategoryPrefix(k).size() > 0); + } +} + +TEST_F(v4_kvbc, add_merkle_keys) { + v4blockchain::detail::LatestKeys latest_keys{db, categories, []() { return 1; }}; + std::vector keys; + std::vector val_is_null_opts; + std::vector ver_is_null_opts; + + std::string no_flags = {0}; + { + uint64_t timestamp = 40; + auto timestamp_str = v4blockchain::detail::Blockchain::generateKey(timestamp); + categorization::Updates updates; + categorization::BlockMerkleUpdates merkle_updates; + auto key1 = std::string("merkle_key1"); + auto val1 = std::string("merkle_value1"); + auto key2 = std::string("merkle_key2"); + auto val2 = std::string("merkle_value2"); + merkle_updates.addUpdate(std::string("merkle_key1"), std::string("merkle_value1")); + merkle_updates.addUpdate(std::string("merkle_key2"), std::string("merkle_value2")); + updates.add("merkle", std::move(merkle_updates)); + + auto wb = db->getBatch(); + latest_keys.addBlockKeys(updates, timestamp, wb); + db->write(std::move(wb)); + testGetValueAndVersion( + latest_keys, timestamp, "merkle", key1, false, false, keys, val_is_null_opts, ver_is_null_opts); + testGetValueAndVersion( + latest_keys, timestamp, "merkle", key2, false, false, keys, val_is_null_opts, ver_is_null_opts); + + std::string out_ts; + // without category prefix + auto val = db->get(v4blockchain::detail::LATEST_KEYS_CF, key1, timestamp_str, &out_ts); + ASSERT_FALSE(val.has_value()); + // with + val = db->get( + v4blockchain::detail::LATEST_KEYS_CF, latest_keys.getCategoryPrefix("merkle") + key1, timestamp_str, &out_ts); + ASSERT_TRUE(val.has_value()); + ASSERT_EQ(*val, val1 + no_flags); + ASSERT_EQ(out_ts, timestamp_str); + auto iout_ts = concordUtils::fromBigEndianBuffer(out_ts.data()); + ASSERT_EQ(timestamp, iout_ts); + } + + { + uint64_t prev_timestamp = 40; + auto prev_timestamp_str = v4blockchain::detail::Blockchain::generateKey(prev_timestamp); + uint64_t higher_timestamp = 400; + auto higher_timestamp_str = v4blockchain::detail::Blockchain::generateKey(higher_timestamp); + uint64_t lower_timestamp = 4; + auto lower_timestamp_str = v4blockchain::detail::Blockchain::generateKey(lower_timestamp); + uint64_t close_timestamp = 41; + auto close_timestamp_str = v4blockchain::detail::Blockchain::generateKey(close_timestamp); + auto prev_val1 = std::string("merkle_value1"); + auto prev_val2 = std::string("merkle_value2"); + uint64_t timestamp = 42; + auto timestamp_str = v4blockchain::detail::Blockchain::generateKey(timestamp); + + categorization::Updates updates; + categorization::BlockMerkleUpdates merkle_updates; + // Update key + auto key1 = std::string("merkle_key1"); + auto val1 = std::string("merkle_updated_value"); + merkle_updates.addUpdate(std::string("merkle_key1"), std::string("merkle_updated_value")); + // delete key2 at timestmap 42 + auto key2 = std::string("merkle_key2"); + merkle_updates.addDelete(std::string("merkle_key2")); + // new key + auto key3 = std::string("merkle_key3"); + auto val3 = std::string("merkle_value3"); + merkle_updates.addUpdate(std::string("merkle_key3"), std::string("merkle_value3")); + + updates.add("merkle", std::move(merkle_updates)); + + auto wb = db->getBatch(); + latest_keys.addBlockKeys(updates, timestamp, wb); + db->write(std::move(wb)); + testGetValueAndVersion( + latest_keys, timestamp, "merkle", key1, false, false, keys, val_is_null_opts, ver_is_null_opts); + testGetValueAndVersion( + latest_keys, timestamp, "merkle", key2, true, true, keys, val_is_null_opts, ver_is_null_opts); + testGetValueAndVersion(latest_keys, 40, "merkle", key2, false, false, keys, val_is_null_opts, ver_is_null_opts); + testGetValueAndVersion( + latest_keys, timestamp, "merkle", key3, false, false, keys, val_is_null_opts, ver_is_null_opts); + + std::string out_ts; + //////////KEY1////////////////////////////// + + // without category prefix + auto val = db->get(v4blockchain::detail::LATEST_KEYS_CF, key1, timestamp_str, &out_ts); + ASSERT_FALSE(val.has_value()); + + // get key1 updated value of this timestamp + val = db->get( + v4blockchain::detail::LATEST_KEYS_CF, latest_keys.getCategoryPrefix("merkle") + key1, timestamp_str, &out_ts); + ASSERT_TRUE(val.has_value()); + ASSERT_EQ(*val, val1 + no_flags); + ASSERT_EQ(out_ts, timestamp_str); + auto iout_ts = concordUtils::fromBigEndianBuffer(out_ts.data()); + ASSERT_EQ(timestamp, iout_ts); + // Get previous version + out_ts.clear(); + val = db->get(v4blockchain::detail::LATEST_KEYS_CF, + latest_keys.getCategoryPrefix("merkle") + key1, + prev_timestamp_str, + &out_ts); + ASSERT_TRUE(val.has_value()); + ASSERT_EQ(*val, prev_val1 + no_flags); + ASSERT_EQ(out_ts, prev_timestamp_str); + iout_ts = concordUtils::fromBigEndianBuffer(out_ts.data()); + ASSERT_EQ(prev_timestamp, iout_ts); + + //////////KEY2////////////////////////////// + out_ts.clear(); + // try to get key2 at current timestamp + val = db->get( + v4blockchain::detail::LATEST_KEYS_CF, latest_keys.getCategoryPrefix("merkle") + key2, timestamp_str, &out_ts); + ASSERT_FALSE(val.has_value()); + // try to get key2 at higher timestamp + out_ts.clear(); + val = db->get(v4blockchain::detail::LATEST_KEYS_CF, + latest_keys.getCategoryPrefix("merkle") + key2, + higher_timestamp_str, + &out_ts); + ASSERT_FALSE(val.has_value()); + // try to get key2 at lower timestamp + out_ts.clear(); + val = db->get(v4blockchain::detail::LATEST_KEYS_CF, + latest_keys.getCategoryPrefix("merkle") + key2, + lower_timestamp_str, + &out_ts); + ASSERT_FALSE(val.has_value()); + // Get previous version + out_ts.clear(); + val = db->get(v4blockchain::detail::LATEST_KEYS_CF, + latest_keys.getCategoryPrefix("merkle") + key2, + prev_timestamp_str, + &out_ts); + ASSERT_TRUE(val.has_value()); + ASSERT_EQ(*val, prev_val2 + no_flags); + ASSERT_EQ(out_ts, prev_timestamp_str); + iout_ts = concordUtils::fromBigEndianBuffer(out_ts.data()); + ASSERT_EQ(prev_timestamp, iout_ts); + // Get close version + out_ts.clear(); + val = db->get(v4blockchain::detail::LATEST_KEYS_CF, + latest_keys.getCategoryPrefix("merkle") + key2, + close_timestamp_str, + &out_ts); + ASSERT_TRUE(val.has_value()); + ASSERT_EQ(*val, prev_val2 + no_flags); + ASSERT_EQ(out_ts, prev_timestamp_str); + iout_ts = concordUtils::fromBigEndianBuffer(out_ts.data()); + ASSERT_EQ(prev_timestamp, iout_ts); + } + + for (size_t i = 0; i < keys.size(); ++i) { + if (keys[i] == "merkle_key2") { + val_is_null_opts[i] = true; + ver_is_null_opts[i] = true; + } + } + testMultiGetValueAndVersion(latest_keys, 400, "merkle", keys, val_is_null_opts, ver_is_null_opts); + + for (size_t i = 0; i < keys.size(); ++i) { + if (keys[i] == "merkle_key2") { // since Key2 was added at 40 + val_is_null_opts[i] = false; + ver_is_null_opts[i] = false; + } + if (keys[i] == "merkle_key3") { // since Key3 was not there at 40 + val_is_null_opts[i] = true; + ver_is_null_opts[i] = true; + } + } + testMultiGetValueAndVersion(latest_keys, 40, "merkle", keys, val_is_null_opts, ver_is_null_opts); +} + +TEST_F(v4_kvbc, add_version_keys) { + v4blockchain::detail::LatestKeys latest_keys{db, categories, []() { return 1; }}; + std::string no_flags = {0}; + std::string stale_on_update_flag = {1}; + std::vector keys; + std::vector val_is_null_opts; + std::vector ver_is_null_opts; + uint64_t block_id1 = 1; + auto block_id1_str = v4blockchain::detail::Blockchain::generateKey(block_id1); + std::string key1 = "ver_key1"; + std::string key2 = "ver_key2"; + std::string val1 = "ver_val1"; + std::string val2 = "ver_val2"; + auto ver_val1 = categorization::ValueWithFlags{"ver_val1", false}; + auto ver_val2 = categorization::ValueWithFlags{"ver_val2", true}; + + // Block 1 + { + categorization::Updates updates; + categorization::VersionedUpdates ver_updates; + ver_updates.calculateRootHash(false); + ver_updates.addUpdate("ver_key1", "ver_val1"); + ver_updates.addUpdate("ver_key2", categorization::VersionedUpdates::Value{"ver_val2", true}); + updates.add("versioned", std::move(ver_updates)); + + auto wb = db->getBatch(); + latest_keys.addBlockKeys(updates, block_id1, wb); + db->write(std::move(wb)); + testGetValueAndVersion( + latest_keys, block_id1, "versioned", key1, false, false, keys, val_is_null_opts, ver_is_null_opts); + testGetValueAndVersion( + latest_keys, block_id1, "versioned", key2, false, false, keys, val_is_null_opts, ver_is_null_opts); + } + + std::string out_ts; + + //////////KEY1////////////////////////////// + + // without category prefix + auto val = db->get(v4blockchain::detail::LATEST_KEYS_CF, key1, block_id1_str, &out_ts); + ASSERT_FALSE(val.has_value()); + + // get key1 updated value of this timestamp + val = db->get( + v4blockchain::detail::LATEST_KEYS_CF, latest_keys.getCategoryPrefix("versioned") + key1, block_id1_str, &out_ts); + ASSERT_TRUE(val.has_value()); + + ASSERT_EQ(*val, ver_val1.data + no_flags); + ASSERT_EQ(out_ts, block_id1_str); + auto iout_ts = concordUtils::fromBigEndianBuffer(out_ts.data()); + ASSERT_EQ(block_id1, iout_ts); + out_ts.clear(); + //////////KEY2////////////////////////////// + val = db->get( + v4blockchain::detail::LATEST_KEYS_CF, latest_keys.getCategoryPrefix("versioned") + key2, block_id1_str, &out_ts); + ASSERT_TRUE(val.has_value()); + ASSERT_EQ(*val, ver_val2.data + stale_on_update_flag); + ASSERT_EQ(out_ts, block_id1_str); + iout_ts = concordUtils::fromBigEndianBuffer(out_ts.data()); + ASSERT_EQ(block_id1, iout_ts); + testMultiGetValueAndVersion(latest_keys, block_id1, "versioned", keys, val_is_null_opts, ver_is_null_opts); +} + +TEST_F(v4_kvbc, add_version_keys_adv) { + v4blockchain::detail::LatestKeys latest_keys{db, categories, []() { return 1; }}; + std::string no_flags = {0}; + std::string stale_on_update_flag = {1}; + std::vector keys; + std::vector val_is_null_opts; + std::vector ver_is_null_opts; + + uint64_t block_id1 = 1; + uint64_t block_id10 = 10; + uint64_t block_id112 = 112; + uint64_t block_id122 = 122; + uint64_t block_id180 = 180; + uint64_t block_id190 = 190; + auto block_id1_str = v4blockchain::detail::Blockchain::generateKey(block_id1); + auto block_id10_str = v4blockchain::detail::Blockchain::generateKey(block_id10); + auto block_id112_str = v4blockchain::detail::Blockchain::generateKey(block_id112); + auto block_id122_str = v4blockchain::detail::Blockchain::generateKey(block_id122); + auto block_id180_str = v4blockchain::detail::Blockchain::generateKey(block_id180); + auto block_id190_str = v4blockchain::detail::Blockchain::generateKey(block_id190); + + std::string key1 = "ver_key1"; + std::string key2 = "ver_key2"; + std::string val1 = "ver_val1"; + std::string val2 = "ver_val2"; + auto ver_val1 = categorization::ValueWithFlags{"ver_val1", false}; + auto ver_val2 = categorization::ValueWithFlags{"ver_val2", true}; + auto ver_val3 = categorization::ValueWithFlags{"ver_val_updated", false}; + auto ver_val4 = categorization::ValueWithFlags{"ver_val_after_delete", false}; + + // Block 1 + { + categorization::Updates updates; + categorization::VersionedUpdates ver_updates; + ver_updates.calculateRootHash(false); + ver_updates.addUpdate("ver_key1", "ver_val1"); + ver_updates.addUpdate("ver_key2", categorization::VersionedUpdates::Value{"ver_val2", true}); + updates.add("versioned", std::move(ver_updates)); + + auto wb = db->getBatch(); + latest_keys.addBlockKeys(updates, block_id1, wb); + db->write(std::move(wb)); + testGetValueAndVersion( + latest_keys, block_id1, "versioned", key1, false, false, keys, val_is_null_opts, ver_is_null_opts); + testGetValueAndVersion( + latest_keys, block_id1, "versioned", key2, false, false, keys, val_is_null_opts, ver_is_null_opts); + } + + // Block 2 + { + categorization::Updates updates; + categorization::VersionedUpdates ver_updates; + ver_updates.calculateRootHash(false); + ver_updates.addUpdate("ver_key1", "ver_val_updated"); + ver_updates.addDelete("ver_key2"); + updates.add("versioned", std::move(ver_updates)); + + auto wb = db->getBatch(); + latest_keys.addBlockKeys(updates, block_id112, wb); + db->write(std::move(wb)); + testGetValueAndVersion( + latest_keys, block_id112, "versioned", key1, false, false, keys, val_is_null_opts, ver_is_null_opts); + testGetValueAndVersion( + latest_keys, block_id112, "versioned", key2, true, true, keys, val_is_null_opts, ver_is_null_opts); + testGetValueAndVersion( + latest_keys, block_id1, "versioned", key2, false, false, keys, val_is_null_opts, ver_is_null_opts); + } + + // Block 3 + { + categorization::Updates updates; + categorization::VersionedUpdates ver_updates; + ver_updates.calculateRootHash(false); + ver_updates.addDelete("ver_key1"); + ver_updates.addUpdate("ver_key2", categorization::VersionedUpdates::Value{"ver_val_after_delete", false}); + updates.add("versioned", std::move(ver_updates)); + + auto wb = db->getBatch(); + latest_keys.addBlockKeys(updates, block_id180, wb); + db->write(std::move(wb)); + testGetValueAndVersion( + latest_keys, block_id180, "versioned", key1, true, true, keys, val_is_null_opts, ver_is_null_opts); + testGetValueAndVersion( + latest_keys, block_id112, "versioned", key1, false, false, keys, val_is_null_opts, ver_is_null_opts); + testGetValueAndVersion( + latest_keys, block_id180, "versioned", key2, false, false, keys, val_is_null_opts, ver_is_null_opts); + } + + std::string out_ts; + + //////////KEY1////////////////////////////// + + // get key1 value of this timestamp + { + testGetValueAndVersion( + latest_keys, block_id1, "versioned", key1, false, false, keys, val_is_null_opts, ver_is_null_opts); + auto val = db->get(v4blockchain::detail::LATEST_KEYS_CF, + latest_keys.getCategoryPrefix("versioned") + key1, + block_id1_str, + &out_ts); + ASSERT_TRUE(val.has_value()); + ASSERT_EQ(*val, ver_val1.data + no_flags); + ASSERT_EQ(out_ts, block_id1_str); + auto iout_ts = concordUtils::fromBigEndianBuffer(out_ts.data()); + ASSERT_EQ(block_id1, iout_ts); + out_ts.clear(); + } + // get key1 value of higer timestamp + { + testGetValueAndVersion( + latest_keys, block_id10, "versioned", key1, false, false, keys, val_is_null_opts, ver_is_null_opts); + auto val = db->get(v4blockchain::detail::LATEST_KEYS_CF, + latest_keys.getCategoryPrefix("versioned") + key1, + block_id10_str, + &out_ts); + ASSERT_TRUE(val.has_value()); + + ASSERT_EQ(*val, ver_val1.data + no_flags); + ASSERT_EQ(out_ts, block_id1_str); + auto iout_ts = concordUtils::fromBigEndianBuffer(out_ts.data()); + ASSERT_EQ(block_id1, iout_ts); + out_ts.clear(); + } + // get key1 updated value + { + testGetValueAndVersion( + latest_keys, block_id112, "versioned", key1, false, false, keys, val_is_null_opts, ver_is_null_opts); + auto val = db->get(v4blockchain::detail::LATEST_KEYS_CF, + latest_keys.getCategoryPrefix("versioned") + key1, + block_id112_str, + &out_ts); + ASSERT_TRUE(val.has_value()); + + ASSERT_EQ(*val, ver_val3.data + no_flags); + ASSERT_EQ(out_ts, block_id112_str); + auto iout_ts = concordUtils::fromBigEndianBuffer(out_ts.data()); + ASSERT_EQ(block_id112, iout_ts); + out_ts.clear(); + } + + ///////////KEY2///////////////// + { + testGetValueAndVersion( + latest_keys, block_id1, "versioned", key2, false, false, keys, val_is_null_opts, ver_is_null_opts); + auto val = db->get(v4blockchain::detail::LATEST_KEYS_CF, + latest_keys.getCategoryPrefix("versioned") + key2, + block_id1_str, + &out_ts); + ASSERT_TRUE(val.has_value()); + + ASSERT_EQ(*val, ver_val2.data + stale_on_update_flag); + ASSERT_EQ(out_ts, block_id1_str); + auto iout_ts = concordUtils::fromBigEndianBuffer(out_ts.data()); + ASSERT_EQ(block_id1, iout_ts); + out_ts.clear(); + } + // get key2 value of higer timestamp + { + testGetValueAndVersion( + latest_keys, block_id10, "versioned", key2, false, false, keys, val_is_null_opts, ver_is_null_opts); + auto val = db->get(v4blockchain::detail::LATEST_KEYS_CF, + latest_keys.getCategoryPrefix("versioned") + key2, + block_id10_str, + &out_ts); + ASSERT_TRUE(val.has_value()); + + ASSERT_EQ(*val, ver_val2.data + stale_on_update_flag); + ASSERT_EQ(out_ts, block_id1_str); + auto iout_ts = concordUtils::fromBigEndianBuffer(out_ts.data()); + ASSERT_EQ(block_id1, iout_ts); + out_ts.clear(); + } + // get key2 deleted value + { + testGetValueAndVersion( + latest_keys, block_id112, "versioned", key2, true, true, keys, val_is_null_opts, ver_is_null_opts); + auto val = db->get(v4blockchain::detail::LATEST_KEYS_CF, + latest_keys.getCategoryPrefix("versioned") + key2, + block_id112_str, + &out_ts); + ASSERT_FALSE(val.has_value()); + } + + // get key2 updates value + { + testGetValueAndVersion( + latest_keys, block_id180, "versioned", key2, false, false, keys, val_is_null_opts, ver_is_null_opts); + auto val = db->get(v4blockchain::detail::LATEST_KEYS_CF, + latest_keys.getCategoryPrefix("versioned") + key2, + block_id180_str, + &out_ts); + ASSERT_TRUE(val.has_value()); + ASSERT_EQ(*val, ver_val4.data + no_flags); + ASSERT_EQ(out_ts, block_id180_str); + auto iout_ts = concordUtils::fromBigEndianBuffer(out_ts.data()); + ASSERT_EQ(block_id180, iout_ts); + out_ts.clear(); + } + + std::vector blocks{block_id1, block_id10, block_id112, block_id122, block_id180, block_id190}; + for (const auto blk : blocks) { + for (size_t i = 0; i < keys.size(); ++i) { + val_is_null_opts[i] = false; + ver_is_null_opts[i] = false; + if ((blk == block_id112) || (blk == block_id122)) { + if (keys[i] == key2) { + val_is_null_opts[i] = true; + ver_is_null_opts[i] = true; + } + } else if ((blk == block_id180) || (blk == block_id190)) { + if (keys[i] == key1) { + val_is_null_opts[i] = true; + ver_is_null_opts[i] = true; + } + } + } + testMultiGetValueAndVersion(latest_keys, blk, "versioned", keys, val_is_null_opts, ver_is_null_opts); + } +} + +TEST_F(v4_kvbc, add_immutable_keys) { + v4blockchain::detail::LatestKeys latest_keys{db, categories, []() { return 1; }}; + std::string no_flags = {0}; + std::string stale_on_update_flag = {1}; + std::vector keys; + std::vector val_is_null_opts; + std::vector ver_is_null_opts; + uint64_t block_id1 = 1; + auto block_id1_str = v4blockchain::detail::Blockchain::generateKey(block_id1); + std::string key1 = "imm_key1"; + std::string val1 = "imm_val1"; + + auto imm_val1 = categorization::ImmutableValueUpdate{"imm_val1", {"1", "2"}}; + + // Block 1 + { + categorization::Updates updates; + categorization::ImmutableUpdates imm_updates; + imm_updates.addUpdate("imm_key1", categorization::ImmutableUpdates::ImmutableValue{"imm_val1", {"1", "2"}}); + updates.add("immutable", std::move(imm_updates)); + + auto wb = db->getBatch(); + latest_keys.addBlockKeys(updates, block_id1, wb); + db->write(std::move(wb)); + } + + std::string out_ts; + + //////////KEY1////////////////////////////// + testGetValueAndVersion( + latest_keys, block_id1, "immutable", key1, false, false, keys, val_is_null_opts, ver_is_null_opts); + // without category prefix + auto val = db->get(v4blockchain::detail::LATEST_KEYS_CF, key1, block_id1_str, &out_ts); + ASSERT_FALSE(val.has_value()); + + // get key1 updated value of this timestamp + val = db->get( + v4blockchain::detail::LATEST_KEYS_CF, latest_keys.getCategoryPrefix("immutable") + key1, block_id1_str, &out_ts); + ASSERT_TRUE(val.has_value()); + ASSERT_EQ(*val, imm_val1.data + stale_on_update_flag); + ASSERT_EQ(out_ts, block_id1_str); + auto iout_ts = concordUtils::fromBigEndianBuffer(out_ts.data()); + ASSERT_EQ(block_id1, iout_ts); + out_ts.clear(); + testMultiGetValueAndVersion(latest_keys, block_id1, "immutable", keys, val_is_null_opts, ver_is_null_opts); +} + +TEST_F(v4_kvbc, add_immutable_keys_adv) { + v4blockchain::detail::LatestKeys latest_keys{db, categories, []() { return 1; }}; + std::string no_flags = {0}; + std::string stale_on_update_flag = {1}; + std::vector keys; + std::vector val_is_null_opts; + std::vector ver_is_null_opts; + uint64_t block_id1 = 1; + auto block_id1_str = v4blockchain::detail::Blockchain::generateKey(block_id1); + uint64_t block_id10 = 10; + auto block_id10_str = v4blockchain::detail::Blockchain::generateKey(block_id10); + uint64_t block_id100 = 100; + auto block_id100_str = v4blockchain::detail::Blockchain::generateKey(block_id100); + std::string key1 = "imm_key1"; + std::string val1 = "imm_val1"; + + auto imm_val1 = categorization::ImmutableValueUpdate{"imm_val1", {"1", "2"}}; + + // Block 1 + { + categorization::Updates updates; + categorization::ImmutableUpdates imm_updates; + imm_updates.addUpdate("imm_key1", categorization::ImmutableUpdates::ImmutableValue{"imm_val1", {"1", "2"}}); + updates.add("immutable", std::move(imm_updates)); + + auto wb = db->getBatch(); + latest_keys.addBlockKeys(updates, block_id1, wb); + db->write(std::move(wb)); + } + + std::string out_ts; + + //////////KEY1////////////////////////////// + testGetValueAndVersion( + latest_keys, block_id1, "immutable", key1, false, false, keys, val_is_null_opts, ver_is_null_opts); + // without category prefix + { + auto val = db->get(v4blockchain::detail::LATEST_KEYS_CF, key1, block_id1_str, &out_ts); + ASSERT_FALSE(val.has_value()); + + // get key1 updated value of this timestamp + val = db->get(v4blockchain::detail::LATEST_KEYS_CF, + latest_keys.getCategoryPrefix("immutable") + key1, + block_id1_str, + &out_ts); + ASSERT_TRUE(val.has_value()); + ASSERT_EQ(*val, imm_val1.data + stale_on_update_flag); + ASSERT_EQ(out_ts, block_id1_str); + auto iout_ts = concordUtils::fromBigEndianBuffer(out_ts.data()); + ASSERT_EQ(block_id1, iout_ts); + out_ts.clear(); + + val = db->get(v4blockchain::detail::LATEST_KEYS_CF, + latest_keys.getCategoryPrefix("immutable") + key1, + block_id10_str, + &out_ts); + ASSERT_TRUE(val.has_value()); + ASSERT_EQ(out_ts, block_id1_str); + iout_ts = concordUtils::fromBigEndianBuffer(out_ts.data()); + ASSERT_EQ(block_id1, iout_ts); + out_ts.clear(); + } + + // Block 2 + { + categorization::Updates updates; + categorization::ImmutableUpdates imm_updates; + imm_updates.addUpdate("imm_key1", + categorization::ImmutableUpdates::ImmutableValue{"imm_val1_updated", {"11", "2"}}); + updates.add("immutable", std::move(imm_updates)); + + auto wb = db->getBatch(); + ASSERT_THROW(latest_keys.addBlockKeys(updates, block_id100, wb), std::runtime_error); + db->write(std::move(wb)); + testGetValueAndVersion( + latest_keys, block_id100, "immutable", key1, false, false, keys, val_is_null_opts, ver_is_null_opts); + } + // Check that original key still exists. + { + auto val = db->get(v4blockchain::detail::LATEST_KEYS_CF, + latest_keys.getCategoryPrefix("immutable") + key1, + block_id1_str, + &out_ts); + + ASSERT_EQ(*val, imm_val1.data + stale_on_update_flag); + ASSERT_EQ(out_ts, block_id1_str); + auto iout_ts = concordUtils::fromBigEndianBuffer(out_ts.data()); + ASSERT_EQ(block_id1, iout_ts); + out_ts.clear(); + + val = db->get(v4blockchain::detail::LATEST_KEYS_CF, + latest_keys.getCategoryPrefix("immutable") + key1, + block_id100_str, + &out_ts); + ASSERT_TRUE(val.has_value()); + ASSERT_EQ(out_ts, block_id1_str); + iout_ts = concordUtils::fromBigEndianBuffer(out_ts.data()); + ASSERT_EQ(block_id1, iout_ts); + out_ts.clear(); + } + testMultiGetValueAndVersion(latest_keys, block_id1, "immutable", keys, val_is_null_opts, ver_is_null_opts); + testMultiGetValueAndVersion(latest_keys, block_id10, "immutable", keys, val_is_null_opts, ver_is_null_opts); + testMultiGetValueAndVersion(latest_keys, block_id100, "immutable", keys, val_is_null_opts, ver_is_null_opts); +} + +TEST_F(v4_kvbc, detect_stale_on_update) { + v4blockchain::detail::LatestKeys latest_keys{db, categories, []() { return 1; }}; + std::vector keys; + std::vector val_is_null_opts; + std::vector ver_is_null_opts; + uint64_t block_id1 = 1; + auto block_id1_str = v4blockchain::detail::Blockchain::generateKey(block_id1); + std::string key1 = "ver_key1"; + std::string key2 = "ver_key2"; + std::string val1 = "ver_val1"; + std::string val2 = "ver_val2"; + auto ver_val1 = categorization::ValueWithFlags{"ver_val1", false}; + auto ver_val2 = categorization::ValueWithFlags{"ver_val2", true}; + + // Block 1 + { + categorization::Updates updates; + categorization::VersionedUpdates ver_updates; + ver_updates.calculateRootHash(false); + ver_updates.addUpdate("ver_key1", "ver_val1"); + ver_updates.addUpdate("ver_key2", categorization::VersionedUpdates::Value{"ver_val2", true}); + updates.add("versioned", std::move(ver_updates)); + + auto wb = db->getBatch(); + latest_keys.addBlockKeys(updates, block_id1, wb); + db->write(std::move(wb)); + testGetValueAndVersion( + latest_keys, block_id1, "versioned", key1, false, false, keys, val_is_null_opts, ver_is_null_opts); + testGetValueAndVersion( + latest_keys, block_id1, "versioned", key2, false, false, keys, val_is_null_opts, ver_is_null_opts); + } + + std::string out_ts; + + //////////KEY1////////////////////////////// + // get key1 updated value of this timestamp + { + auto val = db->get(v4blockchain::detail::LATEST_KEYS_CF, + latest_keys.getCategoryPrefix("versioned") + key1, + block_id1_str, + &out_ts); + ASSERT_TRUE(val.has_value()); + ASSERT_FALSE(v4blockchain::detail::LatestKeys::isStaleOnUpdate(*val)); + } + + //////////KEY2////////////////////////////// + { + auto val = db->get(v4blockchain::detail::LATEST_KEYS_CF, + latest_keys.getCategoryPrefix("versioned") + key2, + block_id1_str, + &out_ts); + ASSERT_TRUE(val.has_value()); + ASSERT_TRUE(v4blockchain::detail::LatestKeys::isStaleOnUpdate(*val)); + } +} + +TEST_F(v4_kvbc, compaction_filter) { + v4blockchain::detail::LatestKeys latest_keys{db, categories, []() { return 100; }}; + std::string no_flags = {0}; + std::string stale_on_update_flag = {1}; + + uint64_t block_id1 = 1; + uint64_t block_id100 = 100; + auto block_id1_str = v4blockchain::detail::Blockchain::generateKey(block_id1); + auto block_id100_str = v4blockchain::detail::Blockchain::generateKey(block_id100); + std::string key1 = "ver_key1"; + std::string key2 = "ver_key2"; + std::string key3 = "ver_key3"; + std::string val1 = "ver_val1"; + std::string val2 = "ver_val2"; + auto ver_val1 = categorization::ValueWithFlags{"ver_val1", false}; + auto ver_val2 = categorization::ValueWithFlags{"ver_val2", true}; + + // Block 1 + { + categorization::Updates updates; + categorization::VersionedUpdates ver_updates; + ver_updates.calculateRootHash(false); + ver_updates.addUpdate("ver_key1", "ver_val1"); + ver_updates.addUpdate("ver_key2", categorization::VersionedUpdates::Value{"ver_val2", true}); + updates.add("versioned", std::move(ver_updates)); + + auto wb = db->getBatch(); + latest_keys.addBlockKeys(updates, block_id1, wb); + db->write(std::move(wb)); + } + + // Block 100 + { + categorization::Updates updates; + categorization::VersionedUpdates ver_updates; + ver_updates.calculateRootHash(false); + ver_updates.addUpdate("ver_key3", categorization::VersionedUpdates::Value{"ver_val3", true}); + updates.add("versioned", std::move(ver_updates)); + + auto wb = db->getBatch(); + latest_keys.addBlockKeys(updates, block_id100, wb); + db->write(std::move(wb)); + } + + std::string out_ts; + + //////////KEY1////////////////////////////// + // get key1 updated value of this timestamp + { + auto val = db->get(v4blockchain::detail::LATEST_KEYS_CF, + latest_keys.getCategoryPrefix("versioned") + key1, + block_id1_str, + &out_ts); + ASSERT_TRUE(val.has_value()); + auto st_key = latest_keys.getCategoryPrefix("versioned") + key1 + block_id1_str; + ASSERT_FALSE(v4blockchain::detail::LatestKeys::isStaleOnUpdate(*val)); + // Not stale on update + bool changed; + ASSERT_FALSE(latest_keys.getCompFilter()->Filter(0, st_key, *val, &out_ts, &changed)); + } + + //////////KEY2////////////////////////////// + { + auto val = db->get(v4blockchain::detail::LATEST_KEYS_CF, + latest_keys.getCategoryPrefix("versioned") + key2, + block_id1_str, + &out_ts); + ASSERT_TRUE(val.has_value()); + ASSERT_TRUE(v4blockchain::detail::LatestKeys::isStaleOnUpdate(*val)); + + auto st_key = latest_keys.getCategoryPrefix("versioned") + key2 + block_id1_str; + // stale on update lower than genesis + bool changed; + ASSERT_TRUE(latest_keys.getCompFilter()->Filter(0, st_key, *val, &out_ts, &changed)); + } + + //////////KEY3////////////////////////////// + { + auto val = db->get(v4blockchain::detail::LATEST_KEYS_CF, + latest_keys.getCategoryPrefix("versioned") + key3, + block_id100_str, + &out_ts); + ASSERT_TRUE(val.has_value()); + ASSERT_TRUE(v4blockchain::detail::LatestKeys::isStaleOnUpdate(*val)); + + auto st_key = latest_keys.getCategoryPrefix("versioned") + key3 + block_id100_str; + // stale on update equal to genesis + bool changed; + ASSERT_FALSE(latest_keys.getCompFilter()->Filter(0, st_key, *val, &out_ts, &changed)); + } +} + +} // end namespace + +int main(int argc, char** argv) { + InitGoogleTest(&argc, argv); + return RUN_ALL_TESTS(); +} diff --git a/kvbc/tools/db_editor/include/kv_blockchain_db_editor.hpp b/kvbc/tools/db_editor/include/kv_blockchain_db_editor.hpp index 40f694a64e..5ac29b4249 100644 --- a/kvbc/tools/db_editor/include/kv_blockchain_db_editor.hpp +++ b/kvbc/tools/db_editor/include/kv_blockchain_db_editor.hpp @@ -678,7 +678,6 @@ struct CompareTo { throw std::invalid_argument{"Missing PATH-TO-OTHER-DB argument"}; } - const auto read_only = true; const auto other_adapter = getAdapter(args.values.front(), read_only); const auto main_genesis = main_adapter.getGenesisBlockId(); diff --git a/logging/include/Logger.hpp b/logging/include/Logger.hpp index 52cc493f49..026af750a5 100644 --- a/logging/include/Logger.hpp +++ b/logging/include/Logger.hpp @@ -31,6 +31,7 @@ extern logging::Logger THRESHSIGN_LOG; extern logging::Logger BLS_LOG; extern logging::Logger KEY_EX_LOG; extern logging::Logger CAT_BLOCK_LOG; +extern logging::Logger V4_BLOCK_LOG; extern logging::Logger VC_LOG; extern logging::Logger ST_SRC_LOG; extern logging::Logger ST_DST_LOG; diff --git a/logging/src/Logger.cpp b/logging/src/Logger.cpp index cceae71a67..044dcdc468 100644 --- a/logging/src/Logger.cpp +++ b/logging/src/Logger.cpp @@ -28,6 +28,7 @@ logging::Logger THRESHSIGN_LOG = logging::getLogger("concord.bft.threshsign"); logging::Logger BLS_LOG = logging::getLogger("concord.bft.threshsign.bls"); logging::Logger KEY_EX_LOG = logging::getLogger("concord.bft.key-exchange"); logging::Logger CAT_BLOCK_LOG = logging::getLogger("concord.kvbc.categorized-blockchain"); +logging::Logger V4_BLOCK_LOG = logging::getLogger("concord.kvbc.v4-blockchain"); logging::Logger VC_LOG = logging::getLogger("concord.bft.viewchange"); logging::Logger ST_DST_LOG = logging::getLogger("concord.bft.st.dst"); logging::Logger ST_SRC_LOG = logging::getLogger("concord.bft.st.src"); diff --git a/storage/CMakeLists.txt b/storage/CMakeLists.txt index ba8777332e..d2b198c12c 100644 --- a/storage/CMakeLists.txt +++ b/storage/CMakeLists.txt @@ -25,7 +25,7 @@ if (BUILD_ROCKSDB_STORAGE) find_library(LIBSNAPPY snappy) #cmake_policy(SET CMP0076 NEW) for cmake 3.14 - target_sources(concordbft_storage PRIVATE src/rocksdb_client.cpp src/rocksdb_key_comparator.cpp) + target_sources(concordbft_storage PRIVATE src/rocksdb_client.cpp src/rocksdb_key_comparator.cpp src/time_stamp_comparator.cpp) target_compile_definitions(concordbft_storage PUBLIC USE_ROCKSDB=1 __BASE=1 SPARSE_STATE=1) target_include_directories(concordbft_storage PUBLIC ${ROCKSDB_INCLUDE_DIR}) target_link_libraries(concordbft_storage PRIVATE ${ROCKSDB_LIBRARY} ${LIBBZ2} ${LIBLZ4} ${LIBZSTD} ${LIBZ} ${LIBSNAPPY} ${CMAKE_DL_LIBS}) diff --git a/storage/include/rocksdb/native_client.h b/storage/include/rocksdb/native_client.h index df40321f3a..fa154defee 100644 --- a/storage/include/rocksdb/native_client.h +++ b/storage/include/rocksdb/native_client.h @@ -79,9 +79,18 @@ class NativeClient : public std::enable_shared_from_this { // Column family single key read-write interface. template void put(const std::string &cFamily, const KeySpan &key, const ValueSpan &value); + template + void put(const std::string &cFamily, const KeySpan &key, const TimestampSpan &ts, const ValueSpan &value); // Returns nullopt if the key is not found. template std::optional get(const std::string &cFamily, const KeySpan &key) const; + // For Column families that use timestamps + template + std::optional get(const std::string &cFamily, + const KeySpan &key, + const TimestampSpan &inTs, + std::string *timestamp) const; + // Returns nullopt if the key is not found. template std::optional<::rocksdb::PinnableSlice> getSlice(const std::string &cFamily, const KeySpan &key) const; @@ -89,9 +98,13 @@ class NativeClient : public std::enable_shared_from_this { template void del(const std::string &cFamily, const KeySpan &key); + template + void del(const std::string &cFamily, const KeySpan &key, const TimestampSpan &ts); + // Single key read-write interface for the default column family. template void put(const KeySpan &key, const ValueSpan &value); + // Returns nullopt if the key is not found. template std::optional get(const KeySpan &key) const; @@ -159,6 +172,10 @@ class NativeClient : public std::enable_shared_from_this { const ::rocksdb::ImportColumnFamilyOptions &importOpts, const ::rocksdb::ExportImportFilesMetaData &metadata, const ::rocksdb::ColumnFamilyOptions &cfOpts = ::rocksdb::ColumnFamilyOptions{}); + + bool createColumnFamilyIfNotExisting(const std::string &cf, + const ::rocksdb::Comparator *comparator = nullptr, + const ::rocksdb::CompactionFilter *filter = nullptr); // 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 0f6c099ff8..976e9d94d5 100644 --- a/storage/include/rocksdb/native_client.ipp +++ b/storage/include/rocksdb/native_client.ipp @@ -87,6 +87,19 @@ void NativeClient::put(const std::string &cFamily, const KeySpan &key, const Val detail::throwOnError("put() failed"sv, std::move(s)); } +template +void NativeClient::put(const std::string &cFamily, + const KeySpan &key, + const TimestampSpan &ts, + const ValueSpan &value) { + auto options = ::rocksdb::WriteOptions{}; + auto slTs = ::rocksdb::Slice(ts); + options.timestamp = &slTs; + auto s = + client_->dbInstance_->Put(options, columnFamilyHandle(cFamily), detail::toSlice(key), detail::toSlice(value)); + detail::throwOnError("put() failed"sv, std::move(s)); +} + template std::optional NativeClient::get(const std::string &cFamily, const KeySpan &key) const { auto value = std::string{}; @@ -99,6 +112,24 @@ std::optional NativeClient::get(const std::string &cFamily, const K return value; } +template +std::optional NativeClient::get(const std::string &cFamily, + const KeySpan &key, + const TimestampSpan &inTs, + std::string *timestamp) const { + auto value = std::string{}; + auto ro = ::rocksdb::ReadOptions{}; + auto slTs = detail::toSlice(inTs); + ro.timestamp = &slTs; + + auto s = client_->dbInstance_->Get(ro, columnFamilyHandle(cFamily), detail::toSlice(key), &value, timestamp); + if (s.IsNotFound()) { + return std::nullopt; + } + detail::throwOnError("get() failed"sv, std::move(s)); + return value; +} + template std::optional<::rocksdb::PinnableSlice> NativeClient::getSlice(const std::string &cFamily, const KeySpan &key) const { auto slice = ::rocksdb::PinnableSlice{}; @@ -117,6 +148,15 @@ void NativeClient::del(const std::string &cFamily, const KeySpan &key) { detail::throwOnError("del() failed"sv, std::move(s)); } +template +void NativeClient::del(const std::string &cFamily, const KeySpan &key, const TimestampSpan &ts) { + auto options = ::rocksdb::WriteOptions{}; + auto slTs = ::rocksdb::Slice(ts); + options.timestamp = &slTs; + auto s = client_->dbInstance_->Delete(options, columnFamilyHandle(cFamily), detail::toSlice(key)); + detail::throwOnError("del() failed"sv, std::move(s)); +} + template void NativeClient::put(const KeySpan &key, const ValueSpan &value) { put(defaultColumnFamily(), key, value); @@ -242,6 +282,24 @@ inline void NativeClient::createColumnFamilyWithImport(const std::string &cFamil client_->cf_handles_[cFamily] = std::move(handleUniquePtr); } +inline bool NativeClient::createColumnFamilyIfNotExisting(const std::string &cf, + const ::rocksdb::Comparator *comparator, + const ::rocksdb::CompactionFilter *filter) { + if (!hasColumnFamily(cf)) { + auto cf_options = ::rocksdb::ColumnFamilyOptions{}; + if (comparator) { + cf_options.comparator = comparator; + } + if (filter) { + cf_options.compaction_filter = filter; + } + createColumnFamily(cf, cf_options); + return true; + } + return false; +} +// Re + inline ::rocksdb::ColumnFamilyOptions NativeClient::columnFamilyOptions(const std::string &cFamily) const { auto family = columnFamilyHandle(cFamily); auto descriptor = ::rocksdb::ColumnFamilyDescriptor{}; diff --git a/storage/include/rocksdb/native_write_batch.h b/storage/include/rocksdb/native_write_batch.h index 1dddf8f746..1b25a292ef 100644 --- a/storage/include/rocksdb/native_write_batch.h +++ b/storage/include/rocksdb/native_write_batch.h @@ -35,18 +35,28 @@ class NativeWriteBatch { template void put(const KeySpan &key, const ValueSpan &value); - // Multi-value put used to eliminate excess copying. + // Multi-key/value put used to eliminate excess copying. template void put(const std::string &cFamily, const KeySpan &key, const std::array<::rocksdb::Slice, N> &value); + + template + void put(const std::string &cFamily, + const std::array<::rocksdb::Slice, K> &key, + const std::array<::rocksdb::Slice, N> &value); template void put(const KeySpan &key, const std::array<::rocksdb::Slice, N> &value); // Deleting a key that doesn't exist is not an error. template void del(const std::string &cFamily, const KeySpan &key); + template void del(const KeySpan &key); + // Multi key used to eliminate excess copying. + template + void del(const std::string &cFamily, const std::array<::rocksdb::Slice, K> &key); + // Remove the DB entries in the range [beginKey, endKey). template void delRange(const std::string &cFamily, const BeginSpan &beginKey, const EndSpan &endKey); diff --git a/storage/include/rocksdb/native_write_batch.ipp b/storage/include/rocksdb/native_write_batch.ipp index cc8bf64a58..8e3eac9580 100644 --- a/storage/include/rocksdb/native_write_batch.ipp +++ b/storage/include/rocksdb/native_write_batch.ipp @@ -44,6 +44,16 @@ void NativeWriteBatch::put(const std::string &cFamily, detail::throwOnError("batch put(multi-value) failed"sv, std::move(s)); } +template +void NativeWriteBatch::put(const std::string &cFamily, + const std::array<::rocksdb::Slice, K> &key, + const std::array<::rocksdb::Slice, N> &value) { + auto s = batch_.Put(client_->columnFamilyHandle(cFamily), + ::rocksdb::SliceParts(key.data(), K), + ::rocksdb::SliceParts(value.data(), N)); + detail::throwOnError("batch put(multi-key-value) failed"sv, std::move(s)); +} + template void put(const KeySpan &key, const std::array<::rocksdb::Slice, N> &value) { put(NativeClient::defaultColumnFamily(), key, value); @@ -61,6 +71,12 @@ inline void NativeWriteBatch::del<::rocksdb::Slice>(const std::string &cFamily, detail::throwOnError("batch del failed"sv, std::move(s)); } +template +void NativeWriteBatch::del(const std::string &cFamily, const std::array<::rocksdb::Slice, K> &key) { + auto s = batch_.Delete(client_->columnFamilyHandle(cFamily), ::rocksdb::SliceParts(key.data(), K)); + detail::throwOnError("batch del failed"sv, std::move(s)); +} + template void NativeWriteBatch::del(const KeySpan &key) { del(NativeClient::defaultColumnFamily(), key); diff --git a/storage/include/rocksdb/time_stamp_comparator.h b/storage/include/rocksdb/time_stamp_comparator.h new file mode 100644 index 0000000000..8f8dad3d9c --- /dev/null +++ b/storage/include/rocksdb/time_stamp_comparator.h @@ -0,0 +1,27 @@ +// Copyright 2022 VMware, all rights reserved + +#pragma once +#ifdef USE_ROCKSDB + +#include + +namespace concord { +namespace storage { +namespace rocksdb { + +/* +Returns a Comparator to compare user defined timestamp for using with +RocksDB timestamp API. +This comparator is used with uint64_t encoded as big endian string using +concordUtils::toBigEndianStringBuffer + +it compares the byte order of two such strings. +*/ +static constexpr size_t TIME_STAMP_SIZE = sizeof(std::uint64_t); +const ::rocksdb::Comparator* getLexicographic64TsComparator(); +::rocksdb::Slice ExtractTimestampFromUserKey(const ::rocksdb::Slice& user_key, size_t ts_sz); +::rocksdb::Slice StripTimestampFromUserKey(const ::rocksdb::Slice& user_key, size_t ts_sz); +} // namespace rocksdb +} // namespace storage +} // namespace concord +#endif diff --git a/storage/src/time_stamp_comparator.cpp b/storage/src/time_stamp_comparator.cpp new file mode 100644 index 0000000000..6ca9cb5940 --- /dev/null +++ b/storage/src/time_stamp_comparator.cpp @@ -0,0 +1,73 @@ +#include "rocksdb/time_stamp_comparator.h" +#include "endianness.hpp" +#include "assertUtils.hpp" + +namespace concord { +namespace storage { +namespace rocksdb { + +#ifdef USE_ROCKSDB + +using namespace ::rocksdb; + +Slice ExtractTimestampFromUserKey(const Slice& user_key, size_t ts_sz) { + ConcordAssert(user_key.size() >= ts_sz); + return Slice(user_key.data() + user_key.size() - ts_sz, ts_sz); +} + +Slice StripTimestampFromUserKey(const Slice& user_key, size_t ts_sz) { + ConcordAssertGE(user_key.size(), ts_sz); + return Slice(user_key.data(), user_key.size() - ts_sz); +} + +class Lexicographic64TsComparator : public Comparator { + public: + Lexicographic64TsComparator() : Comparator(/*ts_sz=*/TIME_STAMP_SIZE), cmp_without_ts_(BytewiseComparator()) { + ConcordAssertNE(cmp_without_ts_, nullptr); + ConcordAssertEQ(cmp_without_ts_->timestamp_size(), 0); + } + const char* Name() const override { return "Lexicographic64TsComparator"; } + void FindShortSuccessor(std::string*) const override {} + void FindShortestSeparator(std::string*, const Slice&) const override {} + int Compare(const Slice& a, const Slice& b) const override { + int ret = CompareWithoutTimestamp(a, b); + size_t ts_sz = timestamp_size(); + if (ret != 0) { + return ret; + } + // Compare timestamp. + // For the same user key with different timestamps, larger (newer) timestamp + // comes first. + return -CompareTimestamp(ExtractTimestampFromUserKey(a, ts_sz), ExtractTimestampFromUserKey(b, ts_sz)); + } + using Comparator::CompareWithoutTimestamp; + int CompareWithoutTimestamp(const Slice& a, bool a_has_ts, const Slice& b, bool b_has_ts) const override { + const size_t ts_sz = timestamp_size(); + if (a_has_ts) ConcordAssertGE(a.size(), ts_sz); + if (b_has_ts) ConcordAssertGE(b.size(), ts_sz); + Slice lhs = a_has_ts ? StripTimestampFromUserKey(a, ts_sz) : a; + Slice rhs = b_has_ts ? StripTimestampFromUserKey(b, ts_sz) : b; + return cmp_without_ts_->Compare(lhs, rhs); + } + int CompareTimestamp(const Slice& ts1, const Slice& ts2) const override { + ConcordAssertEQ(ts1.size(), TIME_STAMP_SIZE); + ConcordAssertEQ(ts2.size(), TIME_STAMP_SIZE); + return ts1.compare(ts2); + } + + virtual ~Lexicographic64TsComparator() = default; + + private: + const Comparator* cmp_without_ts_{nullptr}; +}; + +const Comparator* getLexicographic64TsComparator() { + static Lexicographic64TsComparator instance_; + return &instance_; +} + +#endif + +} // namespace rocksdb +} // namespace storage +} // namespace concord \ No newline at end of file diff --git a/storage/test/native_rocksdb_client_test.cpp b/storage/test/native_rocksdb_client_test.cpp index 930e60dabc..7219caf7f4 100644 --- a/storage/test/native_rocksdb_client_test.cpp +++ b/storage/test/native_rocksdb_client_test.cpp @@ -19,6 +19,8 @@ #include "rocksdb/native_client.h" #include "sliver.hpp" #include "storage/test/storage_test_common.h" +#include "rocksdb/time_stamp_comparator.h" +#include "endianness.hpp" #include #include @@ -453,7 +455,7 @@ TEST_F(native_rocksdb_test, batch_del_invalid_range_in_default_family) { db->put(key2, value2); auto batch = db->getBatch(); batch.delRange(key3, key1); - db->write(std::move(batch)); + EXPECT_THROW(db->write(std::move(batch)), concord::storage::rocksdb::RocksDBException); const auto dbValue1 = db->get(key1); ASSERT_TRUE(dbValue1.has_value()); ASSERT_EQ(*dbValue1, value1); @@ -469,7 +471,7 @@ TEST_F(native_rocksdb_test, batch_del_invalid_range_in_a_family) { db->put(cf, key2, value2); auto batch = db->getBatch(); batch.delRange(cf, key3, key1); - db->write(std::move(batch)); + EXPECT_THROW(db->write(std::move(batch)), concord::storage::rocksdb::RocksDBException); const auto dbValue1 = db->get(cf, key1); ASSERT_TRUE(dbValue1.has_value()); ASSERT_EQ(*dbValue1, value1); @@ -1087,6 +1089,704 @@ TEST_F(native_rocksdb_test, restore_db_from_snapshot) { ASSERT_EQ(checkPoints.size(), 0); } +//////TIMESTAMP SUPPORT///////// + +template +auto getSliceArray(const Sliceable &... sls) { + return std::array<::rocksdb::Slice, sizeof...(sls)>{sls...}; +} + +TEST_F(native_rocksdb_test, put_and_get_with_timestamp_basic) { + const auto cf = "cf"s; + auto cf_options = ::rocksdb::ColumnFamilyOptions{}; + cf_options.comparator = concord::storage::rocksdb::getLexicographic64TsComparator(); + db->createColumnFamily(cf, cf_options); + + uint64_t timestamp = 420; + auto tsStr = concordUtils::toBigEndianStringBuffer(timestamp); + std::string key = "time"; + std::string value = "val"; + db->put(cf, key, tsStr, value); + + std::string outTs; + auto val = db->get(cf, key, tsStr, &outTs); + ASSERT_TRUE(val.has_value()); + ASSERT_EQ(*val, value); + ASSERT_EQ(outTs, tsStr); + auto ioutTs = concordUtils::fromBigEndianBuffer(outTs.data()); + ASSERT_EQ(timestamp, ioutTs); +} + +TEST_F(native_rocksdb_test, put_and_get_with_timestamp_adv) { + const auto cf = "cf"s; + auto cf_options = ::rocksdb::ColumnFamilyOptions{}; + cf_options.comparator = concord::storage::rocksdb::getLexicographic64TsComparator(); + db->createColumnFamily(cf, cf_options); + + uint64_t timestamp = 420; + auto tsStr = concordUtils::toBigEndianStringBuffer(timestamp); + std::string key = "time"; + std::string value = "val"; + db->put(cf, key, tsStr, value); + + // Try to get with lower timestamp + std::string outTs; + uint64_t lowerTimestamp = 419; + auto lowtsStr = concordUtils::toBigEndianStringBuffer(lowerTimestamp); + auto val = db->get(cf, key, lowtsStr, &outTs); + ASSERT_FALSE(val.has_value()); + + // Get with higer + uint64_t highrTimestamp = 425; + auto highsStr = concordUtils::toBigEndianStringBuffer(highrTimestamp); + val = db->get(cf, key, highsStr, &outTs); + ASSERT_TRUE(val.has_value()); + ASSERT_EQ(*val, value); + // Should return 420 + ASSERT_EQ(outTs, tsStr); + auto ioutTs = concordUtils::fromBigEndianBuffer(outTs.data()); + ASSERT_EQ(timestamp, ioutTs); +} + +TEST_F(native_rocksdb_test, put_and_get_multiple_timestamps) { + const auto cf = "cf"s; + auto cf_options = ::rocksdb::ColumnFamilyOptions{}; + cf_options.comparator = concord::storage::rocksdb::getLexicographic64TsComparator(); + db->createColumnFamily(cf, cf_options); + + uint64_t timestamp = 420; + auto tsStr = concordUtils::toBigEndianStringBuffer(timestamp); + std::string key = "time"; + std::string value = "val"; + db->put(cf, key, tsStr, value); + + uint64_t timestamp2 = 430; + auto tsStr2 = concordUtils::toBigEndianStringBuffer(timestamp2); + std::string value2 = "val2"; + db->put(cf, key, tsStr2, value2); + + // Try to get with lower timestamp + std::string outTs; + uint64_t lowerTimestamp = 419; + auto lowtsStr = concordUtils::toBigEndianStringBuffer(lowerTimestamp); + auto val = db->get(cf, key, lowtsStr, &outTs); + ASSERT_FALSE(val.has_value()); + + uint64_t firstTs = 420; + auto ftsStr = concordUtils::toBigEndianStringBuffer(firstTs); + val = db->get(cf, key, ftsStr, &outTs); + ASSERT_TRUE(val.has_value()); + ASSERT_EQ(*val, value); + // Should return 420 + ASSERT_EQ(outTs, tsStr); + auto ioutTs = concordUtils::fromBigEndianBuffer(outTs.data()); + ASSERT_EQ(timestamp, ioutTs); + + // in between shouold return 420 + uint64_t midTs = 424; + auto mtsStr = concordUtils::toBigEndianStringBuffer(midTs); + val = db->get(cf, key, mtsStr, &outTs); + ASSERT_TRUE(val.has_value()); + ASSERT_EQ(*val, value); + // Should return 420 + ASSERT_EQ(outTs, tsStr); + ioutTs = concordUtils::fromBigEndianBuffer(outTs.data()); + ASSERT_EQ(timestamp, ioutTs); + + // second return 430 + uint64_t hTs = 430; + auto htsStr = concordUtils::toBigEndianStringBuffer(hTs); + val = db->get(cf, key, htsStr, &outTs); + ASSERT_TRUE(val.has_value()); + ASSERT_EQ(*val, value2); + ASSERT_EQ(outTs, tsStr2); + ioutTs = concordUtils::fromBigEndianBuffer(outTs.data()); + ASSERT_EQ(timestamp2, ioutTs); + + // higher return 430 + uint64_t hTs2 = 43000; + auto htsStr2 = concordUtils::toBigEndianStringBuffer(hTs2); + val = db->get(cf, key, htsStr2, &outTs); + ASSERT_TRUE(val.has_value()); + ASSERT_EQ(*val, value2); + ASSERT_EQ(outTs, tsStr2); + ioutTs = concordUtils::fromBigEndianBuffer(outTs.data()); + ASSERT_EQ(timestamp2, ioutTs); +} + +TEST_F(native_rocksdb_test, delete_with_timestamp) { + const auto cf = "cf"s; + auto cf_options = ::rocksdb::ColumnFamilyOptions{}; + cf_options.comparator = concord::storage::rocksdb::getLexicographic64TsComparator(); + db->createColumnFamily(cf, cf_options); + + uint64_t timestamp = 420; + auto tsStr = concordUtils::toBigEndianStringBuffer(timestamp); + std::string key = "time"; + std::string value = "val"; + db->put(cf, key, tsStr, value); + + // try to delete with lower timestamp + uint64_t lowerTimestamp = 400; + auto lowerTsStr = concordUtils::toBigEndianStringBuffer(lowerTimestamp); + db->del(cf, key, lowerTsStr); + + // get the value of 420 + { + std::string outTs; + auto val = db->get(cf, key, tsStr, &outTs); + ASSERT_TRUE(val.has_value()); + ASSERT_EQ(*val, value); + ASSERT_EQ(outTs, tsStr); + auto ioutTs = concordUtils::fromBigEndianBuffer(outTs.data()); + ASSERT_EQ(timestamp, ioutTs); + } + + // delete with higher ts + uint64_t higherTimestamp = 500; + auto higherTsStr = concordUtils::toBigEndianStringBuffer(higherTimestamp); + db->del(cf, key, higherTsStr); + + // get the value at 500 + { + std::string outTs; + auto val = db->get(cf, key, higherTsStr, &outTs); + ASSERT_FALSE(val.has_value()); + } + + // get the value of 420 + { + std::string outTs; + auto val = db->get(cf, key, tsStr, &outTs); + ASSERT_TRUE(val.has_value()); + ASSERT_EQ(*val, value); + ASSERT_EQ(outTs, tsStr); + auto ioutTs = concordUtils::fromBigEndianBuffer(outTs.data()); + ASSERT_EQ(timestamp, ioutTs); + } + + // try to delete with ts + db->del(cf, key, tsStr); + + // get the value of 420 + { + std::string outTs; + auto val = db->get(cf, key, tsStr, &outTs); + ASSERT_FALSE(val.has_value()); + } +} + +TEST_F(native_rocksdb_test, put_in_batch_with_timestamps) { + const auto cf1 = "cf1"s; + auto cf_options = ::rocksdb::ColumnFamilyOptions{}; + cf_options.comparator = concord::storage::rocksdb::getLexicographic64TsComparator(); + db->createColumnFamily(cf1, cf_options); + auto batch = db->getBatch(); + + uint64_t timestamp = 420; + auto tsStr = concordUtils::toBigEndianStringBuffer(timestamp); + std::string key = "time"; + std::string value = "val"; + auto key1_ts = getSliceArray(key, tsStr); + auto sl_val = getSliceArray(value); + + std::string key2 = "time2"; + std::string value2 = "val2"; + auto key2_ts = getSliceArray(key2, tsStr); + auto sl_val2 = getSliceArray(value2); + + batch.put(cf1, key1_ts, sl_val); + batch.put(cf1, key2_ts, sl_val2); + db->write(std::move(batch)); + + // try to read with lower timestamp + uint64_t lowerTimestamp = 400; + auto lowerTsStr = concordUtils::toBigEndianStringBuffer(lowerTimestamp); + + { + std::string outTs; + auto val = db->get(cf1, key, lowerTsStr, &outTs); + ASSERT_FALSE(val.has_value()); + val = db->get(cf1, key2, lowerTsStr, &outTs); + ASSERT_FALSE(val.has_value()); + } + + { + // try to read with higher timestamp + uint64_t higherTimestamp = 500; + auto highTsStr = concordUtils::toBigEndianStringBuffer(higherTimestamp); + std::string outTs; + auto val = db->get(cf1, key, highTsStr, &outTs); + ASSERT_TRUE(val.has_value()); + ASSERT_EQ(*val, value); + ASSERT_EQ(outTs, tsStr); + auto ioutTs = concordUtils::fromBigEndianBuffer(outTs.data()); + ASSERT_EQ(timestamp, ioutTs); + } +} + +TEST_F(native_rocksdb_test, 2cf_one_with_timestamp) { + const auto cf1 = "cf1"s; + auto cf_options = ::rocksdb::ColumnFamilyOptions{}; + cf_options.comparator = concord::storage::rocksdb::getLexicographic64TsComparator(); + db->createColumnFamily(cf1, cf_options); + const auto cf2 = "cf2"s; + db->createColumnFamily(cf2); + auto batch = db->getBatch(); + + uint64_t timestamp = 420; + auto tsStr = concordUtils::toBigEndianStringBuffer(timestamp); + std::string key = "time"; + std::string value = "val"; + + std::string key2 = "time2"; + std::string value2 = "val2"; + + batch.put(cf1, getSliceArray(key, tsStr), getSliceArray(value)); + batch.put(cf2, key2, value2); + db->write(std::move(batch)); + + { + // read with timestamp + std::string outTs; + auto val = db->get(cf1, key, tsStr, &outTs); + ASSERT_TRUE(val.has_value()); + ASSERT_EQ(*val, value); + ASSERT_EQ(outTs, tsStr); + auto ioutTs = concordUtils::fromBigEndianBuffer(outTs.data()); + ASSERT_EQ(timestamp, ioutTs); + } + { + // read with without timestamp + auto val = db->get(cf2, key2); + ASSERT_TRUE(val.has_value()); + ASSERT_EQ(*val, value2); + } +} + +TEST_F(native_rocksdb_test, multi_part_write) { + const auto cf1 = "cf1"s; + auto cf_options = ::rocksdb::ColumnFamilyOptions{}; + cf_options.comparator = concord::storage::rocksdb::getLexicographic64TsComparator(); + db->createColumnFamily(cf1, cf_options); + + auto batch = db->getBatch(); + uint64_t timestamp = 420; + auto tsStr = concordUtils::toBigEndianStringBuffer(timestamp); + std::string key = "time"; + std::string value = "val"; + + std::string prefix = "."; + std::string key2 = "time2"; + std::string value2 = "val2"; + + std::string key_with_prefix = prefix + key2; + + batch.put(cf1, getSliceArray(key, tsStr), getSliceArray(value)); + batch.put(cf1, getSliceArray(prefix, key2, tsStr), getSliceArray(value2)); + db->write(std::move(batch)); + + { + std::string outTs; + auto val = db->get(cf1, key, tsStr, &outTs); + ASSERT_TRUE(val.has_value()); + ASSERT_EQ(*val, value); + ASSERT_EQ(outTs, tsStr); + auto ioutTs = concordUtils::fromBigEndianBuffer(outTs.data()); + ASSERT_EQ(timestamp, ioutTs); + + auto val2 = db->get(cf1, key_with_prefix, tsStr, &outTs); + ASSERT_TRUE(val2.has_value()); + ASSERT_EQ(*val2, value2); + ASSERT_EQ(outTs, tsStr); + ioutTs = concordUtils::fromBigEndianBuffer(outTs.data()); + ASSERT_EQ(timestamp, ioutTs); + + // Without the prefix + val2 = db->get(cf1, key2, tsStr, &outTs); + ASSERT_FALSE(val2.has_value()); + val2 = db->get(cf1, prefix, tsStr, &outTs); + ASSERT_FALSE(val2.has_value()); + } +} + +TEST_F(native_rocksdb_test, write_batch_with_Delete) { + const auto cf1 = "cf1"s; + auto cf_options = ::rocksdb::ColumnFamilyOptions{}; + cf_options.comparator = concord::storage::rocksdb::getLexicographic64TsComparator(); + db->createColumnFamily(cf1, cf_options); + + uint64_t timestamp = 420; + auto tsStr = concordUtils::toBigEndianStringBuffer(timestamp); + std::string key = "time"; + std::string value = "val"; + + std::string key2 = "time2"; + std::string value2 = "val2"; + + { + auto batch = db->getBatch(); + batch.put(cf1, getSliceArray(key, tsStr), getSliceArray(value)); + batch.put(cf1, getSliceArray(key2, tsStr), getSliceArray(value2)); + db->write(std::move(batch)); + } + + { + // read with timestamp + std::string outTs; + auto val = db->get(cf1, key, tsStr, &outTs); + ASSERT_TRUE(val.has_value()); + ASSERT_EQ(*val, value); + ASSERT_EQ(outTs, tsStr); + auto ioutTs = concordUtils::fromBigEndianBuffer(outTs.data()); + ASSERT_EQ(timestamp, ioutTs); + } + + std::string value3 = "val3"; + std::string value4 = "val4"; + + uint64_t timestamp2 = 422; + auto tsStr2 = concordUtils::toBigEndianStringBuffer(timestamp2); + { + auto batch = db->getBatch(); + batch.put(cf1, getSliceArray(key2, tsStr2), getSliceArray(value4)); + batch.del(cf1, getSliceArray(key, tsStr2)); + db->write(std::move(batch)); + } + + //// key #1 + { + // read with higher timestamp deleted value at 422 + uint64_t higer_timestamp = 423; + auto higher_ts_str = concordUtils::toBigEndianStringBuffer(higer_timestamp); + std::string outTs; + auto val = db->get(cf1, key, higher_ts_str, &outTs); + ASSERT_FALSE(val.has_value()); + } + + { + // read with deletion timestamp deleted value at 422 + std::string outTs; + auto val = db->get(cf1, key, tsStr2, &outTs); + ASSERT_FALSE(val.has_value()); + } + + { + // read with timestamp 420 + std::string outTs; + auto val = db->get(cf1, key, tsStr, &outTs); + ASSERT_TRUE(val.has_value()); + ASSERT_EQ(*val, value); + ASSERT_EQ(outTs, tsStr); + auto ioutTs = concordUtils::fromBigEndianBuffer(outTs.data()); + ASSERT_EQ(timestamp, ioutTs); + } + + //// key #2 + + { + // read with timestamp 420 + std::string outTs; + auto val = db->get(cf1, key2, tsStr, &outTs); + ASSERT_TRUE(val.has_value()); + ASSERT_EQ(*val, value2); + ASSERT_EQ(outTs, tsStr); + auto ioutTs = concordUtils::fromBigEndianBuffer(outTs.data()); + ASSERT_EQ(timestamp, ioutTs); + } + + { + // read with timestamp 421 + std::string outTs; + auto val = db->get(cf1, key2, tsStr2, &outTs); + ASSERT_TRUE(val.has_value()); + ASSERT_EQ(*val, value4); + ASSERT_EQ(outTs, tsStr2); + auto ioutTs = concordUtils::fromBigEndianBuffer(outTs.data()); + ASSERT_EQ(timestamp2, ioutTs); + } +} + +TEST_F(native_rocksdb_test, write_delete_update) { + const auto cf1 = "cf1"s; + auto cf_options = ::rocksdb::ColumnFamilyOptions{}; + cf_options.comparator = concord::storage::rocksdb::getLexicographic64TsComparator(); + db->createColumnFamily(cf1, cf_options); + + uint64_t timestamp = 420; + auto tsStr = concordUtils::toBigEndianStringBuffer(timestamp); + std::string key = "time"; + std::string value = "val"; + + { + auto batch = db->getBatch(); + batch.put(cf1, getSliceArray(key, tsStr), getSliceArray(value)); + db->write(std::move(batch)); + } + + { + // read with timestamp + std::string outTs; + auto val = db->get(cf1, key, tsStr, &outTs); + ASSERT_TRUE(val.has_value()); + ASSERT_EQ(*val, value); + ASSERT_EQ(outTs, tsStr); + auto ioutTs = concordUtils::fromBigEndianBuffer(outTs.data()); + ASSERT_EQ(timestamp, ioutTs); + } + // UPdate + uint64_t timestamp2 = 422; + auto tsStr2 = concordUtils::toBigEndianStringBuffer(timestamp2); + std::string value2 = "val2"; + { + auto batch = db->getBatch(); + batch.put(cf1, getSliceArray(key, tsStr2), getSliceArray(value2)); + db->write(std::move(batch)); + } + { + // read with timestamp + std::string outTs; + auto val = db->get(cf1, key, tsStr2, &outTs); + ASSERT_TRUE(val.has_value()); + ASSERT_EQ(*val, value2); + ASSERT_EQ(outTs, tsStr2); + auto ioutTs = concordUtils::fromBigEndianBuffer(outTs.data()); + ASSERT_EQ(timestamp2, ioutTs); + } + + // Delete with timestamp2 + { + auto batch = db->getBatch(); + batch.del(cf1, getSliceArray(key, tsStr2)); + db->write(std::move(batch)); + } + + { + // read with deletion timestamp deleted value at 422 + std::string outTs; + auto val = db->get(cf1, key, tsStr2, &outTs); + ASSERT_FALSE(val.has_value()); + } + // update again + { + auto batch = db->getBatch(); + batch.put(cf1, getSliceArray(key, tsStr2), getSliceArray(value2)); + db->write(std::move(batch)); + } + { + // read with timestamp + std::string outTs; + auto val = db->get(cf1, key, tsStr2, &outTs); + ASSERT_TRUE(val.has_value()); + ASSERT_EQ(*val, value2); + ASSERT_EQ(outTs, tsStr2); + auto ioutTs = concordUtils::fromBigEndianBuffer(outTs.data()); + ASSERT_EQ(timestamp2, ioutTs); + } +} + +TEST_F(native_rocksdb_test, lexicographic_timestamp_comparator) { + auto comparator = concord::storage::rocksdb::getLexicographic64TsComparator(); + + // First is less than second + { + uint64_t timestamp = 420; + auto ts1 = concordUtils::toBigEndianStringBuffer(timestamp); + uint64_t timestamp2 = 421; + auto ts2 = concordUtils::toBigEndianStringBuffer(timestamp2); + auto slTs1 = detail::toSlice(ts1); + auto slTs2 = detail::toSlice(ts2); + auto comp = comparator->CompareTimestamp(slTs1, slTs2); + ASSERT_EQ(comp, -1); + } + + // First is larger than second + { + uint64_t timestamp = 422; + auto ts1 = concordUtils::toBigEndianStringBuffer(timestamp); + uint64_t timestamp2 = 421; + auto ts2 = concordUtils::toBigEndianStringBuffer(timestamp2); + auto slTs1 = detail::toSlice(ts1); + auto slTs2 = detail::toSlice(ts2); + auto comp = comparator->CompareTimestamp(slTs1, slTs2); + ASSERT_EQ(comp, 1); + } + + // equal + { + uint64_t timestamp = 420; + auto ts1 = concordUtils::toBigEndianStringBuffer(timestamp); + uint64_t timestamp2 = 420; + auto ts2 = concordUtils::toBigEndianStringBuffer(timestamp2); + auto slTs1 = detail::toSlice(ts1); + auto slTs2 = detail::toSlice(ts2); + auto comp = comparator->CompareTimestamp(slTs1, slTs2); + ASSERT_EQ(comp, 0); + } +} + +TEST_F(native_rocksdb_test, remove_history_on_compaction) { + const auto cf1 = "cf1"s; + auto cf_options = ::rocksdb::ColumnFamilyOptions{}; + cf_options.comparator = concord::storage::rocksdb::getLexicographic64TsComparator(); + db->createColumnFamily(cf1, cf_options); + std::string key = "time"; + std::string key2 = "time100"; + std::string value1 = "val1"; + std::string value2 = "val2"; + uint64_t timestamp1 = 420; + uint64_t timestamp2 = 450; + + { + auto tsStr = concordUtils::toBigEndianStringBuffer(timestamp1); + auto batch = db->getBatch(); + batch.put(cf1, getSliceArray(key, tsStr), getSliceArray(value1)); + db->write(std::move(batch)); + } + // update + { + auto tsStr = concordUtils::toBigEndianStringBuffer(timestamp2); + auto batch = db->getBatch(); + batch.put(cf1, getSliceArray(key, tsStr), getSliceArray(value2)); + db->write(std::move(batch)); + } + + { + // read with timestamp 420 + auto tsStr = concordUtils::toBigEndianStringBuffer(timestamp1); + std::string outTs; + auto val = db->get(cf1, key, tsStr, &outTs); + ASSERT_TRUE(val.has_value()); + ASSERT_EQ(*val, value1); + ASSERT_EQ(outTs, tsStr); + auto ioutTs = concordUtils::fromBigEndianBuffer(outTs.data()); + ASSERT_EQ(timestamp1, ioutTs); + } + + { + // read with timestamp 450 + auto tsStr = concordUtils::toBigEndianStringBuffer(timestamp2); + std::string outTs; + auto val = db->get(cf1, key, tsStr, &outTs); + ASSERT_TRUE(val.has_value()); + ASSERT_EQ(*val, value2); + ASSERT_EQ(outTs, tsStr); + auto ioutTs = concordUtils::fromBigEndianBuffer(outTs.data()); + ASSERT_EQ(timestamp2, ioutTs); + } + + // set low time stamp to 451 + uint64_t threshold1 = 451; + auto threshold1_str = concordUtils::toBigEndianStringBuffer(threshold1); + auto &raw_db = db->rawDB(); + // raw_db.IncreaseFullHistoryTsLow(db->columnFamilyHandle(cf1), threshold1_str); + ::rocksdb::Slice begin(key); + ::rocksdb::Slice end(key2); + ::rocksdb::Slice ts_low(threshold1_str); + ::rocksdb::CompactRangeOptions options; + options.full_history_ts_low = &ts_low; + auto s = raw_db.CompactRange(options, db->columnFamilyHandle(cf1), nullptr, nullptr); + auto status = s.ToString(); + ASSERT_EQ(status, "OK"); + + { + // read with timestamp 420 + auto tsStr = concordUtils::toBigEndianStringBuffer(timestamp1); + std::string outTs; + auto val = db->get(cf1, key, tsStr, &outTs); + ASSERT_FALSE(val.has_value()); + } + + { + // read with timestamp 450 -- latest + auto tsStr = concordUtils::toBigEndianStringBuffer(timestamp2); + std::string outTs; + auto val = db->get(cf1, key, tsStr, &outTs); + ASSERT_TRUE(val.has_value()); + ASSERT_EQ(*val, value2); + ASSERT_EQ(outTs, tsStr); + auto ioutTs = concordUtils::fromBigEndianBuffer(outTs.data()); + ASSERT_EQ(timestamp2, ioutTs); + } +} + +TEST_F(native_rocksdb_test, remove_history_on_compaction_with_delete) { + const auto cf1 = "cf1"s; + auto cf_options = ::rocksdb::ColumnFamilyOptions{}; + cf_options.comparator = concord::storage::rocksdb::getLexicographic64TsComparator(); + db->createColumnFamily(cf1, cf_options); + std::string key = "time"; + std::string key2 = "time100"; + std::string value1 = "val1"; + std::string value2 = "val2"; + uint64_t timestamp1 = 420; + uint64_t timestamp2 = 450; + + { + auto tsStr = concordUtils::toBigEndianStringBuffer(timestamp1); + auto batch = db->getBatch(); + batch.put(cf1, getSliceArray(key, tsStr), getSliceArray(value1)); + db->write(std::move(batch)); + } + // delete + { + auto tsStr = concordUtils::toBigEndianStringBuffer(timestamp2); + auto batch = db->getBatch(); + batch.del(cf1, getSliceArray(key, tsStr)); + db->write(std::move(batch)); + } + + { + // read with timestamp 420 + auto tsStr = concordUtils::toBigEndianStringBuffer(timestamp1); + std::string outTs; + auto val = db->get(cf1, key, tsStr, &outTs); + ASSERT_TRUE(val.has_value()); + ASSERT_EQ(*val, value1); + ASSERT_EQ(outTs, tsStr); + auto ioutTs = concordUtils::fromBigEndianBuffer(outTs.data()); + ASSERT_EQ(timestamp1, ioutTs); + } + + { + // read with timestamp 450 + auto tsStr = concordUtils::toBigEndianStringBuffer(timestamp2); + std::string outTs; + auto val = db->get(cf1, key, tsStr, &outTs); + ASSERT_FALSE(val.has_value()); + } + + // set low time stamp to 451 + uint64_t threshold1 = 451; + auto threshold1_str = concordUtils::toBigEndianStringBuffer(threshold1); + auto &raw_db = db->rawDB(); + // raw_db.IncreaseFullHistoryTsLow(db->columnFamilyHandle(cf1), threshold1_str); + ::rocksdb::Slice begin(key); + ::rocksdb::Slice end(key2); + ::rocksdb::Slice ts_low(threshold1_str); + ::rocksdb::CompactRangeOptions options; + options.full_history_ts_low = &ts_low; + auto s = raw_db.CompactRange(options, db->columnFamilyHandle(cf1), nullptr, nullptr); + auto status = s.ToString(); + ASSERT_EQ(status, "OK"); + + { + // read with timestamp 420 + auto tsStr = concordUtils::toBigEndianStringBuffer(timestamp1); + std::string outTs; + auto val = db->get(cf1, key, tsStr, &outTs); + ASSERT_FALSE(val.has_value()); + } + + { + // read with timestamp 450 -- latest + auto tsStr = concordUtils::toBigEndianStringBuffer(timestamp2); + std::string outTs; + auto val = db->get(cf1, key, tsStr, &outTs); + ASSERT_FALSE(val.has_value()); + } +} + } // namespace int main(int argc, char *argv[]) { diff --git a/tests/apollo/CMakeLists.txt b/tests/apollo/CMakeLists.txt index 55c7563ee2..96a2378ca1 100644 --- a/tests/apollo/CMakeLists.txt +++ b/tests/apollo/CMakeLists.txt @@ -65,9 +65,6 @@ if (BUILD_ROCKSDB_STORAGE) "env ${APOLLO_TEST_ENV} BUILD_COMM_TCP_TLS=${BUILD_COMM_TCP_TLS} TEST_NAME=skvbc_ro_replica_tests python3 -m unittest test_skvbc_ro_replica ${TEST_OUTPUT}" WORKING_DIRECTORY ${CMAKE_CURRENT_SOURCE_DIR}) - add_test(NAME skvbc_s3_integrity_check_tests COMMAND sh -c - "env ${APOLLO_TEST_ENV} BUILD_COMM_TCP_TLS=${BUILD_COMM_TCP_TLS} TEST_NAME=skvbc_s3_integrity_check_tests python3 -m unittest test_skvbc_s3_integrity_check ${TEST_OUTPUT}" - WORKING_DIRECTORY ${CMAKE_CURRENT_SOURCE_DIR}) endif() endif() diff --git a/tests/apollo/test_skvbc_s3_integrity_check.py b/tests/apollo/test_skvbc_s3_integrity_check.py index 39fa63e5ea..57ebd74383 100644 --- a/tests/apollo/test_skvbc_s3_integrity_check.py +++ b/tests/apollo/test_skvbc_s3_integrity_check.py @@ -80,6 +80,7 @@ async def test_integrity_check_validate_all(self, bft_network): self._start_integrity_check(bft_network, keys_config, s3_config) @with_trio + @unittest.skip("unstable scenario") @with_bft_network(start_replica_cmd=start_replica_cmd_prefix, num_ro_replicas=1, selected_configs=lambda n, f, c: n == 7) async def test_integrity_check_validate_all_with_missing_block(self, bft_network): """ diff --git a/tests/simpleKVBC/TesterReplica/internalCommandsHandler.hpp b/tests/simpleKVBC/TesterReplica/internalCommandsHandler.hpp index 0a1f6b4bfb..f110362c6c 100644 --- a/tests/simpleKVBC/TesterReplica/internalCommandsHandler.hpp +++ b/tests/simpleKVBC/TesterReplica/internalCommandsHandler.hpp @@ -26,7 +26,7 @@ #include "skvbc_messages.cmf.hpp" #include "SharedTypes.hpp" #include "categorization/db_categories.h" -#include "categorization/kv_blockchain.h" +#include "kvbc_adapter/replica_adapter.hpp" static const std::string VERSIONED_KV_CAT_ID{concord::kvbc::categorization::kExecutionPrivateCategory}; static const std::string BLOCK_MERKLE_CAT_ID{concord::kvbc::categorization::kExecutionProvableCategory}; @@ -38,7 +38,7 @@ class InternalCommandsHandler : public concord::kvbc::ICommandsHandler { concord::kvbc::IBlockMetadata *blockMetadata, logging::Logger &logger, bool addAllKeysAsPublic = false, - concord::kvbc::categorization::KeyValueBlockchain *kvbc = nullptr) + concord::kvbc::adapter::ReplicaBlockchain *kvbc = nullptr) : m_storage(storage), m_blockAdder(blocksAdder), m_blockMetadata(blockMetadata), @@ -136,5 +136,5 @@ class InternalCommandsHandler : public concord::kvbc::ICommandsHandler { size_t m_getLastBlockCounter = 0; std::shared_ptr perfManager_; bool m_addAllKeysAsPublic{false}; // Add all key-values in the block merkle category as public ones. - concord::kvbc::categorization::KeyValueBlockchain *m_kvbc{nullptr}; + concord::kvbc::adapter::ReplicaBlockchain *m_kvbc{nullptr}; }; diff --git a/tests/simpleKVBC/TesterReplica/main.cpp b/tests/simpleKVBC/TesterReplica/main.cpp index c821cd1300..ad2105db30 100644 --- a/tests/simpleKVBC/TesterReplica/main.cpp +++ b/tests/simpleKVBC/TesterReplica/main.cpp @@ -72,15 +72,15 @@ class STAddRemoveHandlerTest : public concord::client::reconfiguration::IStateHa } }; -void cronSetup(TestSetup& setup, const Replica& replica) { +void cronSetup(TestSetup& setup, const Replica& main_replica) { if (!setup.GetCronEntryNumberOfExecutes()) { return; } const auto numberOfExecutes = *setup.GetCronEntryNumberOfExecutes(); using namespace concord::cron; - const auto cronTableRegistry = replica.cronTableRegistry(); - const auto ticksGenerator = replica.ticksGenerator(); + const auto cronTableRegistry = main_replica.cronTableRegistry(); + const auto ticksGenerator = main_replica.ticksGenerator(); auto& cronTable = cronTableRegistry->operator[](TestSetup::kCronTableComponentId); @@ -145,7 +145,7 @@ void run_replica(int argc, char** argv) { setup->AddAllKeysAsPublic(), replica->kvBlockchain() ? &replica->kvBlockchain().value() : nullptr); replica->set_command_handler(cmdHandler); - replica->setStateSnapshotValueConverter(categorization::KeyValueBlockchain::kNoopConverter); + replica->setStateSnapshotValueConverter([](std::string&& v) -> std::string { return std::move(v); }); replica->start(); auto& replicaConfig = setup->GetReplicaConfig(); diff --git a/thin-replica-server/include/thin-replica-server/replica_state_snapshot_service_impl.hpp b/thin-replica-server/include/thin-replica-server/replica_state_snapshot_service_impl.hpp index 7de63052fb..4ea56b083d 100644 --- a/thin-replica-server/include/thin-replica-server/replica_state_snapshot_service_impl.hpp +++ b/thin-replica-server/include/thin-replica-server/replica_state_snapshot_service_impl.hpp @@ -16,11 +16,12 @@ #include "replica_state_snapshot.grpc.pb.h" #include "bftengine/DbCheckpointManager.hpp" -#include "categorization/kv_blockchain.h" #include "kvbc_app_filter/value_from_kvbc_proto.h" +#include "blockchain_misc.hpp" #include #include +#include namespace concord::thin_replica { @@ -38,9 +39,7 @@ class ReplicaStateSnapshotServiceImpl ::grpc::ServerWriter< ::vmware::concord::replicastatesnapshot::StreamSnapshotResponse>* writer) override; // Allows users to convert state values to any format that is appropriate. - void setStateValueConverter(const kvbc::categorization::KeyValueBlockchain::Converter& c) { - state_value_converter_ = c; - } + void setStateValueConverter(const concord::kvbc::Converter& c) { state_value_converter_ = c; } // Following methods are used for testing only. Please do not use in production. void overrideCheckpointPathForTest(const std::string& path) { overriden_path_for_test_ = path; } @@ -55,7 +54,7 @@ class ReplicaStateSnapshotServiceImpl bool throw_exception_for_test_{false}; // Allows users to convert state values to any format that is appropriate. // The default converter extracts the value from the ValueWithTrids protobuf type. - kvbc::categorization::KeyValueBlockchain::Converter state_value_converter_{kvbc::valueFromKvbcProto}; + concord::kvbc::Converter state_value_converter_{kvbc::valueFromKvbcProto}; }; } // namespace concord::thin_replica diff --git a/thin-replica-server/src/replica_state_snapshot_service_impl.cpp b/thin-replica-server/src/replica_state_snapshot_service_impl.cpp index 1ca3c4b932..748ff65bdc 100644 --- a/thin-replica-server/src/replica_state_snapshot_service_impl.cpp +++ b/thin-replica-server/src/replica_state_snapshot_service_impl.cpp @@ -14,7 +14,7 @@ #include "thin-replica-server/replica_state_snapshot_service_impl.hpp" #include "Logger.hpp" -#include "rocksdb/native_client.h" +#include "kvbc_adapter/replica_adapter.hpp" #include #include @@ -25,8 +25,8 @@ using vmware::concord::replicastatesnapshot::StreamSnapshotRequest; using vmware::concord::replicastatesnapshot::StreamSnapshotResponse; using bftEngine::impl::DbCheckpointManager; -using kvbc::categorization::KeyValueBlockchain; using storage::rocksdb::NativeClient; +using concord::kvbc::adapter::ReplicaBlockchain; grpc::Status ReplicaStateSnapshotServiceImpl::StreamSnapshot(grpc::ServerContext* context, const StreamSnapshotRequest* request, @@ -61,10 +61,10 @@ grpc::Status ReplicaStateSnapshotServiceImpl::StreamSnapshot(grpc::ServerContext const auto snapshot_path = overriden_path_for_test_.has_value() ? *overriden_path_for_test_ : DbCheckpointManager::instance().getPathForCheckpoint(request->snapshot_id()); - const auto link_st_chain = false; const auto read_only = true; - auto db = NativeClient::newClient(snapshot_path, read_only, NativeClient::DefaultOptions{}); - const auto kvbc = KeyValueBlockchain{db, link_st_chain}; + const auto link_st_chain = false; + auto db_client = NativeClient::newClient(snapshot_path, read_only, NativeClient::DefaultOptions{}); + auto kvbc_state_snapshot_ = std::make_unique(db_client, link_st_chain); const auto iterate = [&](std::string&& key, std::string&& value) { auto resp = StreamSnapshotResponse{}; @@ -82,14 +82,14 @@ grpc::Status ReplicaStateSnapshotServiceImpl::StreamSnapshot(grpc::ServerContext }; if (request->has_last_received_key()) { - if (!kvbc.iteratePublicStateKeyValues(iterate, request->last_received_key())) { + if (!kvbc_state_snapshot_->iteratePublicStateKeyValues(iterate, request->last_received_key())) { const auto msg = "Streaming of State Snapshot ID = " + snapshot_id_str + " failed, reason = last_received_key not found"; LOG_INFO(STATE_SNAPSHOT, msg); return grpc::Status{grpc::StatusCode::INVALID_ARGUMENT, msg}; } } else { - kvbc.iteratePublicStateKeyValues(iterate); + kvbc_state_snapshot_->iteratePublicStateKeyValues(iterate); } } catch (const std::exception& e) { const auto err = "Streaming of State Snapshot ID = " + snapshot_id_str + " failed, reason = " + e.what(); diff --git a/thin-replica-server/test/replica_state_snapshot_service_test.cpp b/thin-replica-server/test/replica_state_snapshot_service_test.cpp index dd6f01a5af..3606b8ea51 100644 --- a/thin-replica-server/test/replica_state_snapshot_service_test.cpp +++ b/thin-replica-server/test/replica_state_snapshot_service_test.cpp @@ -15,7 +15,7 @@ #include "gmock/gmock.h" #include "categorization/db_categories.h" -#include "categorization/kv_blockchain.h" +#include "kvbc_adapter/replica_adapter.hpp" #include "kvbc_key_types.hpp" #include "storage/test/storage_test_common.h" #include "thin-replica-server/replica_state_snapshot_service_impl.hpp" @@ -38,6 +38,7 @@ using namespace concord::kvbc; using namespace concord::kvbc::categorization; using namespace concord::thin_replica; using bftEngine::impl::DbCheckpointManager; +using concord::kvbc::adapter::ReplicaBlockchain; using concord::storage::rocksdb::NativeClient; using grpc::Channel; using grpc::ClientContext; @@ -53,10 +54,10 @@ using vmware::concord::replicastatesnapshot::StreamSnapshotResponse; class replica_state_snapshot_service_test : public Test { void SetUp() override { destroyDb(); - service_.setStateValueConverter(KeyValueBlockchain::kNoopConverter); + service_.setStateValueConverter([](std::string &&v) -> std::string { return std::move(v); }); db_ = TestRocksDb::createNative(); const auto link_st_chain = false; - kvbc_ = std::make_unique( + kvbc_ = std::make_unique( db_, link_st_chain, std::map{{kExecutionProvableCategory, CATEGORY_TYPE::block_merkle}, @@ -100,7 +101,7 @@ class replica_state_snapshot_service_test : public Test { std::string{ser_public_state.cbegin(), ser_public_state.cend()}); updates.add(kExecutionProvableCategory, std::move(merkle)); updates.add(kConcordInternalCategoryId, std::move(versioned)); - ASSERT_EQ(kvbc_->addBlock(std::move(updates)), 1); + ASSERT_EQ(kvbc_->add(std::move(updates)), 1); } protected: @@ -111,7 +112,7 @@ class replica_state_snapshot_service_test : public Test { std::unique_ptr server_; std::shared_ptr channel_ = grpc::CreateChannel(grpc_uri_, grpc::InsecureChannelCredentials()); std::unique_ptr stub_ = ReplicaStateSnapshotService::NewStub(channel_); - std::unique_ptr kvbc_; + std::unique_ptr kvbc_; }; TEST_F(replica_state_snapshot_service_test, non_existent_snapshot_id) { diff --git a/thirdparty/rocksdb.cmake b/thirdparty/rocksdb.cmake index fb728fd02d..56debb5a15 100644 --- a/thirdparty/rocksdb.cmake +++ b/thirdparty/rocksdb.cmake @@ -3,7 +3,7 @@ ExternalProject_Add(rocksdb PREFIX rocksdb GIT_REPOSITORY "https://github.com/facebook/rocksdb.git" - GIT_TAG "v6.8.1" + GIT_TAG "v6.29.3" GIT_PROGRESS TRUE LOG_DOWNLOAD 1 LOG_BUILD 1