Implement new experimental cassandra backend (#537)

This commit is contained in:
Alex Kremer
2023-03-29 19:38:38 +01:00
committed by GitHub
parent e9937fab76
commit d86104577b
48 changed files with 7996 additions and 51 deletions

View File

@@ -4,7 +4,7 @@ exec 1>&2
# paths to check and re-format
sources="src unittests"
formatter="clang-format -i"
formatter="clang-format-11 -i"
first=$(git diff $sources)
find $sources -type f \( -name '*.cpp' -o -name '*.h' -o -name '*.ipp' \) -print0 | xargs -0 $formatter

View File

@@ -110,7 +110,7 @@ jobs:
- name: Run Test
run: |
cd clio/build
./clio_tests --gtest_filter="-Backend*"
./clio_tests --gtest_filter="-BackendTest*:BackendCassandraBaseTest*:BackendCassandraTest*"
test_clio:
name: Test Clio
@@ -168,7 +168,7 @@ jobs:
run: |
export BOOST_ROOT=$(pwd)/boost
cd clio
cmake -B build -DCODE_COVERAGE=on -DTEST_PARAMETER='--gtest_filter="-Backend*"'
cmake -B build -DCODE_COVERAGE=on -DTEST_PARAMETER='--gtest_filter="-BackendTest*:BackendCassandraBaseTest*:BackendCassandraTest*"'
if ! cmake --build build -j$(nproc); then
echo '# 🔥Ubuntu build🔥 failed!💥' >> $GITHUB_STEP_SUMMARY
exit 1

View File

@@ -40,6 +40,15 @@ target_sources(clio PRIVATE
src/backend/BackendInterface.cpp
src/backend/CassandraBackend.cpp
src/backend/SimpleCache.cpp
## NextGen Backend
src/backend/cassandra/impl/Future.cpp
src/backend/cassandra/impl/Cluster.cpp
src/backend/cassandra/impl/Batch.cpp
src/backend/cassandra/impl/Result.cpp
src/backend/cassandra/impl/Tuple.cpp
src/backend/cassandra/impl/SslContext.cpp
src/backend/cassandra/Handle.cpp
src/backend/cassandra/SettingsProvider.cpp
## ETL
src/etl/ETLSource.cpp
src/etl/ProbingETLSource.cpp
@@ -126,6 +135,7 @@ if(BUILD_TESTS)
unittests/SubscriptionTest.cpp
unittests/SubscriptionManagerTest.cpp
unittests/util/TestObject.cpp
# RPC
unittests/rpc/ErrorTests.cpp
unittests/rpc/BaseTests.cpp
unittests/rpc/RPCHelpersTest.cpp
@@ -146,9 +156,20 @@ if(BUILD_TESTS)
unittests/rpc/handlers/AccountTxTest.cpp
unittests/rpc/handlers/AccountOffersTest.cpp
unittests/rpc/handlers/TransactionEntryTest.cpp
unittests/rpc/handlers/NFTBuyOffersTest.cpp)
unittests/rpc/handlers/NFTBuyOffersTest.cpp
unittests/rpc/handlers/NFTInfoTest.cpp
# Backend
unittests/backend/cassandra/BaseTests.cpp
unittests/backend/cassandra/BackendTests.cpp
unittests/backend/cassandra/RetryPolicyTests.cpp
unittests/backend/cassandra/SettingsProviderTests.cpp
unittests/backend/cassandra/ExecutionStrategyTests.cpp
unittests/backend/cassandra/AsyncExecutorTests.cpp)
include(CMake/deps/gtest.cmake)
# test for dwarf5 bug on ci
target_compile_options(clio PUBLIC -gdwarf-4)
# if CODE_COVERAGE enable, add clio_test-ccov
if(CODE_COVERAGE)
include(CMake/coverage.cmake)

View File

@@ -21,6 +21,7 @@
#include <backend/BackendInterface.h>
#include <backend/CassandraBackend.h>
#include <backend/CassandraBackendNew.h>
#include <config/Config.h>
#include <log/Logger.h>
@@ -43,6 +44,13 @@ make_Backend(boost::asio::io_context& ioc, clio::Config const& config)
auto ttl = config.valueOr<uint32_t>("online_delete", 0) * 4;
backend = std::make_shared<CassandraBackend>(ioc, cfg, ttl);
}
else if (boost::iequals(type, "cassandra-new"))
{
auto cfg = config.section("database." + type);
auto ttl = config.valueOr<uint16_t>("online_delete", 0) * 4;
backend = std::make_shared<Backend::Cassandra::CassandraBackend>(
Backend::Cassandra::SettingsProvider{cfg, ttl});
}
if (!backend)
throw std::runtime_error("Invalid database type");

View File

@@ -33,9 +33,12 @@ namespace Backend {
bool
BackendInterface::finishWrites(std::uint32_t const ledgerSequence)
{
gLog.debug() << "Want finish writes for " << ledgerSequence;
auto commitRes = doFinishWrites();
if (commitRes)
{
gLog.debug() << "Successfully commited. Updating range now to "
<< ledgerSequence;
updateRange(ledgerSequence);
}
return commitRes;

View File

@@ -182,12 +182,8 @@ protected:
*/
public:
BackendInterface(clio::Config const& config)
{
}
virtual ~BackendInterface()
{
}
BackendInterface() = default;
virtual ~BackendInterface() = default;
/*! @brief LEDGER METHODS */
public:

View File

@@ -708,7 +708,7 @@ public:
boost::asio::io_context& ioc,
clio::Config const& config,
uint32_t ttl)
: BackendInterface(config), config_(config), ttl_(ttl)
: config_(config), ttl_(ttl)
{
work_.emplace(ioContext_);
ioThread_ = std::thread([this]() { ioContext_.run(); });
@@ -935,27 +935,6 @@ public:
std::uint32_t const sequence,
boost::asio::yield_context& yield) const override;
std::optional<int64_t>
getToken(void const* key, boost::asio::yield_context& yield) const
{
log_.trace() << "Fetching from cassandra";
CassandraStatement statement{getToken_};
statement.bindNextBytes(key, 32);
CassandraResult result = executeAsyncRead(statement, yield);
if (!result)
{
log_.error() << "No rows";
return {};
}
int64_t token = result.getInt64();
if (token == INT64_MAX)
return {};
else
return token + 1;
}
std::optional<TransactionAndMetadata>
fetchTransaction(
ripple::uint256 const& hash,

View File

@@ -0,0 +1,979 @@
//------------------------------------------------------------------------------
/*
This file is part of clio: https://github.com/XRPLF/clio
Copyright (c) 2023, the clio developers.
Permission to use, copy, modify, and distribute this software for any
purpose with or without fee is hereby granted, provided that the above
copyright notice and this permission notice appear in all copies.
THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES
WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF
MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR
ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES
WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN
ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF
OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.
*/
//==============================================================================
#pragma once
#include <backend/BackendInterface.h>
#include <backend/cassandra/Concepts.h>
#include <backend/cassandra/Handle.h>
#include <backend/cassandra/Schema.h>
#include <backend/cassandra/SettingsProvider.h>
#include <backend/cassandra/impl/ExecutionStrategy.h>
#include <log/Logger.h>
#include <util/Profiler.h>
#include <ripple/app/tx/impl/details/NFTokenUtils.h>
#include <boost/asio/spawn.hpp>
namespace Backend::Cassandra {
/**
* @brief Implements @ref BackendInterface for Cassandra/Scylladb
*
* Note: this is a safer and more correct rewrite of the original implementation
* of the backend. We deliberately did not change the interface for now so that
* other parts such as ETL do not have to change at all.
* Eventually we should change the interface so that it does not have to know
* about yield_context.
*/
template <
SomeSettingsProvider SettingsProviderType,
SomeExecutionStrategy ExecutionStrategy>
class BasicCassandraBackend : public BackendInterface
{
clio::Logger log_{"Backend"};
SettingsProviderType settingsProvider_;
Schema<SettingsProviderType> schema_;
Handle handle_;
// have to be mutable because BackendInterface constness :(
mutable ExecutionStrategy executor_;
std::atomic_uint32_t ledgerSequence_ = 0u;
public:
/**
* @brief Create a new cassandra/scylla backend instance.
*
* @param settingsProvider
*/
BasicCassandraBackend(SettingsProviderType settingsProvider)
: settingsProvider_{std::move(settingsProvider)}
, schema_{settingsProvider_}
, handle_{settingsProvider_.getSettings()}
, executor_{settingsProvider_.getSettings(), handle_}
{
if (auto const res = handle_.connect(); not res)
throw std::runtime_error(
"Could not connect to Cassandra: " + res.error());
if (auto const res = handle_.execute(schema_.createKeyspace); not res)
throw std::runtime_error(
"Could not create keyspace: " + res.error());
if (auto const res = handle_.executeEach(schema_.createSchema); not res)
throw std::runtime_error("Could not create schema: " + res.error());
schema_.prepareStatements(handle_);
log_.info() << "Created (revamped) CassandraBackend";
}
/*! Not used in this implementation */
void
open([[maybe_unused]] bool readOnly) override
{
}
/*! Not used in this implementation */
void
close() override
{
}
TransactionsAndCursor
fetchAccountTransactions(
ripple::AccountID const& account,
std::uint32_t const limit,
bool forward,
std::optional<TransactionsCursor> const& cursorIn,
boost::asio::yield_context& yield) const override
{
auto rng = fetchLedgerRange();
if (!rng)
return {{}, {}};
Statement statement = [this, forward, &account]() {
if (forward)
return schema_->selectAccountTxForward.bind(account);
else
return schema_->selectAccountTx.bind(account);
}();
auto cursor = cursorIn;
if (cursor)
{
statement.bindAt(1, cursor->asTuple());
log_.debug() << "account = " << ripple::strHex(account)
<< " tuple = " << cursor->ledgerSequence
<< cursor->transactionIndex;
}
else
{
auto const seq = forward ? rng->minSequence : rng->maxSequence;
auto const placeHolder =
forward ? 0u : std::numeric_limits<std::uint32_t>::max();
statement.bindAt(1, std::make_tuple(placeHolder, placeHolder));
log_.debug() << "account = " << ripple::strHex(account)
<< " idx = " << seq << " tuple = " << placeHolder;
}
// FIXME: Limit is a hack to support uint32_t properly for the time
// being. Should be removed later and schema updated to use proper
// types.
statement.bindAt(2, Limit{limit});
auto const res = executor_.read(yield, statement);
auto const& results = res.value();
if (not results.hasRows())
{
log_.debug() << "No rows returned";
return {};
}
std::vector<ripple::uint256> hashes = {};
auto numRows = results.numRows();
log_.info() << "num_rows = " << numRows;
for (auto [hash, data] :
extract<ripple::uint256, std::tuple<uint32_t, uint32_t>>(results))
{
hashes.push_back(hash);
if (--numRows == 0)
{
log_.debug() << "Setting cursor";
cursor = data;
// forward queries by ledger/tx sequence `>=`
// so we have to advance the index by one
if (forward)
++cursor->transactionIndex;
}
}
auto const txns = fetchTransactions(hashes, yield);
log_.debug() << "Txns = " << txns.size();
if (txns.size() == limit)
{
log_.debug() << "Returning cursor";
return {txns, cursor};
}
return {txns, {}};
}
bool
doFinishWrites() override
{
// wait for other threads to finish their writes
executor_.sync();
if (!range)
{
executor_.writeSync(
schema_->updateLedgerRange,
ledgerSequence_,
false,
ledgerSequence_);
}
if (not executeSyncUpdate(schema_->updateLedgerRange.bind(
ledgerSequence_, true, ledgerSequence_ - 1)))
{
log_.warn() << "Update failed for ledger " << ledgerSequence_;
return false;
}
log_.info() << "Committed ledger " << ledgerSequence_;
return true;
}
void
writeLedger(ripple::LedgerInfo const& ledgerInfo, std::string&& header)
override
{
executor_.write(
schema_->insertLedgerHeader, ledgerInfo.seq, std::move(header));
executor_.write(
schema_->insertLedgerHash, ledgerInfo.hash, ledgerInfo.seq);
ledgerSequence_ = ledgerInfo.seq;
}
std::optional<std::uint32_t>
fetchLatestLedgerSequence(boost::asio::yield_context& yield) const override
{
if (auto const res = executor_.read(yield, schema_->selectLatestLedger);
res)
{
if (auto const& result = res.value(); result)
{
if (auto const maybeValue = result.template get<uint32_t>();
maybeValue)
return maybeValue;
log_.error() << "Could not fetch latest ledger - no rows";
return std::nullopt;
}
log_.error() << "Could not fetch latest ledger - no result";
}
else
{
log_.error() << "Could not fetch latest ledger: " << res.error();
}
return std::nullopt;
}
std::optional<ripple::LedgerInfo>
fetchLedgerBySequence(
std::uint32_t const sequence,
boost::asio::yield_context& yield) const override
{
log_.trace() << __func__ << " call for seq " << sequence;
auto const res =
executor_.read(yield, schema_->selectLedgerBySeq, sequence);
if (res)
{
if (auto const& result = res.value(); result)
{
if (auto const maybeValue =
result.template get<std::vector<unsigned char>>();
maybeValue)
{
return deserializeHeader(ripple::makeSlice(*maybeValue));
}
log_.error() << "Could not fetch ledger by sequence - no rows";
return std::nullopt;
}
log_.error() << "Could not fetch ledger by sequence - no result";
}
else
{
log_.error() << "Could not fetch ledger by sequence: "
<< res.error();
}
return std::nullopt;
}
std::optional<ripple::LedgerInfo>
fetchLedgerByHash(
ripple::uint256 const& hash,
boost::asio::yield_context& yield) const override
{
log_.trace() << __func__ << " call";
if (auto const res =
executor_.read(yield, schema_->selectLedgerByHash, hash);
res)
{
if (auto const& result = res.value(); result)
{
if (auto const maybeValue = result.template get<uint32_t>();
maybeValue)
return fetchLedgerBySequence(*maybeValue, yield);
log_.error() << "Could not fetch ledger by hash - no rows";
return std::nullopt;
}
log_.error() << "Could not fetch ledger by hash - no result";
}
else
{
log_.error() << "Could not fetch ledger by hash: " << res.error();
}
return std::nullopt;
}
std::optional<LedgerRange>
hardFetchLedgerRange(boost::asio::yield_context& yield) const override
{
log_.trace() << __func__ << " call";
if (auto const res = executor_.read(yield, schema_->selectLedgerRange);
res)
{
auto const& results = res.value();
if (not results.hasRows())
{
log_.debug() << "Could not fetch ledger range - no rows";
return std::nullopt;
}
// TODO: this is probably a good place to use user type in
// cassandra instead of having two rows with bool flag. or maybe at
// least use tuple<int, int>?
LedgerRange range;
std::size_t idx = 0;
for (auto [seq] : extract<uint32_t>(results))
{
if (idx == 0)
range.maxSequence = range.minSequence = seq;
else if (idx == 1)
range.maxSequence = seq;
++idx;
}
if (range.minSequence > range.maxSequence)
std::swap(range.minSequence, range.maxSequence);
log_.debug() << "After hardFetchLedgerRange range is "
<< range.minSequence << ":" << range.maxSequence;
return range;
}
else
{
log_.error() << "Could not fetch ledger range: " << res.error();
}
return std::nullopt;
}
std::vector<TransactionAndMetadata>
fetchAllTransactionsInLedger(
std::uint32_t const ledgerSequence,
boost::asio::yield_context& yield) const override
{
log_.trace() << __func__ << " call";
auto hashes = fetchAllTransactionHashesInLedger(ledgerSequence, yield);
return fetchTransactions(hashes, yield);
}
std::vector<ripple::uint256>
fetchAllTransactionHashesInLedger(
std::uint32_t const ledgerSequence,
boost::asio::yield_context& yield) const override
{
log_.trace() << __func__ << " call";
auto start = std::chrono::system_clock::now();
auto const res = executor_.read(
yield, schema_->selectAllTransactionHashesInLedger, ledgerSequence);
if (not res)
{
log_.error() << "Could not fetch all transaction hashes: "
<< res.error();
return {};
}
auto const& result = res.value();
if (not result.hasRows())
{
log_.error()
<< "Could not fetch all transaction hashes - no rows; ledger = "
<< std::to_string(ledgerSequence);
return {};
}
std::vector<ripple::uint256> hashes;
for (auto [hash] : extract<ripple::uint256>(result))
hashes.push_back(std::move(hash));
auto end = std::chrono::system_clock::now();
log_.debug() << "Fetched " << hashes.size()
<< " transaction hashes from Cassandra in "
<< std::chrono::duration_cast<std::chrono::milliseconds>(
end - start)
.count()
<< " milliseconds";
return hashes;
}
std::optional<NFT>
fetchNFT(
ripple::uint256 const& tokenID,
std::uint32_t const ledgerSequence,
boost::asio::yield_context& yield) const override
{
log_.trace() << __func__ << " call";
auto const res =
executor_.read(yield, schema_->selectNFT, tokenID, ledgerSequence);
if (not res)
return std::nullopt;
if (auto const maybeRow =
res->template get<uint32_t, ripple::AccountID, bool>();
maybeRow)
{
auto [seq, owner, isBurned] = *maybeRow;
auto result =
std::make_optional<NFT>(tokenID, seq, owner, isBurned);
// now fetch URI. Usually we will have the URI even for burned NFTs,
// but if the first ledger on this clio included NFTokenBurn
// transactions we will not have the URIs for any of those tokens.
// In any other case not having the URI indicates something went
// wrong with our data.
//
// TODO - in the future would be great for any handlers that use
// this could inject a warning in this case (the case of not having
// a URI because it was burned in the first ledger) to indicate that
// even though we are returning a blank URI, the NFT might have had
// one.
auto uriRes = executor_.read(
yield, schema_->selectNFTURI, tokenID, ledgerSequence);
if (uriRes)
{
if (auto const maybeUri = uriRes->template get<ripple::Blob>();
maybeUri)
result->uri = *maybeUri;
}
return result;
}
log_.error() << "Could not fetch NFT - no rows";
return std::nullopt;
}
TransactionsAndCursor
fetchNFTTransactions(
ripple::uint256 const& tokenID,
std::uint32_t const limit,
bool const forward,
std::optional<TransactionsCursor> const& cursorIn,
boost::asio::yield_context& yield) const override
{
log_.trace() << __func__ << " call";
auto rng = fetchLedgerRange();
if (!rng)
return {{}, {}};
Statement statement = [this, forward, &tokenID]() {
if (forward)
return schema_->selectNFTTxForward.bind(tokenID);
else
return schema_->selectNFTTx.bind(tokenID);
}();
auto cursor = cursorIn;
if (cursor)
{
statement.bindAt(1, cursor->asTuple());
log_.debug() << "token_id = " << ripple::strHex(tokenID)
<< " tuple = " << cursor->ledgerSequence
<< cursor->transactionIndex;
}
else
{
auto const seq = forward ? rng->minSequence : rng->maxSequence;
auto const placeHolder =
forward ? 0 : std::numeric_limits<std::uint32_t>::max();
statement.bindAt(1, std::make_tuple(placeHolder, placeHolder));
log_.debug() << "token_id = " << ripple::strHex(tokenID)
<< " idx = " << seq << " tuple = " << placeHolder;
}
statement.bindAt(2, Limit{limit});
auto const res = executor_.read(yield, statement);
auto const& results = res.value();
if (not results.hasRows())
{
log_.debug() << "No rows returned";
return {};
}
std::vector<ripple::uint256> hashes = {};
auto numRows = results.numRows();
log_.info() << "num_rows = " << numRows;
for (auto [hash, data] :
extract<ripple::uint256, std::tuple<uint32_t, uint32_t>>(results))
{
hashes.push_back(hash);
if (--numRows == 0)
{
log_.debug() << "Setting cursor";
cursor = data;
// forward queries by ledger/tx sequence `>=`
// so we have to advance the index by one
if (forward)
++cursor->transactionIndex;
}
}
auto const txns = fetchTransactions(hashes, yield);
log_.debug() << "NFT Txns = " << txns.size();
if (txns.size() == limit)
{
log_.debug() << "Returning cursor";
return {txns, cursor};
}
return {txns, {}};
}
std::optional<Blob>
doFetchLedgerObject(
ripple::uint256 const& key,
std::uint32_t const sequence,
boost::asio::yield_context& yield) const override
{
log_.debug() << "Fetching ledger object for seq " << sequence
<< ", key = " << ripple::to_string(key);
if (auto const res =
executor_.read(yield, schema_->selectObject, key, sequence);
res)
{
if (auto const result = res->template get<Blob>(); result)
{
if (result->size())
return *result;
}
else
{
log_.debug() << "Could not fetch ledger object - no rows";
}
}
else
{
log_.error() << "Could not fetch ledger object: " << res.error();
}
return std::nullopt;
}
std::optional<TransactionAndMetadata>
fetchTransaction(
ripple::uint256 const& hash,
boost::asio::yield_context& yield) const override
{
log_.trace() << __func__ << " call";
if (auto const res =
executor_.read(yield, schema_->selectTransaction, hash);
res)
{
if (auto const maybeValue =
res->template get<Blob, Blob, uint32_t, uint32_t>();
maybeValue)
{
auto [transaction, meta, seq, date] = *maybeValue;
return std::make_optional<TransactionAndMetadata>(
transaction, meta, seq, date);
}
else
{
log_.debug() << "Could not fetch transaction - no rows";
}
}
else
{
log_.error() << "Could not fetch transaction: " << res.error();
}
return std::nullopt;
}
std::optional<ripple::uint256>
doFetchSuccessorKey(
ripple::uint256 key,
std::uint32_t const ledgerSequence,
boost::asio::yield_context& yield) const override
{
log_.trace() << __func__ << " call";
if (auto const res = executor_.read(
yield, schema_->selectSuccessor, key, ledgerSequence);
res)
{
if (auto const result = res->template get<ripple::uint256>();
result)
{
if (*result == lastKey)
return std::nullopt;
return *result;
}
else
{
log_.debug() << "Could not fetch successor - no rows";
}
}
else
{
log_.error() << "Could not fetch successor: " << res.error();
}
return std::nullopt;
}
std::vector<TransactionAndMetadata>
fetchTransactions(
std::vector<ripple::uint256> const& hashes,
boost::asio::yield_context& yield) const override
{
log_.trace() << __func__ << " call";
if (hashes.size() == 0)
return {};
auto const numHashes = hashes.size();
std::vector<TransactionAndMetadata> results;
results.reserve(numHashes);
std::vector<Statement> statements;
statements.reserve(numHashes);
auto const timeDiff = util::timed([this,
&yield,
&results,
&hashes,
&statements]() {
// TODO: seems like a job for "hash IN (list of hashes)" instead?
std::transform(
std::cbegin(hashes),
std::cend(hashes),
std::back_inserter(statements),
[this](auto const& hash) {
return schema_->selectTransaction.bind(hash);
});
auto const entries = executor_.readEach(yield, statements);
std::transform(
std::cbegin(entries),
std::cend(entries),
std::back_inserter(results),
[](auto const& res) -> TransactionAndMetadata {
if (auto const maybeRow =
res.template get<Blob, Blob, uint32_t, uint32_t>();
maybeRow)
return *maybeRow;
else
return {};
});
});
assert(numHashes == results.size());
log_.debug() << "Fetched " << numHashes
<< " transactions from Cassandra in " << timeDiff
<< " milliseconds";
return results;
}
std::vector<Blob>
doFetchLedgerObjects(
std::vector<ripple::uint256> const& keys,
std::uint32_t const sequence,
boost::asio::yield_context& yield) const override
{
log_.trace() << __func__ << " call";
if (keys.size() == 0)
return {};
auto const numKeys = keys.size();
log_.trace() << "Fetching " << numKeys << " objects";
std::vector<Blob> results;
results.reserve(numKeys);
std::vector<Statement> statements;
statements.reserve(numKeys);
// TODO: seems like a job for "key IN (list of keys)" instead?
std::transform(
std::cbegin(keys),
std::cend(keys),
std::back_inserter(statements),
[this, &sequence](auto const& key) {
return schema_->selectObject.bind(key, sequence);
});
auto const entries = executor_.readEach(yield, statements);
std::transform(
std::cbegin(entries),
std::cend(entries),
std::back_inserter(results),
[](auto const& res) -> Blob {
if (auto const maybeValue = res.template get<Blob>();
maybeValue)
return *maybeValue;
else
return {};
});
log_.trace() << "Fetched " << numKeys << " objects";
return results;
}
std::vector<LedgerObject>
fetchLedgerDiff(
std::uint32_t const ledgerSequence,
boost::asio::yield_context& yield) const override
{
log_.trace() << __func__ << " call";
auto const [keys, timeDiff] = util::timed(
[this, &ledgerSequence, &yield]() -> std::vector<ripple::uint256> {
auto const res =
executor_.read(yield, schema_->selectDiff, ledgerSequence);
if (not res)
{
log_.error()
<< "Could not fetch ledger diff: " << res.error()
<< "; ledger = " << ledgerSequence;
return {};
}
auto const& results = res.value();
if (not results)
{
log_.error()
<< "Could not fetch ledger diff - no rows; ledger = "
<< ledgerSequence;
return {};
}
std::vector<ripple::uint256> keys;
for (auto [key] : extract<ripple::uint256>(results))
keys.push_back(key);
return keys;
});
// one of the above errors must have happened
if (keys.empty())
return {};
log_.debug() << "Fetched " << keys.size()
<< " diff hashes from Cassandra in " << timeDiff
<< " milliseconds";
auto const objs = fetchLedgerObjects(keys, ledgerSequence, yield);
std::vector<LedgerObject> results;
results.reserve(keys.size());
std::transform(
std::cbegin(keys),
std::cend(keys),
std::cbegin(objs),
std::back_inserter(results),
[](auto const& key, auto const& obj) {
return LedgerObject{key, obj};
});
return results;
}
void
doWriteLedgerObject(
std::string&& key,
std::uint32_t const seq,
std::string&& blob) override
{
log_.trace() << " Writing ledger object " << key.size() << ":" << seq
<< " [" << blob.size() << " bytes]";
if (range)
executor_.write(schema_->insertDiff, seq, key);
executor_.write(
schema_->insertObject, std::move(key), seq, std::move(blob));
}
void
writeSuccessor(
std::string&& key,
std::uint32_t const seq,
std::string&& successor) override
{
log_.trace() << "Writing successor. key = " << key.size() << " bytes. "
<< " seq = " << std::to_string(seq)
<< " successor = " << successor.size() << " bytes.";
assert(key.size() != 0);
assert(successor.size() != 0);
executor_.write(
schema_->insertSuccessor,
std::move(key),
seq,
std::move(successor));
}
void
writeAccountTransactions(
std::vector<AccountTransactionsData>&& data) override
{
std::vector<Statement> statements;
statements.reserve(data.size() * 10); // assume 10 transactions avg
for (auto& record : data)
{
std::transform(
std::begin(record.accounts),
std::end(record.accounts),
std::back_inserter(statements),
[this, &record](auto&& account) {
return schema_->insertAccountTx.bind(
std::move(account),
std::make_tuple(
record.ledgerSequence, record.transactionIndex),
record.txHash);
});
}
executor_.write(statements);
}
void
writeNFTTransactions(std::vector<NFTTransactionsData>&& data) override
{
std::vector<Statement> statements;
statements.reserve(data.size());
std::transform(
std::cbegin(data),
std::cend(data),
std::back_inserter(statements),
[this](auto const& record) {
return schema_->insertNFTTx.bind(
record.tokenID,
std::make_tuple(
record.ledgerSequence, record.transactionIndex),
record.txHash);
});
executor_.write(statements);
}
void
writeTransaction(
std::string&& hash,
std::uint32_t const seq,
std::uint32_t const date,
std::string&& transaction,
std::string&& metadata) override
{
log_.trace() << "Writing txn to cassandra";
executor_.write(schema_->insertLedgerTransaction, seq, hash);
executor_.write(
schema_->insertTransaction,
std::move(hash),
seq,
date,
std::move(transaction),
std::move(metadata));
}
void
writeNFTs(std::vector<NFTsData>&& data) override
{
std::vector<Statement> statements;
statements.reserve(data.size() * 3);
for (NFTsData const& record : data)
{
statements.push_back(schema_->insertNFT.bind(
record.tokenID,
record.ledgerSequence,
record.owner,
record.isBurned));
// If `uri` is set (and it can be set to an empty uri), we know this
// is a net-new NFT. That is, this NFT has not been seen before by
// us _OR_ it is in the extreme edge case of a re-minted NFT ID with
// the same NFT ID as an already-burned token. In this case, we need
// to record the URI and link to the issuer_nf_tokens table.
if (record.uri)
{
statements.push_back(schema_->insertIssuerNFT.bind(
ripple::nft::getIssuer(record.tokenID),
static_cast<uint32_t>(
ripple::nft::getTaxon(record.tokenID)),
record.tokenID));
statements.push_back(schema_->insertNFTURI.bind(
record.tokenID, record.ledgerSequence, record.uri.value()));
}
}
executor_.write(statements);
}
void
startWrites() const override
{
// Note: no-op in original implementation too.
// probably was used in PG to start a transaction or smth.
}
/*! Unused in this implementation */
bool
doOnlineDelete(
std::uint32_t const numLedgersToKeep,
boost::asio::yield_context& yield) const override
{
log_.trace() << __func__ << " call";
return true;
}
bool
isTooBusy() const override
{
return executor_.isTooBusy();
}
private:
bool
executeSyncUpdate(Statement statement)
{
auto const res = executor_.writeSync(statement);
auto maybeSuccess = res->template get<bool>();
if (not maybeSuccess)
{
log_.error() << "executeSyncUpdate - error getting result - no row";
return false;
}
if (not maybeSuccess.value())
{
log_.warn()
<< "Update failed. Checking if DB state is what we expect";
// error may indicate that another writer wrote something.
// in this case let's just compare the current state of things
// against what we were trying to write in the first place and
// use that as the source of truth for the result.
auto rng = hardFetchLedgerRangeNoThrow();
return rng && rng->maxSequence == ledgerSequence_;
}
return true;
}
};
using CassandraBackend =
BasicCassandraBackend<SettingsProvider, detail::DefaultExecutionStrategy<>>;
} // namespace Backend::Cassandra

View File

@@ -58,6 +58,29 @@ struct TransactionAndMetadata
Blob metadata;
std::uint32_t ledgerSequence;
std::uint32_t date;
TransactionAndMetadata() = default;
TransactionAndMetadata(
Blob const& transaction,
Blob const& metadata,
std::uint32_t ledgerSequence,
std::uint32_t date)
: transaction{transaction}
, metadata{metadata}
, ledgerSequence{ledgerSequence}
, date{date}
{
}
TransactionAndMetadata(
std::tuple<Blob, Blob, std::uint32_t, std::uint32_t> data)
: transaction{std::get<0>(data)}
, metadata{std::get<1>(data)}
, ledgerSequence{std::get<2>(data)}
, date{std::get<3>(data)}
{
}
bool
operator==(const TransactionAndMetadata& other) const
{
@@ -70,8 +93,31 @@ struct TransactionsCursor
{
std::uint32_t ledgerSequence;
std::uint32_t transactionIndex;
TransactionsCursor() = default;
TransactionsCursor(
std::uint32_t ledgerSequence,
std::uint32_t transactionIndex)
: ledgerSequence{ledgerSequence}, transactionIndex{transactionIndex}
{
}
TransactionsCursor(std::tuple<std::uint32_t, std::uint32_t> data)
: ledgerSequence{std::get<0>(data)}, transactionIndex{std::get<1>(data)}
{
}
TransactionsCursor&
operator=(TransactionsCursor const&) = default;
bool
operator==(TransactionsCursor const& other) const = default;
[[nodiscard]] std::tuple<std::uint32_t, std::uint32_t>
asTuple() const
{
return std::make_tuple(ledgerSequence, transactionIndex);
}
};
struct TransactionsAndCursor
@@ -88,6 +134,28 @@ struct NFT
Blob uri;
bool isBurned;
NFT() = default;
NFT(ripple::uint256 const& tokenID,
std::uint32_t ledgerSequence,
ripple::AccountID const& owner,
Blob const& uri,
bool isBurned)
: tokenID{tokenID}
, ledgerSequence{ledgerSequence}
, owner{owner}
, uri{uri}
, isBurned{isBurned}
{
}
NFT(ripple::uint256 const& tokenID,
std::uint32_t ledgerSequence,
ripple::AccountID const& owner,
bool isBurned)
: NFT(tokenID, ledgerSequence, owner, {}, isBurned)
{
}
// clearly two tokens are the same if they have the same ID, but this
// struct stores the state of a given token at a given ledger sequence, so
// we also need to compare with ledgerSequence

View File

@@ -0,0 +1,79 @@
//------------------------------------------------------------------------------
/*
This file is part of clio: https://github.com/XRPLF/clio
Copyright (c) 2023, the clio developers.
Permission to use, copy, modify, and distribute this software for any
purpose with or without fee is hereby granted, provided that the above
copyright notice and this permission notice appear in all copies.
THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES
WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF
MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR
ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES
WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN
ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF
OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.
*/
//==============================================================================
#pragma once
#include <backend/cassandra/Types.h>
#include <boost/asio/spawn.hpp>
#include <chrono>
#include <concepts>
#include <optional>
#include <string>
namespace Backend::Cassandra {
// clang-format off
template <typename T>
concept SomeSettingsProvider = requires(T a) {
{ a.getSettings() } -> std::same_as<Settings>;
{ a.getKeyspace() } -> std::same_as<std::string>;
{ a.getTablePrefix() } -> std::same_as<std::optional<std::string>>;
{ a.getReplicationFactor() } -> std::same_as<uint16_t>;
{ a.getTtl() } -> std::same_as<uint16_t>;
};
// clang-format on
// clang-format off
template <typename T>
concept SomeExecutionStrategy = requires(
T a,
Settings settings,
Handle handle,
Statement statement,
std::vector<Statement> statements,
PreparedStatement prepared,
boost::asio::yield_context token
) {
{ T(settings, handle) };
{ a.sync() } -> std::same_as<void>;
{ a.isTooBusy() } -> std::same_as<bool>;
{ a.writeSync(statement) } -> std::same_as<ResultOrError>;
{ a.writeSync(prepared) } -> std::same_as<ResultOrError>;
{ a.write(prepared) } -> std::same_as<void>;
{ a.write(statements) } -> std::same_as<void>;
{ a.read(token, prepared) } -> std::same_as<ResultOrError>;
{ a.read(token, statement) } -> std::same_as<ResultOrError>;
{ a.read(token, statements) } -> std::same_as<ResultOrError>;
{ a.readEach(token, statements) } -> std::same_as<std::vector<Result>>;
};
// clang-format on
// clang-format off
template <typename T>
concept SomeRetryPolicy = requires(T a, boost::asio::io_context ioc, CassandraError err, uint32_t attempt) {
{ T(ioc) };
{ a.shouldRetry(err) } -> std::same_as<bool>;
{ a.retry([](){}) } -> std::same_as<void>;
{ a.calculateDelay(attempt) } -> std::same_as<std::chrono::milliseconds>;
};
// clang-format on
} // namespace Backend::Cassandra

View File

@@ -0,0 +1,108 @@
//------------------------------------------------------------------------------
/*
This file is part of clio: https://github.com/XRPLF/clio
Copyright (c) 2023, the clio developers.
Permission to use, copy, modify, and distribute this software for any
purpose with or without fee is hereby granted, provided that the above
copyright notice and this permission notice appear in all copies.
THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES
WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF
MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR
ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES
WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN
ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF
OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.
*/
//==============================================================================
#pragma once
#include <cassandra.h>
#include <string>
namespace Backend::Cassandra {
/**
* @brief A simple container for both error message and error code
*/
class CassandraError
{
std::string message_;
uint32_t code_;
public:
CassandraError() = default; // default constructible required by Expected
CassandraError(std::string message, uint32_t code)
: message_{message}, code_{code}
{
}
template <typename T>
friend std::string
operator+(
T const& lhs,
CassandraError const&
rhs) requires std::is_convertible_v<T, std::string>
{
return lhs + rhs.message();
}
template <typename T>
friend bool
operator==(
T const& lhs,
CassandraError const&
rhs) requires std::is_convertible_v<T, std::string>
{
return lhs == rhs.message();
}
template <std::integral T>
friend bool
operator==(T const& lhs, CassandraError const& rhs)
{
return lhs == rhs.code();
}
friend std::ostream&
operator<<(std::ostream& os, CassandraError const& err)
{
os << err.message();
return os;
}
std::string
message() const
{
return message_;
}
uint32_t
code() const
{
return code_;
}
bool
isTimeout() const
{
if (code_ == CASS_ERROR_LIB_NO_HOSTS_AVAILABLE or
code_ == CASS_ERROR_LIB_REQUEST_TIMED_OUT or
code_ == CASS_ERROR_SERVER_UNAVAILABLE or
code_ == CASS_ERROR_SERVER_OVERLOADED or
code_ == CASS_ERROR_SERVER_READ_TIMEOUT)
return true;
return false;
}
bool
isInvalidQuery() const
{
return code_ == CASS_ERROR_SERVER_INVALID_QUERY;
}
};
} // namespace Backend::Cassandra

View File

@@ -0,0 +1,164 @@
//------------------------------------------------------------------------------
/*
This file is part of clio: https://github.com/XRPLF/clio
Copyright (c) 2023, the clio developers.
Permission to use, copy, modify, and distribute this software for any
purpose with or without fee is hereby granted, provided that the above
copyright notice and this permission notice appear in all copies.
THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES
WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF
MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR
ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES
WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN
ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF
OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.
*/
//==============================================================================
#include <backend/cassandra/Handle.h>
namespace Backend::Cassandra {
Handle::Handle(Settings clusterSettings) : cluster_{clusterSettings}
{
}
Handle::Handle(std::string_view contactPoints)
: Handle{Settings::defaultSettings().withContactPoints(contactPoints)}
{
}
Handle::~Handle()
{
[[maybe_unused]] auto _ = disconnect(); // attempt to disconnect
}
Handle::FutureType
Handle::asyncConnect() const
{
return cass_session_connect(session_, cluster_);
}
Handle::MaybeErrorType
Handle::connect() const
{
return asyncConnect().await();
}
Handle::FutureType
Handle::asyncConnect(std::string_view keyspace) const
{
return cass_session_connect_keyspace(session_, cluster_, keyspace.data());
}
Handle::MaybeErrorType
Handle::connect(std::string_view keyspace) const
{
return asyncConnect(keyspace).await();
}
Handle::FutureType
Handle::asyncDisconnect() const
{
return cass_session_close(session_);
}
Handle::MaybeErrorType
Handle::disconnect() const
{
return asyncDisconnect().await();
}
Handle::FutureType
Handle::asyncReconnect(std::string_view keyspace) const
{
if (auto rc = asyncDisconnect().await(); not rc) // sync
throw std::logic_error(
"Reconnect to keyspace '" + std::string{keyspace} +
"' failed: " + rc.error());
return asyncConnect(keyspace);
}
Handle::MaybeErrorType
Handle::reconnect(std::string_view keyspace) const
{
return asyncReconnect(keyspace).await();
}
std::vector<Handle::FutureType>
Handle::asyncExecuteEach(std::vector<Statement> const& statements) const
{
std::vector<Handle::FutureType> futures;
for (auto const& statement : statements)
futures.push_back(cass_session_execute(session_, statement));
return futures;
}
Handle::MaybeErrorType
Handle::executeEach(std::vector<Statement> const& statements) const
{
for (auto futures = asyncExecuteEach(statements);
auto const& future : futures)
{
if (auto const rc = future.await(); not rc)
return rc;
}
return {};
}
Handle::FutureType
Handle::asyncExecute(Statement const& statement) const
{
return cass_session_execute(session_, statement);
}
Handle::FutureWithCallbackType
Handle::asyncExecute(
Statement const& statement,
std::function<void(Handle::ResultOrErrorType)>&& cb) const
{
return Handle::FutureWithCallbackType{
cass_session_execute(session_, statement), std::move(cb)};
}
Handle::ResultOrErrorType
Handle::execute(Statement const& statement) const
{
return asyncExecute(statement).get();
}
Handle::FutureType
Handle::asyncExecute(std::vector<Statement> const& statements) const
{
return cass_session_execute_batch(session_, Batch{statements});
}
Handle::MaybeErrorType
Handle::execute(std::vector<Statement> const& statements) const
{
return asyncExecute(statements).await();
}
Handle::FutureWithCallbackType
Handle::asyncExecute(
std::vector<Statement> const& statements,
std::function<void(Handle::ResultOrErrorType)>&& cb) const
{
return Handle::FutureWithCallbackType{
cass_session_execute_batch(session_, Batch{statements}), std::move(cb)};
}
Handle::PreparedStatementType
Handle::prepare(std::string_view query) const
{
Handle::FutureType future = cass_session_prepare(session_, query.data());
if (auto const rc = future.await(); rc)
return cass_future_get_prepared(future);
else
throw std::runtime_error(rc.error().message());
}
} // namespace Backend::Cassandra

View File

@@ -0,0 +1,300 @@
//------------------------------------------------------------------------------
/*
This file is part of clio: https://github.com/XRPLF/clio
Copyright (c) 2023, the clio developers.
Permission to use, copy, modify, and distribute this software for any
purpose with or without fee is hereby granted, provided that the above
copyright notice and this permission notice appear in all copies.
THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES
WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF
MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR
ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES
WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN
ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF
OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.
*/
//==============================================================================
#pragma once
#include <backend/cassandra/Error.h>
#include <backend/cassandra/Types.h>
#include <backend/cassandra/impl/Batch.h>
#include <backend/cassandra/impl/Cluster.h>
#include <backend/cassandra/impl/Future.h>
#include <backend/cassandra/impl/ManagedObject.h>
#include <backend/cassandra/impl/Result.h>
#include <backend/cassandra/impl/Session.h>
#include <backend/cassandra/impl/Statement.h>
#include <util/Expected.h>
#include <cassandra.h>
#include <chrono>
#include <compare>
#include <iterator>
#include <vector>
namespace Backend::Cassandra {
/**
* @brief Represents a handle to the cassandra database cluster
*/
class Handle
{
detail::Cluster cluster_;
detail::Session session_;
public:
using ResultOrErrorType = ResultOrError;
using MaybeErrorType = MaybeError;
using FutureWithCallbackType = FutureWithCallback;
using FutureType = Future;
using StatementType = Statement;
using PreparedStatementType = PreparedStatement;
using ResultType = Result;
/**
* @brief Construct a new handle from a @ref Settings object
*/
explicit Handle(Settings clusterSettings = Settings::defaultSettings());
/**
* @brief Construct a new handle with default settings and only by setting
* the contact points
*/
explicit Handle(std::string_view contactPoints);
/**
* @brief Disconnects gracefully if possible
*/
~Handle();
/**
* @brief Move is supported
*/
Handle(Handle&&) = default;
/**
* @brief Connect to the cluster asynchronously
*
* @return A future
*/
[[nodiscard]] FutureType
asyncConnect() const;
/**
* @brief Synchonous version of the above
*
* See @ref asyncConnect() const for how this works.
*/
[[nodiscard]] MaybeErrorType
connect() const;
/**
* @brief Connect to the the specified keyspace asynchronously
*
* @return A future
*/
[[nodiscard]] FutureType
asyncConnect(std::string_view keyspace) const;
/**
* @brief Synchonous version of the above
*
* See @ref asyncConnect(std::string_view) const for how this works.
*/
[[nodiscard]] MaybeErrorType
connect(std::string_view keyspace) const;
/**
* @brief Disconnect from the cluster asynchronously
*
* @return A future
*/
[[nodiscard]] FutureType
asyncDisconnect() const;
/**
* @brief Synchonous version of the above
*
* See @ref asyncDisconnect() const for how this works.
*/
[[maybe_unused]] MaybeErrorType
disconnect() const;
/**
* @brief Reconnect to the the specified keyspace asynchronously
*
* @return A future
*/
[[nodiscard]] FutureType
asyncReconnect(std::string_view keyspace) const;
/**
* @brief Synchonous version of the above
*
* See @ref asyncReconnect(std::string_view) const for how this works.
*/
[[nodiscard]] MaybeErrorType
reconnect(std::string_view keyspace) const;
/**
* @brief Execute a simple query with optional args asynchronously
*
* @return A future
*/
template <typename... Args>
[[nodiscard]] FutureType
asyncExecute(std::string_view query, Args&&... args) const
{
auto statement = StatementType{query, std::forward<Args>(args)...};
return cass_session_execute(session_, statement);
}
/**
* @brief Synchonous version of the above
*
* See @ref asyncExecute(std::string_view, Args&&...) const for how this
* works.
*/
template <typename... Args>
[[maybe_unused]] ResultOrErrorType
execute(std::string_view query, Args&&... args) const
{
return asyncExecute<Args...>(query, std::forward<Args>(args)...).get();
}
/**
* @brief Execute each of the statements asynchronously
*
* Batched version is not always the right option. Especially since it only
* supports INSERT, UPDATE and DELETE statements.
* This can be used as an alternative when statements need to execute in
* bulk.
*
* @return A vector of future objects
*/
[[nodiscard]] std::vector<FutureType>
asyncExecuteEach(std::vector<StatementType> const& statements) const;
/**
* @brief Synchonous version of the above
*
* See @ref asyncExecuteEach(std::vector<StatementType> const&) const for
* how this works.
*/
[[maybe_unused]] MaybeErrorType
executeEach(std::vector<StatementType> const& statements) const;
/**
* @brief Execute a prepared statement with optional args asynchronously
*
* @return A future
*/
template <typename... Args>
[[nodiscard]] FutureType
asyncExecute(PreparedStatementType const& statement, Args&&... args) const
{
auto bound = statement.bind<Args...>(std::forward<Args>(args)...);
return cass_session_execute(session_, bound);
}
/**
* @brief Synchonous version of the above
*
* See @ref asyncExecute(std::vector<StatementType> const&, Args&&...) const
* for how this works.
*/
template <typename... Args>
[[maybe_unused]] ResultOrErrorType
execute(PreparedStatementType const& statement, Args&&... args) const
{
return asyncExecute<Args...>(statement, std::forward<Args>(args)...)
.get();
}
/**
* @brief Execute one (bound or simple) statements asynchronously
*
* @return A future
*/
[[nodiscard]] FutureType
asyncExecute(StatementType const& statement) const;
/**
* @brief Execute one (bound or simple) statements asynchronously with a
* callback
*
* @return A future that holds onto the callback provided
*/
[[nodiscard]] FutureWithCallbackType
asyncExecute(
StatementType const& statement,
std::function<void(ResultOrErrorType)>&& cb) const;
/**
* @brief Synchonous version of the above
*
* See @ref asyncExecute(StatementType const&) const for how this
* works.
*/
[[maybe_unused]] ResultOrErrorType
execute(StatementType const& statement) const;
/**
* @brief Execute a batch of (bound or simple) statements asynchronously
*
* @return A future
*/
[[nodiscard]] FutureType
asyncExecute(std::vector<StatementType> const& statements) const;
/**
* @brief Synchonous version of the above
*
* See @ref asyncExecute(std::vector<StatementType> const&) const for how
* this works.
*/
[[maybe_unused]] MaybeErrorType
execute(std::vector<StatementType> const& statements) const;
/**
* @brief Execute a batch of (bound or simple) statements asynchronously
* with a completion callback
*
* @return A future that holds onto the callback provided
*/
[[nodiscard]] FutureWithCallbackType
asyncExecute(
std::vector<StatementType> const& statements,
std::function<void(ResultOrErrorType)>&& cb) const;
/**
* @brief Prepare a statement
*
* @return A @ref PreparedStatementType
* @throws std::runtime_error with underlying error description on failure
*/
[[nodiscard]] PreparedStatementType
prepare(std::string_view query) const;
};
/**
* @brief Extracts the results into series of std::tuple<Types...> by creating a
* simple wrapper with an STL input iterator inside.
*
* You can call .begin() and .end() in order to iterate as usual.
* This also means that you can use it in a range-based for or with some
* algorithms.
*/
template <typename... Types>
[[nodiscard]] detail::ResultExtractor<Types...>
extract(Handle::ResultType const& result)
{
return {result};
}
} // namespace Backend::Cassandra

View File

@@ -0,0 +1,681 @@
//------------------------------------------------------------------------------
/*
This file is part of clio: https://github.com/XRPLF/clio
Copyright (c) 2023, the clio developers.
Permission to use, copy, modify, and distribute this software for any
purpose with or without fee is hereby granted, provided that the above
copyright notice and this permission notice appear in all copies.
THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES
WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF
MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR
ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES
WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN
ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF
OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.
*/
//==============================================================================
#pragma once
#include <backend/cassandra/Concepts.h>
#include <backend/cassandra/Handle.h>
#include <backend/cassandra/SettingsProvider.h>
#include <backend/cassandra/Types.h>
#include <config/Config.h>
#include <log/Logger.h>
#include <util/Expected.h>
#include <fmt/compile.h>
namespace Backend::Cassandra {
template <SomeSettingsProvider SettingsProviderType>
[[nodiscard]] std::string inline qualifiedTableName(
SettingsProviderType const& provider,
std::string_view name)
{
return fmt::format(
"{}.{}{}",
provider.getKeyspace(),
provider.getTablePrefix().value_or(""),
name);
}
/**
* @brief Manages the DB schema and provides access to prepared statements
*/
template <SomeSettingsProvider SettingsProviderType>
class Schema
{
// Current schema version.
// Update this everytime you update the schema.
// Migrations will be ran automatically based on this value.
static constexpr uint16_t version = 1u;
clio::Logger log_{"Backend"};
std::reference_wrapper<SettingsProviderType const> settingsProvider_;
public:
explicit Schema(SettingsProviderType const& settingsProvider)
: settingsProvider_{std::cref(settingsProvider)}
{
}
std::string createKeyspace = [this]() {
return fmt::format(
R"(
CREATE KEYSPACE IF NOT EXISTS {}
WITH replication = {{
'class': 'SimpleStrategy',
'replication_factor': '{}'
}}
AND durable_writes = true
)",
settingsProvider_.get().getKeyspace(),
settingsProvider_.get().getReplicationFactor());
}();
// =======================
// Schema creation queries
// =======================
std::vector<Statement> createSchema = [this]() {
std::vector<Statement> statements;
statements.emplace_back(fmt::format(
R"(
CREATE TABLE IF NOT EXISTS {}
(
key blob,
sequence bigint,
object blob,
PRIMARY KEY (key, sequence)
)
WITH CLUSTERING ORDER BY (sequence DESC)
AND default_time_to_live = {}
)",
qualifiedTableName(settingsProvider_.get(), "objects"),
settingsProvider_.get().getTtl()));
statements.emplace_back(fmt::format(
R"(
CREATE TABLE IF NOT EXISTS {}
(
hash blob PRIMARY KEY,
ledger_sequence bigint,
date bigint,
transaction blob,
metadata blob
)
WITH default_time_to_live = {}
)",
qualifiedTableName(settingsProvider_.get(), "transactions"),
settingsProvider_.get().getTtl()));
statements.emplace_back(fmt::format(
R"(
CREATE TABLE IF NOT EXISTS {}
(
ledger_sequence bigint,
hash blob,
PRIMARY KEY (ledger_sequence, hash)
)
WITH default_time_to_live = {}
)",
qualifiedTableName(settingsProvider_.get(), "ledger_transactions"),
settingsProvider_.get().getTtl()));
statements.emplace_back(fmt::format(
R"(
CREATE TABLE IF NOT EXISTS {}
(
key blob,
seq bigint,
next blob,
PRIMARY KEY (key, seq)
)
WITH default_time_to_live = {}
)",
qualifiedTableName(settingsProvider_.get(), "successor"),
settingsProvider_.get().getTtl()));
statements.emplace_back(fmt::format(
R"(
CREATE TABLE IF NOT EXISTS {}
(
seq bigint,
key blob,
PRIMARY KEY (seq, key)
)
WITH default_time_to_live = {}
)",
qualifiedTableName(settingsProvider_.get(), "diff"),
settingsProvider_.get().getTtl()));
statements.emplace_back(fmt::format(
R"(
CREATE TABLE IF NOT EXISTS {}
(
account blob,
seq_idx tuple<bigint, bigint>,
hash blob,
PRIMARY KEY (account, seq_idx)
)
WITH CLUSTERING ORDER BY (seq_idx DESC)
AND default_time_to_live = {}
)",
qualifiedTableName(settingsProvider_.get(), "account_tx"),
settingsProvider_.get().getTtl()));
statements.emplace_back(fmt::format(
R"(
CREATE TABLE IF NOT EXISTS {}
(
sequence bigint PRIMARY KEY,
header blob
)
WITH default_time_to_live = {}
)",
qualifiedTableName(settingsProvider_.get(), "ledgers"),
settingsProvider_.get().getTtl()));
statements.emplace_back(fmt::format(
R"(
CREATE TABLE IF NOT EXISTS {}
(
hash blob PRIMARY KEY,
sequence bigint
)
WITH default_time_to_live = {}
)",
qualifiedTableName(settingsProvider_.get(), "ledger_hashes"),
settingsProvider_.get().getTtl()));
statements.emplace_back(fmt::format(
R"(
CREATE TABLE IF NOT EXISTS {}
(
is_latest boolean PRIMARY KEY,
sequence bigint
)
)",
qualifiedTableName(settingsProvider_.get(), "ledger_range")));
statements.emplace_back(fmt::format(
R"(
CREATE TABLE IF NOT EXISTS {}
(
token_id blob,
sequence bigint,
owner blob,
is_burned boolean,
PRIMARY KEY (token_id, sequence)
)
WITH CLUSTERING ORDER BY (sequence DESC)
AND default_time_to_live = {}
)",
qualifiedTableName(settingsProvider_.get(), "nf_tokens"),
settingsProvider_.get().getTtl()));
statements.emplace_back(fmt::format(
R"(
CREATE TABLE IF NOT EXISTS {}
(
issuer blob,
taxon bigint,
token_id blob,
PRIMARY KEY (issuer, taxon, token_id)
)
)",
qualifiedTableName(
settingsProvider_.get(), "issuer_nf_tokens_v2")));
statements.emplace_back(fmt::format(
R"(
CREATE TABLE IF NOT EXISTS {}
(
token_id blob,
sequence bigint,
uri blob,
PRIMARY KEY (token_id, sequence)
)
WITH CLUSTERING ORDER BY (sequence DESC)
AND default_time_to_live = {}
)",
qualifiedTableName(settingsProvider_.get(), "nf_token_uris"),
settingsProvider_.get().getTtl()));
statements.emplace_back(fmt::format(
R"(
CREATE TABLE IF NOT EXISTS {}
(
token_id blob,
seq_idx tuple<bigint, bigint>,
hash blob,
PRIMARY KEY (token_id, seq_idx)
)
WITH CLUSTERING ORDER BY (seq_idx DESC)
AND default_time_to_live = {}
)",
qualifiedTableName(
settingsProvider_.get(), "nf_token_transactions"),
settingsProvider_.get().getTtl()));
return statements;
}();
/**
* @brief Prepared statements holder
*/
class Statements
{
std::reference_wrapper<SettingsProviderType const> settingsProvider_;
std::reference_wrapper<Handle const> handle_;
public:
Statements(
SettingsProviderType const& settingsProvider,
Handle const& handle)
: settingsProvider_{settingsProvider}, handle_{std::cref(handle)}
{
}
//
// Insert queries
//
PreparedStatement insertObject = [this]() {
return handle_.get().prepare(fmt::format(
R"(
INSERT INTO {}
(key, sequence, object)
VALUES (?, ?, ?)
)",
qualifiedTableName(settingsProvider_.get(), "objects")));
}();
PreparedStatement insertTransaction = [this]() {
return handle_.get().prepare(fmt::format(
R"(
INSERT INTO {}
(hash, ledger_sequence, date, transaction, metadata)
VALUES (?, ?, ?, ?, ?)
)",
qualifiedTableName(settingsProvider_.get(), "transactions")));
}();
PreparedStatement insertLedgerTransaction = [this]() {
return handle_.get().prepare(fmt::format(
R"(
INSERT INTO {}
(ledger_sequence, hash)
VALUES (?, ?)
)",
qualifiedTableName(
settingsProvider_.get(), "ledger_transactions")));
}();
PreparedStatement insertSuccessor = [this]() {
return handle_.get().prepare(fmt::format(
R"(
INSERT INTO {}
(key, seq, next)
VALUES (?, ?, ?)
)",
qualifiedTableName(settingsProvider_.get(), "successor")));
}();
PreparedStatement insertDiff = [this]() {
return handle_.get().prepare(fmt::format(
R"(
INSERT INTO {}
(seq, key)
VALUES (?, ?)
)",
qualifiedTableName(settingsProvider_.get(), "diff")));
}();
PreparedStatement insertAccountTx = [this]() {
return handle_.get().prepare(fmt::format(
R"(
INSERT INTO {}
(account, seq_idx, hash)
VALUES (?, ?, ?)
)",
qualifiedTableName(settingsProvider_.get(), "account_tx")));
}();
PreparedStatement insertNFT = [this]() {
return handle_.get().prepare(fmt::format(
R"(
INSERT INTO {}
(token_id, sequence, owner, is_burned)
VALUES (?, ?, ?, ?)
)",
qualifiedTableName(settingsProvider_.get(), "nf_tokens")));
}();
PreparedStatement insertIssuerNFT = [this]() {
return handle_.get().prepare(fmt::format(
R"(
INSERT INTO {}
(issuer, taxon, token_id)
VALUES (?, ?, ?)
)",
qualifiedTableName(
settingsProvider_.get(), "issuer_nf_tokens_v2")));
}();
PreparedStatement insertNFTURI = [this]() {
return handle_.get().prepare(fmt::format(
R"(
INSERT INTO {}
(token_id, sequence, uri)
VALUES (?, ?, ?)
)",
qualifiedTableName(settingsProvider_.get(), "nf_token_uris")));
}();
PreparedStatement insertNFTTx = [this]() {
return handle_.get().prepare(fmt::format(
R"(
INSERT INTO {}
(token_id, seq_idx, hash)
VALUES (?, ?, ?)
)",
qualifiedTableName(
settingsProvider_.get(), "nf_token_transactions")));
}();
PreparedStatement insertLedgerHeader = [this]() {
return handle_.get().prepare(fmt::format(
R"(
INSERT INTO {}
(sequence, header)
VALUES (?, ?)
)",
qualifiedTableName(settingsProvider_.get(), "ledgers")));
}();
PreparedStatement insertLedgerHash = [this]() {
return handle_.get().prepare(fmt::format(
R"(
INSERT INTO {}
(hash, sequence)
VALUES (?, ?)
)",
qualifiedTableName(settingsProvider_.get(), "ledger_hashes")));
}();
//
// Update (and "delete") queries
//
PreparedStatement updateLedgerRange = [this]() {
return handle_.get().prepare(fmt::format(
R"(
UPDATE {}
SET sequence = ?
WHERE is_latest = ?
IF sequence IN (?, null)
)",
qualifiedTableName(settingsProvider_.get(), "ledger_range")));
}();
PreparedStatement deleteLedgerRange = [this]() {
return handle_.get().prepare(fmt::format(
R"(
UPDATE {}
SET sequence = ?
WHERE is_latest = false
)",
qualifiedTableName(settingsProvider_.get(), "ledger_range")));
}();
//
// Select queries
//
PreparedStatement selectSuccessor = [this]() {
return handle_.get().prepare(fmt::format(
R"(
SELECT next
FROM {}
WHERE key = ?
AND seq <= ?
ORDER BY seq DESC
LIMIT 1
)",
qualifiedTableName(settingsProvider_.get(), "successor")));
}();
PreparedStatement selectDiff = [this]() {
return handle_.get().prepare(fmt::format(
R"(
SELECT key
FROM {}
WHERE seq = ?
)",
qualifiedTableName(settingsProvider_.get(), "diff")));
}();
PreparedStatement selectObject = [this]() {
return handle_.get().prepare(fmt::format(
R"(
SELECT object, sequence
FROM {}
WHERE key = ?
AND sequence <= ?
ORDER BY sequence DESC
LIMIT 1
)",
qualifiedTableName(settingsProvider_.get(), "objects")));
}();
PreparedStatement selectTransaction = [this]() {
return handle_.get().prepare(fmt::format(
R"(
SELECT transaction, metadata, ledger_sequence, date
FROM {}
WHERE hash = ?
)",
qualifiedTableName(settingsProvider_.get(), "transactions")));
}();
PreparedStatement selectAllTransactionHashesInLedger = [this]() {
return handle_.get().prepare(fmt::format(
R"(
SELECT hash
FROM {}
WHERE ledger_sequence = ?
)",
qualifiedTableName(
settingsProvider_.get(), "ledger_transactions")));
}();
PreparedStatement selectLedgerPageKeys = [this]() {
return handle_.get().prepare(fmt::format(
R"(
SELECT key
FROM {}
WHERE TOKEN(key) >= ?
AND sequence <= ?
PER PARTITION LIMIT 1
LIMIT ?
ALLOW FILTERING
)",
qualifiedTableName(settingsProvider_.get(), "objects")));
}();
PreparedStatement selectLedgerPage = [this]() {
return handle_.get().prepare(fmt::format(
R"(
SELECT object, key
FROM {}
WHERE TOKEN(key) >= ?
AND sequence <= ?
PER PARTITION LIMIT 1
LIMIT ?
ALLOW FILTERING
)",
qualifiedTableName(settingsProvider_.get(), "objects")));
}();
PreparedStatement getToken = [this]() {
return handle_.get().prepare(fmt::format(
R"(
SELECT TOKEN(key)
FROM {}
WHERE key = ?
LIMIT 1
)",
qualifiedTableName(settingsProvider_.get(), "objects")));
}();
PreparedStatement selectAccountTx = [this]() {
return handle_.get().prepare(fmt::format(
R"(
SELECT hash, seq_idx
FROM {}
WHERE account = ?
AND seq_idx <= ?
LIMIT ?
)",
qualifiedTableName(settingsProvider_.get(), "account_tx")));
}();
PreparedStatement selectAccountTxForward = [this]() {
return handle_.get().prepare(fmt::format(
R"(
SELECT hash, seq_idx
FROM {}
WHERE account = ?
AND seq_idx >= ?
ORDER BY seq_idx ASC
LIMIT ?
)",
qualifiedTableName(settingsProvider_.get(), "account_tx")));
}();
PreparedStatement selectNFT = [this]() {
return handle_.get().prepare(fmt::format(
R"(
SELECT sequence, owner, is_burned
FROM {}
WHERE token_id = ?
AND sequence <= ?
ORDER BY sequence DESC
LIMIT 1
)",
qualifiedTableName(settingsProvider_.get(), "nf_tokens")));
}();
PreparedStatement selectNFTURI = [this]() {
return handle_.get().prepare(fmt::format(
R"(
SELECT uri
FROM {}
WHERE token_id = ?
AND sequence <= ?
ORDER BY sequence DESC
LIMIT 1
)",
qualifiedTableName(settingsProvider_.get(), "nf_token_uris")));
}();
PreparedStatement selectNFTTx = [this]() {
return handle_.get().prepare(fmt::format(
R"(
SELECT hash, seq_idx
FROM {}
WHERE token_id = ?
AND seq_idx < ?
ORDER BY seq_idx DESC
LIMIT ?
)",
qualifiedTableName(
settingsProvider_.get(), "nf_token_transactions")));
}();
PreparedStatement selectNFTTxForward = [this]() {
return handle_.get().prepare(fmt::format(
R"(
SELECT hash, seq_idx
FROM {}
WHERE token_id = ?
AND seq_idx >= ?
ORDER BY seq_idx ASC
LIMIT ?
)",
qualifiedTableName(
settingsProvider_.get(), "nf_token_transactions")));
}();
PreparedStatement selectLedgerByHash = [this]() {
return handle_.get().prepare(fmt::format(
R"(
SELECT sequence
FROM {}
WHERE hash = ?
LIMIT 1
)",
qualifiedTableName(settingsProvider_.get(), "ledger_hashes")));
}();
PreparedStatement selectLedgerBySeq = [this]() {
return handle_.get().prepare(fmt::format(
R"(
SELECT header
FROM {}
WHERE sequence = ?
)",
qualifiedTableName(settingsProvider_.get(), "ledgers")));
}();
PreparedStatement selectLatestLedger = [this]() {
return handle_.get().prepare(fmt::format(
R"(
SELECT sequence
FROM {}
WHERE is_latest = true
)",
qualifiedTableName(settingsProvider_.get(), "ledger_range")));
}();
PreparedStatement selectLedgerRange = [this]() {
return handle_.get().prepare(fmt::format(
R"(
SELECT sequence
FROM {}
)",
qualifiedTableName(settingsProvider_.get(), "ledger_range")));
}();
};
/**
* @brief Recreates the prepared statements
*/
void
prepareStatements(Handle const& handle)
{
log_.info() << "Preparing cassandra statements";
statements_ = std::make_unique<Statements>(settingsProvider_, handle);
log_.info() << "Finished preparing statements";
}
/**
* @brief Provides access to statements
*/
std::unique_ptr<Statements> const&
operator->() const
{
return statements_;
}
private:
std::unique_ptr<Statements> statements_{nullptr};
};
} // namespace Backend::Cassandra

View File

@@ -0,0 +1,142 @@
//------------------------------------------------------------------------------
/*
This file is part of clio: https://github.com/XRPLF/clio
Copyright (c) 2023, the clio developers.
Permission to use, copy, modify, and distribute this software for any
purpose with or without fee is hereby granted, provided that the above
copyright notice and this permission notice appear in all copies.
THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES
WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF
MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR
ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES
WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN
ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF
OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.
*/
//==============================================================================
#include <backend/cassandra/SettingsProvider.h>
#include <backend/cassandra/impl/Cluster.h>
#include <backend/cassandra/impl/Statement.h>
#include <config/Config.h>
#include <boost/json.hpp>
#include <string>
#include <thread>
namespace Backend::Cassandra {
namespace detail {
inline Settings::ContactPoints
tag_invoke(
boost::json::value_to_tag<Settings::ContactPoints>,
boost::json::value const& value)
{
if (not value.is_object())
throw std::runtime_error(
"Feed entire Cassandra section to parse "
"Settings::ContactPoints instead");
clio::Config obj{value};
Settings::ContactPoints out;
out.contactPoints = obj.valueOrThrow<std::string>(
"contact_points", "`contact_points` must be a string");
out.port = obj.maybeValue<uint16_t>("port");
return out;
}
inline Settings::SecureConnectionBundle
tag_invoke(
boost::json::value_to_tag<Settings::SecureConnectionBundle>,
boost::json::value const& value)
{
if (not value.is_string())
throw std::runtime_error("`secure_connect_bundle` must be a string");
return Settings::SecureConnectionBundle{value.as_string().data()};
}
} // namespace detail
SettingsProvider::SettingsProvider(clio::Config const& cfg, uint16_t ttl)
: config_{cfg}
, keyspace_{cfg.valueOr<std::string>("keyspace", "clio")}
, tablePrefix_{cfg.maybeValue<std::string>("table_prefix")}
, replicationFactor_{cfg.valueOr<uint16_t>("replication_factor", 3)}
, ttl_{ttl}
, settings_{parseSettings()}
{
}
Settings
SettingsProvider::getSettings() const
{
return settings_;
}
std::optional<std::string>
SettingsProvider::parseOptionalCertificate() const
{
if (auto const certPath = config_.maybeValue<std::string>("certfile");
certPath)
{
auto const path = std::filesystem::path(*certPath);
std::ifstream fileStream(path.string(), std::ios::in);
if (!fileStream)
{
throw std::system_error(
errno,
std::generic_category(),
"Opening certificate " + path.string());
}
std::string contents(
std::istreambuf_iterator<char>{fileStream},
std::istreambuf_iterator<char>{});
if (fileStream.bad())
{
throw std::system_error(
errno,
std::generic_category(),
"Reading certificate " + path.string());
}
return contents;
}
return std::nullopt;
}
Settings
SettingsProvider::parseSettings() const
{
auto settings = Settings::defaultSettings();
if (auto const bundle =
config_.maybeValue<Settings::SecureConnectionBundle>(
"secure_connect_bundle");
bundle)
{
settings.connectionInfo = *bundle;
}
else
{
settings.connectionInfo = config_.valueOrThrow<Settings::ContactPoints>(
"Missing contact_points in Cassandra config");
}
settings.threads = config_.valueOr<uint32_t>("threads", settings.threads);
settings.maxWriteRequestsOutstanding = config_.valueOr<uint32_t>(
"max_write_requests_outstanding", settings.maxWriteRequestsOutstanding);
settings.maxReadRequestsOutstanding = config_.valueOr<uint32_t>(
"max_read_requests_outstanding", settings.maxReadRequestsOutstanding);
settings.certificate = parseOptionalCertificate();
settings.username = config_.maybeValue<std::string>("username");
settings.password = config_.maybeValue<std::string>("password");
return settings;
}
} // namespace Backend::Cassandra

View File

@@ -0,0 +1,86 @@
//------------------------------------------------------------------------------
/*
This file is part of clio: https://github.com/XRPLF/clio
Copyright (c) 2023, the clio developers.
Permission to use, copy, modify, and distribute this software for any
purpose with or without fee is hereby granted, provided that the above
copyright notice and this permission notice appear in all copies.
THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES
WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF
MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR
ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES
WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN
ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF
OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.
*/
//==============================================================================
#pragma once
#include <backend/cassandra/Handle.h>
#include <backend/cassandra/Types.h>
#include <config/Config.h>
#include <log/Logger.h>
#include <util/Expected.h>
namespace Backend::Cassandra {
/**
* @brief Provides settings for @ref CassandraBackend
*/
class SettingsProvider
{
clio::Config config_;
std::string keyspace_;
std::optional<std::string> tablePrefix_;
uint16_t replicationFactor_;
uint16_t ttl_;
Settings settings_;
public:
explicit SettingsProvider(clio::Config const& cfg, uint16_t ttl = 0);
/*! Get the cluster settings */
[[nodiscard]] Settings
getSettings() const;
/*! Get the specified keyspace */
[[nodiscard]] inline std::string
getKeyspace() const
{
return keyspace_;
}
/*! Get an optional table prefix to use in all queries */
[[nodiscard]] inline std::optional<std::string>
getTablePrefix() const
{
return tablePrefix_;
}
/*! Get the replication factor */
[[nodiscard]] inline uint16_t
getReplicationFactor() const
{
return replicationFactor_;
}
/*! Get the default time to live to use in all `create` queries */
[[nodiscard]] inline uint16_t
getTtl() const
{
return ttl_;
}
private:
[[nodiscard]] std::optional<std::string>
parseOptionalCertificate() const;
[[nodiscard]] Settings
parseSettings() const;
};
} // namespace Backend::Cassandra

View File

@@ -0,0 +1,67 @@
//------------------------------------------------------------------------------
/*
This file is part of clio: https://github.com/XRPLF/clio
Copyright (c) 2023, the clio developers.
Permission to use, copy, modify, and distribute this software for any
purpose with or without fee is hereby granted, provided that the above
copyright notice and this permission notice appear in all copies.
THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES
WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF
MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR
ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES
WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN
ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF
OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.
*/
//==============================================================================
#pragma once
#include <util/Expected.h>
#include <string>
namespace Backend::Cassandra {
namespace detail {
struct Settings;
class Session;
class Cluster;
struct Future;
class FutureWithCallback;
struct Result;
class Statement;
class PreparedStatement;
struct Batch;
} // namespace detail
using Settings = detail::Settings;
using Future = detail::Future;
using FutureWithCallback = detail::FutureWithCallback;
using Result = detail::Result;
using Statement = detail::Statement;
using PreparedStatement = detail::PreparedStatement;
using Batch = detail::Batch;
/**
* @brief A strong type wrapper for int32_t
*
* This is unfortunately needed right now to support uint32_t properly
* because clio uses bigint (int64) everywhere except for when one need
* to specify LIMIT, which needs an int32 :-/
*/
struct Limit
{
int32_t limit;
};
class Handle;
class CassandraError;
using MaybeError = util::Expected<void, CassandraError>;
using ResultOrError = util::Expected<Result, CassandraError>;
using Error = util::Unexpected<CassandraError>;
} // namespace Backend::Cassandra

View File

@@ -0,0 +1,134 @@
//------------------------------------------------------------------------------
/*
This file is part of clio: https://github.com/XRPLF/clio
Copyright (c) 2023, the clio developers.
Permission to use, copy, modify, and distribute this software for any
purpose with or without fee is hereby granted, provided that the above
copyright notice and this permission notice appear in all copies.
THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES
WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF
MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR
ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES
WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN
ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF
OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.
*/
//==============================================================================
#pragma once
#include <backend/cassandra/Concepts.h>
#include <backend/cassandra/Handle.h>
#include <backend/cassandra/Types.h>
#include <backend/cassandra/impl/RetryPolicy.h>
#include <log/Logger.h>
#include <util/Expected.h>
#include <boost/asio.hpp>
#include <functional>
#include <memory>
namespace Backend::Cassandra::detail {
/**
* @brief A query executor with a changable retry policy
*
* Note: this is a bit of an anti-pattern and should be done differently
* eventually.
*
* Currently it's basically a saner implementation of the previous design that
* was used in production without much issue but was using raw new/delete and
* could leak easily. This version is slightly better but the overall design is
* flawed and should be reworked.
*/
template <
typename StatementType,
typename HandleType = Handle,
SomeRetryPolicy RetryPolicyType = ExponentialBackoffRetryPolicy>
class AsyncExecutor
: public std::enable_shared_from_this<
AsyncExecutor<StatementType, HandleType, RetryPolicyType>>
{
using FutureWithCallbackType = typename HandleType::FutureWithCallbackType;
using CallbackType =
std::function<void(typename HandleType::ResultOrErrorType)>;
clio::Logger log_{"Backend"};
StatementType data_;
RetryPolicyType retryPolicy_;
CallbackType onComplete_;
// does not exist during initial construction, hence optional
std::optional<FutureWithCallbackType> future_;
std::mutex mtx_;
public:
/**
* @brief Create a new instance of the AsyncExecutor and execute it.
*/
static void
run(boost::asio::io_context& ioc,
HandleType const& handle,
StatementType data,
CallbackType&& onComplete)
{
// this is a helper that allows us to use std::make_shared below
struct EnableMakeShared
: public AsyncExecutor<StatementType, HandleType, RetryPolicyType>
{
EnableMakeShared(
boost::asio::io_context& ioc,
StatementType&& data,
CallbackType&& onComplete)
: AsyncExecutor(ioc, std::move(data), std::move(onComplete))
{
}
};
auto ptr = std::make_shared<EnableMakeShared>(
ioc, std::move(data), std::move(onComplete));
ptr->execute(handle);
}
private:
AsyncExecutor(
boost::asio::io_context& ioc,
StatementType&& data,
CallbackType&& onComplete)
: data_{std::move(data)}
, retryPolicy_{ioc}
, onComplete_{std::move(onComplete)}
{
}
void
execute(HandleType const& handle)
{
auto self = this->shared_from_this();
// lifetime is extended by capturing self ptr
auto handler = [this, &handle, self](auto&& res) mutable {
if (res)
{
onComplete_(std::move(res));
}
else
{
if (retryPolicy_.shouldRetry(res.error()))
retryPolicy_.retry(
[self, &handle]() { self->execute(handle); });
else
onComplete_(std::move(res)); // report error
}
};
std::scoped_lock lck{mtx_};
future_.emplace(handle.asyncExecute(data_, std::move(handler)));
}
};
} // namespace Backend::Cassandra::detail

View File

@@ -0,0 +1,60 @@
//------------------------------------------------------------------------------
/*
This file is part of clio: https://github.com/XRPLF/clio
Copyright (c) 2023, the clio developers.
Permission to use, copy, modify, and distribute this software for any
purpose with or without fee is hereby granted, provided that the above
copyright notice and this permission notice appear in all copies.
THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES
WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF
MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR
ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES
WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN
ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF
OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.
*/
//==============================================================================
#include <backend/cassandra/Error.h>
#include <backend/cassandra/impl/Batch.h>
#include <backend/cassandra/impl/Statement.h>
#include <util/Expected.h>
#include <exception>
#include <vector>
namespace {
static constexpr auto batchDeleter = [](CassBatch* ptr) {
cass_batch_free(ptr);
};
};
namespace Backend::Cassandra::detail {
// todo: use an appropritae value instead of CASS_BATCH_TYPE_LOGGED for
// different use cases
Batch::Batch(std::vector<Statement> const& statements)
: ManagedObject{cass_batch_new(CASS_BATCH_TYPE_LOGGED), batchDeleter}
{
cass_batch_set_is_idempotent(*this, cass_true);
for (auto const& statement : statements)
if (auto const res = add(statement); not res)
throw std::runtime_error(
"Failed to add statement to batch: " + res.error());
}
MaybeError
Batch::add(Statement const& statement)
{
if (auto const rc = cass_batch_add_statement(*this, statement);
rc != CASS_OK)
{
return Error{CassandraError{cass_error_desc(rc), rc}};
}
return {};
}
} // namespace Backend::Cassandra::detail

View File

@@ -0,0 +1,37 @@
//------------------------------------------------------------------------------
/*
This file is part of clio: https://github.com/XRPLF/clio
Copyright (c) 2023, the clio developers.
Permission to use, copy, modify, and distribute this software for any
purpose with or without fee is hereby granted, provided that the above
copyright notice and this permission notice appear in all copies.
THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES
WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF
MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR
ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES
WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN
ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF
OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.
*/
//==============================================================================
#pragma once
#include <backend/cassandra/Types.h>
#include <backend/cassandra/impl/ManagedObject.h>
#include <cassandra.h>
namespace Backend::Cassandra::detail {
struct Batch : public ManagedObject<CassBatch>
{
Batch(std::vector<Statement> const& statements);
MaybeError
add(Statement const& statement);
};
} // namespace Backend::Cassandra::detail

View File

@@ -0,0 +1,180 @@
//------------------------------------------------------------------------------
/*
This file is part of clio: https://github.com/XRPLF/clio
Copyright (c) 2023, the clio developers.
Permission to use, copy, modify, and distribute this software for any
purpose with or without fee is hereby granted, provided that the above
copyright notice and this permission notice appear in all copies.
THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES
WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF
MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR
ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES
WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN
ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF
OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.
*/
//==============================================================================
#include <backend/cassandra/impl/Cluster.h>
#include <backend/cassandra/impl/SslContext.h>
#include <backend/cassandra/impl/Statement.h>
#include <util/Expected.h>
#include <exception>
#include <vector>
namespace {
static constexpr auto clusterDeleter = [](CassCluster* ptr) {
cass_cluster_free(ptr);
};
template <class... Ts>
struct overloadSet : Ts...
{
using Ts::operator()...;
};
// explicit deduction guide (not needed as of C++20, but clang be clang)
template <class... Ts>
overloadSet(Ts...) -> overloadSet<Ts...>;
}; // namespace
namespace Backend::Cassandra::detail {
Cluster::Cluster(Settings const& settings)
: ManagedObject{cass_cluster_new(), clusterDeleter}
{
using std::to_string;
cass_cluster_set_token_aware_routing(*this, cass_true);
if (auto const rc =
cass_cluster_set_protocol_version(*this, CASS_PROTOCOL_VERSION_V4);
rc != CASS_OK)
{
throw std::runtime_error(
std::string{"Error setting cassandra protocol version to v4: "} +
cass_error_desc(rc));
}
if (auto const rc =
cass_cluster_set_num_threads_io(*this, settings.threads);
rc != CASS_OK)
{
throw std::runtime_error(
std::string{"Error setting cassandra io threads to "} +
to_string(settings.threads) + ": " + cass_error_desc(rc));
}
cass_log_set_level(settings.enableLog ? CASS_LOG_TRACE : CASS_LOG_DISABLED);
cass_cluster_set_connect_timeout(*this, settings.connectionTimeout.count());
cass_cluster_set_request_timeout(*this, settings.requestTimeout.count());
// TODO: other options to experiment with and consider later:
// cass_cluster_set_max_concurrent_requests_threshold(*this, 10000);
// cass_cluster_set_queue_size_event(*this, 100000);
// cass_cluster_set_queue_size_io(*this, 100000);
// cass_cluster_set_write_bytes_high_water_mark(
// *this, 16 * 1024 * 1024); // 16mb
// cass_cluster_set_write_bytes_low_water_mark(
// *this, 8 * 1024 * 1024); // half of allowance
// cass_cluster_set_pending_requests_high_water_mark(*this, 5000);
// cass_cluster_set_pending_requests_low_water_mark(*this, 2500); // half
// cass_cluster_set_max_requests_per_flush(*this, 1000);
// cass_cluster_set_max_concurrent_creation(*this, 8);
// cass_cluster_set_max_connections_per_host(*this, 6);
// cass_cluster_set_core_connections_per_host(*this, 4);
// cass_cluster_set_constant_speculative_execution_policy(*this, 1000,
// 1024);
if (auto const rc = cass_cluster_set_queue_size_io(
*this,
settings.maxWriteRequestsOutstanding +
settings.maxReadRequestsOutstanding);
rc != CASS_OK)
{
throw std::runtime_error(
std::string{"Could not set queue size for IO per host: "} +
cass_error_desc(rc));
}
setupConnection(settings);
setupCertificate(settings);
setupCredentials(settings);
}
void
Cluster::setupConnection(Settings const& settings)
{
std::visit(
overloadSet{
[this](Settings::ContactPoints const& points) {
setupContactPoints(points);
},
[this](Settings::SecureConnectionBundle const& bundle) {
setupSecureBundle(bundle);
}},
settings.connectionInfo);
}
void
Cluster::setupContactPoints(Settings::ContactPoints const& points)
{
using std::to_string;
auto throwErrorIfNeeded =
[](CassError rc, std::string const label, std::string const value) {
if (rc != CASS_OK)
throw std::runtime_error(
"Cassandra: Error setting " + label + " [" + value +
"]: " + cass_error_desc(rc));
};
{
auto const rc =
cass_cluster_set_contact_points(*this, points.contactPoints.data());
throwErrorIfNeeded(rc, "contact_points", points.contactPoints);
}
if (points.port)
{
auto const rc = cass_cluster_set_port(*this, points.port.value());
throwErrorIfNeeded(rc, "port", to_string(points.port.value()));
}
}
void
Cluster::setupSecureBundle(Settings::SecureConnectionBundle const& bundle)
{
if (auto const rc = cass_cluster_set_cloud_secure_connection_bundle(
*this, bundle.bundle.data());
rc != CASS_OK)
{
throw std::runtime_error(
"Failed to connect using secure connection bundle" + bundle.bundle);
}
}
void
Cluster::setupCertificate(Settings const& settings)
{
if (not settings.certificate)
return;
SslContext context = SslContext(*settings.certificate);
cass_cluster_set_ssl(*this, context);
}
void
Cluster::setupCredentials(Settings const& settings)
{
if (not settings.username || not settings.password)
return;
cass_cluster_set_credentials(
*this,
settings.username.value().c_str(),
settings.password.value().c_str());
}
} // namespace Backend::Cassandra::detail

View File

@@ -0,0 +1,99 @@
//------------------------------------------------------------------------------
/*
This file is part of clio: https://github.com/XRPLF/clio
Copyright (c) 2023, the clio developers.
Permission to use, copy, modify, and distribute this software for any
purpose with or without fee is hereby granted, provided that the above
copyright notice and this permission notice appear in all copies.
THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES
WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF
MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR
ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES
WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN
ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF
OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.
*/
//==============================================================================
#pragma once
#include <backend/cassandra/impl/ManagedObject.h>
#include <cassandra.h>
#include <chrono>
#include <optional>
#include <string>
#include <string_view>
#include <thread>
#include <variant>
namespace Backend::Cassandra::detail {
struct Settings
{
struct ContactPoints
{
std::string contactPoints = "127.0.0.1"; // defaults to localhost
std::optional<uint16_t> port;
};
struct SecureConnectionBundle
{
std::string bundle; // no meaningful default
};
bool enableLog = false;
std::chrono::milliseconds connectionTimeout =
std::chrono::milliseconds{1000};
std::chrono::milliseconds requestTimeout =
std::chrono::milliseconds{0}; // no timeout at all
std::variant<ContactPoints, SecureConnectionBundle> connectionInfo =
ContactPoints{};
uint32_t threads = std::thread::hardware_concurrency();
uint32_t maxWriteRequestsOutstanding = 10'000;
uint32_t maxReadRequestsOutstanding = 100'000;
std::optional<std::string> certificate; // ssl context
std::optional<std::string> username;
std::optional<std::string> password;
Settings
withContactPoints(std::string_view contactPoints)
{
auto tmp = *this;
tmp.connectionInfo = ContactPoints{std::string{contactPoints}};
return tmp;
}
static Settings
defaultSettings()
{
return Settings();
}
};
class Cluster : public ManagedObject<CassCluster>
{
public:
Cluster(Settings const& settings);
private:
void
setupConnection(Settings const& settings);
void
setupContactPoints(Settings::ContactPoints const& points);
void
setupSecureBundle(Settings::SecureConnectionBundle const& bundle);
void
setupCertificate(Settings const& settings);
void
setupCredentials(Settings const& settings);
};
} // namespace Backend::Cassandra::detail

View File

@@ -0,0 +1,465 @@
//------------------------------------------------------------------------------
/*
This file is part of clio: https://github.com/XRPLF/clio
Copyright (c) 2023, the clio developers.
Permission to use, copy, modify, and distribute this software for any
purpose with or without fee is hereby granted, provided that the above
copyright notice and this permission notice appear in all copies.
THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES
WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF
MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR
ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES
WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN
ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF
OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.
*/
//==============================================================================
#pragma once
#include <backend/cassandra/Handle.h>
#include <backend/cassandra/Types.h>
#include <backend/cassandra/impl/AsyncExecutor.h>
#include <log/Logger.h>
#include <util/Expected.h>
#include <boost/asio/async_result.hpp>
#include <boost/asio/spawn.hpp>
#include <atomic>
#include <condition_variable>
#include <functional>
#include <memory>
#include <mutex>
#include <optional>
#include <thread>
namespace Backend::Cassandra::detail {
/**
* @brief Implements async and sync querying against the cassandra DB with
* support for throttling.
*
* Note: A lot of the code that uses yield is repeated below. This is ok for now
* because we are hopefully going to be getting rid of it entirely later on.
*/
template <typename HandleType = Handle>
class DefaultExecutionStrategy
{
clio::Logger log_{"Backend"};
std::uint32_t maxWriteRequestsOutstanding_;
std::atomic_uint32_t numWriteRequestsOutstanding_ = 0;
std::uint32_t maxReadRequestsOutstanding_;
std::atomic_uint32_t numReadRequestsOutstanding_ = 0;
std::mutex throttleMutex_;
std::condition_variable throttleCv_;
std::mutex syncMutex_;
std::condition_variable syncCv_;
boost::asio::io_context ioc_;
std::optional<boost::asio::io_service::work> work_;
std::reference_wrapper<HandleType const> handle_;
std::thread thread_;
public:
using ResultOrErrorType = typename HandleType::ResultOrErrorType;
using StatementType = typename HandleType::StatementType;
using PreparedStatementType = typename HandleType::PreparedStatementType;
using FutureType = typename HandleType::FutureType;
using FutureWithCallbackType = typename HandleType::FutureWithCallbackType;
using ResultType = typename HandleType::ResultType;
using CompletionTokenType = boost::asio::yield_context;
using FunctionType = void(boost::system::error_code);
using AsyncResultType =
boost::asio::async_result<CompletionTokenType, FunctionType>;
using HandlerType = typename AsyncResultType::completion_handler_type;
DefaultExecutionStrategy(Settings settings, HandleType const& handle)
: maxWriteRequestsOutstanding_{settings.maxWriteRequestsOutstanding}
, maxReadRequestsOutstanding_{settings.maxReadRequestsOutstanding}
, work_{ioc_}
, handle_{std::cref(handle)}
, thread_{[this]() { ioc_.run(); }}
{
log_.info() << "Max write requests outstanding is "
<< maxWriteRequestsOutstanding_
<< "; Max read requests outstanding is "
<< maxReadRequestsOutstanding_;
}
~DefaultExecutionStrategy()
{
work_.reset();
ioc_.stop();
thread_.join();
}
/**
* @brief Wait for all async writes to finish before unblocking
*/
void
sync()
{
log_.debug() << "Waiting to sync all writes...";
std::unique_lock<std::mutex> lck(syncMutex_);
syncCv_.wait(lck, [this]() { return finishedAllWriteRequests(); });
log_.debug() << "Sync done.";
}
bool
isTooBusy() const
{
return numReadRequestsOutstanding_ >= maxReadRequestsOutstanding_;
}
/**
* @brief Blocking query execution used for writing data
*
* Retries forever sleeping for 5 milliseconds between attempts.
*/
ResultOrErrorType
writeSync(StatementType const& statement)
{
while (true)
{
if (auto res = handle_.get().execute(statement); res)
{
return res;
}
else
{
log_.warn()
<< "Cassandra sync write error, retrying: " << res.error();
std::this_thread::sleep_for(std::chrono::milliseconds(5));
}
}
}
/**
* @brief Blocking query execution used for writing data
*
* Retries forever sleeping for 5 milliseconds between attempts.
*/
template <typename... Args>
ResultOrErrorType
writeSync(PreparedStatementType const& preparedStatement, Args&&... args)
{
return writeSync(preparedStatement.bind(std::forward<Args>(args)...));
}
/**
* @brief Non-blocking query execution used for writing data
*
* Retries forever with retry policy specified by @ref AsyncExecutor
*
* @param prepradeStatement Statement to prepare and execute
* @param args Args to bind to the prepared statement
* @throw DatabaseTimeout on timeout
*/
template <typename... Args>
void
write(PreparedStatementType const& preparedStatement, Args&&... args)
{
auto statement = preparedStatement.bind(std::forward<Args>(args)...);
incrementOutstandingRequestCount();
// Note: lifetime is controlled by std::shared_from_this internally
AsyncExecutor<decltype(statement), HandleType>::run(
ioc_, handle_.get(), std::move(statement), [this](auto const&) {
decrementOutstandingRequestCount();
});
}
/**
* @brief Non-blocking batched query execution used for writing data
*
* Retries forever with retry policy specified by @ref AsyncExecutor.
*
* @param statements Vector of statements to execute as a batch
* @throw DatabaseTimeout on timeout
*/
void
write(std::vector<StatementType> const& statements)
{
incrementOutstandingRequestCount();
// Note: lifetime is controlled by std::shared_from_this internally
AsyncExecutor<decltype(statements), HandleType>::run(
ioc_, handle_.get(), statements, [this](auto const&) {
decrementOutstandingRequestCount();
});
}
/**
* @brief Coroutine-based query execution used for reading data.
*
* Retries forever until successful or throws an exception on timeout.
*
* @param token Completion token (yield_context)
* @param prepradeStatement Statement to prepare and execute
* @param args Args to bind to the prepared statement
* @throw DatabaseTimeout on timeout
* @return ResultType or error wrapped in Expected
*/
template <typename... Args>
[[maybe_unused]] ResultOrErrorType
read(
CompletionTokenType token,
PreparedStatementType const& preparedStatement,
Args&&... args)
{
return read(token, preparedStatement.bind(std::forward<Args>(args)...));
}
/**
* @brief Coroutine-based query execution used for reading data.
*
* Retries forever until successful or throws an exception on timeout.
*
* @param token Completion token (yield_context)
* @param statements Statements to execute in a batch
* @throw DatabaseTimeout on timeout
* @return ResultType or error wrapped in Expected
*/
[[maybe_unused]] ResultOrErrorType
read(
CompletionTokenType token,
std::vector<StatementType> const& statements)
{
auto handler = HandlerType{token};
auto result = AsyncResultType{handler};
// todo: perhaps use policy instead
while (true)
{
numReadRequestsOutstanding_ += statements.size();
auto const future = handle_.get().asyncExecute(
statements, [handler](auto&&) mutable {
boost::asio::post(
boost::asio::get_associated_executor(handler),
[handler]() mutable {
handler(boost::system::error_code{});
});
});
// suspend coroutine until completion handler is called
result.get();
numReadRequestsOutstanding_ -= statements.size();
// it's safe to call blocking get on future here as we already
// waited for the coroutine to resume above.
if (auto res = future.get(); res)
{
return res;
}
else
{
log_.error()
<< "Failed batch read in coroutine: " << res.error();
throwErrorIfNeeded(res.error());
}
}
}
/**
* @brief Coroutine-based query execution used for reading data.
*
* Retries forever until successful or throws an exception on timeout.
*
* @param token Completion token (yield_context)
* @param statement Statement to execute
* @throw DatabaseTimeout on timeout
* @return ResultType or error wrapped in Expected
*/
[[maybe_unused]] ResultOrErrorType
read(CompletionTokenType token, StatementType const& statement)
{
auto handler = HandlerType{token};
auto result = AsyncResultType{handler};
// todo: perhaps use policy instead
while (true)
{
++numReadRequestsOutstanding_;
auto const future = handle_.get().asyncExecute(
statement, [handler](auto const&) mutable {
boost::asio::post(
boost::asio::get_associated_executor(handler),
[handler]() mutable {
handler(boost::system::error_code{});
});
});
// suspend coroutine until completion handler is called
result.get();
--numReadRequestsOutstanding_;
// it's safe to call blocking get on future here as we already
// waited for the coroutine to resume above.
if (auto res = future.get(); res)
{
return res;
}
else
{
log_.error() << "Failed read in coroutine: " << res.error();
throwErrorIfNeeded(res.error());
}
}
}
/**
* @brief Coroutine-based query execution used for reading data.
*
* Attempts to execute each statement. On any error the whole vector will be
* discarded and exception will be thrown.
*
* @param token Completion token (yield_context)
* @param statements Statements to execute
* @throw DatabaseTimeout on db error
* @return Vector of results
*/
std::vector<ResultType>
readEach(
CompletionTokenType token,
std::vector<StatementType> const& statements)
{
auto handler = HandlerType{token};
auto result = AsyncResultType{handler};
std::atomic_bool hadError = false;
std::atomic_int numOutstanding = statements.size();
numReadRequestsOutstanding_ += statements.size();
auto futures = std::vector<FutureWithCallbackType>{};
futures.reserve(numOutstanding);
// used as the handler for each async statement individually
auto executionHandler =
[handler, &hadError, &numOutstanding](auto const& res) mutable {
if (not res)
hadError = true;
// when all async operations complete unblock the result
if (--numOutstanding == 0)
boost::asio::post(
boost::asio::get_associated_executor(handler),
[handler]() mutable {
handler(boost::system::error_code{});
});
};
std::transform(
std::cbegin(statements),
std::cend(statements),
std::back_inserter(futures),
[this, &executionHandler](auto const& statement) {
return handle_.get().asyncExecute(statement, executionHandler);
});
// suspend coroutine until completion handler is called
result.get();
numReadRequestsOutstanding_ -= statements.size();
if (hadError)
throw DatabaseTimeout{};
std::vector<ResultType> results;
results.reserve(futures.size());
// it's safe to call blocking get on futures here as we already
// waited for the coroutine to resume above.
std::transform(
std::make_move_iterator(std::begin(futures)),
std::make_move_iterator(std::end(futures)),
std::back_inserter(results),
[](auto&& future) {
auto entry = future.get();
auto&& res = entry.value();
return std::move(res);
});
assert(futures.size() == statements.size());
assert(results.size() == statements.size());
return results;
}
private:
void
incrementOutstandingRequestCount()
{
{
std::unique_lock<std::mutex> lck(throttleMutex_);
if (!canAddWriteRequest())
{
log_.trace() << "Max outstanding requests reached. "
<< "Waiting for other requests to finish";
throttleCv_.wait(
lck, [this]() { return canAddWriteRequest(); });
}
}
++numWriteRequestsOutstanding_;
}
void
decrementOutstandingRequestCount()
{
// sanity check
if (numWriteRequestsOutstanding_ == 0)
{
assert(false);
throw std::runtime_error("decrementing num outstanding below 0");
}
size_t cur = (--numWriteRequestsOutstanding_);
{
// mutex lock required to prevent race condition around spurious
// wakeup
std::lock_guard lck(throttleMutex_);
throttleCv_.notify_one();
}
if (cur == 0)
{
// mutex lock required to prevent race condition around spurious
// wakeup
std::lock_guard lck(syncMutex_);
syncCv_.notify_one();
}
}
bool
canAddWriteRequest() const
{
return numWriteRequestsOutstanding_ < maxWriteRequestsOutstanding_;
}
bool
finishedAllWriteRequests() const
{
return numWriteRequestsOutstanding_ == 0;
}
void
throwErrorIfNeeded(CassandraError err) const
{
if (err.isTimeout())
throw DatabaseTimeout();
if (err.isInvalidQuery())
throw std::runtime_error("Invalid query");
}
};
} // namespace Backend::Cassandra::detail

View File

@@ -0,0 +1,111 @@
//------------------------------------------------------------------------------
/*
This file is part of clio: https://github.com/XRPLF/clio
Copyright (c) 2023, the clio developers.
Permission to use, copy, modify, and distribute this software for any
purpose with or without fee is hereby granted, provided that the above
copyright notice and this permission notice appear in all copies.
THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES
WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF
MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR
ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES
WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN
ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF
OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.
*/
//==============================================================================
#include <backend/cassandra/Error.h>
#include <backend/cassandra/impl/Future.h>
#include <backend/cassandra/impl/Result.h>
#include <exception>
#include <vector>
namespace {
static constexpr auto futureDeleter = [](CassFuture* ptr) {
cass_future_free(ptr);
};
} // namespace
namespace Backend::Cassandra::detail {
/* implicit */ Future::Future(CassFuture* ptr)
: ManagedObject{ptr, futureDeleter}
{
}
MaybeError
Future::await() const
{
if (auto const rc = cass_future_error_code(*this); rc)
{
auto errMsg = [this](std::string label) {
char const* message;
std::size_t len;
cass_future_error_message(*this, &message, &len);
return label + ": " + std::string{message, len};
}(cass_error_desc(rc));
return Error{CassandraError{errMsg, rc}};
}
return {};
}
ResultOrError
Future::get() const
{
if (Result result = cass_future_get_result(*this); not result)
{
auto [errMsg, code] = [this](std::string label) {
char const* message;
std::size_t len;
cass_future_error_message(*this, &message, &len);
return std::make_pair(
label + ": " + std::string{message, len},
cass_future_error_code(*this));
}("future::get()");
return Error{CassandraError{errMsg, code}};
}
else
{
return result;
}
}
void
invokeHelper(CassFuture* ptr, void* cbPtr)
{
// Note: can't use Future{ptr}.get() because double free will occur :/
auto* cb = static_cast<FutureWithCallback::fn_t*>(cbPtr);
if (Result result = cass_future_get_result(ptr); not result)
{
auto [errMsg, code] = [&ptr](std::string label) {
char const* message;
std::size_t len;
cass_future_error_message(ptr, &message, &len);
return std::make_pair(
label + ": " + std::string{message, len},
cass_future_error_code(ptr));
}("invokeHelper");
(*cb)(Error{CassandraError{errMsg, code}});
}
else
{
(*cb)(std::move(result));
}
}
/* implicit */ FutureWithCallback::FutureWithCallback(
CassFuture* ptr,
fn_t&& cb)
: Future{ptr}, cb_{std::make_unique<fn_t>(std::move(cb))}
{
// Instead of passing `this` as the userdata void*, we pass the address of
// the callback itself which will survive std::move of the
// FutureWithCallback parent. Not ideal but I have no better solution atm.
cass_future_set_callback(*this, &invokeHelper, cb_.get());
}
} // namespace Backend::Cassandra::detail

View File

@@ -0,0 +1,58 @@
//------------------------------------------------------------------------------
/*
This file is part of clio: https://github.com/XRPLF/clio
Copyright (c) 2023, the clio developers.
Permission to use, copy, modify, and distribute this software for any
purpose with or without fee is hereby granted, provided that the above
copyright notice and this permission notice appear in all copies.
THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES
WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF
MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR
ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES
WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN
ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF
OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.
*/
//==============================================================================
#pragma once
#include <backend/cassandra/Types.h>
#include <backend/cassandra/impl/ManagedObject.h>
#include <cassandra.h>
namespace Backend::Cassandra::detail {
struct Future : public ManagedObject<CassFuture>
{
/* implicit */ Future(CassFuture* ptr);
MaybeError
await() const;
ResultOrError
get() const;
};
void
invokeHelper(CassFuture* ptr, void* self);
class FutureWithCallback : public Future
{
public:
using fn_t = std::function<void(ResultOrError)>;
using fn_ptr_t = std::unique_ptr<fn_t>;
/* implicit */ FutureWithCallback(CassFuture* ptr, fn_t&& cb);
FutureWithCallback(FutureWithCallback const&) = delete;
FutureWithCallback(FutureWithCallback&&) = default;
private:
/*! Wrapped in a unique_ptr so it can survive std::move :/ */
fn_ptr_t cb_;
};
} // namespace Backend::Cassandra::detail

View File

@@ -0,0 +1,49 @@
//------------------------------------------------------------------------------
/*
This file is part of clio: https://github.com/XRPLF/clio
Copyright (c) 2023, the clio developers.
Permission to use, copy, modify, and distribute this software for any
purpose with or without fee is hereby granted, provided that the above
copyright notice and this permission notice appear in all copies.
THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES
WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF
MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR
ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES
WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN
ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF
OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.
*/
//==============================================================================
#pragma once
#include <memory>
namespace Backend::Cassandra::detail {
template <typename Managed>
class ManagedObject
{
protected:
std::unique_ptr<Managed, void (*)(Managed*)> ptr_;
public:
template <typename deleterCallable>
ManagedObject(Managed* rawPtr, deleterCallable deleter)
: ptr_{rawPtr, deleter}
{
if (rawPtr == nullptr)
throw std::runtime_error(
"Could not create DB object - got nullptr");
}
ManagedObject(ManagedObject&&) = default;
operator Managed* const() const
{
return ptr_.get();
}
};
} // namespace Backend::Cassandra::detail

View File

@@ -0,0 +1,75 @@
//------------------------------------------------------------------------------
/*
This file is part of clio: https://github.com/XRPLF/clio
Copyright (c) 2023, the clio developers.
Permission to use, copy, modify, and distribute this software for any
purpose with or without fee is hereby granted, provided that the above
copyright notice and this permission notice appear in all copies.
THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES
WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF
MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR
ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES
WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN
ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF
OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.
*/
//==============================================================================
#include <backend/cassandra/impl/Result.h>
namespace {
static constexpr auto resultDeleter = [](CassResult const* ptr) {
cass_result_free(ptr);
};
static constexpr auto resultIteratorDeleter = [](CassIterator* ptr) {
cass_iterator_free(ptr);
};
} // namespace
namespace Backend::Cassandra::detail {
/* implicit */ Result::Result(CassResult const* ptr)
: ManagedObject{ptr, resultDeleter}
{
}
[[nodiscard]] std::size_t
Result::numRows() const
{
return cass_result_row_count(*this);
}
[[nodiscard]] bool
Result::hasRows() const
{
return numRows() > 0;
}
/* implicit */ ResultIterator::ResultIterator(CassIterator* ptr)
: ManagedObject{ptr, resultIteratorDeleter}
, hasMore_{cass_iterator_next(ptr)}
{
}
[[nodiscard]] ResultIterator
ResultIterator::fromResult(Result const& result)
{
return {cass_iterator_from_result(result)};
}
[[maybe_unused]] bool
ResultIterator::moveForward()
{
hasMore_ = cass_iterator_next(*this);
return hasMore_;
}
[[nodiscard]] bool
ResultIterator::hasMore() const
{
return hasMore_;
}
} // namespace Backend::Cassandra::detail

View File

@@ -0,0 +1,265 @@
//------------------------------------------------------------------------------
/*
This file is part of clio: https://github.com/XRPLF/clio
Copyright (c) 2023, the clio developers.
Permission to use, copy, modify, and distribute this software for any
purpose with or without fee is hereby granted, provided that the above
copyright notice and this permission notice appear in all copies.
THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES
WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF
MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR
ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES
WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN
ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF
OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.
*/
//==============================================================================
#pragma once
#include <backend/cassandra/impl/ManagedObject.h>
#include <backend/cassandra/impl/Tuple.h>
#include <util/Expected.h>
#include <ripple/basics/base_uint.h>
#include <ripple/protocol/AccountID.h>
#include <cassandra.h>
#include <compare>
#include <iterator>
#include <tuple>
namespace Backend::Cassandra::detail {
template <typename>
static constexpr bool unsupported_v = false;
template <typename Type>
inline Type
extractColumn(CassRow const* row, std::size_t idx)
{
using std::to_string;
Type output;
auto throwErrorIfNeeded = [](CassError rc, std::string_view label) {
if (rc != CASS_OK)
{
auto const tag = '[' + std::string{label} + ']';
throw std::logic_error(tag + ": " + cass_error_desc(rc));
}
};
using decayed_t = std::decay_t<Type>;
using uint_tuple_t = std::tuple<uint32_t, uint32_t>;
using uchar_vector_t = std::vector<unsigned char>;
if constexpr (std::is_same_v<decayed_t, ripple::uint256>)
{
cass_byte_t const* buf;
std::size_t bufSize;
auto const rc =
cass_value_get_bytes(cass_row_get_column(row, idx), &buf, &bufSize);
throwErrorIfNeeded(rc, "Extract ripple::uint256");
output = ripple::uint256::fromVoid(buf);
}
else if constexpr (std::is_same_v<decayed_t, ripple::AccountID>)
{
cass_byte_t const* buf;
std::size_t bufSize;
auto const rc =
cass_value_get_bytes(cass_row_get_column(row, idx), &buf, &bufSize);
throwErrorIfNeeded(rc, "Extract ripple::AccountID");
output = ripple::AccountID::fromVoid(buf);
}
else if constexpr (std::is_same_v<decayed_t, uchar_vector_t>)
{
cass_byte_t const* buf;
std::size_t bufSize;
auto const rc =
cass_value_get_bytes(cass_row_get_column(row, idx), &buf, &bufSize);
throwErrorIfNeeded(rc, "Extract vector<unsigned char>");
output = uchar_vector_t{buf, buf + bufSize};
}
else if constexpr (std::is_same_v<decayed_t, uint_tuple_t>)
{
auto const* tuple = cass_row_get_column(row, idx);
output = TupleIterator::fromTuple(tuple).extract<uint32_t, uint32_t>();
}
else if constexpr (std::is_convertible_v<decayed_t, std::string>)
{
char const* value;
std::size_t len;
auto const rc =
cass_value_get_string(cass_row_get_column(row, idx), &value, &len);
throwErrorIfNeeded(rc, "Extract string");
output = std::string{value, len};
}
else if constexpr (std::is_same_v<decayed_t, bool>)
{
cass_bool_t flag;
auto const rc =
cass_value_get_bool(cass_row_get_column(row, idx), &flag);
throwErrorIfNeeded(rc, "Extract bool");
output = flag ? true : false;
}
// clio only uses bigint (int64_t) so we convert any incoming type
else if constexpr (std::is_convertible_v<decayed_t, int64_t>)
{
int64_t out;
auto const rc =
cass_value_get_int64(cass_row_get_column(row, idx), &out);
throwErrorIfNeeded(rc, "Extract int64");
output = static_cast<decayed_t>(out);
}
else
{
// type not supported for extraction
static_assert(unsupported_v<decayed_t>);
}
return output;
}
struct Result : public ManagedObject<CassResult const>
{
/* implicit */ Result(CassResult const* ptr);
[[nodiscard]] std::size_t
numRows() const;
[[nodiscard]] bool
hasRows() const;
template <typename... RowTypes>
std::optional<std::tuple<RowTypes...>>
get() const requires(std::tuple_size<std::tuple<RowTypes...>>{} > 1)
{
// row managed internally by cassandra driver, hence no ManagedObject.
auto const* row = cass_result_first_row(*this);
if (row == nullptr)
return std::nullopt;
std::size_t idx = 0;
auto advanceId = [&idx]() { return idx++; };
return std::make_optional<std::tuple<RowTypes...>>(
{extractColumn<RowTypes>(row, advanceId())...});
}
template <typename RowType>
std::optional<RowType>
get() const
{
// row managed internally by cassandra driver, hence no ManagedObject.
auto const* row = cass_result_first_row(*this);
if (row == nullptr)
return std::nullopt;
return std::make_optional<RowType>(extractColumn<RowType>(row, 0));
}
};
class ResultIterator : public ManagedObject<CassIterator>
{
bool hasMore_ = false;
public:
/* implicit */ ResultIterator(CassIterator* ptr);
[[nodiscard]] static ResultIterator
fromResult(Result const& result);
[[maybe_unused]] bool
moveForward();
[[nodiscard]] bool
hasMore() const;
template <typename... RowTypes>
std::tuple<RowTypes...>
extractCurrentRow() const
{
// note: row is invalidated on each iteration.
// managed internally by cassandra driver, hence no ManagedObject.
auto const* row = cass_iterator_get_row(*this);
std::size_t idx = 0;
auto advanceId = [&idx]() { return idx++; };
return {extractColumn<RowTypes>(row, advanceId())...};
}
};
template <typename... Types>
class ResultExtractor
{
std::reference_wrapper<Result const> ref_;
public:
struct Sentinel
{
};
struct Iterator
{
using iterator_category = std::input_iterator_tag;
using difference_type = std::size_t; // rows count
using value_type = std::tuple<Types...>;
/* implicit */ Iterator(ResultIterator iterator)
: iterator_{std::move(iterator)}
{
}
Iterator(Iterator const&) = delete;
Iterator&
operator=(Iterator const&) = delete;
value_type
operator*() const
{
return iterator_.extractCurrentRow<Types...>();
}
value_type
operator->()
{
return iterator_.extractCurrentRow<Types...>();
}
Iterator&
operator++()
{
iterator_.moveForward();
return *this;
}
bool
operator==(Sentinel const&) const
{
return not iterator_.hasMore();
}
private:
ResultIterator iterator_;
};
ResultExtractor(Result const& result) : ref_{std::cref(result)}
{
}
Iterator
begin()
{
return ResultIterator::fromResult(ref_);
}
Sentinel
end()
{
return {};
}
};
} // namespace Backend::Cassandra::detail

View File

@@ -0,0 +1,97 @@
//------------------------------------------------------------------------------
/*
This file is part of clio: https://github.com/XRPLF/clio
Copyright (c) 2023, the clio developers.
Permission to use, copy, modify, and distribute this software for any
purpose with or without fee is hereby granted, provided that the above
copyright notice and this permission notice appear in all copies.
THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES
WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF
MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR
ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES
WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN
ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF
OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.
*/
//==============================================================================
#pragma once
#include <backend/cassandra/Handle.h>
#include <backend/cassandra/Types.h>
#include <log/Logger.h>
#include <util/Expected.h>
#include <boost/asio.hpp>
#include <algorithm>
#include <chrono>
#include <cmath>
namespace Backend::Cassandra::detail {
/**
* @brief A retry policy that employs exponential backoff
*/
class ExponentialBackoffRetryPolicy
{
clio::Logger log_{"Backend"};
boost::asio::steady_timer timer_;
uint32_t attempt_ = 0u;
public:
/**
* @brief Create a new retry policy instance with the io_context provided
*/
ExponentialBackoffRetryPolicy(boost::asio::io_context& ioc) : timer_{ioc}
{
}
/**
* @brief Computes next retry delay and returns true unconditionally
*
* @param err The cassandra error that triggered the retry
*/
[[nodiscard]] bool
shouldRetry([[maybe_unused]] CassandraError err)
{
auto const delay = calculateDelay(attempt_);
log_.error() << "Cassandra write error: " << err << ", current retries "
<< attempt_ << ", retrying in " << delay.count()
<< " milliseconds";
return true; // keep retrying forever
}
/**
* @brief Schedules next retry
*
* @param fn The callable to execute
*/
template <typename Fn>
void
retry(Fn&& fn)
{
timer_.expires_after(calculateDelay(attempt_++));
timer_.async_wait(
[fn = std::move(fn)]([[maybe_unused]] const auto& err) {
// todo: deal with cancellation (thru err)
fn();
});
}
/**
* @brief Calculates the wait time before attempting another retry
*/
std::chrono::milliseconds
calculateDelay(uint32_t attempt)
{
return std::chrono::milliseconds{
lround(std::pow(2, std::min(10u, attempt)))};
}
};
} // namespace Backend::Cassandra::detail

View File

@@ -0,0 +1,40 @@
//------------------------------------------------------------------------------
/*
This file is part of clio: https://github.com/XRPLF/clio
Copyright (c) 2023, the clio developers.
Permission to use, copy, modify, and distribute this software for any
purpose with or without fee is hereby granted, provided that the above
copyright notice and this permission notice appear in all copies.
THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES
WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF
MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR
ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES
WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN
ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF
OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.
*/
//==============================================================================
#pragma once
#include <backend/cassandra/impl/ManagedObject.h>
#include <cassandra.h>
namespace Backend::Cassandra::detail {
class Session : public ManagedObject<CassSession>
{
static constexpr auto deleter = [](CassSession* ptr) {
cass_session_free(ptr);
};
public:
Session() : ManagedObject{cass_session_new(), deleter}
{
}
};
} // namespace Backend::Cassandra::detail

View File

@@ -0,0 +1,41 @@
//------------------------------------------------------------------------------
/*
This file is part of clio: https://github.com/XRPLF/clio
Copyright (c) 2023, the clio developers.
Permission to use, copy, modify, and distribute this software for any
purpose with or without fee is hereby granted, provided that the above
copyright notice and this permission notice appear in all copies.
THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES
WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF
MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR
ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES
WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN
ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF
OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.
*/
//==============================================================================
#include <backend/cassandra/impl/SslContext.h>
namespace {
static constexpr auto contextDeleter = [](CassSsl* ptr) { cass_ssl_free(ptr); };
} // namespace
namespace Backend::Cassandra::detail {
SslContext::SslContext(std::string const& certificate)
: ManagedObject{cass_ssl_new(), contextDeleter}
{
cass_ssl_set_verify_flags(*this, CASS_SSL_VERIFY_NONE);
if (auto const rc = cass_ssl_add_trusted_cert(*this, certificate.c_str());
rc != CASS_OK)
{
throw std::runtime_error(
std::string{"Error setting Cassandra SSL Context: "} +
cass_error_desc(rc));
}
}
} // namespace Backend::Cassandra::detail

View File

@@ -0,0 +1,35 @@
//------------------------------------------------------------------------------
/*
This file is part of clio: https://github.com/XRPLF/clio
Copyright (c) 2023, the clio developers.
Permission to use, copy, modify, and distribute this software for any
purpose with or without fee is hereby granted, provided that the above
copyright notice and this permission notice appear in all copies.
THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES
WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF
MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR
ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES
WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN
ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF
OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.
*/
//==============================================================================
#pragma once
#include <backend/cassandra/impl/ManagedObject.h>
#include <cassandra.h>
#include <string>
namespace Backend::Cassandra::detail {
struct SslContext : public ManagedObject<CassSsl>
{
explicit SslContext(std::string const& certificate);
};
} // namespace Backend::Cassandra::detail

View File

@@ -0,0 +1,177 @@
//------------------------------------------------------------------------------
/*
This file is part of clio: https://github.com/XRPLF/clio
Copyright (c) 2023, the clio developers.
Permission to use, copy, modify, and distribute this software for any
purpose with or without fee is hereby granted, provided that the above
copyright notice and this permission notice appear in all copies.
THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES
WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF
MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR
ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES
WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN
ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF
OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.
*/
//==============================================================================
#pragma once
#include <backend/cassandra/Types.h>
#include <backend/cassandra/impl/ManagedObject.h>
#include <backend/cassandra/impl/Tuple.h>
#include <util/Expected.h>
#include <ripple/basics/base_uint.h>
#include <ripple/protocol/STAccount.h>
#include <cassandra.h>
#include <fmt/core.h>
#include <chrono>
#include <compare>
#include <iterator>
namespace Backend::Cassandra::detail {
class Statement : public ManagedObject<CassStatement>
{
static constexpr auto deleter = [](CassStatement* ptr) {
cass_statement_free(ptr);
};
template <typename>
static constexpr bool unsupported_v = false;
public:
/**
* @brief Construct a new statement with optionally provided arguments
*
* Note: it's up to the user to make sure the bound parameters match
* the format of the query (e.g. amount of '?' matches count of args).
*/
template <typename... Args>
explicit Statement(std::string_view query, Args&&... args)
: ManagedObject{
cass_statement_new(query.data(), sizeof...(args)),
deleter}
{
cass_statement_set_consistency(*this, CASS_CONSISTENCY_QUORUM);
cass_statement_set_is_idempotent(*this, cass_true);
bind<Args...>(std::forward<Args>(args)...);
}
/* implicit */ Statement(CassStatement* ptr) : ManagedObject{ptr, deleter}
{
cass_statement_set_consistency(*this, CASS_CONSISTENCY_QUORUM);
cass_statement_set_is_idempotent(*this, cass_true);
}
Statement(Statement&&) = default;
template <typename... Args>
void
bind(Args&&... args) const
{
std::size_t idx = 0;
(this->bindAt<Args>(idx++, std::forward<Args>(args)), ...);
}
template <typename Type>
void
bindAt(std::size_t const idx, Type&& value) const
{
using std::to_string;
auto throwErrorIfNeeded = [idx](CassError rc, std::string_view label) {
if (rc != CASS_OK)
throw std::logic_error(fmt::format(
"[{}] at idx {}: {}", label, idx, cass_error_desc(rc)));
};
auto bindBytes = [this, idx](auto const* data, size_t size) {
return cass_statement_bind_bytes(
*this, idx, static_cast<cass_byte_t const*>(data), size);
};
using decayed_t = std::decay_t<Type>;
using uchar_vec_t = std::vector<unsigned char>;
using uint_tuple_t = std::tuple<uint32_t, uint32_t>;
if constexpr (std::is_same_v<decayed_t, ripple::uint256>)
{
auto const rc = bindBytes(value.data(), value.size());
throwErrorIfNeeded(rc, "Bind ripple::uint256");
}
else if constexpr (std::is_same_v<decayed_t, ripple::AccountID>)
{
auto const rc = bindBytes(value.data(), value.size());
throwErrorIfNeeded(rc, "Bind ripple::AccountID");
}
else if constexpr (std::is_same_v<decayed_t, uchar_vec_t>)
{
auto const rc = bindBytes(value.data(), value.size());
throwErrorIfNeeded(rc, "Bind vector<unsigned char>");
}
else if constexpr (std::is_convertible_v<decayed_t, std::string>)
{
// reinterpret_cast is needed here :'(
auto const rc = bindBytes(
reinterpret_cast<unsigned char const*>(value.data()),
value.size());
throwErrorIfNeeded(rc, "Bind string (as bytes)");
}
else if constexpr (std::is_same_v<decayed_t, uint_tuple_t>)
{
auto const rc =
cass_statement_bind_tuple(*this, idx, Tuple{std::move(value)});
throwErrorIfNeeded(rc, "Bind tuple<uint32, uint32>");
}
else if constexpr (std::is_same_v<decayed_t, bool>)
{
auto const rc = cass_statement_bind_bool(
*this, idx, value ? cass_true : cass_false);
throwErrorIfNeeded(rc, "Bind bool");
}
else if constexpr (std::is_same_v<decayed_t, Limit>)
{
auto const rc = cass_statement_bind_int32(*this, idx, value.limit);
throwErrorIfNeeded(rc, "Bind limit (int32)");
}
// clio only uses bigint (int64_t) so we convert any incoming type
else if constexpr (std::is_convertible_v<decayed_t, int64_t>)
{
auto const rc = cass_statement_bind_int64(*this, idx, value);
throwErrorIfNeeded(rc, "Bind int64");
}
else
{
// type not supported for binding
static_assert(unsupported_v<decayed_t>);
}
}
};
class PreparedStatement : public ManagedObject<CassPrepared const>
{
static constexpr auto deleter = [](CassPrepared const* ptr) {
cass_prepared_free(ptr);
};
public:
/* implicit */ PreparedStatement(CassPrepared const* ptr)
: ManagedObject{ptr, deleter}
{
}
template <typename... Args>
Statement
bind(Args&&... args) const
{
Statement statement = cass_prepared_bind(*this);
statement.bind<Args...>(std::forward<Args>(args)...);
return statement;
}
};
} // namespace Backend::Cassandra::detail

View File

@@ -0,0 +1,48 @@
//------------------------------------------------------------------------------
/*
This file is part of clio: https://github.com/XRPLF/clio
Copyright (c) 2023, the clio developers.
Permission to use, copy, modify, and distribute this software for any
purpose with or without fee is hereby granted, provided that the above
copyright notice and this permission notice appear in all copies.
THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES
WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF
MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR
ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES
WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN
ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF
OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.
*/
//==============================================================================
#include <backend/cassandra/impl/Tuple.h>
namespace {
static constexpr auto tupleDeleter = [](CassTuple* ptr) {
cass_tuple_free(ptr);
};
static constexpr auto tupleIteratorDeleter = [](CassIterator* ptr) {
cass_iterator_free(ptr);
};
} // namespace
namespace Backend::Cassandra::detail {
/* implicit */ Tuple::Tuple(CassTuple* ptr) : ManagedObject{ptr, tupleDeleter}
{
}
/* implicit */ TupleIterator::TupleIterator(CassIterator* ptr)
: ManagedObject{ptr, tupleIteratorDeleter}
{
}
[[nodiscard]] TupleIterator
TupleIterator::fromTuple(CassValue const* value)
{
return {cass_iterator_from_tuple(value)};
}
} // namespace Backend::Cassandra::detail

View File

@@ -0,0 +1,159 @@
//------------------------------------------------------------------------------
/*
This file is part of clio: https://github.com/XRPLF/clio
Copyright (c) 2023, the clio developers.
Permission to use, copy, modify, and distribute this software for any
purpose with or without fee is hereby granted, provided that the above
copyright notice and this permission notice appear in all copies.
THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES
WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF
MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR
ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES
WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN
ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF
OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.
*/
//==============================================================================
#pragma once
#include <backend/cassandra/impl/ManagedObject.h>
#include <cassandra.h>
#include <functional>
#include <string>
#include <string_view>
#include <tuple>
namespace Backend::Cassandra::detail {
class Tuple : public ManagedObject<CassTuple>
{
static constexpr auto deleter = [](CassTuple* ptr) {
cass_tuple_free(ptr);
};
template <typename>
static constexpr bool unsupported_v = false;
public:
/* implicit */ Tuple(CassTuple* ptr);
template <typename... Types>
explicit Tuple(std::tuple<Types...>&& value)
: ManagedObject{
cass_tuple_new(std::tuple_size<std::tuple<Types...>>{}),
deleter}
{
std::apply(
std::bind_front(&Tuple::bind<Types...>, this), std::move(value));
}
template <typename... Args>
void
bind(Args&&... args) const
{
std::size_t idx = 0;
(this->bindAt<Args>(idx++, std::forward<Args>(args)), ...);
}
template <typename Type>
void
bindAt(std::size_t const idx, Type&& value) const
{
using std::to_string;
auto throwErrorIfNeeded = [idx](CassError rc, std::string_view label) {
if (rc != CASS_OK)
{
auto const tag = '[' + std::string{label} + ']';
throw std::logic_error(
tag + " at idx " + to_string(idx) + ": " +
cass_error_desc(rc));
}
};
using decayed_t = std::decay_t<Type>;
if constexpr (std::is_same_v<decayed_t, bool>)
{
auto const rc =
cass_tuple_set_bool(*this, idx, value ? cass_true : cass_false);
throwErrorIfNeeded(rc, "Bind bool");
}
// clio only uses bigint (int64_t) so we convert any incoming type
else if constexpr (std::is_convertible_v<decayed_t, int64_t>)
{
auto const rc = cass_tuple_set_int64(*this, idx, value);
throwErrorIfNeeded(rc, "Bind int64");
}
else
{
// type not supported for binding
static_assert(unsupported_v<decayed_t>);
}
}
};
class TupleIterator : public ManagedObject<CassIterator>
{
template <typename>
static constexpr bool unsupported_v = false;
public:
/* implicit */ TupleIterator(CassIterator* ptr);
[[nodiscard]] static TupleIterator
fromTuple(CassValue const* value);
template <typename... Types>
[[nodiscard]] std::tuple<Types...>
extract() const
{
return {extractNext<Types>()...};
}
private:
template <typename Type>
Type
extractNext() const
{
using std::to_string;
Type output;
if (not cass_iterator_next(*this))
throw std::logic_error(
"Could not extract next value from tuple iterator");
auto throwErrorIfNeeded = [](CassError rc, std::string_view label) {
if (rc != CASS_OK)
{
auto const tag = '[' + std::string{label} + ']';
throw std::logic_error(tag + ": " + cass_error_desc(rc));
}
};
using decayed_t = std::decay_t<Type>;
// clio only uses bigint (int64_t) so we convert any incoming type
if constexpr (std::is_convertible_v<decayed_t, int64_t>)
{
int64_t out;
auto const rc =
cass_value_get_int64(cass_iterator_get_value(*this), &out);
throwErrorIfNeeded(rc, "Extract int64 from tuple");
output = static_cast<decayed_t>(out);
}
else
{
// type not supported for extraction
static_assert(unsupported_v<decayed_t>);
}
return output;
}
};
} // namespace Backend::Cassandra::detail

View File

@@ -349,6 +349,8 @@ private:
[[nodiscard]] Return
checkedAs(key_type key, boost::json::value const& value) const
{
using boost::json::value_to;
auto has_error = false;
if constexpr (std::is_same_v<Return, bool>)
{
@@ -379,7 +381,7 @@ private:
std::string{to_string(value.kind())} +
"' in JSON but requested '" + detail::typeName<Return>() + "'");
return boost::json::value_to<Return>(value);
return value_to<Return>(value);
}
std::optional<boost::json::value>

View File

@@ -659,10 +659,13 @@ public:
call(stub, cq);
}
log_.trace() << "Writing objects";
auto const numObjects = cur_->ledger_objects().objects_size();
log_.debug() << "Writing " << numObjects << " objects";
std::vector<Backend::LedgerObject> cacheUpdates;
cacheUpdates.reserve(cur_->ledger_objects().objects_size());
for (int i = 0; i < cur_->ledger_objects().objects_size(); ++i)
cacheUpdates.reserve(numObjects);
for (int i = 0; i < numObjects; ++i)
{
auto& obj = *(cur_->mutable_ledger_objects()->mutable_objects(i));
if (!more && nextPrefix_ != 0x00)
@@ -691,7 +694,8 @@ public:
}
backend.cache().update(
cacheUpdates, request_.ledger().sequence(), cacheOnly);
log_.trace() << "Wrote objects";
log_.debug() << "Wrote " << numObjects
<< " objects. Got more: " << (more ? "YES" : "NO");
return more ? CallStatus::MORE : CallStatus::DONE;
}

View File

@@ -138,8 +138,7 @@ Unexpected(E (&)[N]) -> Unexpected<E const*>;
// Definition of Expected. All of the machinery comes from boost::result.
template <class T, class E>
class [[nodiscard]] Expected
: private boost::outcome_v2::result<T, E, detail::throw_policy>
class Expected : private boost::outcome_v2::result<T, E, detail::throw_policy>
{
using Base = boost::outcome_v2::result<T, E, detail::throw_policy>;

View File

@@ -0,0 +1,208 @@
//------------------------------------------------------------------------------
/*
This file is part of clio: https://github.com/XRPLF/clio
Copyright (c) 2023, the clio developers.
Permission to use, copy, modify, and distribute this software for any
purpose with or without fee is hereby granted, provided that the above
copyright notice and this permission notice appear in all copies.
THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES
WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF
MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR
ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES
WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN
ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF
OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.
*/
//==============================================================================
#include <backend/cassandra/impl/FakesAndMocks.h>
#include <util/Fixtures.h>
#include <backend/cassandra/Error.h>
#include <backend/cassandra/impl/AsyncExecutor.h>
#include <gmock/gmock.h>
using namespace Backend::Cassandra;
using namespace Backend::Cassandra::detail;
using namespace testing;
class BackendCassandraAsyncExecutorTest : public SyncAsioContextTest
{
};
TEST_F(BackendCassandraAsyncExecutorTest, CompletionCalledOnSuccess)
{
auto statement = FakeStatement{};
auto handle = MockHandle{};
ON_CALL(
handle,
asyncExecute(
An<FakeStatement const&>(),
An<std::function<void(FakeResultOrError)>&&>()))
.WillByDefault([this](auto const&, auto&& cb) {
ctx.post([cb = std::move(cb)]() { cb({}); });
return FakeFutureWithCallback{};
});
EXPECT_CALL(
handle,
asyncExecute(
An<FakeStatement const&>(),
An<std::function<void(FakeResultOrError)>&&>()))
.Times(1);
auto called = std::atomic_bool{false};
auto work = std::optional<boost::asio::io_context::work>{ctx};
AsyncExecutor<FakeStatement, MockHandle>::run(
ctx, handle, statement, [&called, &work](auto&&) {
called = true;
work.reset();
});
ctx.run();
ASSERT_TRUE(called);
}
TEST_F(
BackendCassandraAsyncExecutorTest,
ExecutedMultipleTimesByRetryPolicyOnMainThread)
{
auto callCount = std::atomic_int{0};
auto statement = FakeStatement{};
auto handle = MockHandle{};
// emulate successfull execution after some attempts
ON_CALL(
handle,
asyncExecute(
An<FakeStatement const&>(),
An<std::function<void(FakeResultOrError)>&&>()))
.WillByDefault([&callCount](auto const&, auto&& cb) {
++callCount;
if (callCount >= 3)
cb({});
else
cb({CassandraError{
"timeout", CASS_ERROR_LIB_REQUEST_TIMED_OUT}});
return FakeFutureWithCallback{};
});
EXPECT_CALL(
handle,
asyncExecute(
An<FakeStatement const&>(),
An<std::function<void(FakeResultOrError)>&&>()))
.Times(3);
auto called = std::atomic_bool{false};
auto work = std::optional<boost::asio::io_context::work>{ctx};
AsyncExecutor<FakeStatement, MockHandle>::run(
ctx, handle, statement, [&called, &work](auto&&) {
called = true;
work.reset();
});
ctx.run();
ASSERT_TRUE(callCount >= 3);
ASSERT_TRUE(called);
}
TEST_F(
BackendCassandraAsyncExecutorTest,
ExecutedMultipleTimesByRetryPolicyOnOtherThread)
{
auto callCount = std::atomic_int{0};
auto statement = FakeStatement{};
auto handle = MockHandle{};
auto threadedCtx = boost::asio::io_context{};
auto work = std::optional<boost::asio::io_context::work>{threadedCtx};
auto thread = std::thread{[&threadedCtx] { threadedCtx.run(); }};
// emulate successfull execution after some attempts
ON_CALL(
handle,
asyncExecute(
An<FakeStatement const&>(),
An<std::function<void(FakeResultOrError)>&&>()))
.WillByDefault([&callCount](auto const&, auto&& cb) {
++callCount;
if (callCount >= 3)
cb({});
else
cb({CassandraError{
"timeout", CASS_ERROR_LIB_REQUEST_TIMED_OUT}});
return FakeFutureWithCallback{};
});
EXPECT_CALL(
handle,
asyncExecute(
An<FakeStatement const&>(),
An<std::function<void(FakeResultOrError)>&&>()))
.Times(3);
auto called = std::atomic_bool{false};
auto work2 = std::optional<boost::asio::io_context::work>{ctx};
AsyncExecutor<FakeStatement, MockHandle>::run(
threadedCtx, handle, statement, [&called, &work, &work2](auto&&) {
called = true;
work.reset();
work2.reset();
});
ctx.run();
ASSERT_TRUE(callCount >= 3);
ASSERT_TRUE(called);
threadedCtx.stop();
thread.join();
}
TEST_F(
BackendCassandraAsyncExecutorTest,
CompletionCalledOnFailureAfterRetryCountExceeded)
{
auto statement = FakeStatement{};
auto handle = MockHandle{};
// FakeRetryPolicy returns false for shouldRetry in which case we should
// still call onComplete giving it whatever error we have raised internally.
ON_CALL(
handle,
asyncExecute(
An<FakeStatement const&>(),
An<std::function<void(FakeResultOrError)>&&>()))
.WillByDefault([](auto const&, auto&& cb) {
cb({CassandraError{
"not a timeout", CASS_ERROR_LIB_INTERNAL_ERROR}});
return FakeFutureWithCallback{};
});
EXPECT_CALL(
handle,
asyncExecute(
An<FakeStatement const&>(),
An<std::function<void(FakeResultOrError)>&&>()))
.Times(1);
auto called = std::atomic_bool{false};
auto work = std::optional<boost::asio::io_context::work>{ctx};
AsyncExecutor<FakeStatement, MockHandle, FakeRetryPolicy>::run(
ctx, handle, statement, [&called, &work](auto&& res) {
EXPECT_FALSE(res);
EXPECT_EQ(res.error().code(), CASS_ERROR_LIB_INTERNAL_ERROR);
EXPECT_EQ(res.error().message(), "not a timeout");
called = true;
work.reset();
});
ctx.run();
ASSERT_TRUE(called);
}

File diff suppressed because it is too large Load Diff

View File

@@ -0,0 +1,419 @@
//------------------------------------------------------------------------------
/*
This file is part of clio: https://github.com/XRPLF/clio
Copyright (c) 2023, the clio developers.
Permission to use, copy, modify, and distribute this software for any
purpose with or without fee is hereby granted, provided that the above
copyright notice and this permission notice appear in all copies.
THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES
WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF
MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR
ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES
WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN
ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF
OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.
*/
//==============================================================================
#include <util/Fixtures.h>
#include <backend/cassandra/Handle.h>
#include <gtest/gtest.h>
#include <semaphore>
using namespace clio;
using namespace std;
using namespace Backend::Cassandra;
namespace json = boost::json;
class BackendCassandraBaseTest : public NoLoggerFixture
{
protected:
Handle
createHandle(std::string_view contactPoints, std::string_view keyspace)
{
Handle handle{contactPoints};
EXPECT_TRUE(handle.connect());
std::string query = "CREATE KEYSPACE IF NOT EXISTS " +
std::string{keyspace} +
" WITH replication = {'class': "
"'SimpleStrategy', 'replication_factor': '1'} AND "
"durable_writes = "
"true";
EXPECT_TRUE(handle.execute(query));
EXPECT_TRUE(handle.reconnect(keyspace));
return handle;
}
void
dropKeyspace(Handle const& handle, std::string_view keyspace)
{
std::string query = "DROP KEYSPACE " + std::string{keyspace};
ASSERT_TRUE(handle.execute(query));
}
void
prepStringsTable(Handle const& handle)
{
auto const entries = std::vector<std::string>{
"first",
"second",
"third",
"fourth",
"fifth",
};
std::string q1 =
"CREATE TABLE IF NOT EXISTS strings "
"(hash blob PRIMARY KEY, sequence bigint) "
"WITH default_time_to_live = " +
to_string(5000);
auto f1 = handle.asyncExecute(q1);
if (auto const rc = f1.await(); not rc)
std::cout << "oops: " << rc.error() << '\n';
std::string q2 = "INSERT INTO strings (hash, sequence) VALUES (?, ?)";
auto insert = handle.prepare(q2);
std::vector<Statement> statements;
int64_t idx = 1000;
for (auto const& entry : entries)
statements.push_back(
insert.bind(entry, static_cast<int64_t>(idx++)));
EXPECT_EQ(statements.size(), entries.size());
EXPECT_TRUE(handle.execute(statements));
}
};
TEST_F(BackendCassandraBaseTest, ConnectionSuccess)
{
Handle handle{"127.0.0.1"};
auto f = handle.asyncConnect();
auto res = f.await();
ASSERT_TRUE(res);
}
TEST_F(BackendCassandraBaseTest, ConnectionFailFormat)
{
Handle handle{"127.0.0."};
auto f = handle.asyncConnect();
auto res = f.await();
ASSERT_FALSE(res);
EXPECT_EQ(
res.error(),
"No hosts available: Unable to connect to any contact points");
EXPECT_EQ(res.error().code(), CASS_ERROR_LIB_NO_HOSTS_AVAILABLE);
}
TEST_F(BackendCassandraBaseTest, ConnectionFailTimeout)
{
Settings settings;
settings.connectionTimeout = std::chrono::milliseconds{30};
settings.connectionInfo = Settings::ContactPoints{"127.0.0.2"};
Handle handle{settings};
auto f = handle.asyncConnect();
auto res = f.await();
ASSERT_FALSE(res);
// scylla and cassandra produce different text
EXPECT_TRUE(res.error().message().starts_with(
"No hosts available: Underlying connection error:"));
EXPECT_EQ(res.error().code(), CASS_ERROR_LIB_NO_HOSTS_AVAILABLE);
}
TEST_F(BackendCassandraBaseTest, FutureCallback)
{
Handle handle{"127.0.0.1"};
ASSERT_TRUE(handle.connect());
auto const statement =
handle.prepare("SELECT keyspace_name FROM system_schema.keyspaces")
.bind();
bool complete = false;
auto f = handle.asyncExecute(statement, [&complete](auto const res) {
complete = true;
EXPECT_TRUE(res.value().hasRows());
for (auto [ks] : extract<std::string>(res.value()))
std::cout << "keyspace: " << ks << '\n';
});
auto const res = f.await(); // callback should still be called
ASSERT_TRUE(res);
ASSERT_TRUE(complete);
}
TEST_F(BackendCassandraBaseTest, FutureCallbackSurviveMove)
{
Handle handle{"127.0.0.1"};
ASSERT_TRUE(handle.connect());
auto const statement =
handle.prepare("SELECT keyspace_name FROM system_schema.keyspaces")
.bind();
bool complete = false;
std::vector<FutureWithCallback> futures;
std::binary_semaphore sem{0};
futures.push_back(
handle.asyncExecute(statement, [&complete, &sem](auto const res) {
complete = true;
EXPECT_TRUE(res.value().hasRows());
for (auto [ks] : extract<std::string>(res.value()))
std::cout << "keyspace: " << ks << '\n';
sem.release();
}));
sem.acquire();
for (auto const& f : futures)
ASSERT_TRUE(f.await());
ASSERT_TRUE(complete);
}
TEST_F(BackendCassandraBaseTest, KeyspaceManipulation)
{
Handle handle{"127.0.0.1"};
std::string keyspace = "test_keyspace_manipulation";
{
auto const f = handle.asyncConnect(keyspace);
auto const rc = f.await();
ASSERT_FALSE(rc); // initially expecting the keyspace does not exist
}
{
auto const f = handle.asyncConnect();
auto const rc = f.await();
ASSERT_TRUE(rc); // expect that we can still connect without keyspace
}
{
std::string query = "CREATE KEYSPACE " + keyspace +
" WITH replication = {'class': "
"'SimpleStrategy', 'replication_factor': '1'} AND durable_writes = "
"true";
auto const f = handle.asyncExecute(query);
auto const rc = f.await();
ASSERT_TRUE(rc); // keyspace created
}
{
auto const rc = handle.reconnect(keyspace);
ASSERT_TRUE(rc); // connect to the keyspace we created earlier
}
{
auto const f = handle.asyncExecute("DROP KEYSPACE " + keyspace);
auto const rc = f.await();
ASSERT_TRUE(rc); // dropped the keyspace
}
{
auto const f = handle.asyncExecute("DROP KEYSPACE " + keyspace);
auto const rc = f.await();
ASSERT_FALSE(rc); // keyspace already does not exist
}
}
TEST_F(BackendCassandraBaseTest, CreateTableWithStrings)
{
using std::to_string;
auto const entries = std::vector<std::string>{
"first",
"second",
"third",
"fourth",
"fifth",
};
auto handle = createHandle("127.0.0.1", "test");
std::string q1 =
"CREATE TABLE IF NOT EXISTS strings "
"(hash blob PRIMARY KEY, sequence bigint) "
"WITH default_time_to_live = " +
to_string(5000);
auto f1 = handle.asyncExecute(q1);
if (auto const rc = f1.await(); not rc)
std::cout << "oops: " << rc.error() << '\n';
std::string q2 = "INSERT INTO strings (hash, sequence) VALUES (?, ?)";
auto insert = handle.prepare(q2);
// write data
{
std::vector<Future> futures;
int64_t idx = 1000;
for (auto const& entry : entries)
futures.push_back(handle.asyncExecute(
insert, entry, static_cast<int64_t>(idx++)));
ASSERT_EQ(futures.size(), entries.size());
for (auto const& f : futures)
{
auto const rc = f.await();
if (not rc)
std::cout << rc.error() << '\n';
ASSERT_TRUE(rc);
}
}
// read data back
{
auto const res = handle.execute("SELECT hash, sequence FROM strings");
ASSERT_TRUE(res);
if (not res)
std::cout << "oops: " << res.error() << '\n';
else
{
auto const& results = res.value();
auto const totalRows = results.numRows();
std::cout << "total rows: " << totalRows << '\n';
for (auto [hash, seq] : extract<std::string, int64_t>(results))
std::cout << "row: " << seq << ":" << hash << '\n';
}
}
// delete everything
{
auto const res = handle.execute("DROP TABLE strings");
ASSERT_TRUE(res);
if (not res)
std::cout << "oops: " << res.error() << '\n';
dropKeyspace(handle, "test");
}
}
TEST_F(BackendCassandraBaseTest, BatchInsert)
{
using std::to_string;
auto const entries = std::vector<std::string>{
"first",
"second",
"third",
"fourth",
"fifth",
};
auto handle = createHandle("127.0.0.1", "test");
std::string q1 =
"CREATE TABLE IF NOT EXISTS strings "
"(hash blob PRIMARY KEY, sequence bigint) "
"WITH default_time_to_live = " +
to_string(5000);
auto f1 = handle.asyncExecute(q1);
if (auto const rc = f1.await(); not rc)
std::cout << "oops: " << rc.error() << '\n';
std::string q2 = "INSERT INTO strings (hash, sequence) VALUES (?, ?)";
auto insert = handle.prepare(q2);
// write data in bulk
{
std::vector<Statement> statements;
int64_t idx = 1000;
for (auto const& entry : entries)
statements.push_back(
insert.bind(entry, static_cast<int64_t>(idx++)));
ASSERT_EQ(statements.size(), entries.size());
auto rc = handle.execute(statements);
if (not rc)
std::cout << rc.error() << '\n';
ASSERT_TRUE(rc);
}
// read data back
{
auto const res = handle.execute("SELECT hash, sequence FROM strings");
ASSERT_TRUE(res);
if (not res)
std::cout << "oops: " << res.error() << '\n';
else
{
auto const& results = res.value();
auto const totalRows = results.numRows();
std::cout << "total rows: " << totalRows << '\n';
for (auto [hash, seq] : extract<std::string, int64_t>(results))
std::cout << "row: " << seq << ":" << hash << '\n';
}
}
dropKeyspace(handle, "test");
}
TEST_F(BackendCassandraBaseTest, AlterTableAddColumn)
{
auto handle = createHandle("127.0.0.1", "test");
std::string q1 =
"CREATE TABLE IF NOT EXISTS strings "
"(hash blob PRIMARY KEY, sequence bigint) "
"WITH default_time_to_live = " +
to_string(5000);
ASSERT_TRUE(handle.execute(q1));
std::string update = "ALTER TABLE strings ADD tmp blob";
ASSERT_TRUE(handle.execute(update));
dropKeyspace(handle, "test");
}
TEST_F(BackendCassandraBaseTest, AlterTableMoveToNewTable)
{
auto handle = createHandle("127.0.0.1", "test");
prepStringsTable(handle);
std::string newTable =
"CREATE TABLE IF NOT EXISTS strings_v2 "
"(hash blob PRIMARY KEY, sequence bigint, tmp bigint) "
"WITH default_time_to_live = " +
to_string(5000);
ASSERT_TRUE(handle.execute(newTable));
// now migrate data; tmp column will just get the sequence number + 1 stored
std::vector<Statement> migrationStatements;
auto const migrationInsert = handle.prepare(
"INSERT INTO strings_v2 (hash, sequence, tmp) VALUES (?, ?, ?)");
auto const res = handle.execute("SELECT hash, sequence FROM strings");
ASSERT_TRUE(res);
auto const& results = res.value();
for (auto [hash, seq] : extract<std::string, int64_t>(results))
{
static_assert(std::is_same_v<decltype(hash), std::string>);
static_assert(std::is_same_v<decltype(seq), int64_t>);
migrationStatements.push_back(migrationInsert.bind(
hash, static_cast<int64_t>(seq), static_cast<int64_t>(seq + 1u)));
}
EXPECT_TRUE(handle.execute(migrationStatements));
// now let's read back the v2 table and compare
auto const resV2 = handle.execute("SELECT sequence, tmp FROM strings_v2");
EXPECT_TRUE(resV2);
auto const& resultsV2 = resV2.value();
EXPECT_EQ(results.numRows(), resultsV2.numRows());
for (auto [seq, tmp] : extract<int64_t, int64_t>(resultsV2))
{
static_assert(std::is_same_v<decltype(seq), int64_t>);
static_assert(std::is_same_v<decltype(tmp), int64_t>);
EXPECT_EQ(seq + 1, tmp);
}
dropKeyspace(handle, "test");
}

View File

@@ -0,0 +1,477 @@
//------------------------------------------------------------------------------
/*
This file is part of clio: https://github.com/XRPLF/clio
Copyright (c) 2023, the clio developers.
Permission to use, copy, modify, and distribute this software for any
purpose with or without fee is hereby granted, provided that the above
copyright notice and this permission notice appear in all copies.
THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES
WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF
MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR
ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES
WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN
ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF
OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.
*/
//==============================================================================
#include <backend/cassandra/impl/FakesAndMocks.h>
#include <util/Fixtures.h>
#include <backend/cassandra/impl/ExecutionStrategy.h>
#include <gtest/gtest.h>
using namespace Backend::Cassandra;
using namespace Backend::Cassandra::detail;
using namespace testing;
class BackendCassandraExecutionStrategyTest : public SyncAsioContextTest
{
};
TEST_F(BackendCassandraExecutionStrategyTest, ReadOneInCoroutineSuccessful)
{
auto handle = MockHandle{};
auto strat = DefaultExecutionStrategy{Settings{}, handle};
ON_CALL(
handle,
asyncExecute(
An<FakeStatement const&>(),
An<std::function<void(FakeResultOrError)>&&>()))
.WillByDefault([](auto const& statement, auto&& cb) {
cb({}); // pretend we got data
return FakeFutureWithCallback{};
});
EXPECT_CALL(
handle,
asyncExecute(
An<FakeStatement const&>(),
An<std::function<void(FakeResultOrError)>&&>()))
.Times(1);
auto called = std::atomic_bool{false};
auto work = std::optional<boost::asio::io_context::work>{ctx};
boost::asio::spawn(
ctx, [&work, &called, &strat](boost::asio::yield_context yield) {
auto statement = FakeStatement{};
strat.read(yield, statement);
called = true;
work.reset();
});
ctx.run();
ASSERT_TRUE(called);
}
TEST_F(
BackendCassandraExecutionStrategyTest,
ReadOneInCoroutineThrowsOnTimeoutFailure)
{
auto handle = MockHandle{};
auto strat = DefaultExecutionStrategy{Settings{}, handle};
ON_CALL(
handle,
asyncExecute(
An<FakeStatement const&>(),
An<std::function<void(FakeResultOrError)>&&>()))
.WillByDefault([](auto const&, auto&& cb) {
cb({}); // notify that item is ready
return FakeFutureWithCallback{FakeResultOrError{
CassandraError{"timeout", CASS_ERROR_LIB_REQUEST_TIMED_OUT}}};
});
EXPECT_CALL(
handle,
asyncExecute(
An<FakeStatement const&>(),
An<std::function<void(FakeResultOrError)>&&>()))
.Times(1);
auto called = std::atomic_bool{false};
auto work = std::optional<boost::asio::io_context::work>{ctx};
boost::asio::spawn(
ctx, [&work, &called, &strat](boost::asio::yield_context yield) {
auto statement = FakeStatement{};
EXPECT_THROW(strat.read(yield, statement), DatabaseTimeout);
called = true;
work.reset();
});
ctx.run();
ASSERT_TRUE(called);
}
TEST_F(
BackendCassandraExecutionStrategyTest,
ReadOneInCoroutineThrowsOnInvalidQueryFailure)
{
auto handle = MockHandle{};
auto strat = DefaultExecutionStrategy{Settings{}, handle};
ON_CALL(
handle,
asyncExecute(
An<FakeStatement const&>(),
An<std::function<void(FakeResultOrError)>&&>()))
.WillByDefault([](auto const&, auto&& cb) {
cb({}); // notify that item is ready
return FakeFutureWithCallback{FakeResultOrError{
CassandraError{"invalid", CASS_ERROR_SERVER_INVALID_QUERY}}};
});
EXPECT_CALL(
handle,
asyncExecute(
An<FakeStatement const&>(),
An<std::function<void(FakeResultOrError)>&&>()))
.Times(1);
auto called = std::atomic_bool{false};
auto work = std::optional<boost::asio::io_context::work>{ctx};
boost::asio::spawn(
ctx, [&work, &called, &strat](boost::asio::yield_context yield) {
auto statement = FakeStatement{};
EXPECT_THROW(strat.read(yield, statement), std::runtime_error);
called = true;
work.reset();
});
ctx.run();
ASSERT_TRUE(called);
}
TEST_F(BackendCassandraExecutionStrategyTest, ReadBatchInCoroutineSuccessful)
{
auto handle = MockHandle{};
auto strat = DefaultExecutionStrategy{Settings{}, handle};
ON_CALL(
handle,
asyncExecute(
An<std::vector<FakeStatement> const&>(),
An<std::function<void(FakeResultOrError)>&&>()))
.WillByDefault([](auto const& statements, auto&& cb) {
EXPECT_EQ(statements.size(), 3);
cb({}); // pretend we got data
return FakeFutureWithCallback{};
});
EXPECT_CALL(
handle,
asyncExecute(
An<std::vector<FakeStatement> const&>(),
An<std::function<void(FakeResultOrError)>&&>()))
.Times(1);
auto called = std::atomic_bool{false};
auto work = std::optional<boost::asio::io_context::work>{ctx};
boost::asio::spawn(
ctx, [&work, &called, &strat](boost::asio::yield_context yield) {
auto statements = std::vector<FakeStatement>(3);
strat.read(yield, statements);
called = true;
work.reset();
});
ctx.run();
ASSERT_TRUE(called);
}
TEST_F(
BackendCassandraExecutionStrategyTest,
ReadBatchInCoroutineThrowsOnTimeoutFailure)
{
auto handle = MockHandle{};
auto strat = DefaultExecutionStrategy{Settings{}, handle};
ON_CALL(
handle,
asyncExecute(
An<std::vector<FakeStatement> const&>(),
An<std::function<void(FakeResultOrError)>&&>()))
.WillByDefault([](auto const& statements, auto&& cb) {
EXPECT_EQ(statements.size(), 3);
cb({}); // notify that item is ready
return FakeFutureWithCallback{FakeResultOrError{
CassandraError{"timeout", CASS_ERROR_LIB_REQUEST_TIMED_OUT}}};
});
EXPECT_CALL(
handle,
asyncExecute(
An<std::vector<FakeStatement> const&>(),
An<std::function<void(FakeResultOrError)>&&>()))
.Times(1);
auto called = std::atomic_bool{false};
auto work = std::optional<boost::asio::io_context::work>{ctx};
boost::asio::spawn(
ctx, [&work, &called, &strat](boost::asio::yield_context yield) {
auto statements = std::vector<FakeStatement>(3);
EXPECT_THROW(strat.read(yield, statements), DatabaseTimeout);
called = true;
work.reset();
});
ctx.run();
ASSERT_TRUE(called);
}
TEST_F(
BackendCassandraExecutionStrategyTest,
ReadBatchInCoroutineThrowsOnInvalidQueryFailure)
{
auto handle = MockHandle{};
auto strat = DefaultExecutionStrategy{Settings{}, handle};
ON_CALL(
handle,
asyncExecute(
An<std::vector<FakeStatement> const&>(),
An<std::function<void(FakeResultOrError)>&&>()))
.WillByDefault([](auto const& statements, auto&& cb) {
EXPECT_EQ(statements.size(), 3);
cb({}); // notify that item is ready
return FakeFutureWithCallback{FakeResultOrError{
CassandraError{"invalid", CASS_ERROR_SERVER_INVALID_QUERY}}};
});
EXPECT_CALL(
handle,
asyncExecute(
An<std::vector<FakeStatement> const&>(),
An<std::function<void(FakeResultOrError)>&&>()))
.Times(1);
auto called = std::atomic_bool{false};
auto work = std::optional<boost::asio::io_context::work>{ctx};
boost::asio::spawn(
ctx, [&work, &called, &strat](boost::asio::yield_context yield) {
auto statements = std::vector<FakeStatement>(3);
EXPECT_THROW(strat.read(yield, statements), std::runtime_error);
called = true;
work.reset();
});
ctx.run();
ASSERT_TRUE(called);
}
TEST_F(
BackendCassandraExecutionStrategyTest,
ReadBatchInCoroutineMarksBusyIfRequestsOutstandingExceeded)
{
auto handle = MockHandle{};
auto settings = Settings{};
settings.maxReadRequestsOutstanding = 2;
auto strat = DefaultExecutionStrategy{settings, handle};
ON_CALL(
handle,
asyncExecute(
An<std::vector<FakeStatement> const&>(),
An<std::function<void(FakeResultOrError)>&&>()))
.WillByDefault([&strat](auto const& statements, auto&& cb) {
EXPECT_EQ(statements.size(), 3);
EXPECT_TRUE(strat.isTooBusy()); // 2 was the limit, we sent 3
cb({}); // notify that item is ready
return FakeFutureWithCallback{};
});
EXPECT_CALL(
handle,
asyncExecute(
An<std::vector<FakeStatement> const&>(),
An<std::function<void(FakeResultOrError)>&&>()))
.Times(1);
auto called = std::atomic_bool{false};
auto work = std::optional<boost::asio::io_context::work>{ctx};
boost::asio::spawn(
ctx, [&work, &called, &strat](boost::asio::yield_context yield) {
EXPECT_FALSE(strat.isTooBusy()); // 2 was the limit, 0 atm
auto statements = std::vector<FakeStatement>(3);
strat.read(yield, statements);
EXPECT_FALSE(
strat.isTooBusy()); // after read completes it's 0 again
called = true;
work.reset();
});
ctx.run();
ASSERT_TRUE(called);
}
TEST_F(BackendCassandraExecutionStrategyTest, ReadEachInCoroutineSuccessful)
{
auto handle = MockHandle{};
auto strat = DefaultExecutionStrategy{Settings{}, handle};
ON_CALL(
handle,
asyncExecute(
An<FakeStatement const&>(),
An<std::function<void(FakeResultOrError)>&&>()))
.WillByDefault([](auto const&, auto&& cb) {
cb({}); // pretend we got data
return FakeFutureWithCallback{};
});
EXPECT_CALL(
handle,
asyncExecute(
An<FakeStatement const&>(),
An<std::function<void(FakeResultOrError)>&&>()))
.Times(3); // once per statement
auto called = std::atomic_bool{false};
auto work = std::optional<boost::asio::io_context::work>{ctx};
boost::asio::spawn(
ctx, [&work, &called, &strat](boost::asio::yield_context yield) {
auto statements = std::vector<FakeStatement>(3);
auto res = strat.readEach(yield, statements);
EXPECT_EQ(res.size(), statements.size());
called = true;
work.reset();
});
ctx.run();
ASSERT_TRUE(called);
}
TEST_F(
BackendCassandraExecutionStrategyTest,
ReadEachInCoroutineThrowsOnFailure)
{
auto handle = MockHandle{};
auto strat = DefaultExecutionStrategy{Settings{}, handle};
auto callCount = std::atomic_int{0};
ON_CALL(
handle,
asyncExecute(
An<FakeStatement const&>(),
An<std::function<void(FakeResultOrError)>&&>()))
.WillByDefault([&callCount](auto const&, auto&& cb) {
if (callCount == 1) // error happens on one of the entries
cb({CassandraError{
"invalid data", CASS_ERROR_LIB_INVALID_DATA}});
else
cb({}); // pretend we got data
++callCount;
return FakeFutureWithCallback{};
});
EXPECT_CALL(
handle,
asyncExecute(
An<FakeStatement const&>(),
An<std::function<void(FakeResultOrError)>&&>()))
.Times(3); // once per statement
auto called = std::atomic_bool{false};
auto work = std::optional<boost::asio::io_context::work>{ctx};
boost::asio::spawn(
ctx, [&work, &called, &strat](boost::asio::yield_context yield) {
auto statements = std::vector<FakeStatement>(3);
EXPECT_THROW(strat.readEach(yield, statements), DatabaseTimeout);
called = true;
work.reset();
});
ctx.run();
ASSERT_TRUE(called);
}
TEST_F(BackendCassandraExecutionStrategyTest, WriteSyncFirstTrySuccessful)
{
auto handle = MockHandle{};
auto strat = DefaultExecutionStrategy{Settings{}, handle};
ON_CALL(handle, execute(An<FakeStatement const&>()))
.WillByDefault([](auto const&) { return FakeResultOrError{}; });
EXPECT_CALL(
handle,
execute(An<FakeStatement const&>()))
.Times(1); // first one will succeed
EXPECT_TRUE(strat.writeSync({}));
}
TEST_F(BackendCassandraExecutionStrategyTest, WriteSyncRetrySuccessful)
{
auto handle = MockHandle{};
auto strat = DefaultExecutionStrategy{Settings{}, handle};
auto callCount = 0;
ON_CALL(handle, execute(An<FakeStatement const&>()))
.WillByDefault([&callCount](auto const&) {
if (callCount++ == 1)
return FakeResultOrError{};
return FakeResultOrError{
CassandraError{"invalid data", CASS_ERROR_LIB_INVALID_DATA}};
});
EXPECT_CALL(
handle,
execute(An<FakeStatement const&>()))
.Times(2); // first one will fail, second will succeed
EXPECT_TRUE(strat.writeSync({}));
}
TEST_F(BackendCassandraExecutionStrategyTest, WriteMultipleAndCallSyncSucceeds)
{
auto handle = MockHandle{};
auto strat = DefaultExecutionStrategy{Settings{}, handle};
auto totalRequests = 1024u;
auto callCount = std::atomic_uint{0u};
auto work = std::optional<boost::asio::io_context::work>{ctx};
auto thread = std::thread{[this]() { ctx.run(); }};
ON_CALL(
handle,
asyncExecute(
An<std::vector<FakeStatement> const&>(),
An<std::function<void(FakeResultOrError)>&&>()))
.WillByDefault([this, &callCount](auto const&, auto&& cb) {
// run on thread to emulate concurrency model of real asyncExecute
boost::asio::post(ctx, [&callCount, cb = std::move(cb)] {
++callCount;
cb({}); // pretend we got data
});
return FakeFutureWithCallback{};
});
EXPECT_CALL(
handle,
asyncExecute(
An<std::vector<FakeStatement> const&>(),
An<std::function<void(FakeResultOrError)>&&>()))
.Times(totalRequests); // one per write call
auto statements = std::vector<FakeStatement>(16);
for (auto i = 0u; i < totalRequests; ++i)
strat.write(statements);
strat.sync(); // make sure all above writes are finished
ASSERT_EQ(callCount, totalRequests); // all requests should finish
work.reset();
thread.join();
}

View File

@@ -0,0 +1,87 @@
//------------------------------------------------------------------------------
/*
This file is part of clio: https://github.com/XRPLF/clio
Copyright (c) 2023, the clio developers.
Permission to use, copy, modify, and distribute this software for any
purpose with or without fee is hereby granted, provided that the above
copyright notice and this permission notice appear in all copies.
THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES
WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF
MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR
ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES
WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN
ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF
OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.
*/
//==============================================================================
#include <util/Fixtures.h>
#include <backend/cassandra/Error.h>
#include <backend/cassandra/impl/RetryPolicy.h>
#include <gtest/gtest.h>
using namespace Backend::Cassandra;
using namespace Backend::Cassandra::detail;
using namespace testing;
class BackendCassandraRetryPolicyTest : public SyncAsioContextTest
{
};
TEST_F(BackendCassandraRetryPolicyTest, ShouldRetryAlwaysTrue)
{
auto retryPolicy = ExponentialBackoffRetryPolicy{ctx};
EXPECT_TRUE(retryPolicy.shouldRetry(
CassandraError{"timeout", CASS_ERROR_LIB_REQUEST_TIMED_OUT}));
EXPECT_TRUE(retryPolicy.shouldRetry(
CassandraError{"invalid data", CASS_ERROR_LIB_INVALID_DATA}));
EXPECT_TRUE(retryPolicy.shouldRetry(
CassandraError{"invalid query", CASS_ERROR_SERVER_INVALID_QUERY}));
// this policy actually always returns true
auto const err = CassandraError{"ok", CASS_OK};
for (auto i = 0; i < 1024; ++i)
{
EXPECT_TRUE(retryPolicy.shouldRetry(err));
}
}
TEST_F(BackendCassandraRetryPolicyTest, CheckComputedBackoffDelayIsCorrect)
{
auto retryPolicy = ExponentialBackoffRetryPolicy{ctx};
EXPECT_EQ(retryPolicy.calculateDelay(0).count(), 1);
EXPECT_EQ(retryPolicy.calculateDelay(1).count(), 2);
EXPECT_EQ(retryPolicy.calculateDelay(2).count(), 4);
EXPECT_EQ(retryPolicy.calculateDelay(3).count(), 8);
EXPECT_EQ(retryPolicy.calculateDelay(4).count(), 16);
EXPECT_EQ(retryPolicy.calculateDelay(5).count(), 32);
EXPECT_EQ(retryPolicy.calculateDelay(6).count(), 64);
EXPECT_EQ(retryPolicy.calculateDelay(7).count(), 128);
EXPECT_EQ(retryPolicy.calculateDelay(8).count(), 256);
EXPECT_EQ(retryPolicy.calculateDelay(9).count(), 512);
EXPECT_EQ(retryPolicy.calculateDelay(10).count(), 1024);
EXPECT_EQ(
retryPolicy.calculateDelay(11).count(),
1024); // 10 is max, same after that
}
TEST_F(BackendCassandraRetryPolicyTest, RetryCorrectlyExecuted)
{
auto callCount = std::atomic_int{0};
auto work = std::optional<boost::asio::io_context::work>{ctx};
auto retryPolicy = ExponentialBackoffRetryPolicy{ctx};
retryPolicy.retry([&callCount]() { ++callCount; });
retryPolicy.retry([&callCount]() { ++callCount; });
retryPolicy.retry([&callCount, &work]() {
++callCount;
work.reset();
});
ctx.run();
ASSERT_EQ(callCount, 3);
}

View File

@@ -0,0 +1,118 @@
//------------------------------------------------------------------------------
/*
This file is part of clio: https://github.com/XRPLF/clio
Copyright (c) 2023, the clio developers.
Permission to use, copy, modify, and distribute this software for any
purpose with or without fee is hereby granted, provided that the above
copyright notice and this permission notice appear in all copies.
THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES
WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF
MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR
ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES
WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN
ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF
OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.
*/
//==============================================================================
#include <util/Fixtures.h>
#include <util/TmpFile.h>
#include <backend/cassandra/SettingsProvider.h>
#include <config/Config.h>
#include <boost/json/parse.hpp>
#include <gtest/gtest.h>
#include <thread>
#include <variant>
using namespace clio;
using namespace std;
namespace json = boost::json;
using namespace Backend::Cassandra;
class SettingsProviderTest : public NoLoggerFixture
{
};
TEST_F(SettingsProviderTest, Defaults)
{
Config cfg{json::parse(R"({"contact_points": "127.0.0.1"})")};
SettingsProvider provider{cfg};
auto const settings = provider.getSettings();
EXPECT_EQ(settings.threads, std::thread::hardware_concurrency());
EXPECT_EQ(settings.enableLog, false);
EXPECT_EQ(settings.connectionTimeout, std::chrono::milliseconds{1000});
EXPECT_EQ(settings.requestTimeout, std::chrono::milliseconds{0});
EXPECT_EQ(settings.certificate, std::nullopt);
EXPECT_EQ(settings.username, std::nullopt);
EXPECT_EQ(settings.password, std::nullopt);
auto const* cp =
std::get_if<Settings::ContactPoints>(&settings.connectionInfo);
ASSERT_TRUE(cp != nullptr);
EXPECT_EQ(cp->contactPoints, "127.0.0.1");
EXPECT_FALSE(cp->port);
EXPECT_EQ(provider.getKeyspace(), "clio");
EXPECT_EQ(provider.getReplicationFactor(), 3);
EXPECT_EQ(provider.getTablePrefix(), std::nullopt);
}
TEST_F(SettingsProviderTest, SimpleConfig)
{
Config cfg{json::parse(R"({
"contact_points": "123.123.123.123",
"port": 1234,
"keyspace": "test",
"replication_factor": 42,
"table_prefix": "prefix",
"threads": 24
})")};
SettingsProvider provider{cfg};
auto const settings = provider.getSettings();
EXPECT_EQ(settings.threads, 24);
auto const* cp =
std::get_if<Settings::ContactPoints>(&settings.connectionInfo);
ASSERT_TRUE(cp != nullptr);
EXPECT_EQ(cp->contactPoints, "123.123.123.123");
EXPECT_EQ(cp->port, 1234);
EXPECT_EQ(provider.getKeyspace(), "test");
EXPECT_EQ(provider.getReplicationFactor(), 42);
EXPECT_EQ(provider.getTablePrefix(), "prefix");
}
TEST_F(SettingsProviderTest, SecureBundleConfig)
{
Config cfg{json::parse(R"({"secure_connect_bundle": "bundleData"})")};
SettingsProvider provider{cfg};
auto const settings = provider.getSettings();
auto const* sb =
std::get_if<Settings::SecureConnectionBundle>(&settings.connectionInfo);
ASSERT_TRUE(sb != nullptr);
EXPECT_EQ(sb->bundle, "bundleData");
}
TEST_F(SettingsProviderTest, CertificateConfig)
{
TmpFile file{"certificateData"};
Config cfg{json::parse(
R"({
"contact_points": "127.0.0.1",
"certfile": ")" +
file.path + "\"}")};
SettingsProvider provider{cfg};
auto const settings = provider.getSettings();
EXPECT_EQ(settings.certificate, "certificateData");
}

View File

@@ -0,0 +1,130 @@
//------------------------------------------------------------------------------
/*
This file is part of clio: https://github.com/XRPLF/clio
Copyright (c) 2023, the clio developers.
Permission to use, copy, modify, and distribute this software for any
purpose with or without fee is hereby granted, provided that the above
copyright notice and this permission notice appear in all copies.
THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES
WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF
MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR
ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES
WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN
ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF
OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.
*/
//==============================================================================
#include <backend/cassandra/Error.h>
#include <backend/cassandra/impl/AsyncExecutor.h>
#include <gmock/gmock.h>
#include <vector>
using namespace Backend::Cassandra;
using namespace Backend::Cassandra::detail;
struct FakeResult
{
};
struct FakeResultOrError
{
CassandraError err{"<default>", CASS_OK};
operator bool() const
{
return err.code() == CASS_OK;
}
CassandraError
error() const
{
return err;
}
FakeResult
value() const
{
return FakeResult{};
}
};
struct FakeMaybeError
{
};
struct FakeStatement
{
};
struct FakePreparedStatement
{
};
struct FakeFuture
{
FakeResultOrError data;
FakeResultOrError
get() const
{
return data;
}
};
struct FakeFutureWithCallback : public FakeFuture
{
};
struct MockHandle
{
using ResultOrErrorType = FakeResultOrError;
using MaybeErrorType = FakeMaybeError;
using FutureWithCallbackType = FakeFutureWithCallback;
using FutureType = FakeFuture;
using StatementType = FakeStatement;
using PreparedStatementType = FakePreparedStatement;
using ResultType = FakeResult;
MOCK_METHOD(
FutureWithCallbackType,
asyncExecute,
(StatementType const&, std::function<void(ResultOrErrorType)>&&),
(const));
MOCK_METHOD(
FutureWithCallbackType,
asyncExecute,
(std::vector<StatementType> const&,
std::function<void(ResultOrErrorType)>&&),
(const));
MOCK_METHOD(ResultOrErrorType, execute, (StatementType const&), (const));
};
struct FakeRetryPolicy
{
FakeRetryPolicy(boost::asio::io_context&){}; // required by concept
std::chrono::milliseconds
calculateDelay(uint32_t attempt)
{
return std::chrono::milliseconds{1};
}
bool shouldRetry(CassandraError) const
{
return false;
}
template <typename Fn>
void
retry(Fn&& fn)
{
fn();
}
};

View File

@@ -2,11 +2,9 @@
/*
This file is part of clio: https://github.com/XRPLF/clio
Copyright (c) 2023, the clio developers.
Permission to use, copy, modify, and distribute this software for any
purpose with or without fee is hereby granted, provided that the above
copyright notice and this permission notice appear in all copies.
THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES
WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF
MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR
@@ -235,18 +233,23 @@ TEST_F(RPCBaseTest, IfTypeValidator)
{
// clang-format off
auto spec = RpcSpec{
{"mix", Required{},
Type<std::string,json::object>{},
IfType<json::object>{
Section{{ "limit", Required{}, Type<uint32_t>{}, Between<uint32_t>{0, 100}}},
Section{{ "limit2", Required{}, Type<uint32_t>{}, Between<uint32_t>{0, 100}}}
},
IfType<std::string>{Uint256HexStringValidator,}
{"mix",
Required{}, Type<std::string, json::object>{},
IfType<json::object>{
Section{{"limit", Required{}, Type<uint32_t>{}, Between<uint32_t>{0, 100}}},
Section{{"limit2", Required{}, Type<uint32_t>{}, Between<uint32_t>{0, 100}}}
},
IfType<std::string>{
Uint256HexStringValidator
}
},
{"mix2",
Section{{"limit", Required{}, Type<uint32_t>{}, Between<uint32_t>{0, 100}}},
Type<std::string, json::object>{}
},
{"mix2",Section{{ "limit", Required{}, Type<uint32_t>{}, Between<uint32_t>{0, 100}}},
Type<std::string,json::object>{}}
};
// clang-format on
// if json object pass
auto passingInput =
json::parse(R"({ "mix": {"limit": 42, "limit2": 22} })");

View File

@@ -27,7 +27,7 @@ using namespace Backend;
class MockBackend : public BackendInterface
{
public:
MockBackend(clio::Config cfg) : BackendInterface(cfg)
MockBackend(clio::Config)
{
}
MOCK_METHOD(

41
unittests/util/TmpFile.h Normal file
View File

@@ -0,0 +1,41 @@
//------------------------------------------------------------------------------
/*
This file is part of clio: https://github.com/XRPLF/clio
Copyright (c) 2023, the clio developers.
Permission to use, copy, modify, and distribute this software for any
purpose with or without fee is hereby granted, provided that the above
copyright notice and this permission notice appear in all copies.
THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES
WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF
MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR
ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES
WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN
ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF
OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.
*/
//==============================================================================
#pragma once
#include <cstdio>
#include <filesystem>
#include <fstream>
struct TmpFile
{
std::string const path;
TmpFile(std::string_view content) : path{std::tmpnam(nullptr)}
{
std::ofstream ofs;
ofs.open(path, std::ios::out);
ofs << content;
}
~TmpFile()
{
std::filesystem::remove(path);
}
};