Skip to content

Commit

Permalink
Move digest files
Browse files Browse the repository at this point in the history
  • Loading branch information
WildFireFlum committed Sep 12, 2022
1 parent 6031a05 commit 9702db0
Show file tree
Hide file tree
Showing 55 changed files with 616 additions and 500 deletions.
9 changes: 5 additions & 4 deletions bftengine/src/bftengine/KeyStore.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -63,18 +63,19 @@ void ClusterKeyStore::saveReplicaKeyStoreToReserevedPages(const uint16_t& repID)

// Save clients keys to res pages and sets `published` to true.
void ClientKeyStore::save(const std::string& keys) {
auto hashed_keys = concord::util::SHA3_256().digest(keys.c_str(), keys.size());
auto hashed_keys = concord::crypto::openssl::SHA3_256().digest(keys.c_str(), keys.size());
auto strHashed_keys = std::string(hashed_keys.begin(), hashed_keys.end());
ConcordAssertEQ(strHashed_keys.size(), concord::util::SHA3_256::SIZE_IN_BYTES);
ConcordAssertEQ(strHashed_keys.size(), concord::crypto::openssl::SHA3_256::SIZE_IN_BYTES);
saveReservedPage(0, strHashed_keys.size(), strHashed_keys.c_str());
published_ = true;
LOG_INFO(KEY_EX_LOG, "Clients keys were updated, size " << keys.size());
}

std::string ClientKeyStore::load() {
std::string res_page_version(concord::util::SHA3_256::SIZE_IN_BYTES, '\0');
std::string res_page_version(concord::crypto::openssl::SHA3_256::SIZE_IN_BYTES, '\0');
loadReservedPage(0, res_page_version.length(), res_page_version.data());
return res_page_version == std::string(concord::util::SHA3_256::SIZE_IN_BYTES, '\0') ? "" : res_page_version;
return res_page_version == std::string(concord::crypto::openssl::SHA3_256::SIZE_IN_BYTES, '\0') ? ""
: res_page_version;
}

} // namespace bftEngine::impl
29 changes: 15 additions & 14 deletions bftengine/src/preprocessor/RequestProcessingState.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,6 @@ namespace preprocessor {
using namespace std;
using namespace bftEngine;
using namespace chrono;
using namespace concord::util;
using namespace concord::kvbc::sparse_merkle;

uint16_t RequestProcessingState::numOfRequiredEqualReplies_ = 0;
Expand Down Expand Up @@ -124,9 +123,8 @@ void RequestProcessingState::releaseResources() {
preProcessRequestMsg_.reset();
}

void RequestProcessingState::detectNonDeterministicPreProcessing(const SHA3_256::Digest &newHash,
NodeIdType newSenderId,
uint64_t reqRetryId) const {
void RequestProcessingState::detectNonDeterministicPreProcessing(
const concord::crypto::openssl::SHA3_256::Digest &newHash, NodeIdType newSenderId, uint64_t reqRetryId) const {
for (auto &hashArray : preProcessingResultHashes_)
if ((newHash != hashArray.first) && reqRetryId_ && (reqRetryId_ == reqRetryId)) {
// Compare only between matching request/reply retry ids
Expand Down Expand Up @@ -170,9 +168,10 @@ void RequestProcessingState::handlePreProcessReplyMsg(const PreProcessReplyMsgSh
}
}

SHA3_256::Digest RequestProcessingState::convertToArray(const uint8_t resultsHash[SHA3_256::SIZE_IN_BYTES]) {
SHA3_256::Digest hashArray;
for (uint64_t i = 0; i < SHA3_256::SIZE_IN_BYTES; i++) hashArray[i] = resultsHash[i];
concord::crypto::openssl::SHA3_256::Digest RequestProcessingState::convertToArray(
const uint8_t resultsHash[concord::crypto::openssl::SHA3_256::SIZE_IN_BYTES]) {
concord::crypto::openssl::SHA3_256::Digest hashArray;
for (uint64_t i = 0; i < concord::crypto::openssl::SHA3_256::SIZE_IN_BYTES; i++) hashArray[i] = resultsHash[i];
return hashArray;
}

Expand Down Expand Up @@ -205,8 +204,8 @@ bool RequestProcessingState::isReqTimedOut() const {
return false;
}

std::pair<std::string, concord::util::SHA3_256::Digest> RequestProcessingState::detectFailureDueToBlockID(
const concord::util::SHA3_256::Digest &other, uint64_t blockId) {
std::pair<std::string, concord::crypto::openssl::SHA3_256::Digest> RequestProcessingState::detectFailureDueToBlockID(
const concord::crypto::openssl::SHA3_256::Digest &other, uint64_t blockId) {
// Since this scenario is rare, a new string is allocated for safety.
std::string modifiedResult(primaryPreProcessResultData_, primaryPreProcessResultLen_);
ConcordAssertGT(modifiedResult.size(), sizeof(uint64_t));
Expand All @@ -221,11 +220,11 @@ std::pair<std::string, concord::util::SHA3_256::Digest> RequestProcessingState::
"Primary hash is different from quorum due to mismatch in block id" << KVLOG(batchCid_, reqSeqNum_, reqCid_));
return {modifiedResult, modifiedHash};
}
return {"", concord::util::SHA3_256::Digest{}};
return {"", concord::crypto::openssl::SHA3_256::Digest{}};
}

void RequestProcessingState::modifyPrimaryResult(
const std::pair<std::string, concord::util::SHA3_256::Digest> &result) {
const std::pair<std::string, concord::crypto::openssl::SHA3_256::Digest> &result) {
memcpy(const_cast<char *>(primaryPreProcessResultData_), result.first.c_str(), primaryPreProcessResultLen_);
primaryPreProcessResultHash_ = result.second;
auto sm = SigManager::instance();
Expand All @@ -241,9 +240,11 @@ void RequestProcessingState::modifyPrimaryResult(
void RequestProcessingState::reportNonEqualHashes(const unsigned char *chosenData, uint32_t chosenSize) const {
// Primary replica calculated hash is different from a hash that passed pre-execution consensus => we don't have
// correct pre-processed results.
const auto &primaryHash =
Hash(SHA3_256().digest(primaryPreProcessResultHash_.data(), primaryPreProcessResultHash_.size())).toString();
const auto &hashPassedConsensus = Hash(SHA3_256().digest(chosenData, chosenSize)).toString();
const auto &primaryHash = Hash(concord::crypto::openssl::SHA3_256().digest(primaryPreProcessResultHash_.data(),
primaryPreProcessResultHash_.size()))
.toString();
const auto &hashPassedConsensus =
Hash(concord::crypto::openssl::SHA3_256().digest(chosenData, chosenSize)).toString();
LOG_WARN(logger(),
"Primary replica pre-processing result hash: "
<< primaryHash << " is different from one passed the consensus: " << hashPassedConsensus
Expand Down
18 changes: 9 additions & 9 deletions bftengine/src/preprocessor/RequestProcessingState.hpp
Original file line number Diff line number Diff line change
Expand Up @@ -80,21 +80,21 @@ class RequestProcessingState {
void resetRejectedReplicasList() { rejectedReplicaIds_.clear(); }
void setPreprocessingRightNow(bool set) { preprocessingRightNow_ = set; }
const std::set<PreProcessResultSignature>& getPreProcessResultSignatures();
const concord::util::SHA3_256::Digest& getResultHash() { return primaryPreProcessResultHash_; };
const concord::crypto::openssl::SHA3_256::Digest& getResultHash() { return primaryPreProcessResultHash_; };
const bftEngine::OperationResult getAgreedPreProcessResult() const { return agreedPreProcessResult_; }

static void init(uint16_t numOfRequiredReplies, preprocessor::PreProcessorRecorder* histograms);
uint16_t getNumOfReceivedReplicas() { return numOfReceivedReplies_; };

private:
static concord::util::SHA3_256::Digest convertToArray(
const uint8_t resultsHash[concord::util::SHA3_256::SIZE_IN_BYTES]);
static concord::crypto::openssl::SHA3_256::Digest convertToArray(
const uint8_t resultsHash[concord::crypto::openssl::SHA3_256::SIZE_IN_BYTES]);
static logging::Logger& logger() {
static logging::Logger logger_ = logging::getLogger("concord.preprocessor");
return logger_;
}
auto calculateMaxNbrOfEqualHashes(uint16_t& maxNumOfEqualHashes) const;
void detectNonDeterministicPreProcessing(const concord::util::SHA3_256::Digest& newHash,
void detectNonDeterministicPreProcessing(const concord::crypto::openssl::SHA3_256::Digest& newHash,
NodeIdType newSenderId,
uint64_t reqRetryId) const;
void setupPreProcessResultData(bftEngine::OperationResult preProcessResult);
Expand All @@ -103,11 +103,11 @@ class RequestProcessingState {
void reportNonEqualHashes(const unsigned char* chosenData, uint32_t chosenSize) const;

// Detect if a hash is different from the input parameter because of the appended block id.
std::pair<std::string, concord::util::SHA3_256::Digest> detectFailureDueToBlockID(
const concord::util::SHA3_256::Digest&, uint64_t);
std::pair<std::string, concord::crypto::openssl::SHA3_256::Digest> detectFailureDueToBlockID(
const concord::crypto::openssl::SHA3_256::Digest&, uint64_t);

// Set a new block id at the end of the result.
void modifyPrimaryResult(const std::pair<std::string, concord::util::SHA3_256::Digest>&);
void modifyPrimaryResult(const std::pair<std::string, concord::crypto::openssl::SHA3_256::Digest>&);

private:
static uint16_t numOfRequiredEqualReplies_;
Expand All @@ -132,10 +132,10 @@ class RequestProcessingState {
uint32_t primaryPreProcessResultLen_ = 0;
bftEngine::OperationResult primaryPreProcessResult_ = bftEngine::OperationResult::UNKNOWN;
bftEngine::OperationResult agreedPreProcessResult_ = bftEngine::OperationResult::UNKNOWN;
concord::util::SHA3_256::Digest primaryPreProcessResultHash_ = {};
concord::crypto::openssl::SHA3_256::Digest primaryPreProcessResultHash_ = {};
// Maps result hash to a list of replica signatures sent for this hash. This also implicitly gives the number of
// replicas returning a specific hash.
std::map<concord::util::SHA3_256::Digest, std::set<PreProcessResultSignature>> preProcessingResultHashes_;
std::map<concord::crypto::openssl::SHA3_256::Digest, std::set<PreProcessResultSignature>> preProcessingResultHashes_;
bool preprocessingRightNow_ = false;
uint64_t reqRetryId_ = 0;
};
Expand Down
10 changes: 6 additions & 4 deletions bftengine/src/preprocessor/messages/PreProcessReplyMsg.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -84,7 +84,7 @@ void PreProcessReplyMsg::validate(const ReplicasInfo& repInfo) const {
concord::diagnostics::TimeRecorder scoped_timer(*preProcessorHistograms_->verifyPreProcessReplySig);
if (!sigManager->verifySig(msgHeader.senderId,
msgBody()->resultsHash,
SHA3_256::SIZE_IN_BYTES,
concord::crypto::openssl::SHA3_256::SIZE_IN_BYTES,
reinterpret_cast<concord::Byte*>(msgBody()) + headerSize,
sigLen))
throw runtime_error(__PRETTY_FUNCTION__ + string(": verifySig failed"));
Expand Down Expand Up @@ -151,17 +151,19 @@ void PreProcessReplyMsg::setupMsgBody(const char* preProcessResultBuf,
msgBody()->preProcessResult,
msgBody()->clientId,
msgBody()->reqSeqNum);
memcpy(msgBody()->resultsHash, hash.data(), SHA3_256::SIZE_IN_BYTES);
memcpy(msgBody()->resultsHash, hash.data(), concord::crypto::openssl::SHA3_256::SIZE_IN_BYTES);
{
concord::diagnostics::TimeRecorder scoped_timer(*preProcessorHistograms_->signPreProcessReplyHash);
sigManager->sign(hash.data(), SHA3_256::SIZE_IN_BYTES, reinterpret_cast<concord::Byte*>(body() + sizeof(Header)));
sigManager->sign(hash.data(),
concord::crypto::openssl::SHA3_256::SIZE_IN_BYTES,
reinterpret_cast<concord::Byte*>(body() + sizeof(Header)));
}
setLeftMsgParams(reqCid, sigSize);
}

// Used by PreProcessBatchReplyMsg while retrieving PreProcessReplyMsgs from the batch
void PreProcessReplyMsg::setupMsgBody(const uint8_t* resultsHash, const char* signature, const string& reqCid) {
memcpy(msgBody()->resultsHash, resultsHash, SHA3_256::SIZE_IN_BYTES);
memcpy(msgBody()->resultsHash, resultsHash, concord::crypto::openssl::SHA3_256::SIZE_IN_BYTES);
const uint16_t sigLen = SigManager::instance()->getMySigLength();
memcpy(body() + sizeof(Header), signature, sigLen);
setLeftMsgParams(reqCid, sigLen);
Expand Down
2 changes: 1 addition & 1 deletion bftengine/src/preprocessor/messages/PreProcessReplyMsg.hpp
Original file line number Diff line number Diff line change
Expand Up @@ -75,7 +75,7 @@ class PreProcessReplyMsg : public MessageBase {
uint16_t reqOffsetInBatch = 0;
ReplyStatus status = STATUS_GOOD;
bftEngine::OperationResult preProcessResult = bftEngine::OperationResult::UNKNOWN;
uint8_t resultsHash[concord::util::SHA3_256::SIZE_IN_BYTES];
uint8_t resultsHash[concord::crypto::openssl::SHA3_256::SIZE_IN_BYTES];
uint32_t replyLength = 0;
uint32_t cidLength = 0;
uint64_t reqRetryId = 0;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,12 +18,12 @@ namespace preprocessor {

class PreProcessResultHashCreator {
public:
static concord::util::SHA3_256::Digest create(const char* preProcessResultData,
uint32_t preProcessResultLen,
bftEngine::OperationResult preProcessResult,
uint16_t clientId,
ReqId reqSeqNum) {
auto dataDigest = concord::util::SHA3_256();
static concord::crypto::openssl::SHA3_256::Digest create(const char* preProcessResultData,
uint32_t preProcessResultLen,
bftEngine::OperationResult preProcessResult,
uint16_t clientId,
ReqId reqSeqNum) {
auto dataDigest = concord::crypto::openssl::SHA3_256();
dataDigest.init();
if (preProcessResult == bftEngine::OperationResult::SUCCESS)
dataDigest.update(preProcessResultData, preProcessResultLen);
Expand Down
3 changes: 2 additions & 1 deletion bftengine/src/preprocessor/tests/preprocessor_test.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -493,7 +493,8 @@ TEST(requestPreprocessingState_test, changePrimaryBlockId) {
memcpy(buf + bufLen - sizeof(uint64_t), reinterpret_cast<char*>(&primaryBlockId), sizeof(uint64_t));
reqState.handlePrimaryPreProcessed(buf, bufLen, OperationResult::SUCCESS);

concord::util::SHA3_256::Digest replicasHash = *((concord::util::SHA3_256::Digest*)reply->resultsHash());
concord::crypto::openssl::SHA3_256::Digest replicasHash =
*((concord::crypto::openssl::SHA3_256::Digest*)reply->resultsHash());
ConcordAssertNE(replicasHash, reqState.getResultHash());
ConcordAssertEQ(reqState.definePreProcessingConsensusResult(), COMPLETE);
ConcordAssertEQ(replicasHash, reqState.getResultHash());
Expand Down
2 changes: 1 addition & 1 deletion bftengine/tests/KeyStore/KeyStore_test.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -52,7 +52,7 @@ TEST(checkAndSetState_set_true, ClientKeyStore) {
cks.save(str_ser);
ASSERT_TRUE(cks.published());
auto loaded_keys = cks.load();
auto hashed_keys = concord::util::SHA3_256().digest(str_ser.c_str(), str_ser.size());
auto hashed_keys = concord::crypto::openssl::SHA3_256().digest(str_ser.c_str(), str_ser.size());
auto strHashed_keys = std::string(hashed_keys.begin(), hashed_keys.end());
ASSERT_EQ(loaded_keys, strHashed_keys);
}
Expand Down
2 changes: 1 addition & 1 deletion bftengine/tests/bcstatetransfer/bcstatetransfer_tests.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -3028,7 +3028,7 @@ TEST_F(BcStTest, bkpTestRvbDataConflictDetection) {
ASSERT_TRUE(datastore_->hasCheckpointDesc(i));

DataStore::CheckpointDesc desc = datastore_->getCheckpointDesc(i);
digest::Digest stateDigest, reservedPagesDigest, rvbDataDigest;
Digest stateDigest, reservedPagesDigest, rvbDataDigest;
uint64_t outBlockId;

stateTransfer_->getDigestOfCheckpoint(
Expand Down
3 changes: 0 additions & 3 deletions bftengine/tests/messages/ReplicaAsksToLeaveViewMsg_test.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -12,14 +12,11 @@
#include "gtest/gtest.h"

#include "helper.hpp"
#include "crypto/digest_type.hpp"
#include "ViewsManager.hpp"
#include "ReplicasInfo.hpp"
#include "SigManager.hpp"
#include "messages/MsgCode.hpp"
#include "messages/ReplicaAsksToLeaveViewMsg.hpp"
#include "bftengine/ClientMsgs.hpp"
#include "bftengine/ReplicaConfig.hpp"
#include "ReservedPagesMock.hpp"
#include "EpochManager.hpp"

Expand Down
2 changes: 1 addition & 1 deletion bftengine/tests/messages/ViewChangeMsg_test.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -12,7 +12,7 @@
#include "gtest/gtest.h"

#include "helper.hpp"
#include "crypto/digest_type.hpp"
#include "crypto/digest_parametes.hpp"
#include "ViewsManager.hpp"
#include "ReplicasInfo.hpp"
#include "SigManager.hpp"
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -41,7 +41,7 @@ class StateSnapshotServiceImpl final

private:
void isHashValid(uint64_t snapshot_id,
const concord::util::SHA3_256::Digest& final_hash,
const concord::crypto::openssl::SHA3_256::Digest& final_hash,
const std::chrono::milliseconds& timeout,
grpc::Status& return_status);

Expand Down
12 changes: 7 additions & 5 deletions client/clientservice/src/state_snapshot_service.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -59,12 +59,14 @@ using namespace std;

namespace concord::client::clientservice {

static concord::util::SHA3_256::Digest singleHash(const std::string& key) {
return concord::util::SHA3_256{}.digest(key.data(), key.size());
static concord::crypto::openssl::SHA3_256::Digest singleHash(const std::string& key) {
return concord::crypto::openssl::SHA3_256{}.digest(key.data(), key.size());
}

static void nextHash(const std::string& key, const std::string& value, concord::util::SHA3_256::Digest& prev_hash) {
auto hasher = concord::util::SHA3_256{};
static void nextHash(const std::string& key,
const std::string& value,
concord::crypto::openssl::SHA3_256::Digest& prev_hash) {
auto hasher = concord::crypto::openssl::SHA3_256{};
hasher.init();
hasher.update(prev_hash.data(), prev_hash.size());
const auto key_hash = singleHash(key);
Expand Down Expand Up @@ -362,7 +364,7 @@ Status StateSnapshotServiceImpl::StreamSnapshot(ServerContext* context,
}

void StateSnapshotServiceImpl::isHashValid(uint64_t snapshot_id,
const concord::util::SHA3_256::Digest& final_hash,
const concord::crypto::openssl::SHA3_256::Digest& final_hash,
const chrono::milliseconds& timeout,
Status& return_status) {
auto read_config = std::shared_ptr<ReadConfig>(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,13 +17,13 @@
#define TRC_HASH_HPP_

#include <list>
#include "openssl_crypto.hpp"
#include "crypto/openssl/crypto.hpp"
#include "thin_replica.grpc.pb.h"
#include "client/concordclient/event_update.hpp"

namespace client::thin_replica_client {

const size_t kThinReplicaHashLength = concord::util::openssl_utils::kExpectedSHA256HashLengthInBytes;
const size_t kThinReplicaHashLength = concord::crypto::openssl::kExpectedSHA256HashLengthInBytes;

// Compute the Thin Replica Mechanism hash of a given update; the hash is
// returned as a byte string stored in an std::string and should be of length
Expand Down
2 changes: 1 addition & 1 deletion client/thin-replica-client/src/trc_hash.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,7 @@ using com::vmware::concord::thin_replica::Data;
using concord::client::concordclient::EventVariant;
using concord::client::concordclient::EventGroup;
using concord::client::concordclient::Update;
using concord::util::openssl_utils::computeSHA256Hash;
using concord::crypto::openssl::computeSHA256Hash;
using std::invalid_argument;
using std::list;
using std::map;
Expand Down
2 changes: 1 addition & 1 deletion examples/replica/src/KVCommandHandler.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -42,7 +42,7 @@ using concord::osexample::kv::messages::KVRequest;
using concord::osexample::kv::messages::KVWriteReply;
using concord::osexample::kv::messages::KVWriteRequest;

using Hasher = concord::util::SHA3_256;
using Hasher = concord::crypto::openssl::SHA3_256;
using Hash = Hasher::Digest;

const uint64_t LONG_EXEC_CMD_TIME_IN_SEC = 11;
Expand Down
2 changes: 1 addition & 1 deletion kvbc/benchmark/sparse_merkle_benchmark.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -264,7 +264,7 @@ void calculateSha2(benchmark::State &state) {

void calculateSha3(benchmark::State &state) {
const auto value = randomBuffer(state.range(0));
auto hasher = SHA3_256{};
auto hasher = concord::crypto::openssl::SHA3_256{};

for (auto _ : state) {
const auto hash = hasher.digest(value.data(), value.size());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -199,7 +199,7 @@ int run(int argc, char* argv[]) {
auto db = NativeClient::newClient(config["rocksdb-path"].as<std::string>(), read_only, opts);

// Start with an arbitrary hash - SHA2-256('a').
auto current_hash = SHA2_256{}.digest("a", 1);
auto current_hash = concord::crypto::openssl::SHA2_256{}.digest("a", 1);
const auto time = Time{};
auto multi_get_batch = MultiGetBatch<Buffer>{static_cast<std::uint64_t>(point_lookup_batch_size),
static_cast<std::uint32_t>(point_lookup_threads)};
Expand Down Expand Up @@ -243,7 +243,7 @@ int run(int argc, char* argv[]) {
for (auto j = 0ull; j < serialized_keys.size(); ++j) {
ConcordAssert(statuses[j].ok());
bytes_read += (serialized_keys[j].size() + value_slices[j].size());
auto h = SHA2_256{};
auto h = concord::crypto::openssl::SHA2_256{};
h.init();
h.update(current_hash.data(), current_hash.size());
const auto& ver_key = multi_get_batch[key_idx];
Expand Down
2 changes: 1 addition & 1 deletion kvbc/include/categorization/base_types.h
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,7 @@

namespace concord::kvbc::categorization {

using Hasher = concord::util::SHA3_256;
using Hasher = concord::crypto::openssl::SHA3_256;
using Hash = Hasher::Digest;

struct BasicValue {
Expand Down
2 changes: 1 addition & 1 deletion kvbc/include/direct_kv_block.h
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,7 @@

#include "kv_types.hpp"
#include "sliver.hpp"
#include "crypto/digest_type.hpp"
#include "crypto/digest_parametes.hpp"
#include "crypto/digest.hpp"

#include <cstddef>
Expand Down
Loading

0 comments on commit 9702db0

Please sign in to comment.