aboutsummaryrefslogtreecommitdiff
path: root/src
diff options
context:
space:
mode:
Diffstat (limited to 'src')
-rw-r--r--src/client.cpp2
-rw-r--r--src/client.h66
-rw-r--r--src/core.cpp2
-rw-r--r--src/core.h631
-rw-r--r--src/crypto.cpp4
-rw-r--r--src/crypto.h386
-rw-r--r--src/entity.cpp4
-rw-r--r--src/entity.h309
-rw-r--r--src/hotstuff.cpp13
-rw-r--r--src/hotstuff_client.cpp7
-rw-r--r--src/hotstuff_keygen.cpp2
-rw-r--r--src/promise.hpp745
-rw-r--r--src/type.h46
-rw-r--r--src/util.cpp2
-rw-r--r--src/util.h119
15 files changed, 19 insertions, 2319 deletions
diff --git a/src/client.cpp b/src/client.cpp
index f787003..482c23d 100644
--- a/src/client.cpp
+++ b/src/client.cpp
@@ -1,4 +1,4 @@
-#include "client.h"
+#include "hotstuff/client.h"
namespace hotstuff {
diff --git a/src/client.h b/src/client.h
deleted file mode 100644
index dd1cfee..0000000
--- a/src/client.h
+++ /dev/null
@@ -1,66 +0,0 @@
-#ifndef _HOTSTUFF_CLIENT_H
-#define _HOTSTUFF_CLIENT_H
-
-#include "type.h"
-#include "salticidae/msg.h"
-#include "entity.h"
-
-namespace hotstuff {
-
-enum {
- REQ_CMD = 0x4,
- RESP_CMD = 0x5,
- CHK_CMD = 0x6
-};
-
-class CommandDummy: public Command {
- static uint64_t cnt;
- uint64_t n;
- uint256_t hash;
-
- public:
-
- CommandDummy() {}
-
- ~CommandDummy() override {}
-
- CommandDummy(uint64_t n):
- n(n), hash(salticidae::get_hash(*this)) {}
-
- static command_t make_cmd() {
- return new CommandDummy(cnt++);
- }
-
- void serialize(DataStream &s) const override {
- s << n;
- }
-
- void unserialize(DataStream &s) override {
- s >> n;
- hash = salticidae::get_hash(*this);
- }
-
- const uint256_t &get_hash() const override {
- return hash;
- }
-
- bool verify() const override {
- return true;
- }
-};
-
-struct MsgClient: public salticidae::MsgBase<> {
- using MsgBase::MsgBase;
- void gen_reqcmd(const Command &cmd);
- void parse_reqcmd(CommandDummy &cmd) const;
-
- void gen_respcmd(const uint256_t &cmd_hash, const Finality &fin);
- void parse_respcmd(uint256_t &cmd_hash, Finality &fin) const;
-
- void gen_chkcmd(const uint256_t &cmd_hash);
- void parse_chkcmd(uint256_t &cmd_hash) const;
-};
-
-}
-
-#endif
diff --git a/src/core.cpp b/src/core.cpp
index d6a4cc7..125e168 100644
--- a/src/core.cpp
+++ b/src/core.cpp
@@ -1,5 +1,5 @@
#include <stack>
-#include "core.h"
+#include "hotstuff/core.h"
using salticidae::DataStream;
using salticidae::static_pointer_cast;
diff --git a/src/core.h b/src/core.h
deleted file mode 100644
index c7e1fe6..0000000
--- a/src/core.h
+++ /dev/null
@@ -1,631 +0,0 @@
-#ifndef _HOTSTUFF_CORE_H
-#define _HOTSTUFF_CORE_H
-
-#include <queue>
-#include <set>
-#include <unordered_map>
-#include <unordered_set>
-
-#include "salticidae/stream.h"
-#include "salticidae/util.h"
-#include "salticidae/network.h"
-#include "salticidae/msg.h"
-
-#include "promise.hpp"
-#include "util.h"
-#include "entity.h"
-#include "crypto.h"
-
-using salticidae::EventContext;
-using salticidae::Event;
-using salticidae::NetAddr;
-using salticidae::MsgNetwork;
-using salticidae::PeerNetwork;
-using salticidae::ElapsedTime;
-using salticidae::_1;
-using salticidae::_2;
-
-namespace hotstuff {
-
-const double ent_waiting_timeout = 10;
-const double double_inf = 1e10;
-
-enum {
- PROPOSE = 0x0,
- VOTE = 0x1,
- QUERY_FETCH_BLK = 0x2,
- RESP_FETCH_BLK = 0x3,
-};
-
-using promise::promise_t;
-
-struct Proposal;
-struct Vote;
-
-/** Abstraction for HotStuff protocol state machine (without network implementation). */
-class HotStuffCore {
- block_t b0; /** the genesis block */
- /* === state variables === */
- /** block containing the QC for the highest block having one */
- block_t bqc;
- block_t bexec; /**< last executed block */
- uint32_t vheight; /**< height of the block last voted for */
- /* === auxilliary variables === */
- privkey_bt priv_key; /**< private key for signing votes */
- std::set<block_t, BlockHeightCmp> tails; /**< set of tail blocks */
- ReplicaConfig config; /**< replica configuration */
- /* === async event queues === */
- std::unordered_map<block_t, promise_t> qc_waiting;
- promise_t propose_waiting;
-
- block_t sanity_check_delivered(const uint256_t &blk_hash);
- void sanity_check_delivered(const block_t &blk);
- void check_commit(const block_t &_bqc);
- bool update(const uint256_t &bqc_hash);
- void on_qc_finish(const block_t &blk);
- void on_propose_(const block_t &blk);
-
- protected:
- ReplicaID id; /**< identity of the replica itself */
- const int32_t parent_limit; /**< maximum number of parents */
-
- public:
- BoxObj<EntityStorage> storage;
-
- HotStuffCore(ReplicaID id,
- privkey_bt &&priv_key,
- int32_t parent_limit);
- virtual ~HotStuffCore() = default;
-
- /* Inputs of the state machine triggered by external events, should called
- * by the class user, with proper invariants. */
-
- /** Call to initialize the protocol, should be called once before all other
- * functions. */
- void on_init(uint32_t nfaulty) { config.nmajority = 2 * nfaulty + 1; }
-
- /** Call to deliver a block.
- * A block is only delivered if itself is fetched, the block for the
- * contained qc is fetched and all parents are delivered. The user should
- * always ensure this invariant. The invalid blocks will be dropped by this
- * function.
- * @return true if valid */
- bool on_deliver_blk(const block_t &blk);
-
- /** Call upon the delivery of a proposal message.
- * The block mentioned in the message should be already delivered. */
- void on_receive_proposal(const Proposal &prop);
-
- /** Call upon the delivery of a vote message.
- * The block mentioned in the message should be already delivered. */
- void on_receive_vote(const Vote &vote);
-
- /** Call to submit new commands to be decided (executed). */
- void on_propose(const std::vector<command_t> &cmds);
-
- /* Functions required to construct concrete instances for abstract classes.
- * */
-
- /* Outputs of the state machine triggering external events. The virtual
- * functions should be implemented by the user to specify the behavior upon
- * the events. */
- protected:
- /** Called by HotStuffCore upon the decision being made for cmd. */
- virtual void do_decide(const command_t &cmd) = 0;
- /** Called by HotStuffCore upon broadcasting a new proposal.
- * The user should send the proposal message to all replicas except for
- * itself. */
- virtual void do_broadcast_proposal(const Proposal &prop) = 0;
- /** Called upon sending out a new vote to the next proposer. The user
- * should send the vote message to a *good* proposer to have good liveness,
- * while safety is always guaranteed by HotStuffCore. */
- virtual void do_vote(ReplicaID last_proposer, const Vote &vote) = 0;
-
- /* The user plugs in the detailed instances for those
- * polymorphic data types. */
- public:
- /** Create a partial certificate that proves the vote for a block. */
- virtual part_cert_bt create_part_cert(const PrivKey &priv_key, const uint256_t &blk_hash) = 0;
- /** Create a partial certificate from its seralized form. */
- virtual part_cert_bt parse_part_cert(DataStream &s) = 0;
- /** Create a quorum certificate that proves 2f+1 votes for a block. */
- virtual quorum_cert_bt create_quorum_cert(const uint256_t &blk_hash) = 0;
- /** Create a quorum certificate from its serialized form. */
- virtual quorum_cert_bt parse_quorum_cert(DataStream &s) = 0;
- /** Create a command object from its serialized form. */
- virtual command_t parse_cmd(DataStream &s) = 0;
-
- public:
- /** Add a replica to the current configuration. This should only be called
- * before running HotStuffCore protocol. */
- void add_replica(ReplicaID rid, const NetAddr &addr, pubkey_bt &&pub_key);
- /** Try to prune blocks lower than last committed height - staleness. */
- void prune(uint32_t staleness);
-
- /* PaceMaker can use these functions to monitor the core protocol state
- * transition */
- /** Get a promise resolved when the block gets a QC. */
- promise_t async_qc_finish(const block_t &blk);
- /** Get a promise resolved when a new block is proposed. */
- promise_t async_wait_propose();
-
- /* Other useful functions */
- block_t get_genesis() { return b0; }
- const ReplicaConfig &get_config() { return config; }
- int8_t get_cmd_decision(const uint256_t &cmd_hash);
- ReplicaID get_id() { return id; }
- operator std::string () const;
-};
-
-/** Abstraction for proposal messages. */
-struct Proposal: public Serializable {
- ReplicaID proposer;
- /** hash for the block containing the highest QC */
- uint256_t bqc_hash;
- /** block being proposed */
- block_t blk;
-
- /** handle of the core object to allow polymorphism. The user should use
- * a pointer to the object of the class derived from HotStuffCore */
- HotStuffCore *hsc;
-
- Proposal(HotStuffCore *hsc): blk(nullptr), hsc(hsc) {}
- Proposal(ReplicaID proposer,
- const uint256_t &bqc_hash,
- block_t &blk,
- HotStuffCore *hsc):
- proposer(proposer),
- bqc_hash(bqc_hash),
- blk(blk), hsc(hsc) {}
-
- void serialize(DataStream &s) const override {
- s << proposer
- << bqc_hash
- << *blk;
- }
-
- void unserialize(DataStream &s) override {
- assert(hsc != nullptr);
- s >> proposer
- >> bqc_hash;
- Block _blk;
- _blk.unserialize(s, hsc);
- blk = hsc->storage->add_blk(std::move(_blk));
- }
-
- operator std::string () const {
- DataStream s;
- s << "<proposal "
- << "rid=" << std::to_string(proposer) << " "
- << "bqc=" << get_hex10(bqc_hash) << " "
- << "blk=" << get_hex10(blk->get_hash()) << ">";
- return std::string(std::move(s));
- }
-};
-
-/** Abstraction for vote messages. */
-struct Vote: public Serializable {
- ReplicaID voter;
- /** hash for the block containing the highest QC */
- uint256_t bqc_hash;
- /** block being voted */
- uint256_t blk_hash;
- /** proof of validity for the vote (nullptr for a negative vote) */
- part_cert_bt cert;
-
- /** handle of the core object to allow polymorphism */
- HotStuffCore *hsc;
-
- Vote(HotStuffCore *hsc): cert(nullptr), hsc(hsc) {}
- Vote(ReplicaID voter,
- const uint256_t &bqc_hash,
- const uint256_t &blk_hash,
- part_cert_bt &&cert,
- HotStuffCore *hsc):
- voter(voter),
- bqc_hash(bqc_hash),
- blk_hash(blk_hash),
- cert(std::move(cert)), hsc(hsc) {}
-
- Vote(const Vote &other):
- voter(other.voter),
- bqc_hash(other.bqc_hash),
- blk_hash(other.blk_hash),
- cert(other.cert->clone()),
- hsc(other.hsc) {}
-
- Vote(Vote &&other) = default;
-
- void serialize(DataStream &s) const override {
- s << voter
- << bqc_hash
- << blk_hash;
- if (cert == nullptr)
- s << (uint8_t)0;
- else
- s << (uint8_t)1 << *cert;
- }
-
- void unserialize(DataStream &s) override {
- assert(hsc != nullptr);
- uint8_t has_cert;
- s >> voter
- >> bqc_hash
- >> blk_hash
- >> has_cert;
- cert = has_cert ? hsc->parse_part_cert(s) : nullptr;
- }
-
- bool verify() const {
- assert(hsc != nullptr);
- return cert->verify(hsc->get_config().get_pubkey(voter)) &&
- cert->get_blk_hash() == blk_hash;
- }
-
- operator std::string () const {
- DataStream s;
- s << "<vote "
- << "rid=" << std::to_string(voter) << " "
- << "bqc=" << get_hex10(bqc_hash) << " "
- << "blk=" << get_hex10(blk_hash) << " "
- << "cert=" << (cert ? "yes" : "no") << ">";
- return std::string(std::move(s));
- }
-};
-
-/** Abstraction for liveness gadget (oracle). */
-class PaceMaker {
- public:
- virtual ~PaceMaker() = default;
- /** Get a promise resolved when the pace maker thinks it is a *good* time
- * to issue new commands. When promise is resolved with the ID of itself,
- * the replica should propose the command, otherwise it will forward the
- * command to the proposer indicated by the ID. */
- virtual promise_t beat() = 0;
- /** Get a promise resolved when the pace maker thinks it is a *good* time
- * to vote for a block. The promise is resolved with the next proposer's ID
- * */
- virtual promise_t next_proposer(ReplicaID last_proposer) = 0;
-};
-
-using pacemaker_bt = BoxObj<PaceMaker>;
-
-/** A pace maker that waits for the qc of the last proposed block. */
-class PaceMakerDummy: public PaceMaker {
- HotStuffCore *hsc;
- std::queue<promise_t> pending_beats;
- block_t last_proposed;
- bool locked;
-
- void schedule_next() {
- if (!pending_beats.empty() && !locked)
- {
- auto pm = pending_beats.front();
- pending_beats.pop();
- hsc->async_qc_finish(last_proposed).then(
- [id = hsc->get_id(), pm]() {
- pm.resolve(id);
- });
- locked = true;
- }
- }
-
- void update_last_proposed() {
- hsc->async_wait_propose().then([this](block_t blk) {
- update_last_proposed();
- last_proposed = blk;
- locked = false;
- schedule_next();
- });
- }
-
- public:
- PaceMakerDummy(HotStuffCore *hsc):
- hsc(hsc),
- last_proposed(hsc->get_genesis()),
- locked(false) {
- update_last_proposed();
- }
-
- promise_t beat() override {
- promise_t pm;
- pending_beats.push(pm);
- schedule_next();
- return pm;
- }
-
- promise_t next_proposer(ReplicaID last_proposer) override {
- return promise_t([last_proposer](promise_t &pm) {
- pm.resolve(last_proposer);
- });
- }
-};
-
-/** Network message format for HotStuff. */
-struct MsgHotStuff: public salticidae::MsgBase<> {
- using MsgBase::MsgBase;
- void gen_propose(const Proposal &);
- void parse_propose(Proposal &) const;
-
- void gen_vote(const Vote &);
- void parse_vote(Vote &) const;
-
- void gen_qfetchblk(const std::vector<uint256_t> &blk_hashes);
- void parse_qfetchblk(std::vector<uint256_t> &blk_hashes) const;
-
- void gen_rfetchblk(const std::vector<block_t> &blks);
- void parse_rfetchblk(std::vector<block_t> &blks, HotStuffCore *hsc) const;
-};
-
-using promise::promise_t;
-
-class HotStuffBase;
-
-template<EntityType ent_type>
-class FetchContext: public promise_t {
- Event timeout;
- HotStuffBase *hs;
- MsgHotStuff fetch_msg;
- const uint256_t ent_hash;
- std::unordered_set<NetAddr> replica_ids;
- inline void timeout_cb(evutil_socket_t, short);
- public:
- FetchContext(const FetchContext &) = delete;
- FetchContext &operator=(const FetchContext &) = delete;
- FetchContext(FetchContext &&other);
-
- FetchContext(const uint256_t &ent_hash, HotStuffBase *hs);
- ~FetchContext() {}
-
- inline void send(const NetAddr &replica_id);
- inline void reset_timeout();
- inline void add_replica(const NetAddr &replica_id, bool fetch_now = true);
-};
-
-class BlockDeliveryContext: public promise_t {
- public:
- ElapsedTime elapsed;
- BlockDeliveryContext &operator=(const BlockDeliveryContext &) = delete;
- BlockDeliveryContext(const BlockDeliveryContext &other):
- promise_t(static_cast<const promise_t &>(other)),
- elapsed(other.elapsed) {}
- BlockDeliveryContext(BlockDeliveryContext &&other):
- promise_t(static_cast<const promise_t &>(other)),
- elapsed(std::move(other.elapsed)) {}
- template<typename Func>
- BlockDeliveryContext(Func callback): promise_t(callback) {
- elapsed.start();
- }
-};
-
-
-/** HotStuff protocol (with network implementation). */
-class HotStuffBase: public HotStuffCore {
- using BlockFetchContext = FetchContext<ENT_TYPE_BLK>;
- using CmdFetchContext = FetchContext<ENT_TYPE_CMD>;
- using conn_t = MsgNetwork<MsgHotStuff>::conn_t;
-
- friend BlockFetchContext;
- friend CmdFetchContext;
-
- protected:
- /** the binding address in replica network */
- NetAddr listen_addr;
- /** the block size */
- size_t blk_size;
- /** libevent handle */
- EventContext eb;
- pacemaker_bt pmaker;
-
- private:
- /** whether libevent handle is owned by itself */
- bool eb_loop;
- /** network stack */
- PeerNetwork<MsgHotStuff> pn;
-#ifdef HOTSTUFF_ENABLE_BLK_PROFILE
- BlockProfiler blk_profiler;
-#endif
- /* queues for async tasks */
- std::unordered_map<const uint256_t, BlockFetchContext> blk_fetch_waiting;
- std::unordered_map<const uint256_t, BlockDeliveryContext> blk_delivery_waiting;
- std::unordered_map<const uint256_t, CmdFetchContext> cmd_fetch_waiting;
- std::unordered_map<const uint256_t, promise_t> decision_waiting;
- std::queue<command_t> cmd_pending;
-
- /* statistics */
- uint64_t fetched;
- uint64_t delivered;
- mutable uint64_t nsent;
- mutable uint64_t nrecv;
-
- mutable uint32_t part_parent_size;
- mutable uint32_t part_fetched;
- mutable uint32_t part_delivered;
- mutable uint32_t part_decided;
- mutable uint32_t part_gened;
- mutable double part_delivery_time;
- mutable double part_delivery_time_min;
- mutable double part_delivery_time_max;
- mutable std::unordered_map<const NetAddr, uint32_t> part_fetched_replica;
-
- void on_fetch_cmd(const command_t &cmd);
- void on_fetch_blk(const block_t &blk);
- void on_deliver_blk(const block_t &blk);
-
- /** deliver consensus message: <propose> */
- inline void propose_handler(const MsgHotStuff &, conn_t);
- /** deliver consensus message: <vote> */
- inline void vote_handler(const MsgHotStuff &, conn_t);
- /** fetches full block data */
- inline void query_fetch_blk_handler(const MsgHotStuff &, conn_t);
- /** receives a block */
- inline void resp_fetch_blk_handler(const MsgHotStuff &, conn_t);
-
- void do_broadcast_proposal(const Proposal &) override;
- void do_vote(ReplicaID, const Vote &) override;
- void do_decide(const command_t &) override;
-
- public:
- HotStuffBase(uint32_t blk_size,
- int32_t parent_limit,
- ReplicaID rid,
- privkey_bt &&priv_key,
- NetAddr listen_addr,
- EventContext eb = EventContext(),
- pacemaker_bt pmaker = nullptr);
-
- ~HotStuffBase();
-
- /* the API for HotStuffBase */
-
- /* Submit the command to be decided. */
- void add_command(command_t cmd) {
- cmd_pending.push(storage->add_cmd(cmd));
- if (cmd_pending.size() >= blk_size)
- {
- std::vector<command_t> cmds;
- for (uint32_t i = 0; i < blk_size; i++)
- {
- cmds.push_back(cmd_pending.front());
- cmd_pending.pop();
- }
- pmaker->beat().then([this, cmds = std::move(cmds)]() {
- on_propose(cmds);
- });
- }
- }
-
- void add_replica(ReplicaID idx, const NetAddr &addr, pubkey_bt &&pub_key);
- void start(bool eb_loop = false);
-
- size_t size() const { return pn.all_peers().size(); }
- void print_stat() const;
-
- /* Helper functions */
- /** Returns a promise resolved (with command_t cmd) when Command is fetched. */
- promise_t async_fetch_cmd(const uint256_t &cmd_hash, const NetAddr *replica_id, bool fetch_now = true);
- /** Returns a promise resolved (with block_t blk) when Block is fetched. */
- promise_t async_fetch_blk(const uint256_t &blk_hash, const NetAddr *replica_id, bool fetch_now = true);
- /** Returns a promise resolved (with block_t blk) when Block is delivered (i.e. prefix is fetched). */
- promise_t async_deliver_blk(const uint256_t &blk_hash, const NetAddr &replica_id);
- /** Returns a promise resolved (with command_t cmd) when Command is decided. */
- promise_t async_decide(const uint256_t &cmd_hash);
-};
-
-/** HotStuff protocol (templated by cryptographic implementation). */
-template<typename PrivKeyType = PrivKeyDummy,
- typename PubKeyType = PubKeyDummy,
- typename PartCertType = PartCertDummy,
- typename QuorumCertType = QuorumCertDummy>
-class HotStuff: public HotStuffBase {
- using HotStuffBase::HotStuffBase;
- protected:
-
- part_cert_bt create_part_cert(const PrivKey &priv_key, const uint256_t &blk_hash) override {
- return new PartCertType(
- static_cast<const PrivKeyType &>(priv_key),
- blk_hash);
- }
-
- part_cert_bt parse_part_cert(DataStream &s) override {
- PartCert *pc = new PartCertType();
- s >> *pc;
- return pc;
- }
-
- quorum_cert_bt create_quorum_cert(const uint256_t &blk_hash) override {
- return new QuorumCertType(get_config(), blk_hash);
- }
-
- quorum_cert_bt parse_quorum_cert(DataStream &s) override {
- QuorumCert *qc = new QuorumCertType();
- s >> *qc;
- return qc;
- }
-
- public:
- HotStuff(uint32_t blk_size,
- int32_t parent_limit,
- ReplicaID rid,
- const bytearray_t &raw_privkey,
- NetAddr listen_addr,
- EventContext eb = nullptr):
- HotStuffBase(blk_size,
- parent_limit,
- rid,
- new PrivKeyType(raw_privkey),
- listen_addr,
- eb) {}
-
- void add_replica(ReplicaID idx, const NetAddr &addr, const bytearray_t &pubkey_raw) {
- DataStream s(pubkey_raw);
- HotStuffBase::add_replica(idx, addr, new PubKeyType(pubkey_raw));
- }
-};
-
-using HotStuffNoSig = HotStuff<>;
-using HotStuffSecp256k1 = HotStuff<PrivKeySecp256k1, PubKeySecp256k1,
- PartCertSecp256k1, QuorumCertSecp256k1>;
-
-template<EntityType ent_type>
-FetchContext<ent_type>::FetchContext(FetchContext && other):
- promise_t(static_cast<const promise_t &>(other)),
- hs(other.hs),
- fetch_msg(std::move(other.fetch_msg)),
- ent_hash(other.ent_hash),
- replica_ids(std::move(other.replica_ids)) {
- other.timeout.del();
- timeout = Event(hs->eb, -1, 0,
- std::bind(&FetchContext::timeout_cb, this, _1, _2));
- reset_timeout();
-}
-
-template<>
-inline void FetchContext<ENT_TYPE_CMD>::timeout_cb(evutil_socket_t, short) {
- HOTSTUFF_LOG_WARN("cmd fetching %.10s timeout", get_hex(ent_hash).c_str());
- for (const auto &replica_id: replica_ids)
- send(replica_id);
- reset_timeout();
-}
-
-template<>
-inline void FetchContext<ENT_TYPE_BLK>::timeout_cb(evutil_socket_t, short) {
- HOTSTUFF_LOG_WARN("block fetching %.10s timeout", get_hex(ent_hash).c_str());
- for (const auto &replica_id: replica_ids)
- send(replica_id);
- reset_timeout();
-}
-
-template<EntityType ent_type>
-FetchContext<ent_type>::FetchContext(
- const uint256_t &ent_hash, HotStuffBase *hs):
- promise_t([](promise_t){}),
- hs(hs), ent_hash(ent_hash) {
- fetch_msg.gen_qfetchblk(std::vector<uint256_t>{ent_hash});
-
- timeout = Event(hs->eb, -1, 0,
- std::bind(&FetchContext::timeout_cb, this, _1, _2));
- reset_timeout();
-}
-
-template<EntityType ent_type>
-void FetchContext<ent_type>::send(const NetAddr &replica_id) {
- hs->part_fetched_replica[replica_id]++;
- hs->pn.send_msg(fetch_msg, replica_id);
-}
-
-template<EntityType ent_type>
-void FetchContext<ent_type>::reset_timeout() {
- timeout.add_with_timeout(salticidae::gen_rand_timeout(ent_waiting_timeout));
-}
-
-template<EntityType ent_type>
-void FetchContext<ent_type>::add_replica(const NetAddr &replica_id, bool fetch_now) {
- if (replica_ids.empty() && fetch_now)
- send(replica_id);
- replica_ids.insert(replica_id);
-}
-
-}
-
-#endif
diff --git a/src/crypto.cpp b/src/crypto.cpp
index 335a521..7450229 100644
--- a/src/crypto.cpp
+++ b/src/crypto.cpp
@@ -1,5 +1,5 @@
-#include "entity.h"
-#include "crypto.h"
+#include "hotstuff/entity.h"
+#include "hotstuff/crypto.h"
namespace hotstuff {
diff --git a/src/crypto.h b/src/crypto.h
deleted file mode 100644
index 2fbf745..0000000
--- a/src/crypto.h
+++ /dev/null
@@ -1,386 +0,0 @@
-#ifndef _HOTSTUFF_CRYPTO_H
-#define _HOTSTUFF_CRYPTO_H
-
-#include "salticidae/crypto.h"
-#include "salticidae/ref.h"
-#include "secp256k1.h"
-#include <openssl/rand.h>
-#include "type.h"
-
-using salticidae::RcObj;
-using salticidae::BoxObj;
-
-namespace hotstuff {
-
-using salticidae::SHA256;
-
-class PubKey: public Serializable, Cloneable {
- public:
- virtual ~PubKey() = default;
- virtual PubKey *clone() override = 0;
-};
-
-using pubkey_bt = BoxObj<PubKey>;
-
-class PrivKey: public Serializable {
- public:
- virtual ~PrivKey() = default;
- virtual pubkey_bt get_pubkey() const = 0;
- virtual void from_rand() = 0;
-};
-
-using privkey_bt = BoxObj<PrivKey>;
-
-class PartCert: public Serializable, public Cloneable {
- public:
- virtual ~PartCert() = default;
- virtual bool verify(const PubKey &pubkey) = 0;
- virtual const uint256_t &get_blk_hash() const = 0;
- virtual PartCert *clone() override = 0;
-};
-
-class ReplicaConfig;
-
-class QuorumCert: public Serializable, public Cloneable {
- public:
- virtual ~QuorumCert() = default;
- virtual void add_part(ReplicaID replica, const PartCert &pc) = 0;
- virtual void compute() = 0;
- virtual bool verify(const ReplicaConfig &config) = 0;
- virtual const uint256_t &get_blk_hash() const = 0;
- virtual QuorumCert *clone() override = 0;
-};
-
-using part_cert_bt = BoxObj<PartCert>;
-using quorum_cert_bt = BoxObj<QuorumCert>;
-
-class PubKeyDummy: public PubKey {
- PubKeyDummy *clone() override { return new PubKeyDummy(*this); }
- void serialize(DataStream &) const override {}
- void unserialize(DataStream &) override {}
-};
-
-class PrivKeyDummy: public PrivKey {
- pubkey_bt get_pubkey() const override { return new PubKeyDummy(); }
- void serialize(DataStream &) const override {}
- void unserialize(DataStream &) override {}
- void from_rand() override {}
-};
-
-class PartCertDummy: public PartCert {
- uint256_t blk_hash;
- public:
- PartCertDummy() {}
- PartCertDummy(const uint256_t &blk_hash):
- blk_hash(blk_hash) {}
-
- void serialize(DataStream &s) const override {
- s << (uint32_t)0 << blk_hash;
- }
-
- void unserialize(DataStream &s) override {
- uint32_t tmp;
- s >> tmp >> blk_hash;
- }
-
- PartCert *clone() override {
- return new PartCertDummy(blk_hash);
- }
-
- bool verify(const PubKey &) override { return true; }
-
- const uint256_t &get_blk_hash() const override { return blk_hash; }
-};
-
-class QuorumCertDummy: public QuorumCert {
- uint256_t blk_hash;
- public:
- QuorumCertDummy() {}
- QuorumCertDummy(const ReplicaConfig &, const uint256_t &blk_hash):
- blk_hash(blk_hash) {}
-
- void serialize(DataStream &s) const override {
- s << (uint32_t)1 << blk_hash;
- }
-
- void unserialize(DataStream &s) override {
- uint32_t tmp;
- s >> tmp >> blk_hash;
- }
-
- QuorumCert *clone() override {
- return new QuorumCertDummy(*this);
- }
-
- void add_part(ReplicaID, const PartCert &) override {}
- void compute() override {}
- bool verify(const ReplicaConfig &) override { return true; }
-
- const uint256_t &get_blk_hash() const override { return blk_hash; }
-};
-
-
-class Secp256k1Context {
- secp256k1_context *ctx;
- friend class PubKeySecp256k1;
- friend class SigSecp256k1;
- public:
- Secp256k1Context(bool sign = false):
- ctx(secp256k1_context_create(
- sign ? SECP256K1_CONTEXT_SIGN :
- SECP256K1_CONTEXT_VERIFY)) {}
-
- Secp256k1Context(const Secp256k1Context &) = delete;
-
- Secp256k1Context(Secp256k1Context &&other): ctx(other.ctx) {
- other.ctx = nullptr;
- }
-
- ~Secp256k1Context() {
- if (ctx) secp256k1_context_destroy(ctx);
- }
-};
-
-using secp256k1_context_t = RcObj<Secp256k1Context>;
-
-extern secp256k1_context_t secp256k1_default_sign_ctx;
-extern secp256k1_context_t secp256k1_default_verify_ctx;
-
-class PrivKeySecp256k1;
-
-class PubKeySecp256k1: public PubKey {
- static const auto _olen = 33;
- friend class SigSecp256k1;
- secp256k1_pubkey data;
- secp256k1_context_t ctx;
-
- public:
- PubKeySecp256k1(const secp256k1_context_t &ctx =
- secp256k1_default_sign_ctx):
- PubKey(), ctx(ctx) {}
-
- PubKeySecp256k1(const bytearray_t &raw_bytes,
- const secp256k1_context_t &ctx =
- secp256k1_default_sign_ctx):
- PubKeySecp256k1(ctx) { from_bytes(raw_bytes); }
-
- inline PubKeySecp256k1(const PrivKeySecp256k1 &priv_key,
- const secp256k1_context_t &ctx =
- secp256k1_default_sign_ctx);
-
- void serialize(DataStream &s) const override {
- static uint8_t output[_olen];
- size_t olen = _olen;
- (void)secp256k1_ec_pubkey_serialize(
- ctx->ctx, (unsigned char *)output,
- &olen, &data, SECP256K1_EC_COMPRESSED);
- s.put_data(output, output + _olen);
- }
-
- void unserialize(DataStream &s) override {
- static const auto _exc = std::invalid_argument("ill-formed public key");
- try {
- if (!secp256k1_ec_pubkey_parse(
- ctx->ctx, &data, s.get_data_inplace(_olen), _olen))
- throw _exc;
- } catch (std::ios_base::failure &) {
- throw _exc;
- }
- }
-
- PubKeySecp256k1 *clone() override {
- return new PubKeySecp256k1(*this);
- }
-};
-
-class PrivKeySecp256k1: public PrivKey {
- static const auto nbytes = 32;
- friend class PubKeySecp256k1;
- friend class SigSecp256k1;
- uint8_t data[nbytes];
- secp256k1_context_t ctx;
-
- public:
- PrivKeySecp256k1(const secp256k1_context_t &ctx =
- secp256k1_default_sign_ctx):
- PrivKey(), ctx(ctx) {}
-
- PrivKeySecp256k1(const bytearray_t &raw_bytes,
- const secp256k1_context_t &ctx =
- secp256k1_default_sign_ctx):
- PrivKeySecp256k1(ctx) { from_bytes(raw_bytes); }
-
- void serialize(DataStream &s) const override {
- s.put_data(data, data + nbytes);
- }
-
- void unserialize(DataStream &s) override {
- static const auto _exc = std::invalid_argument("ill-formed private key");
- try {
- memmove(data, s.get_data_inplace(nbytes), nbytes);
- } catch (std::ios_base::failure &) {
- throw _exc;
- }
- }
-
- void from_rand() override {
- if (!RAND_bytes(data, nbytes))
- throw std::runtime_error("cannot get rand bytes from openssl");
- }
-
- inline pubkey_bt get_pubkey() const override;
-};
-
-pubkey_bt PrivKeySecp256k1::get_pubkey() const {
- return new PubKeySecp256k1(*this, ctx);
-}
-
-PubKeySecp256k1::PubKeySecp256k1(
- const PrivKeySecp256k1 &priv_key,
- const secp256k1_context_t &ctx): PubKey(), ctx(ctx) {
- if (!secp256k1_ec_pubkey_create(ctx->ctx, &data, priv_key.data))
- throw std::invalid_argument("invalid secp256k1 private key");
-}
-
-class SigSecp256k1: public Serializable {
- secp256k1_ecdsa_signature data;
- secp256k1_context_t ctx;
-
- void check_msg_length(const bytearray_t &msg) {
- if (msg.size() != 32)
- throw std::invalid_argument("the message should be 32-bytes");
- }
-
- public:
- SigSecp256k1(const secp256k1_context_t &ctx =
- secp256k1_default_sign_ctx):
- Serializable(), ctx(ctx) {}
- SigSecp256k1(const uint256_t &digest,
- const PrivKeySecp256k1 &priv_key,
- secp256k1_context_t &ctx =
- secp256k1_default_sign_ctx):
- Serializable(), ctx(ctx) {
- sign(digest, priv_key);
- }
-
- void serialize(DataStream &s) const override {
- static uint8_t output[64];
- (void)secp256k1_ecdsa_signature_serialize_compact(
- ctx->ctx, (unsigned char *)output,
- &data);
- s.put_data(output, output + 64);
- }
-
- void unserialize(DataStream &s) override {
- static const auto _exc = std::invalid_argument("ill-formed signature");
- try {
- if (!secp256k1_ecdsa_signature_parse_compact(
- ctx->ctx, &data, s.get_data_inplace(64)))
- throw _exc;
- } catch (std::ios_base::failure &) {
- throw _exc;
- }
- }
-
- void sign(const bytearray_t &msg, const PrivKeySecp256k1 &priv_key) {
- check_msg_length(msg);
- if (!secp256k1_ecdsa_sign(
- ctx->ctx, &data,
- (unsigned char *)&*msg.begin(),
- (unsigned char *)priv_key.data,
- NULL, // default nonce function
- NULL))
- throw std::invalid_argument("failed to create secp256k1 signature");
- }
-
- bool verify(const bytearray_t &msg, const PubKeySecp256k1 &pub_key,
- const secp256k1_context_t &_ctx) {
- check_msg_length(msg);
- return secp256k1_ecdsa_verify(
- _ctx->ctx, &data,
- (unsigned char *)&*msg.begin(),
- &pub_key.data) == 1;
- }
-
- bool verify(const bytearray_t &msg, const PubKeySecp256k1 &pub_key) {
- return verify(msg, pub_key, ctx);
- }
-};
-
-class PartCertSecp256k1: public SigSecp256k1, public PartCert {
- uint256_t blk_hash;
-
- public:
- PartCertSecp256k1() = default;
- PartCertSecp256k1(const PrivKeySecp256k1 &priv_key, const uint256_t &blk_hash):
- SigSecp256k1(blk_hash, priv_key),
- PartCert(),
- blk_hash(blk_hash) {}
-
- bool verify(const PubKey &pub_key) override {
- return SigSecp256k1::verify(blk_hash,
- static_cast<const PubKeySecp256k1 &>(pub_key),
- secp256k1_default_verify_ctx);
- }
-
- const uint256_t &get_blk_hash() const override { return blk_hash; }
-
- PartCertSecp256k1 *clone() override {
- return new PartCertSecp256k1(*this);
- }
-
- void serialize(DataStream &s) const override {
- s << blk_hash;
- this->SigSecp256k1::serialize(s);
- }
-
- void unserialize(DataStream &s) override {
- s >> blk_hash;
- this->SigSecp256k1::unserialize(s);
- }
-};
-
-class QuorumCertSecp256k1: public QuorumCert {
- uint256_t blk_hash;
- salticidae::Bits rids;
- std::vector<SigSecp256k1> sigs;
-
- public:
- QuorumCertSecp256k1() = default;
- QuorumCertSecp256k1(const ReplicaConfig &config, const uint256_t &blk_hash);
-
- void add_part(ReplicaID rid, const PartCert &pc) override {
- if (pc.get_blk_hash() != blk_hash)
- throw std::invalid_argument("PartCert does match the block hash");
- if (!rids.get(rid))
- {
- rids.set(rid);
- sigs.push_back(static_cast<const PartCertSecp256k1 &>(pc));
- }
- }
-
- void compute() override {}
-
- bool verify(const ReplicaConfig &config) override;
-
- const uint256_t &get_blk_hash() const override { return blk_hash; }
-
- QuorumCertSecp256k1 *clone() override {
- return new QuorumCertSecp256k1(*this);
- }
-
- void serialize(DataStream &s) const override {
- s << blk_hash << rids;
- for (const auto &sig: sigs) s << sig;
- }
-
- void unserialize(DataStream &s) override {
- s >> blk_hash >> rids;
- sigs.resize(rids.size());
- for (auto &sig: sigs) s >> sig;
- }
-};
-
-}
-
-#endif
diff --git a/src/entity.cpp b/src/entity.cpp
index 1294484..1d2e926 100644
--- a/src/entity.cpp
+++ b/src/entity.cpp
@@ -1,5 +1,5 @@
-#include "entity.h"
-#include "core.h"
+#include "hotstuff/entity.h"
+#include "hotstuff/core.h"
namespace hotstuff {
diff --git a/src/entity.h b/src/entity.h
deleted file mode 100644
index b3a0df4..0000000
--- a/src/entity.h
+++ /dev/null
@@ -1,309 +0,0 @@
-#ifndef _HOTSTUFF_ENT_H
-#define _HOTSTUFF_ENT_H
-
-#include <vector>
-#include <unordered_map>
-#include <unordered_set>
-#include <string>
-#include <cstddef>
-#include <ios>
-
-#include "salticidae/netaddr.h"
-#include "salticidae/ref.h"
-#include "type.h"
-#include "util.h"
-#include "crypto.h"
-
-namespace hotstuff {
-
-enum EntityType {
- ENT_TYPE_CMD = 0x0,
- ENT_TYPE_BLK = 0x1
-};
-
-struct ReplicaInfo {
- ReplicaID id;
- salticidae::NetAddr addr;
- pubkey_bt pubkey;
-
- ReplicaInfo(ReplicaID id,
- const salticidae::NetAddr &addr,
- pubkey_bt &&pubkey):
- id(id), addr(addr), pubkey(std::move(pubkey)) {}
-
- ReplicaInfo(const ReplicaInfo &other):
- id(other.id), addr(other.addr),
- pubkey(other.pubkey->clone()) {}
-
- ReplicaInfo(ReplicaInfo &&other):
- id(other.id), addr(other.addr),
- pubkey(std::move(other.pubkey)) {}
-};
-
-class ReplicaConfig {
- std::unordered_map<ReplicaID, ReplicaInfo> replica_map;
-
- public:
- size_t nmajority;
-
- void add_replica(ReplicaID rid, const ReplicaInfo &info) {
- replica_map.insert(std::make_pair(rid, info));
- }
-
- const ReplicaInfo &get_info(ReplicaID rid) const {
- auto it = replica_map.find(rid);
- if (it == replica_map.end())
- throw HotStuffError("rid %s not found",
- get_hex(rid).c_str());
- return it->second;
- }
-
- const PubKey &get_pubkey(ReplicaID rid) const {
- return *(get_info(rid).pubkey);
- }
-
- const salticidae::NetAddr &get_addr(ReplicaID rid) const {
- return get_info(rid).addr;
- }
-};
-
-class Block;
-class HotStuffCore;
-
-using block_t = salticidae::RcObj<Block>;
-using block_weak_t = salticidae::WeakObj<Block>;
-
-struct Finality: public Serializable {
- int8_t decision;
- uint256_t blk_hash;
-
- public:
- Finality(): decision(0) {}
- Finality(int8_t decision, uint256_t blk_hash):
- decision(decision), blk_hash(blk_hash) {}
-
- void serialize(DataStream &s) const override {
- s << decision;
- if (decision == 1) s << blk_hash;
- }
-
- void unserialize(DataStream &s) override {
- s >> decision;
- if (decision == 1) s >> blk_hash;
- }
-};
-
-class Command: public Serializable {
- friend HotStuffCore;
- block_weak_t container;
- public:
- virtual ~Command() = default;
- virtual const uint256_t &get_hash() const = 0;
- virtual bool verify() const = 0;
- inline int8_t get_decision() const;
- inline Finality get_finality() const;
- block_t get_container() const {
- return container;
- }
-};
-
-using command_t = RcObj<Command>;
-
-template<typename Hashable>
-inline static std::vector<uint256_t>
-get_hashes(const std::vector<Hashable> &plist) {
- std::vector<uint256_t> hashes;
- for (const auto &p: plist)
- hashes.push_back(p->get_hash());
- return std::move(hashes);
-}
-
-class Block {
- friend HotStuffCore;
- std::vector<uint256_t> parent_hashes;
- std::vector<command_t> cmds;
- quorum_cert_bt qc;
- uint256_t hash;
-
- std::vector<block_t> parents;
- block_t qc_ref;
- quorum_cert_bt self_qc;
- uint32_t height;
- bool delivered;
- int8_t decision;
-
- std::unordered_set<ReplicaID> voted;
-
- public:
- Block():
- qc(nullptr),
- qc_ref(nullptr),
- self_qc(nullptr), height(0),
- delivered(false), decision(0) {}
-
- Block(bool delivered, int8_t decision):
- qc(nullptr),
- hash(salticidae::get_hash(*this)),
- qc_ref(nullptr),
- self_qc(nullptr), height(0),
- delivered(delivered), decision(decision) {}
-
- Block(const std::vector<block_t> &parents,
- const std::vector<command_t> &cmds,
- uint32_t height,
- quorum_cert_bt &&qc,
- const block_t &qc_ref,
- quorum_cert_bt &&self_qc,
- int8_t decision = 0):
- parent_hashes(get_hashes(parents)),
- cmds(cmds),
- qc(std::move(qc)),
- hash(salticidae::get_hash(*this)),
- parents(parents),
- qc_ref(qc_ref),
- self_qc(std::move(self_qc)),
- height(height),
- delivered(0),
- decision(decision) {}
-
- void serialize(DataStream &s) const;
-
- void unserialize(DataStream &s, HotStuffCore *hsc);
-
- const std::vector<command_t> &get_cmds() const {
- return cmds;
- }
-
- const std::vector<block_t> &get_parents() const {
- return parents;
- }
-
- const std::vector<uint256_t> &get_parent_hashes() const {
- return parent_hashes;
- }
-
- const uint256_t &get_hash() const { return hash; }
-
- bool verify(const ReplicaConfig &config) const {
- if (qc && !qc->verify(config)) return false;
- for (auto cmd: cmds)
- if (!cmd->verify()) return false;
- return true;
- }
-
- int8_t get_decision() const { return decision; }
-
- bool is_delivered() const { return delivered; }
-
- uint32_t get_height() const { return height; }
-
- const quorum_cert_bt &get_qc() const { return qc; }
-
- operator std::string () const {
- DataStream s;
- s << "<block "
- << "id=" << get_hex10(hash) << " "
- << "height=" << std::to_string(height) << " "
- << "parent=" << get_hex10(parent_hashes[0]) << ">";
- return std::string(std::move(s));
- }
-};
-
-struct BlockHeightCmp {
- bool operator()(const block_t &a, const block_t &b) const {
- return a->get_height() < b->get_height();
- }
-};
-
-int8_t Command::get_decision() const {
- block_t cptr = container;
- return cptr ? cptr->get_decision() : 0;
-}
-
-Finality Command::get_finality() const {
- block_t blk = get_container();
- return Finality(get_decision(),
- blk ? blk->get_hash() : uint256_t());
-}
-
-class EntityStorage {
- std::unordered_map<const uint256_t, block_t> blk_cache;
- std::unordered_map<const uint256_t, command_t> cmd_cache;
- public:
- bool is_blk_delivered(const uint256_t &blk_hash) {
- auto it = blk_cache.find(blk_hash);
- if (it == blk_cache.end()) return false;
- return it->second->is_delivered();
- }
-
- bool is_blk_fetched(const uint256_t &blk_hash) {
- return blk_cache.count(blk_hash);
- }
-
- const block_t &add_blk(Block &&_blk) {
- block_t blk = new Block(std::move(_blk));
- return blk_cache.insert(std::make_pair(blk->get_hash(), blk)).first->second;
- }
-
- const block_t &add_blk(const block_t &blk) {
- return blk_cache.insert(std::make_pair(blk->get_hash(), blk)).first->second;
- }
-
- block_t find_blk(const uint256_t &blk_hash) {
- auto it = blk_cache.find(blk_hash);
- return it == blk_cache.end() ? nullptr : it->second;
- }
-
- bool is_cmd_fetched(const uint256_t &cmd_hash) {
- return cmd_cache.count(cmd_hash);
- }
-
- const command_t &add_cmd(const command_t &cmd) {
- return cmd_cache.insert(std::make_pair(cmd->get_hash(), cmd)).first->second;
- }
-
- command_t find_cmd(const uint256_t &cmd_hash) {
- auto it = cmd_cache.find(cmd_hash);
- return it == cmd_cache.end() ? nullptr: it->second;
- }
-
- size_t get_cmd_cache_size() {
- return cmd_cache.size();
- }
- size_t get_blk_cache_size() {
- return blk_cache.size();
- }
-
- bool try_release_cmd(const command_t &cmd) {
- if (cmd.get_cnt() == 2) /* only referred by cmd and the storage */
- {
- const auto &cmd_hash = cmd->get_hash();
- cmd_cache.erase(cmd_hash);
- return true;
- }
- return false;
- }
-
- bool try_release_blk(const block_t &blk) {
- if (blk.get_cnt() == 2) /* only referred by blk and the storage */
- {
- const auto &blk_hash = blk->get_hash();
-#ifdef HOTSTUFF_ENABLE_LOG_PROTO
- HOTSTUFF_LOG_INFO("releasing blk %.10s", get_hex(blk_hash).c_str());
-#endif
- for (const auto &cmd: blk->get_cmds())
- try_release_cmd(cmd);
- blk_cache.erase(blk_hash);
- return true;
- }
-#ifdef HOTSTUFF_ENABLE_LOG_PROTO
- else
- HOTSTUFF_LOG_INFO("cannot release (%lu)", blk.get_cnt());
-#endif
- return false;
- }
-};
-
-}
-
-#endif
diff --git a/src/hotstuff.cpp b/src/hotstuff.cpp
index b10f103..74d2f04 100644
--- a/src/hotstuff.cpp
+++ b/src/hotstuff.cpp
@@ -11,12 +11,13 @@
#include "salticidae/util.h"
#include "salticidae/network.h"
#include "salticidae/msg.h"
-#include "promise.hpp"
-#include "type.h"
-#include "core.h"
-#include "entity.h"
-#include "util.h"
-#include "client.h"
+
+#include "hotstuff/promise.hpp"
+#include "hotstuff/type.h"
+#include "hotstuff/core.h"
+#include "hotstuff/entity.h"
+#include "hotstuff/util.h"
+#include "hotstuff/client.h"
using promise::promise_t;
using salticidae::NetAddr;
diff --git a/src/hotstuff_client.cpp b/src/hotstuff_client.cpp
index 5c04e5f..6b26c9c 100644
--- a/src/hotstuff_client.cpp
+++ b/src/hotstuff_client.cpp
@@ -3,9 +3,10 @@
#include "salticidae/netaddr.h"
#include "salticidae/network.h"
#include "salticidae/util.h"
-#include "util.h"
-#include "type.h"
-#include "client.h"
+
+#include "hotstuff/util.h"
+#include "hotstuff/type.h"
+#include "hotstuff/client.h"
using salticidae::NetAddr;
using salticidae::Config;
diff --git a/src/hotstuff_keygen.cpp b/src/hotstuff_keygen.cpp
index 7a7e615..100cb01 100644
--- a/src/hotstuff_keygen.cpp
+++ b/src/hotstuff_keygen.cpp
@@ -1,6 +1,6 @@
#include <error.h>
#include "salticidae/util.h"
-#include "crypto.h"
+#include "hotstuff/crypto.h"
using salticidae::Config;
using hotstuff::privkey_bt;
diff --git a/src/promise.hpp b/src/promise.hpp
deleted file mode 100644
index 593d5c1..0000000
--- a/src/promise.hpp
+++ /dev/null
@@ -1,745 +0,0 @@
-#ifndef _CPPROMISE_HPP
-#define _CPPROMISE_HPP
-
-/**
- * MIT License
- * Copyright (c) 2018 Ted Yin <[email protected]>
- *
- * Permission is hereby granted, free of charge, to any person obtaining a copy
- * of this software and associated documentation files (the "Software"), to deal
- * in the Software without restriction, including without limitation the rights
- * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
- * copies of the Software, and to permit persons to whom the Software is
- * furnished to do so, subject to the following conditions:
- *
- * The above copyright notice and this permission notice shall be included in all
- * copies or substantial portions of the Software.
- *
- * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
- * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
- * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
- * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
- * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
- * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
- * SOFTWARE.
- */
-
-#include <stack>
-#include <vector>
-#include <memory>
-#include <functional>
-#include <type_traits>
-
-#if __cplusplus >= 201703L
-#ifdef __has_include
-# if __has_include(<any>)
-# include <any>
-# ifdef __cpp_lib_any
-# define _CPPROMISE_STD_ANY
-# endif
-# endif
-#endif
-#endif
-
-#ifndef _CPPROMISE_STD_ANY
-#include <boost/any.hpp>
-#endif
-
-/**
- * Implement type-safe Promise primitives similar to the ones specified by
- * Javascript Promise/A+.
- */
-namespace promise {
-#ifdef _CPPROMISE_STD_ANY
- using pm_any_t = std::any;
- template<typename T>
- constexpr auto any_cast = static_cast<T(*)(const std::any&)>(std::any_cast<T>);
- using bad_any_cast = std::bad_any_cast;
-#else
-# warning "using boost::any"
-# pragma message "using boost::any"
- using pm_any_t = boost::any;
- template<typename T>
- constexpr auto any_cast = static_cast<T(*)(const boost::any&)>(boost::any_cast<T>);
- using bad_any_cast = boost::bad_any_cast;
-#endif
- using callback_t = std::function<void()>;
- using values_t = std::vector<pm_any_t>;
-
- /* match lambdas */
- template<typename T>
- struct function_traits:
- public function_traits<decltype(&T::operator())> {};
-
- template<typename ReturnType>
- struct function_traits<ReturnType()> {
- using ret_type = ReturnType;
- using arg_type = void;
- using empty_arg = void;
- };
-
- /* match plain functions */
- template<typename ReturnType, typename ArgType>
- struct function_traits<ReturnType(ArgType)> {
- using ret_type = ReturnType;
- using arg_type = ArgType;
- using non_empty_arg = void;
- };
-
- /* match function pointers */
- template<typename ReturnType, typename... ArgType>
- struct function_traits<ReturnType(*)(ArgType...)>:
- public function_traits<ReturnType(ArgType...)> {};
-
- /* match const member functions */
- template<typename ClassType, typename ReturnType, typename... ArgType>
- struct function_traits<ReturnType(ClassType::*)(ArgType...) const>:
- public function_traits<ReturnType(ArgType...)> {};
-
- /* match member functions */
- template<typename ClassType, typename ReturnType, typename... ArgType>
- struct function_traits<ReturnType(ClassType::*)(ArgType...)>:
- public function_traits<ReturnType(ArgType...)> {};
-
- template<typename Func, typename ReturnType>
- using enable_if_return = typename std::enable_if<
- std::is_same<typename function_traits<Func>::ret_type,
- ReturnType>::value>;
-
- template<typename Func, typename ReturnType>
- using disable_if_return = typename std::enable_if<
- !std::is_same<typename function_traits<Func>::ret_type,
- ReturnType>::value>;
-
- template<typename Func, typename ArgType>
- using enable_if_arg = typename std::enable_if<
- std::is_same<typename function_traits<Func>::arg_type,
- ArgType>::value>;
-
- template<typename Func, typename ArgType>
- using disable_if_arg = typename std::enable_if<
- !std::is_same<typename function_traits<Func>::arg_type,
- ArgType>::value>;
-
- class Promise;
- //class promise_t: public std::shared_ptr<Promise> {
- class promise_t {
- Promise *pm;
- size_t *ref_cnt;
- public:
- friend Promise;
- template<typename PList> friend promise_t all(const PList &promise_list);
- template<typename PList> friend promise_t race(const PList &promise_list);
-
- inline promise_t();
- inline ~promise_t();
- template<typename Func> inline promise_t(Func callback);
-
- void swap(promise_t &other) {
- std::swap(pm, other.pm);
- std::swap(ref_cnt, other.ref_cnt);
- }
-
- promise_t &operator=(const promise_t &other) {
- if (this != &other)
- {
- promise_t tmp(other);
- tmp.swap(*this);
- }
- return *this;
- }
-
- promise_t &operator=(promise_t &&other) {
- if (this != &other)
- {
- promise_t tmp(std::move(other));
- tmp.swap(*this);
- }
- return *this;
- }
-
- promise_t(const promise_t &other):
- pm(other.pm),
- ref_cnt(other.ref_cnt) {
- ++*ref_cnt;
- }
-
- promise_t(promise_t &&other):
- pm(other.pm),
- ref_cnt(other.ref_cnt) {
- other.pm = nullptr;
- }
-
- Promise *operator->() const {
- return pm;
- }
-
- template<typename T> inline void resolve(T result) const;
- template<typename T> inline void reject(T reason) const;
- inline void resolve() const;
- inline void reject() const;
-
- template<typename FuncFulfilled>
- inline promise_t then(FuncFulfilled on_fulfilled) const;
-
- template<typename FuncFulfilled, typename FuncRejected>
- inline promise_t then(FuncFulfilled on_fulfilled,
- FuncRejected on_rejected) const;
-
- template<typename FuncRejected>
- inline promise_t fail(FuncRejected on_rejected) const;
- };
-
-#define PROMISE_ERR_INVALID_STATE do {throw std::runtime_error("invalid promise state");} while (0)
-#define PROMISE_ERR_MISMATCH_TYPE do {throw std::runtime_error("mismatching promise value types");} while (0)
-
- class Promise {
- template<typename PList> friend promise_t all(const PList &promise_list);
- template<typename PList> friend promise_t race(const PList &promise_list);
- std::vector<callback_t> fulfilled_callbacks;
- std::vector<callback_t> rejected_callbacks;
-#ifdef CPPROMISE_USE_STACK_FREE
- std::vector<Promise *> fulfilled_pms;
- std::vector<Promise *> rejected_pms;
-#endif
- enum class State {
- Pending,
-#ifdef CPPROMISE_USE_STACK_FREE
- PreFulfilled,
- PreRejected,
-#endif
- Fulfilled,
- Rejected,
- } state;
- pm_any_t result;
- pm_any_t reason;
-
- void add_on_fulfilled(callback_t &&cb) {
- fulfilled_callbacks.push_back(std::move(cb));
- }
-
- void add_on_rejected(callback_t &&cb) {
- rejected_callbacks.push_back(std::move(cb));
- }
-
- template<typename Func,
- typename function_traits<Func>::non_empty_arg * = nullptr>
- static constexpr auto cps_transform(
- Func f, const pm_any_t &result, const promise_t &npm) {
- return [&result, npm, f = std::forward<Func>(f)]() mutable {
-#ifndef CPPROMISE_USE_STACK_FREE
- f(result)->then(
- [npm] (pm_any_t result) {npm->resolve(result);},
- [npm] (pm_any_t reason) {npm->reject(reason);});
-#else
- promise_t rpm{f(result)};
- rpm->then(
- [rpm, npm] (pm_any_t result) {
- npm->_resolve(result);
- },
- [rpm, npm] (pm_any_t reason) {
- npm->_reject(reason);
- });
- rpm->_dep_resolve(npm);
- rpm->_dep_reject(npm);
-#endif
- };
- }
-
- template<typename Func,
- typename function_traits<Func>::empty_arg * = nullptr>
- static constexpr auto cps_transform(
- Func f, const pm_any_t &, const promise_t &npm) {
- return [npm, f = std::forward<Func>(f)]() mutable {
-#ifndef CPPROMISE_USE_STACK_FREE
- f()->then(
- [npm] (pm_any_t result) {npm->resolve(result);},
- [npm] (pm_any_t reason) {npm->reject(reason);});
-#else
- promise_t rpm{f()};
- rpm->then(
- [rpm, npm] (pm_any_t result) {
- npm->_resolve(result);
- },
- [rpm, npm] (pm_any_t reason) {
- npm->_reject(reason);
- });
- rpm->_dep_resolve(npm);
- rpm->_dep_reject(npm);
-#endif
- };
- }
-
- template<typename Func,
- typename enable_if_return<Func, promise_t>::type * = nullptr>
- constexpr auto gen_on_fulfilled(Func on_fulfilled, const promise_t &npm) {
- return cps_transform(std::forward<Func>(on_fulfilled), this->result, npm);
- }
-
- template<typename Func,
- typename enable_if_return<Func, promise_t>::type * = nullptr>
- constexpr auto gen_on_rejected(Func on_rejected, const promise_t &npm) {
- return cps_transform(std::forward<Func>(on_rejected), this->reason, npm);
- }
-
-
- template<typename Func,
- typename enable_if_return<Func, void>::type * = nullptr,
- typename function_traits<Func>::non_empty_arg * = nullptr>
- constexpr auto gen_on_fulfilled(Func on_fulfilled, const promise_t &npm) {
- return [this, npm,
- on_fulfilled = std::forward<Func>(on_fulfilled)]() mutable {
- on_fulfilled(result);
- npm->_resolve();
- };
- }
-
- template<typename Func,
- typename enable_if_return<Func, void>::type * = nullptr,
- typename function_traits<Func>::empty_arg * = nullptr>
- constexpr auto gen_on_fulfilled(Func on_fulfilled, const promise_t &npm) {
- return [on_fulfilled = std::forward<Func>(on_fulfilled), npm]() mutable {
- on_fulfilled();
- npm->_resolve();
- };
- }
-
- template<typename Func,
- typename enable_if_return<Func, void>::type * = nullptr,
- typename function_traits<Func>::non_empty_arg * = nullptr>
- constexpr auto gen_on_rejected(Func on_rejected, const promise_t &npm) {
- return [this, npm,
- on_rejected = std::forward<Func>(on_rejected)]() mutable {
- on_rejected(reason);
- npm->_reject();
- };
- }
-
- template<typename Func,
- typename enable_if_return<Func, void>::type * = nullptr,
- typename function_traits<Func>::empty_arg * = nullptr>
- constexpr auto gen_on_rejected(Func on_rejected, const promise_t &npm) {
- return [npm,
- on_rejected = std::forward<Func>(on_rejected)]() mutable {
- on_rejected();
- npm->_reject();
- };
- }
-
- template<typename Func,
- typename enable_if_return<Func, pm_any_t>::type * = nullptr,
- typename function_traits<Func>::non_empty_arg * = nullptr>
- constexpr auto gen_on_fulfilled(Func on_fulfilled, const promise_t &npm) {
- return [this, npm,
- on_fulfilled = std::forward<Func>(on_fulfilled)]() mutable {
- npm->_resolve(on_fulfilled(result));
- };
- }
-
- template<typename Func,
- typename enable_if_return<Func, pm_any_t>::type * = nullptr,
- typename function_traits<Func>::empty_arg * = nullptr>
- constexpr auto gen_on_fulfilled(Func on_fulfilled, const promise_t &npm) {
- return [npm, on_fulfilled = std::forward<Func>(on_fulfilled)]() mutable {
- npm->_resolve(on_fulfilled());
- };
- }
-
- template<typename Func,
- typename enable_if_return<Func, pm_any_t>::type * = nullptr,
- typename function_traits<Func>::non_empty_arg * = nullptr>
- constexpr auto gen_on_rejected(Func on_rejected, const promise_t &npm) {
- return [this, npm, on_rejected = std::forward<Func>(on_rejected)]() mutable {
- npm->_reject(on_rejected(reason));
- };
- }
-
- template<typename Func,
- typename enable_if_return<Func, pm_any_t>::type * = nullptr,
- typename function_traits<Func>::empty_arg * = nullptr>
- constexpr auto gen_on_rejected(Func on_rejected, const promise_t &npm) {
- return [npm, on_rejected = std::forward<Func>(on_rejected)]() mutable {
- npm->_reject(on_rejected());
- };
- }
-
-#ifdef CPPROMISE_USE_STACK_FREE
- void _trigger() {
- std::stack<std::tuple<
- std::vector<Promise *>::const_iterator,
- std::vector<Promise *> *,
- Promise *>> s;
- auto push_frame = [&s](Promise *pm) {
- if (pm->state == State::PreFulfilled)
- {
- pm->state = State::Fulfilled;
- for (auto &cb: pm->fulfilled_callbacks) cb();
- s.push(std::make_tuple(pm->fulfilled_pms.begin(),
- &pm->fulfilled_pms,
- pm));
- }
- else if (pm->state == State::PreRejected)
- {
- pm->state = State::Rejected;
- for (auto &cb: pm->rejected_callbacks) cb();
- s.push(std::make_tuple(pm->rejected_pms.begin(),
- &pm->rejected_pms,
- pm));
- }
- };
- push_frame(this);
- while (!s.empty())
- {
- auto &u = s.top();
- auto &it = std::get<0>(u);
- auto vec = std::get<1>(u);
- auto pm = std::get<2>(u);
- if (it == vec->end())
- {
- s.pop();
- vec->clear();
- pm->fulfilled_callbacks.clear();
- pm->rejected_callbacks.clear();
- continue;
- }
- push_frame(*it++);
- }
- }
-
- void trigger_fulfill() {
- state = State::PreFulfilled;
- _trigger();
- }
-
- void trigger_reject() {
- state = State::PreRejected;
- _trigger();
- }
-
- void _resolve() {
- if (state == State::Pending) state = State::PreFulfilled;
- }
-
- void _reject() {
- if (state == State::Pending) state = State::PreRejected;
- }
-
- void _dep_resolve(const promise_t &npm) {
- if (state == State::Pending)
- fulfilled_pms.push_back(npm.pm);
- else
- npm->_trigger();
- }
-
- void _dep_reject(const promise_t &npm) {
- if (state == State::Pending)
- rejected_pms.push_back(npm.pm);
- else
- npm->_trigger();
- }
-
- void _resolve(pm_any_t _result) {
- if (state == State::Pending)
- {
- result = _result;
- state = State::PreFulfilled;
- }
- }
-
- void _reject(pm_any_t _reason) {
- if (state == State::Pending)
- {
- reason = _reason;
- state = State::PreRejected;
- }
- }
-#else
- void _resolve() { resolve(); }
- void _reject() { reject(); }
- void _resolve(pm_any_t result) { resolve(result); }
- void _reject(pm_any_t reason) { reject(reason); }
-
- void trigger_fulfill() {
- state = State::Fulfilled;
- for (const auto &cb: fulfilled_callbacks) cb();
- fulfilled_callbacks.clear();
- }
-
- void trigger_reject() {
- state = State::Rejected;
- for (const auto &cb: rejected_callbacks) cb();
- rejected_callbacks.clear();
- }
-#endif
- public:
-
- Promise(): state(State::Pending) {}
- ~Promise() {}
-
- template<typename FuncFulfilled, typename FuncRejected>
- promise_t then(FuncFulfilled on_fulfilled,
- FuncRejected on_rejected) {
- switch (state)
- {
- case State::Pending:
- return promise_t([this,
- on_fulfilled = std::forward<FuncFulfilled>(on_fulfilled),
- on_rejected = std::forward<FuncRejected>(on_rejected)
- ](promise_t &npm) {
- add_on_fulfilled(gen_on_fulfilled(std::move(on_fulfilled), npm));
- add_on_rejected(gen_on_rejected(std::move(on_rejected), npm));
-#ifdef CPPROMISE_USE_STACK_FREE
- _dep_resolve(npm);
- _dep_reject(npm);
-#endif
- });
- case State::Fulfilled:
- return promise_t([this,
- on_fulfilled = std::forward<FuncFulfilled>(on_fulfilled)
- ](promise_t &npm) {
- gen_on_fulfilled(std::move(on_fulfilled), npm)();
- });
- case State::Rejected:
- return promise_t([this,
- on_rejected = std::forward<FuncRejected>(on_rejected)
- ](promise_t &npm) {
- gen_on_rejected(std::move(on_rejected), npm)();
- });
- default: PROMISE_ERR_INVALID_STATE;
- }
- }
-
- template<typename FuncFulfilled>
- promise_t then(FuncFulfilled &&on_fulfilled) {
- switch (state)
- {
- case State::Pending:
- return promise_t([this,
- on_fulfilled = std::forward<FuncFulfilled>(on_fulfilled)
- ](promise_t &npm) {
- add_on_fulfilled(gen_on_fulfilled(std::move(on_fulfilled), npm));
- add_on_rejected([this, npm]() {npm->_reject(reason);});
-#ifdef CPPROMISE_USE_STACK_FREE
- _dep_resolve(npm);
- _dep_reject(npm);
-#endif
- });
- case State::Fulfilled:
- return promise_t([this,
- on_fulfilled = std::forward<FuncFulfilled>(on_fulfilled)
- ](promise_t &npm) {
- gen_on_fulfilled(std::move(on_fulfilled), npm)();
- });
- case State::Rejected:
- return promise_t([this](promise_t &npm) {npm->_reject(reason);});
- default: PROMISE_ERR_INVALID_STATE;
- }
- }
-
- template<typename FuncRejected>
- promise_t fail(FuncRejected &&on_rejected) {
- switch (state)
- {
- case State::Pending:
- return promise_t([this,
- on_rejected = std::forward<FuncRejected>(on_rejected)
- ](promise_t &npm) {
- callback_t ret;
- add_on_rejected(gen_on_rejected(std::move(on_rejected), npm));
- add_on_fulfilled([this, npm]() {npm->_resolve(result);});
-#ifdef CPPROMISE_USE_STACK_FREE
- _dep_resolve(npm);
- _dep_reject(npm);
-#endif
- });
- case State::Fulfilled:
- return promise_t([this](promise_t &npm) {npm->_resolve(result);});
- case State::Rejected:
- return promise_t([this,
- on_rejected = std::forward<FuncRejected>(on_rejected)
- ](promise_t &npm) {
- gen_on_rejected(std::move(on_rejected), npm)();
- });
- default: PROMISE_ERR_INVALID_STATE;
- }
- }
-
- void resolve() {
- if (state == State::Pending) trigger_fulfill();
- }
-
- void reject() {
- if (state == State::Pending) trigger_reject();
- }
-
- void resolve(pm_any_t _result) {
- if (state == State::Pending)
- {
- result = _result;
- trigger_fulfill();
- }
- }
-
- void reject(pm_any_t _reason) {
- if (state == State::Pending)
- {
- reason = _reason;
- trigger_reject();
- }
- }
- };
-
- template<typename PList> promise_t all(const PList &promise_list) {
- return promise_t([&promise_list] (promise_t &npm) {
- auto size = std::make_shared<size_t>(promise_list.size());
- auto results = std::make_shared<values_t>();
- if (!*size) PROMISE_ERR_MISMATCH_TYPE;
- results->resize(*size);
- size_t idx = 0;
- for (const auto &pm: promise_list) {
- pm->then(
- [results, size, idx, npm](pm_any_t result) {
- (*results)[idx] = result;
- if (!--(*size))
- npm->_resolve(*results);
- },
- [npm](pm_any_t reason) {npm->_reject(reason);});
-#ifdef CPPROMISE_USE_STACK_FREE
- pm->_dep_resolve(npm);
- pm->_dep_reject(npm);
-#endif
- idx++;
- }
- });
- }
-
- template<typename PList> promise_t race(const PList &promise_list) {
- return promise_t([&promise_list] (promise_t &npm) {
- for (const auto &pm: promise_list) {
- pm->then([npm](pm_any_t result) {npm->_resolve(result);},
- [npm](pm_any_t reason) {npm->_reject(reason);});
-#ifdef CPPROMISE_USE_STACK_FREE
- pm->_dep_resolve(npm);
- pm->_dep_reject(npm);
-#endif
- }
- });
- }
-
- template<typename Func>
- inline promise_t::promise_t(Func callback):
- pm(new Promise()),
- ref_cnt(new size_t(1)) {
- callback(*this);
- }
-
- inline promise_t::promise_t():
- pm(new Promise()),
- ref_cnt(new size_t(1)) {}
-
- inline promise_t::~promise_t() {
- if (pm)
- {
- if (--*ref_cnt) return;
- delete pm;
- delete ref_cnt;
- }
- }
-
- template<typename T>
- inline void promise_t::resolve(T result) const { (*this)->resolve(result); }
-
- template<typename T>
- inline void promise_t::reject(T reason) const { (*this)->reject(reason); }
-
- inline void promise_t::resolve() const { (*this)->resolve(); }
- inline void promise_t::reject() const { (*this)->reject(); }
-
- template<typename T>
- struct callback_types {
- using arg_type = typename function_traits<T>::arg_type;
- using ret_type = typename std::conditional<
- std::is_same<typename function_traits<T>::ret_type, promise_t>::value,
- promise_t, pm_any_t>::type;
- };
-
- template<typename Func,
- typename disable_if_arg<Func, pm_any_t>::type * = nullptr,
- typename enable_if_return<Func, void>::type * = nullptr,
- typename function_traits<Func>::non_empty_arg * = nullptr>
- constexpr auto gen_any_callback(Func f) {
- using func_t = callback_types<Func>;
- return [f = std::forward<Func>(f)](pm_any_t v) mutable {
- try {
- f(any_cast<typename func_t::arg_type>(v));
- } catch (bad_any_cast &e) { PROMISE_ERR_MISMATCH_TYPE; }
- };
- }
-
- template<typename Func,
- typename enable_if_arg<Func, pm_any_t>::type * = nullptr,
- typename enable_if_return<Func, void>::type * = nullptr,
- typename function_traits<Func>::non_empty_arg * = nullptr>
- constexpr auto gen_any_callback(Func f) {
- return [f = std::forward<Func>(f)](pm_any_t v) mutable {f(v);};
- }
-
- template<typename Func,
- typename enable_if_return<Func, void>::type * = nullptr,
- typename function_traits<Func>::empty_arg * = nullptr>
- constexpr auto gen_any_callback(Func f) { return std::forward<Func>(f); }
-
- template<typename Func,
- typename enable_if_arg<Func, pm_any_t>::type * = nullptr,
- typename disable_if_return<Func, void>::type * = nullptr,
- typename function_traits<Func>::non_empty_arg * = nullptr>
- constexpr auto gen_any_callback(Func f) {
- using func_t = callback_types<Func>;
- return [f = std::forward<Func>(f)](pm_any_t v) mutable {
- return typename func_t::ret_type(f(v));
- };
- }
-
- template<typename Func,
- typename disable_if_arg<Func, pm_any_t>::type * = nullptr,
- typename disable_if_return<Func, void>::type * = nullptr,
- typename function_traits<Func>::non_empty_arg * = nullptr>
- constexpr auto gen_any_callback(Func f) {
- using func_t = callback_types<Func>;
- return [f = std::forward<Func>(f)](pm_any_t v) mutable {
- try {
- return typename func_t::ret_type(
- f(any_cast<typename func_t::arg_type>(v)));
- } catch (bad_any_cast &e) { PROMISE_ERR_MISMATCH_TYPE; }
- };
- }
-
- template<typename Func,
- typename disable_if_return<Func, void>::type * = nullptr,
- typename function_traits<Func>::empty_arg * = nullptr>
- constexpr auto gen_any_callback(Func f) {
- using func_t = callback_types<Func>;
- return [f = std::forward<Func>(f)]() mutable {
- return typename func_t::ret_type(f());
- };
- }
-
- template<typename FuncFulfilled>
- inline promise_t promise_t::then(FuncFulfilled on_fulfilled) const {
- return (*this)->then(gen_any_callback(std::forward<FuncFulfilled>(on_fulfilled)));
- }
-
- template<typename FuncFulfilled, typename FuncRejected>
- inline promise_t promise_t::then(FuncFulfilled on_fulfilled,
- FuncRejected on_rejected) const {
- return (*this)->then(gen_any_callback(std::forward<FuncFulfilled>(on_fulfilled)),
- gen_any_callback(std::forward<FuncRejected>(on_rejected)));
- }
-
- template<typename FuncRejected>
- inline promise_t promise_t::fail(FuncRejected on_rejected) const {
- return (*this)->fail(gen_any_callback(std::forward<FuncRejected>(on_rejected)));
- }
-}
-
-#endif
diff --git a/src/type.h b/src/type.h
deleted file mode 100644
index 4665979..0000000
--- a/src/type.h
+++ /dev/null
@@ -1,46 +0,0 @@
-#ifndef _HOTSTUFF_TYPE_H
-#define _HOTSTUFF_TYPE_H
-
-#include "salticidae/stream.h"
-#include "salticidae/type.h"
-#include "salticidae/util.h"
-
-namespace hotstuff {
-
-using salticidae::uint256_t;
-using salticidae::DataStream;
-using salticidae::htole;
-using salticidae::letoh;
-using salticidae::get_hex;
-using salticidae::from_hex;
-using salticidae::bytearray_t;
-
-inline std::string get_hex10(const uint256_t &x) {
- return get_hex(x).substr(0, 10);
-}
-
-class HotStuffError: public salticidae::SalticidaeError {
- public:
- template<typename... Args>
- HotStuffError(Args... args): salticidae::SalticidaeError(args...) {}
-};
-
-class HotStuffInvalidEntity: public HotStuffError {
- public:
- template<typename... Args>
- HotStuffInvalidEntity(Args... args): HotStuffError(args...) {}
-};
-
-using salticidae::Serializable;
-
-class Cloneable {
- public:
- virtual ~Cloneable() = default;
- virtual Cloneable *clone() = 0;
-};
-
-using ReplicaID = uint16_t;
-
-}
-
-#endif
diff --git a/src/util.cpp b/src/util.cpp
index dc509f1..bed6e5a 100644
--- a/src/util.cpp
+++ b/src/util.cpp
@@ -1,4 +1,4 @@
-#include "util.h"
+#include "hotstuff/util.h"
namespace hotstuff {
diff --git a/src/util.h b/src/util.h
deleted file mode 100644
index 42c0135..0000000
--- a/src/util.h
+++ /dev/null
@@ -1,119 +0,0 @@
-#ifndef _HOTSTUFF_UTIL_H
-#define _HOTSTUFF_UTIL_H
-
-#include "salticidae/util.h"
-
-namespace hotstuff {
-
-class Logger: public salticidae::Logger {
- public:
- using salticidae::Logger::Logger;
-};
-
-extern Logger logger;
-
-#ifdef HOTSTUFF_DEBUG_LOG
-#define HOTSTUFF_NORMAL_LOG
-#define HOTSTUFF_ENABLE_LOG_DEBUG
-#endif
-
-#ifdef HOTSTUFF_NORMAL_LOG
-#define HOTSTUFF_ENABLE_LOG_INFO
-#define HOTSTUFF_ENABLE_LOG_WARN
-#endif
-
-#ifdef HOTSTUFF_ENABLE_LOG_INFO
-#define HOTSTUFF_LOG_INFO(...) hotstuff::logger.info(__VA_ARGS__)
-#else
-#define HOTSTUFF_LOG_INFO(...) ((void)0)
-#endif
-
-#ifdef HOTSTUFF_ENABLE_LOG_DEBUG
-#define HOTSTUFF_LOG_DEBUG(...) hotstuff::logger.debug(__VA_ARGS__)
-#else
-#define HOTSTUFF_LOG_DEBUG(...) ((void)0)
-#endif
-
-#ifdef HOTSTUFF_ENABLE_LOG_WARN
-#define HOTSTUFF_LOG_WARN(...) hotstuff::logger.warning(__VA_ARGS__)
-#else
-#define HOTSTUFF_LOG_WARN(...) ((void)0)
-#endif
-
-#define HOTSTUFF_LOG_ERROR(...) hotstuff::logger.error(__VA_ARGS__)
-
-#ifdef HOTSTUFF_ENABLE_BLK_PROFILE
-class BlockProfiler {
- enum BlockState {
- BLK_SEEN,
- BLK_FETCH,
- BLK_CC
- };
-
- struct BlockProfile {
- bool is_local; /* is the block proposed by the replica itself? */
- BlockState state;
- double hash_seen_time; /* the first time to see block hash */
- double full_fetch_time; /* the first time to get full block content */
- double cc_time; /* the time when it receives cc */
- double commit_time; /* the time when it commits */
- };
-
- std::unordered_map<const uint256, BlockProfile> blocks;
- ElapsedTime timer;
-
- public:
- BlockProfiler() { timer.start(); }
-
- auto rec_blk(const uint256 &blk_hash, bool is_local) {
- auto it = blocks.find(blk_hash);
- assert(it == blocks.end());
- timer.stop(false);
- return blocks.insert(std::make_pair(blk_hash,
- BlockProfile{is_local, BLK_SEEN, timer.elapsed_sec, 0, 0, 0})).first;
- }
-
- void get_blk(const uint256 &blk_hash) {
- auto it = blocks.find(blk_hash);
- if (it == blocks.end())
- it = rec_blk(blk_hash, false);
- BlockProfile &blkp = it->second;
- assert(blkp.state == BLK_SEEN);
- timer.stop(false);
- blkp.full_fetch_time = timer.elapsed_sec;
- blkp.state = BLK_FETCH;
- }
-
- void have_cc(const uint256 &blk_hash) {
- auto it = blocks.find(blk_hash);
- assert(it != blocks.end());
- BlockProfile &blkp = it->second;
- assert(blkp.state == BLK_FETCH);
- timer.stop(false);
- blkp.polling_start_time = timer.elapsed_sec;
- blkp.state = BLK_CC;
- }
-
- const char *decide_blk(const uint256 &blk_hash) {
- static char buff[1024];
- auto it = blocks.find(blk_hash);
- assert(it != blocks.end());
- BlockProfile &blkp = it->second;
- assert(blkp.state == BLK_CC);
- timer.stop(false);
- blkp.commit_time = timer.elapsed_sec;
- snprintf(buff, sizeof buff, "(%d,%.4f,%.4f,%.4f,%.4f,%.4f)",
- blkp.is_local,
- blkp.hash_seen_time, blkp.full_fetch_time,
- blkp.polling_start_time, blkp.polling_end_time,
- blkp.commit_time);
- blocks.erase(it);
- return buff;
- }
-};
-
-#endif
-
-}
-
-#endif