refactor: Replace all old instances of Config with New Config (#1627)

Fixes #1184 
Previous PR's found [here](https://github.com/XRPLF/clio/pull/1593) and
[here](https://github.com/XRPLF/clio/pull/1544)
This commit is contained in:
Peter Chen
2024-12-16 15:33:32 -08:00
committed by GitHub
parent b53cfd0ec1
commit 3c4903a339
103 changed files with 1624 additions and 898 deletions

View File

@@ -1,43 +0,0 @@
/*
* This is an example configuration file. Please do not use without modifying to suit your needs.
*/
{
"database": {
"type": "cassandra",
"cassandra": {
// This option can be used to setup a secure connect bundle connection
"secure_connect_bundle": "[path/to/zip. ignore if using contact_points]",
// The following options are used only if using contact_points
"contact_points": "[ip. ignore if using secure_connect_bundle]",
"port": "[port. ignore if using_secure_connect_bundle]",
// Authentication settings
"username": "[username, if any]",
"password": "[password, if any]",
// Other common settings
"keyspace": "clio",
"max_write_requests_outstanding": 25000,
"max_read_requests_outstanding": 30000,
"threads": 8
}
},
"etl_sources": [
{
"ip": "[rippled ip]",
"ws_port": "6006",
"grpc_port": "50051"
}
],
"dos_guard": {
"whitelist": [
"127.0.0.1"
]
},
"server": {
"ip": "0.0.0.0",
"port": 8080
},
"log_level": "debug",
"log_file": "./clio.log",
"extractor_threads": 8,
"read_only": false
}

View File

@@ -31,8 +31,8 @@
#include "rpc/WorkQueue.hpp"
#include "rpc/common/impl/HandlerProvider.hpp"
#include "util/build/Build.hpp"
#include "util/config/Config.hpp"
#include "util/log/Logger.hpp"
#include "util/newconfig/ConfigDefinition.hpp"
#include "util/prometheus/Prometheus.hpp"
#include "web/AdminVerificationStrategy.hpp"
#include "web/RPCServerHandler.hpp"
@@ -78,7 +78,8 @@ start(boost::asio::io_context& ioc, std::uint32_t numThreads)
} // namespace
ClioApplication::ClioApplication(util::Config const& config) : config_(config), signalsHandler_{config_}
ClioApplication::ClioApplication(util::config::ClioConfigDefinition const& config)
: config_(config), signalsHandler_{config_}
{
LOG(util::LogService::info()) << "Clio version: " << util::build::getClioFullVersionString();
PrometheusService::init(config);
@@ -87,11 +88,7 @@ ClioApplication::ClioApplication(util::Config const& config) : config_(config),
int
ClioApplication::run(bool const useNgWebServer)
{
auto const threads = config_.valueOr("io_threads", 2);
if (threads <= 0) {
LOG(util::LogService::fatal()) << "io_threads is less than 1";
return EXIT_FAILURE;
}
auto const threads = config_.get<uint16_t>("io_threads");
LOG(util::LogService::info()) << "Number of io threads = " << threads;
// IO context to handle all incoming requests, as well as other things.
@@ -132,7 +129,7 @@ ClioApplication::run(bool const useNgWebServer)
auto const rpcEngine =
RPCEngineType::make_RPCEngine(config_, backend, balancer, dosGuard, workQueue, counters, handlerProvider);
if (useNgWebServer or config_.valueOr("server.__ng_web_server", false)) {
if (useNgWebServer or config_.get<bool>("server.__ng_web_server")) {
web::ng::RPCServerHandler<RPCEngineType, etl::ETLService> handler{config_, backend, rpcEngine, etl};
auto expectedAdminVerifier = web::make_AdminVerificationStrategy(config_);

View File

@@ -18,9 +18,8 @@
//==============================================================================
#pragma once
#include "util/SignalsHandler.hpp"
#include "util/config//Config.hpp"
#include "util/newconfig/ConfigDefinition.hpp"
namespace app {
@@ -28,7 +27,7 @@ namespace app {
* @brief The main application class
*/
class ClioApplication {
util::Config const& config_;
util::config::ClioConfigDefinition const& config_;
util::SignalsHandler signalsHandler_;
public:
@@ -37,7 +36,7 @@ public:
*
* @param config The configuration of the application
*/
ClioApplication(util::Config const& config);
ClioApplication(util::config::ClioConfigDefinition const& config);
/**
* @brief Run the application

View File

@@ -22,8 +22,8 @@
#include "data/BackendInterface.hpp"
#include "data/CassandraBackend.hpp"
#include "data/cassandra/SettingsProvider.hpp"
#include "util/config/Config.hpp"
#include "util/log/Logger.hpp"
#include "util/newconfig/ConfigDefinition.hpp"
#include <boost/algorithm/string.hpp>
#include <boost/algorithm/string/predicate.hpp>
@@ -41,18 +41,18 @@ namespace data {
* @return A shared_ptr<BackendInterface> with the selected implementation
*/
inline std::shared_ptr<BackendInterface>
make_Backend(util::Config const& config)
make_Backend(util::config::ClioConfigDefinition const& config)
{
static util::Logger const log{"Backend"};
LOG(log.info()) << "Constructing BackendInterface";
auto const readOnly = config.valueOr("read_only", false);
auto const readOnly = config.get<bool>("read_only");
auto const type = config.value<std::string>("database.type");
auto const type = config.get<std::string>("database.type");
std::shared_ptr<BackendInterface> backend = nullptr;
if (boost::iequals(type, "cassandra")) {
auto cfg = config.section("database." + type);
auto const cfg = config.getObject("database." + type);
backend = std::make_shared<data::cassandra::CassandraBackend>(data::cassandra::SettingsProvider{cfg}, readOnly);
}

View File

@@ -93,7 +93,7 @@ public:
, executor_{settingsProvider_.getSettings(), handle_}
{
if (auto const res = handle_.connect(); not res)
throw std::runtime_error("Could not connect to databse: " + res.error());
throw std::runtime_error("Could not connect to database: " + res.error());
if (not readOnly) {
if (auto const res = handle_.execute(schema_.createKeyspace); not res) {

View File

@@ -22,7 +22,7 @@
#include "data/cassandra/Types.hpp"
#include "data/cassandra/impl/Cluster.hpp"
#include "util/Constants.hpp"
#include "util/config/Config.hpp"
#include "util/newconfig/ObjectView.hpp"
#include <boost/json/conversion.hpp>
#include <boost/json/value.hpp>
@@ -36,43 +36,17 @@
#include <ios>
#include <iterator>
#include <optional>
#include <stdexcept>
#include <string>
#include <string_view>
#include <system_error>
namespace data::cassandra {
namespace impl {
inline static 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");
}
util::Config const 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 static 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 impl
SettingsProvider::SettingsProvider(util::Config const& cfg)
SettingsProvider::SettingsProvider(util::config::ObjectView const& cfg)
: config_{cfg}
, keyspace_{cfg.valueOr<std::string>("keyspace", "clio")}
, keyspace_{cfg.get<std::string>("keyspace")}
, tablePrefix_{cfg.maybeValue<std::string>("table_prefix")}
, replicationFactor_{cfg.valueOr<uint16_t>("replication_factor", 3)}
, replicationFactor_{cfg.get<uint16_t>("replication_factor")}
, settings_{parseSettings()}
{
}
@@ -86,8 +60,8 @@ SettingsProvider::getSettings() const
std::optional<std::string>
SettingsProvider::parseOptionalCertificate() const
{
if (auto const certPath = config_.maybeValue<std::string>("certfile"); certPath) {
auto const path = std::filesystem::path(*certPath);
if (auto const certPath = config_.getValueView("certfile"); certPath.hasValue()) {
auto const path = std::filesystem::path(certPath.asString());
std::ifstream fileStream(path.string(), std::ios::in);
if (!fileStream) {
throw std::system_error(errno, std::generic_category(), "Opening certificate " + path.string());
@@ -108,30 +82,34 @@ Settings
SettingsProvider::parseSettings() const
{
auto settings = Settings::defaultSettings();
if (auto const bundle = config_.maybeValue<Settings::SecureConnectionBundle>("secure_connect_bundle"); bundle) {
settings.connectionInfo = *bundle;
// all config values used in settings is under "database.cassandra" prefix
if (config_.getValueView("secure_connect_bundle").hasValue()) {
auto const bundle = Settings::SecureConnectionBundle{(config_.get<std::string>("secure_connect_bundle"))};
settings.connectionInfo = bundle;
} else {
settings.connectionInfo =
config_.valueOrThrow<Settings::ContactPoints>("Missing contact_points in Cassandra config");
Settings::ContactPoints out;
out.contactPoints = config_.get<std::string>("contact_points");
out.port = config_.maybeValue<uint32_t>("port");
settings.connectionInfo = out;
}
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.coreConnectionsPerHost =
config_.valueOr<uint32_t>("core_connections_per_host", settings.coreConnectionsPerHost);
settings.threads = config_.get<uint32_t>("threads");
settings.maxWriteRequestsOutstanding = config_.get<uint32_t>("max_write_requests_outstanding");
settings.maxReadRequestsOutstanding = config_.get<uint32_t>("max_read_requests_outstanding");
settings.coreConnectionsPerHost = config_.get<uint32_t>("core_connections_per_host");
settings.queueSizeIO = config_.maybeValue<uint32_t>("queue_size_io");
settings.writeBatchSize = config_.valueOr<std::size_t>("write_batch_size", settings.writeBatchSize);
settings.writeBatchSize = config_.get<std::size_t>("write_batch_size");
auto const connectTimeoutSecond = config_.maybeValue<uint32_t>("connect_timeout");
if (connectTimeoutSecond)
settings.connectionTimeout = std::chrono::milliseconds{*connectTimeoutSecond * util::MILLISECONDS_PER_SECOND};
if (config_.getValueView("connect_timeout").hasValue()) {
auto const connectTimeoutSecond = config_.get<uint32_t>("connect_timeout");
settings.connectionTimeout = std::chrono::milliseconds{connectTimeoutSecond * util::MILLISECONDS_PER_SECOND};
}
auto const requestTimeoutSecond = config_.maybeValue<uint32_t>("request_timeout");
if (requestTimeoutSecond)
settings.requestTimeout = std::chrono::milliseconds{*requestTimeoutSecond * util::MILLISECONDS_PER_SECOND};
if (config_.getValueView("request_timeout").hasValue()) {
auto const requestTimeoutSecond = config_.get<uint32_t>("request_timeout");
settings.requestTimeout = std::chrono::milliseconds{requestTimeoutSecond * util::MILLISECONDS_PER_SECOND};
}
settings.certificate = parseOptionalCertificate();
settings.username = config_.maybeValue<std::string>("username");

View File

@@ -19,10 +19,9 @@
#pragma once
#include "data/cassandra/Handle.hpp"
#include "data/cassandra/Types.hpp"
#include "util/config/Config.hpp"
#include "util/log/Logger.hpp"
#include "data/cassandra/impl/Cluster.hpp"
#include "util/newconfig/ObjectView.hpp"
#include <cstdint>
#include <optional>
@@ -34,7 +33,7 @@ namespace data::cassandra {
* @brief Provides settings for @ref BasicCassandraBackend.
*/
class SettingsProvider {
util::Config config_;
util::config::ObjectView config_;
std::string keyspace_;
std::optional<std::string> tablePrefix_;
@@ -47,7 +46,7 @@ public:
*
* @param cfg The config of Clio to use
*/
explicit SettingsProvider(util::Config const& cfg);
explicit SettingsProvider(util::config::ObjectView const& cfg);
/**
* @return The cluster settings

View File

@@ -64,7 +64,11 @@ public:
* @param backend The backend to use
* @param cache The cache to load into
*/
CacheLoader(util::Config const& config, std::shared_ptr<BackendInterface> const& backend, CacheType& cache)
CacheLoader(
util::config::ClioConfigDefinition const& config,
std::shared_ptr<BackendInterface> const& backend,
CacheType& cache
)
: backend_{backend}, cache_{cache}, settings_{make_CacheLoaderSettings(config)}, ctx_{settings_.numThreads}
{
}

View File

@@ -19,11 +19,12 @@
#include "etl/CacheLoaderSettings.hpp"
#include "util/config/Config.hpp"
#include "util/newconfig/ConfigDefinition.hpp"
#include <boost/algorithm/string/predicate.hpp>
#include <cstddef>
#include <cstdint>
#include <string>
namespace etl {
@@ -47,31 +48,29 @@ CacheLoaderSettings::isDisabled() const
}
[[nodiscard]] CacheLoaderSettings
make_CacheLoaderSettings(util::Config const& config)
make_CacheLoaderSettings(util::config::ClioConfigDefinition const& config)
{
CacheLoaderSettings settings;
settings.numThreads = config.valueOr("io_threads", settings.numThreads);
if (config.contains("cache")) {
auto const cache = config.section("cache");
// Given diff number to generate cursors
settings.numCacheDiffs = cache.valueOr<size_t>("num_diffs", settings.numCacheDiffs);
// Given cursors number fetching from diff
settings.numCacheCursorsFromDiff = cache.valueOr<size_t>("num_cursors_from_diff", 0);
// Given cursors number fetching from account
settings.numCacheCursorsFromAccount = cache.valueOr<size_t>("num_cursors_from_account", 0);
settings.numThreads = config.get<uint16_t>("io_threads");
auto const cache = config.getObject("cache");
// Given diff number to generate cursors
settings.numCacheDiffs = cache.get<std::size_t>("num_diffs");
// Given cursors number fetching from diff
settings.numCacheCursorsFromDiff = cache.get<std::size_t>("num_cursors_from_diff");
// Given cursors number fetching from account
settings.numCacheCursorsFromAccount = cache.get<std::size_t>("num_cursors_from_account");
settings.numCacheMarkers = cache.valueOr<size_t>("num_markers", settings.numCacheMarkers);
settings.cachePageFetchSize = cache.valueOr<size_t>("page_fetch_size", settings.cachePageFetchSize);
settings.numCacheMarkers = cache.get<std::size_t>("num_markers");
settings.cachePageFetchSize = cache.get<std::size_t>("page_fetch_size");
auto const entry = cache.get<std::string>("load");
if (boost::iequals(entry, "sync"))
settings.loadStyle = CacheLoaderSettings::LoadStyle::SYNC;
if (boost::iequals(entry, "async"))
settings.loadStyle = CacheLoaderSettings::LoadStyle::ASYNC;
if (boost::iequals(entry, "none") or boost::iequals(entry, "no"))
settings.loadStyle = CacheLoaderSettings::LoadStyle::NONE;
if (auto entry = cache.maybeValue<std::string>("load"); entry) {
if (boost::iequals(*entry, "sync"))
settings.loadStyle = CacheLoaderSettings::LoadStyle::SYNC;
if (boost::iequals(*entry, "async"))
settings.loadStyle = CacheLoaderSettings::LoadStyle::ASYNC;
if (boost::iequals(*entry, "none") or boost::iequals(*entry, "no"))
settings.loadStyle = CacheLoaderSettings::LoadStyle::NONE;
}
}
return settings;
}

View File

@@ -19,7 +19,7 @@
#pragma once
#include "util/config/Config.hpp"
#include "util/newconfig/ConfigDefinition.hpp"
#include <cstddef>
@@ -64,6 +64,6 @@ struct CacheLoaderSettings {
* @returns The CacheLoaderSettings object
*/
[[nodiscard]] CacheLoaderSettings
make_CacheLoaderSettings(util::Config const& config);
make_CacheLoaderSettings(util::config::ClioConfigDefinition const& config);
} // namespace etl

View File

@@ -26,8 +26,8 @@
#include "feed/SubscriptionManagerInterface.hpp"
#include "util/Assert.hpp"
#include "util/Constants.hpp"
#include "util/config/Config.hpp"
#include "util/log/Logger.hpp"
#include "util/newconfig/ConfigDefinition.hpp"
#include <boost/asio/io_context.hpp>
#include <xrpl/beast/core/CurrentThreadName.h>
@@ -262,7 +262,7 @@ ETLService::doWork()
}
ETLService::ETLService(
util::Config const& config,
util::config::ClioConfigDefinition const& config,
boost::asio::io_context& ioc,
std::shared_ptr<BackendInterface> backend,
std::shared_ptr<feed::SubscriptionManagerInterface> subscriptions,
@@ -280,9 +280,9 @@ ETLService::ETLService(
{
startSequence_ = config.maybeValue<uint32_t>("start_sequence");
finishSequence_ = config.maybeValue<uint32_t>("finish_sequence");
state_.isReadOnly = config.valueOr("read_only", static_cast<bool>(state_.isReadOnly));
extractorThreads_ = config.valueOr<uint32_t>("extractor_threads", extractorThreads_);
txnThreshold_ = config.valueOr<size_t>("txn_threshold", txnThreshold_);
state_.isReadOnly = config.get<bool>("read_only");
extractorThreads_ = config.get<uint32_t>("extractor_threads");
txnThreshold_ = config.get<std::size_t>("txn_threshold");
// This should probably be done in the backend factory but we don't have state available until here
backend_->setCorruptionDetector(CorruptionDetector<data::LedgerCache>{state_, backend->cache()});

View File

@@ -119,7 +119,7 @@ public:
* @param ledgers The network validated ledgers datastructure
*/
ETLService(
util::Config const& config,
util::config::ClioConfigDefinition const& config,
boost::asio::io_context& ioc,
std::shared_ptr<BackendInterface> backend,
std::shared_ptr<feed::SubscriptionManagerInterface> subscriptions,
@@ -142,7 +142,7 @@ public:
*/
static std::shared_ptr<ETLService>
make_ETLService(
util::Config const& config,
util::config::ClioConfigDefinition const& config,
boost::asio::io_context& ioc,
std::shared_ptr<BackendInterface> backend,
std::shared_ptr<feed::SubscriptionManagerInterface> subscriptions,

View File

@@ -29,6 +29,9 @@
#include "util/Random.hpp"
#include "util/ResponseExpirationCache.hpp"
#include "util/log/Logger.hpp"
#include "util/newconfig/ArrayView.hpp"
#include "util/newconfig/ConfigDefinition.hpp"
#include "util/newconfig/ObjectView.hpp"
#include <boost/asio/io_context.hpp>
#include <boost/asio/spawn.hpp>
@@ -51,13 +54,13 @@
#include <utility>
#include <vector>
using namespace util;
using namespace util::config;
namespace etl {
std::shared_ptr<LoadBalancer>
LoadBalancer::make_LoadBalancer(
Config const& config,
ClioConfigDefinition const& config,
boost::asio::io_context& ioc,
std::shared_ptr<BackendInterface> backend,
std::shared_ptr<feed::SubscriptionManagerInterface> subscriptions,
@@ -71,7 +74,7 @@ LoadBalancer::make_LoadBalancer(
}
LoadBalancer::LoadBalancer(
Config const& config,
ClioConfigDefinition const& config,
boost::asio::io_context& ioc,
std::shared_ptr<BackendInterface> backend,
std::shared_ptr<feed::SubscriptionManagerInterface> subscriptions,
@@ -79,23 +82,23 @@ LoadBalancer::LoadBalancer(
SourceFactory sourceFactory
)
{
auto const forwardingCacheTimeout = config.valueOr<float>("forwarding.cache_timeout", 0.f);
auto const forwardingCacheTimeout = config.get<float>("forwarding.cache_timeout");
if (forwardingCacheTimeout > 0.f) {
forwardingCache_ = util::ResponseExpirationCache{
Config::toMilliseconds(forwardingCacheTimeout),
util::config::ClioConfigDefinition::toMilliseconds(forwardingCacheTimeout),
{"server_info", "server_state", "server_definitions", "fee", "ledger_closed"}
};
}
static constexpr std::uint32_t MAX_DOWNLOAD = 256;
if (auto value = config.maybeValue<uint32_t>("num_markers"); value) {
ASSERT(*value > 0 and *value <= MAX_DOWNLOAD, "'num_markers' value in config must be in range 1-256");
downloadRanges_ = *value;
auto const numMarkers = config.getValueView("num_markers");
if (numMarkers.hasValue()) {
auto const value = numMarkers.asIntType<uint32_t>();
downloadRanges_ = value;
} else if (backend->fetchLedgerRange()) {
downloadRanges_ = 4;
}
auto const allowNoEtl = config.valueOr("allow_no_etl", false);
auto const allowNoEtl = config.get<bool>("allow_no_etl");
auto const checkOnETLFailure = [this, allowNoEtl](std::string const& log) {
LOG(log_.warn()) << log;
@@ -106,10 +109,12 @@ LoadBalancer::LoadBalancer(
}
};
auto const forwardingTimeout = Config::toMilliseconds(config.valueOr<float>("forwarding.request_timeout", 10.));
for (auto const& entry : config.array("etl_sources")) {
auto const forwardingTimeout =
ClioConfigDefinition::toMilliseconds(config.get<float>("forwarding.request_timeout"));
auto const etlArray = config.getArray("etl_sources");
for (auto it = etlArray.begin<ObjectView>(); it != etlArray.end<ObjectView>(); ++it) {
auto source = sourceFactory(
entry,
*it,
ioc,
backend,
subscriptions,

View File

@@ -27,8 +27,8 @@
#include "rpc/Errors.hpp"
#include "util/Mutex.hpp"
#include "util/ResponseExpirationCache.hpp"
#include "util/config/Config.hpp"
#include "util/log/Logger.hpp"
#include "util/newconfig/ConfigDefinition.hpp"
#include <boost/asio.hpp>
#include <boost/asio/io_context.hpp>
@@ -103,7 +103,7 @@ public:
* @param sourceFactory A factory function to create a source
*/
LoadBalancer(
util::Config const& config,
util::config::ClioConfigDefinition const& config,
boost::asio::io_context& ioc,
std::shared_ptr<BackendInterface> backend,
std::shared_ptr<feed::SubscriptionManagerInterface> subscriptions,
@@ -124,7 +124,7 @@ public:
*/
static std::shared_ptr<LoadBalancer>
make_LoadBalancer(
util::Config const& config,
util::config::ClioConfigDefinition const& config,
boost::asio::io_context& ioc,
std::shared_ptr<BackendInterface> backend,
std::shared_ptr<feed::SubscriptionManagerInterface> subscriptions,

View File

@@ -26,7 +26,8 @@
#include "etl/impl/SourceImpl.hpp"
#include "etl/impl/SubscriptionSource.hpp"
#include "feed/SubscriptionManagerInterface.hpp"
#include "util/config/Config.hpp"
#include "util/newconfig/ConfigDefinition.hpp"
#include "util/newconfig/ObjectView.hpp"
#include <boost/asio/io_context.hpp>
@@ -39,7 +40,7 @@ namespace etl {
SourcePtr
make_Source(
util::Config const& config,
util::config::ObjectView const& config,
boost::asio::io_context& ioc,
std::shared_ptr<BackendInterface> backend,
std::shared_ptr<feed::SubscriptionManagerInterface> subscriptions,
@@ -50,9 +51,9 @@ make_Source(
SourceBase::OnLedgerClosedHook onLedgerClosed
)
{
auto const ip = config.valueOr<std::string>("ip", {});
auto const wsPort = config.valueOr<std::string>("ws_port", {});
auto const grpcPort = config.valueOr<std::string>("grpc_port", {});
auto const ip = config.get<std::string>("ip");
auto const wsPort = config.get<std::string>("ws_port");
auto const grpcPort = config.get<std::string>("grpc_port");
impl::ForwardingSource forwardingSource{ip, wsPort, forwardingTimeout};
impl::GrpcSource grpcSource{ip, grpcPort, std::move(backend)};

View File

@@ -23,8 +23,9 @@
#include "etl/NetworkValidatedLedgersInterface.hpp"
#include "feed/SubscriptionManagerInterface.hpp"
#include "rpc/Errors.hpp"
#include "util/config/Config.hpp"
#include "util/log/Logger.hpp"
#include "util/newconfig/ConfigDefinition.hpp"
#include "util/newconfig/ObjectView.hpp"
#include <boost/asio/io_context.hpp>
#include <boost/asio/spawn.hpp>
@@ -147,7 +148,7 @@ public:
using SourcePtr = std::unique_ptr<SourceBase>;
using SourceFactory = std::function<SourcePtr(
util::Config const& config,
util::config::ObjectView const& config,
boost::asio::io_context& ioc,
std::shared_ptr<BackendInterface> backend,
std::shared_ptr<feed::SubscriptionManagerInterface> subscriptions,
@@ -175,7 +176,7 @@ using SourceFactory = std::function<SourcePtr(
*/
SourcePtr
make_Source(
util::Config const& config,
util::config::ObjectView const& config,
boost::asio::io_context& ioc,
std::shared_ptr<BackendInterface> backend,
std::shared_ptr<feed::SubscriptionManagerInterface> subscriptions,

View File

@@ -30,8 +30,8 @@
#include "feed/impl/TransactionFeed.hpp"
#include "util/async/AnyExecutionContext.hpp"
#include "util/async/context/BasicExecutionContext.hpp"
#include "util/config/Config.hpp"
#include "util/log/Logger.hpp"
#include "util/newconfig/ConfigDefinition.hpp"
#include <boost/asio/executor_work_guard.hpp>
#include <boost/asio/io_context.hpp>
@@ -77,9 +77,12 @@ public:
* @return A shared pointer to a new instance of SubscriptionManager
*/
static std::shared_ptr<SubscriptionManager>
make_SubscriptionManager(util::Config const& config, std::shared_ptr<data::BackendInterface const> const& backend)
make_SubscriptionManager(
util::config::ClioConfigDefinition const& config,
std::shared_ptr<data::BackendInterface const> const& backend
)
{
auto const workersNum = config.valueOr<std::uint64_t>("subscription_workers", 1);
auto const workersNum = config.get<uint64_t>("subscription_workers");
util::Logger const logger{"Subscriptions"};
LOG(logger.info()) << "Starting subscription manager with " << workersNum << " workers";

View File

@@ -21,13 +21,16 @@
#include "app/ClioApplication.hpp"
#include "rpc/common/impl/HandlerProvider.hpp"
#include "util/TerminationHandler.hpp"
#include "util/config/Config.hpp"
#include "util/log/Logger.hpp"
#include "util/newconfig/ConfigDefinition.hpp"
#include "util/newconfig/ConfigFileJson.hpp"
#include <cstdlib>
#include <exception>
#include <iostream>
using namespace util::config;
int
main(int argc, char const* argv[])
try {
@@ -37,14 +40,19 @@ try {
return action.apply(
[](app::CliArgs::Action::Exit const& exit) { return exit.exitCode; },
[](app::CliArgs::Action::Run const& run) {
auto const config = util::ConfigReader::open(run.configPath);
if (!config) {
std::cerr << "Couldnt parse config '" << run.configPath << "'." << std::endl;
auto const json = ConfigFileJson::make_ConfigFileJson(run.configPath);
if (!json.has_value()) {
std::cerr << json.error().error << std::endl;
return EXIT_FAILURE;
}
util::LogService::init(config);
app::ClioApplication clio{config};
auto const errors = ClioConfig.parse(json.value());
if (errors.has_value()) {
for (auto const& err : errors.value())
std::cerr << err.error << std::endl;
return EXIT_FAILURE;
}
util::LogService::init(ClioConfig);
app::ClioApplication clio{ClioConfig};
return clio.run(run.useNgWebServer);
}
);

View File

@@ -84,7 +84,7 @@ public:
* @param handlerProvider The handler provider to use
*/
RPCEngine(
util::Config const& config,
util::config::ClioConfigDefinition const& config,
std::shared_ptr<BackendInterface> const& backend,
std::shared_ptr<LoadBalancerType> const& balancer,
web::dosguard::DOSGuardInterface const& dosGuard,
@@ -100,13 +100,14 @@ public:
, forwardingProxy_{balancer, counters, handlerProvider}
{
// Let main thread catch the exception if config type is wrong
auto const cacheTimeout = config.valueOr<float>("rpc.cache_timeout", 0.f);
auto const cacheTimeout = config.get<float>("rpc.cache_timeout");
if (cacheTimeout > 0.f) {
LOG(log_.info()) << fmt::format("Init RPC Cache, timeout: {} seconds", cacheTimeout);
responseCache_.emplace(
util::Config::toMilliseconds(cacheTimeout), std::unordered_set<std::string>{"server_info"}
util::config::ClioConfigDefinition::toMilliseconds(cacheTimeout),
std::unordered_set<std::string>{"server_info"}
);
}
}
@@ -125,7 +126,7 @@ public:
*/
static std::shared_ptr<RPCEngine>
make_RPCEngine(
util::Config const& config,
util::config::ClioConfigDefinition const& config,
std::shared_ptr<BackendInterface> const& backend,
std::shared_ptr<LoadBalancerType> const& balancer,
web::dosguard::DOSGuardInterface const& dosGuard,

View File

@@ -19,7 +19,6 @@
#include "rpc/WorkQueue.hpp"
#include "util/config/Config.hpp"
#include "util/log/Logger.hpp"
#include "util/prometheus/Label.hpp"
#include "util/prometheus/Prometheus.hpp"
@@ -29,7 +28,6 @@
#include <cstddef>
#include <cstdint>
#include <functional>
#include <thread>
#include <utility>
namespace rpc {
@@ -92,12 +90,12 @@ WorkQueue::stop(std::function<void()> onQueueEmpty)
}
WorkQueue
WorkQueue::make_WorkQueue(util::Config const& config)
WorkQueue::make_WorkQueue(util::config::ClioConfigDefinition const& config)
{
static util::Logger const log{"RPC"};
auto const serverConfig = config.section("server");
auto const numThreads = config.valueOr<uint32_t>("workers", std::thread::hardware_concurrency());
auto const maxQueueSize = serverConfig.valueOr<uint32_t>("max_queue_size", 0); // 0 is no limit
auto const serverConfig = config.getObject("server");
auto const numThreads = config.get<uint32_t>("workers");
auto const maxQueueSize = serverConfig.get<uint32_t>("max_queue_size");
LOG(log.info()) << "Number of workers = " << numThreads << ". Max queue size = " << maxQueueSize;
return WorkQueue{numThreads, maxQueueSize};

View File

@@ -19,10 +19,9 @@
#pragma once
#include "util/Assert.hpp"
#include "util/Mutex.hpp"
#include "util/config/Config.hpp"
#include "util/log/Logger.hpp"
#include "util/newconfig/ConfigDefinition.hpp"
#include "util/prometheus/Counter.hpp"
#include "util/prometheus/Gauge.hpp"
@@ -97,7 +96,7 @@ public:
* @return The work queue
*/
static WorkQueue
make_WorkQueue(util::Config const& config);
make_WorkQueue(util::config::ClioConfigDefinition const& config);
/**
* @brief Submit a job to the work queue.

View File

@@ -19,8 +19,7 @@
#include "rpc/common/impl/APIVersionParser.hpp"
#include "rpc/common/APIVersion.hpp"
#include "util/log/Logger.hpp"
#include "util/newconfig/ObjectView.hpp"
#include <boost/json/object.hpp>
#include <fmt/core.h>
@@ -33,11 +32,22 @@ using namespace std;
namespace rpc::impl {
ProductionAPIVersionParser::ProductionAPIVersionParser(util::Config const& config)
ProductionAPIVersionParser::ProductionAPIVersionParser(
uint32_t defaultVersion,
uint32_t minVersion,
uint32_t maxVersion
)
: defaultVersion_{defaultVersion}, minVersion_{minVersion}, maxVersion_{maxVersion}
{
LOG(log_.info()) << "API version settings: [min = " << minVersion_ << "; max = " << maxVersion_
<< "; default = " << defaultVersion_ << "]";
}
ProductionAPIVersionParser::ProductionAPIVersionParser(util::config::ObjectView const& config)
: ProductionAPIVersionParser(
config.valueOr("default", API_VERSION_DEFAULT),
config.valueOr("min", API_VERSION_MIN),
config.valueOr("max", API_VERSION_MAX)
config.get<uint32_t>("default"),
config.get<uint32_t>("min"),
config.get<uint32_t>("max")
)
{
}

View File

@@ -20,12 +20,11 @@
#pragma once
#include "rpc/common/APIVersion.hpp"
#include "util/config/Config.hpp"
#include "util/log/Logger.hpp"
#include "util/newconfig/ObjectView.hpp"
#include <boost/json/object.hpp>
#include <algorithm>
#include <cstdint>
#include <expected>
#include <string>
@@ -40,39 +39,13 @@ class ProductionAPIVersionParser : public APIVersionParser {
uint32_t maxVersion_;
public:
// Note: this constructor must remain in the header because of UNITTEST_BUILD definition below
ProductionAPIVersionParser(
uint32_t defaultVersion = API_VERSION_DEFAULT,
uint32_t minVersion = API_VERSION_MIN,
uint32_t maxVersion = API_VERSION_MAX
)
: defaultVersion_{defaultVersion}, minVersion_{minVersion}, maxVersion_{maxVersion}
{
#ifndef UNITTEST_BUILD
// in production, we don't want the ability to misconfigure clio with bogus versions
// that are not actually supported by the code itself. for testing it is desired however.
auto checkRange = [this](uint32_t version, std::string label) {
if (std::clamp(version, API_VERSION_MIN, API_VERSION_MAX) != version) {
LOG(log_.error()) << "API version settings issue detected: " << label << " version with value "
<< version << " is outside of supported range " << API_VERSION_MIN << "-"
<< API_VERSION_MAX << "; Falling back to hardcoded values.";
);
defaultVersion_ = API_VERSION_DEFAULT;
minVersion_ = API_VERSION_MIN;
maxVersion_ = API_VERSION_MAX;
}
};
checkRange(defaultVersion, "default");
checkRange(minVersion, "minimum");
checkRange(maxVersion, "maximum");
#endif
LOG(log_.info()) << "API version settings: [min = " << minVersion_ << "; max = " << maxVersion_
<< "; default = " << defaultVersion_ << "]";
}
ProductionAPIVersionParser(util::Config const& config);
ProductionAPIVersionParser(util::config::ObjectView const& config);
std::expected<uint32_t, std::string>
parse(boost::json::object const& request) const override;

View File

@@ -60,7 +60,7 @@
#include "rpc/handlers/Tx.hpp"
#include "rpc/handlers/Unsubscribe.hpp"
#include "rpc/handlers/VersionHandler.hpp"
#include "util/config/Config.hpp"
#include "util/newconfig/ConfigDefinition.hpp"
#include <memory>
#include <optional>
@@ -69,7 +69,7 @@
namespace rpc::impl {
ProductionHandlerProvider::ProductionHandlerProvider(
util::Config const& config,
util::config::ClioConfigDefinition const& config,
std::shared_ptr<BackendInterface> const& backend,
std::shared_ptr<feed::SubscriptionManagerInterface> const& subscriptionManager,
std::shared_ptr<etl::LoadBalancer> const& balancer,

View File

@@ -25,7 +25,7 @@
#include "rpc/common/AnyHandler.hpp"
#include "rpc/common/HandlerProvider.hpp"
#include "rpc/common/Types.hpp"
#include "util/config/Config.hpp"
#include "util/log/Logger.hpp"
#include <memory>
#include <optional>
@@ -52,7 +52,7 @@ class ProductionHandlerProvider final : public HandlerProvider {
public:
ProductionHandlerProvider(
util::Config const& config,
util::config::ClioConfigDefinition const& config,
std::shared_ptr<BackendInterface> const& backend,
std::shared_ptr<feed::SubscriptionManagerInterface> const& subscriptionManager,
std::shared_ptr<etl::LoadBalancer> const& balancer,

View File

@@ -22,7 +22,7 @@
#include "rpc/common/APIVersion.hpp"
#include "rpc/common/Types.hpp"
#include "rpc/common/impl/APIVersionParser.hpp"
#include "util/config/Config.hpp"
#include "util/newconfig/ConfigDefinition.hpp"
#include <boost/json/conversion.hpp>
#include <boost/json/value.hpp>
@@ -53,11 +53,11 @@ public:
*
* @param config The configuration to use
*/
explicit VersionHandler(util::Config const& config)
explicit VersionHandler(util::config::ClioConfigDefinition const& config)
: apiVersionParser_(
config.valueOr("default", API_VERSION_DEFAULT),
config.valueOr("min", API_VERSION_MIN),
config.valueOr("max", API_VERSION_MAX)
config.get<uint32_t>("api_version.default"),
config.get<uint32_t>("api_version.min"),
config.get<uint32_t>("api_version.max")
)
{
}

View File

@@ -20,8 +20,8 @@
#include "util/SignalsHandler.hpp"
#include "util/Assert.hpp"
#include "util/config/Config.hpp"
#include "util/log/Logger.hpp"
#include "util/newconfig/ConfigDefinition.hpp"
#include <chrono>
#include <csignal>
@@ -70,7 +70,7 @@ SignalsHandler* SignalsHandlerStatic::handler_ = nullptr;
} // namespace impl
SignalsHandler::SignalsHandler(Config const& config, std::function<void()> forceExitHandler)
SignalsHandler::SignalsHandler(config::ClioConfigDefinition const& config, std::function<void()> forceExitHandler)
: gracefulPeriod_(0)
, context_(1)
, stopHandler_([this, forceExitHandler](int) mutable {
@@ -99,9 +99,7 @@ SignalsHandler::SignalsHandler(Config const& config, std::function<void()> force
{
impl::SignalsHandlerStatic::registerHandler(*this);
gracefulPeriod_ = Config::toMilliseconds(config.valueOr("graceful_period", 10.f));
ASSERT(gracefulPeriod_.count() >= 0, "Graceful period must be non-negative");
gracefulPeriod_ = util::config::ClioConfigDefinition::toMilliseconds(config.get<float>("graceful_period"));
setHandler(impl::SignalsHandlerStatic::handleSignal);
}

View File

@@ -20,8 +20,8 @@
#pragma once
#include "util/async/context/BasicExecutionContext.hpp"
#include "util/config/Config.hpp"
#include "util/log/Logger.hpp"
#include "util/newconfig/ConfigDefinition.hpp"
#include <boost/asio/executor_work_guard.hpp>
#include <boost/asio/io_context.hpp>
@@ -35,7 +35,6 @@
#include <cstdlib>
#include <functional>
#include <optional>
#include <string>
namespace util {
@@ -70,7 +69,10 @@ public:
* @param config The configuration.
* @param forceExitHandler The handler for forced exit.
*/
SignalsHandler(Config const& config, std::function<void()> forceExitHandler = defaultForceExitHandler_);
SignalsHandler(
util::config::ClioConfigDefinition const& config,
std::function<void()> forceExitHandler = defaultForceExitHandler_
);
SignalsHandler(SignalsHandler const&) = delete;
SignalsHandler(SignalsHandler&&) = delete;

View File

@@ -19,7 +19,8 @@
#pragma once
#include "util/config/Config.hpp"
#include "util/Assert.hpp"
#include "util/newconfig/ConfigDefinition.hpp"
#include <boost/algorithm/string/predicate.hpp>
#include <boost/json.hpp>
@@ -33,7 +34,9 @@
#include <memory>
#include <optional>
#include <ostream>
#include <stdexcept>
#include <string>
#include <string_view>
#include <utility>
namespace util {
namespace impl {
@@ -178,6 +181,22 @@ class TagDecoratorFactory final {
Type type_; /*< The type of TagDecorator this factory produces */
ParentType parent_ = std::nullopt; /*< The parent tag decorator to bind */
static Type
getLogTagType(std::string_view style)
{
if (boost::iequals(style, "int") || boost::iequals(style, "uint"))
return TagDecoratorFactory::Type::UINT;
if (boost::iequals(style, "null") || boost::iequals(style, "none"))
return TagDecoratorFactory::Type::NONE;
if (boost::iequals(style, "uuid"))
return TagDecoratorFactory::Type::UUID;
ASSERT(false, "log_tag_style does not have valid value");
std::unreachable();
}
public:
~TagDecoratorFactory() = default;
@@ -186,7 +205,8 @@ public:
*
* @param config The configuration as a json object
*/
explicit TagDecoratorFactory(util::Config const& config) : type_{config.valueOr<Type>("log_tag_style", Type::NONE)}
explicit TagDecoratorFactory(util::config::ClioConfigDefinition const& config)
: type_{getLogTagType(config.get<std::string>("log_tag_style"))}
{
}
@@ -212,26 +232,6 @@ public:
*/
TagDecoratorFactory
with(ParentType parent) const noexcept;
private:
friend Type
tag_invoke(boost::json::value_to_tag<Type>, boost::json::value const& value)
{
if (not value.is_string())
throw std::runtime_error("`log_tag_style` must be a string");
auto const& style = value.as_string();
if (boost::iequals(style, "int") || boost::iequals(style, "uint"))
return TagDecoratorFactory::Type::UINT;
if (boost::iequals(style, "null") || boost::iequals(style, "none"))
return TagDecoratorFactory::Type::NONE;
if (boost::iequals(style, "uuid"))
return TagDecoratorFactory::Type::UUID;
throw std::runtime_error("Could not parse `log_tag_style`: expected `uint`, `uuid` or `null`");
}
};
/**

View File

@@ -19,8 +19,11 @@
#include "util/log/Logger.hpp"
#include "util/Assert.hpp"
#include "util/SourceLocation.hpp"
#include "util/config/Config.hpp"
#include "util/newconfig/ArrayView.hpp"
#include "util/newconfig/ConfigDefinition.hpp"
#include "util/newconfig/ObjectView.hpp"
#include <boost/algorithm/string/predicate.hpp>
#include <boost/date_time/posix_time/posix_time_duration.hpp>
@@ -53,9 +56,11 @@
#include <cstdint>
#include <ios>
#include <iostream>
#include <optional>
#include <ostream>
#include <stdexcept>
#include <string>
#include <string_view>
#include <unordered_map>
#include <utility>
@@ -80,13 +85,15 @@ operator<<(std::ostream& stream, Severity sev)
return stream << labels.at(static_cast<int>(sev));
}
/**
* @brief converts the loglevel to string to a corresponding Severity enum value.
*
* @param logLevel A string representing the log level
* @return Severity The corresponding Severity enum value.
*/
Severity
tag_invoke(boost::json::value_to_tag<Severity>, boost::json::value const& value)
getSeverityLevel(std::string_view logLevel)
{
if (not value.is_string())
throw std::runtime_error("`log_level` must be a string");
auto const& logLevel = value.as_string();
if (boost::iequals(logLevel, "trace"))
return Severity::TRC;
if (boost::iequals(logLevel, "debug"))
@@ -100,23 +107,22 @@ tag_invoke(boost::json::value_to_tag<Severity>, boost::json::value const& value)
if (boost::iequals(logLevel, "fatal"))
return Severity::FTL;
throw std::runtime_error(
"Could not parse `log_level`: expected `trace`, `debug`, `info`, `warning`, `error` or `fatal`"
);
// already checked during parsing of config that value must be valid
ASSERT(false, "Parsing of log_level is incorrect");
std::unreachable();
}
void
LogService::init(util::Config const& config)
LogService::init(config::ClioConfigDefinition const& config)
{
namespace keywords = boost::log::keywords;
namespace sinks = boost::log::sinks;
boost::log::add_common_attributes();
boost::log::register_simple_formatter_factory<Severity, char>("Severity");
auto const defaultFormat = "%TimeStamp% (%SourceLocation%) [%ThreadID%] %Channel%:%Severity% %Message%";
std::string format = config.valueOr<std::string>("log_format", defaultFormat);
std::string format = config.get<std::string>("log_format");
if (config.valueOr("log_to_console", false)) {
if (config.get<bool>("log_to_console")) {
boost::log::add_console_log(
std::cout, keywords::format = format, keywords::filter = log_severity < Severity::FTL
);
@@ -125,13 +131,14 @@ LogService::init(util::Config const& config)
// Always print fatal logs to cerr
boost::log::add_console_log(std::cerr, keywords::format = format, keywords::filter = log_severity >= Severity::FTL);
if (auto logDir = config.maybeValue<std::string>("log_directory"); logDir) {
auto const logDir = config.maybeValue<std::string>("log_directory");
if (logDir) {
boost::filesystem::path dirPath{logDir.value()};
if (!boost::filesystem::exists(dirPath))
boost::filesystem::create_directories(dirPath);
auto const rotationSize = config.valueOr<uint64_t>("log_rotation_size", 2048u) * 1024u * 1024u;
auto const rotationPeriod = config.valueOr<uint32_t>("log_rotation_hour_interval", 12u);
auto const dirSize = config.valueOr<uint64_t>("log_directory_max_size", 50u * 1024u) * 1024u * 1024u;
auto const rotationSize = config.get<uint64_t>("log_rotation_size");
auto const rotationPeriod = config.get<uint32_t>("log_rotation_hour_interval");
auto const dirSize = config.get<uint64_t>("log_directory_max_size");
auto fileSink = boost::log::add_file_log(
keywords::file_name = dirPath / "clio.log",
keywords::target_file_name = dirPath / "clio_%Y-%m-%d_%H-%M-%S.log",
@@ -149,19 +156,22 @@ LogService::init(util::Config const& config)
}
// get default severity, can be overridden per channel using the `log_channels` array
auto defaultSeverity = config.valueOr<Severity>("log_level", Severity::NFO);
auto const defaultSeverity = getSeverityLevel(config.get<std::string>("log_level"));
std::unordered_map<std::string, Severity> min_severity;
for (auto const& channel : Logger::CHANNELS)
min_severity[channel] = defaultSeverity;
min_severity["Alert"] = Severity::WRN; // Channel for alerts, always warning severity
for (auto const overrides = config.arrayOr("log_channels", {}); auto const& cfg : overrides) {
auto name = cfg.valueOrThrow<std::string>("channel", "Channel name is required");
auto const overrides = config.getArray("log_channels");
for (auto it = overrides.begin<util::config::ObjectView>(); it != overrides.end<util::config::ObjectView>(); ++it) {
auto const& cfg = *it;
auto name = cfg.get<std::string>("channel");
if (std::count(std::begin(Logger::CHANNELS), std::end(Logger::CHANNELS), name) == 0)
throw std::runtime_error("Can't override settings for log channel " + name + ": invalid channel");
min_severity[name] = cfg.valueOr<Severity>("log_level", defaultSeverity);
min_severity[name] = getSeverityLevel(config.get<std::string>("log_level"));
}
auto log_filter = [min_severity = std::move(min_severity),

View File

@@ -52,7 +52,9 @@
namespace util {
class Config;
namespace config {
class ClioConfigDefinition;
} // namespace config
/**
* @brief Skips evaluation of expensive argument lists if the given logger is disabled for the required severity level.
@@ -164,16 +166,6 @@ class Logger final {
private:
[[nodiscard]] static std::string
pretty_path(SourceLocationType const& loc, size_t max_depth = 3);
/**
* @brief Custom JSON parser for @ref Severity.
*
* @param value The JSON string to parse
* @return The parsed severity
* @throws std::runtime_error Thrown if severity is not in the right format
*/
friend Severity
tag_invoke(boost::json::value_to_tag<Severity>, boost::json::value const& value);
};
public:
@@ -285,7 +277,7 @@ public:
* @param config The configuration to use
*/
static void
init(Config const& config);
init(config::ClioConfigDefinition const& config);
/**
* @brief Globally accesible General logger at Severity::TRC severity

View File

@@ -95,9 +95,11 @@ PositiveDouble::checkTypeImpl(Value const& num) const
std::optional<Error>
PositiveDouble::checkValueImpl(Value const& num) const
{
if (std::holds_alternative<int64_t>(num) && std::get<int64_t>(num) >= 0)
return std::nullopt;
if (std::get<double>(num) >= 0)
return std::nullopt;
return Error{"Double number must be greater than 0"};
return Error{"Double number must be greater than or equal to 0"};
}
} // namespace util::config

View File

@@ -79,6 +79,11 @@ static constexpr std::array<char const*, 3> LOAD_CACHE_MODE = {
*/
static constexpr std::array<char const*, 1> DATABASE_TYPE = {"cassandra"};
/**
* @brief specific values that are accepted for server's processing_policy in config.
*/
static constexpr std::array<char const*, 2> PROCESSING_POLICY = {"parallel", "sequent"};
/**
* @brief An interface to enforce constraints on certain values within ClioConfigDefinition.
*/
@@ -346,9 +351,13 @@ static constinit OneOf validateLogLevelName{"log_level", LOG_LEVELS};
static constinit OneOf validateCassandraName{"database.type", DATABASE_TYPE};
static constinit OneOf validateLoadMode{"cache.load", LOAD_CACHE_MODE};
static constinit OneOf validateLogTag{"log_tag_style", LOG_TAGS};
static constinit OneOf validateProcessingPolicy{"server.processing_policy", PROCESSING_POLICY};
static constinit PositiveDouble validatePositiveDouble{};
static constinit NumberValueConstraint<uint32_t> validateNumMarkers{1, 256};
static constinit NumberValueConstraint<uint32_t> validateIOThreads{1, std::numeric_limits<uint16_t>::max()};
static constinit NumberValueConstraint<uint16_t> validateUint16{
std::numeric_limits<uint16_t>::min(),
std::numeric_limits<uint16_t>::max()

View File

@@ -20,6 +20,7 @@
#include "util/newconfig/ConfigDefinition.hpp"
#include "util/Assert.hpp"
#include "util/Constants.hpp"
#include "util/OverloadSet.hpp"
#include "util/newconfig/Array.hpp"
#include "util/newconfig/ArrayView.hpp"
@@ -34,100 +35,18 @@
#include <fmt/core.h>
#include <algorithm>
#include <chrono>
#include <cmath>
#include <cstddef>
#include <cstdint>
#include <initializer_list>
#include <optional>
#include <string>
#include <string_view>
#include <thread>
#include <utility>
#include <variant>
#include <vector>
namespace util::config {
/**
* @brief Full Clio Configuration definition.
*
* Specifies which keys are valid in Clio Config and provides default values if user's do not specify one. Those
* without default values must be present in the user's config file.
*/
static ClioConfigDefinition ClioConfig = ClioConfigDefinition{
{{"database.type", ConfigValue{ConfigType::String}.defaultValue("cassandra").withConstraint(validateCassandraName)},
{"database.cassandra.contact_points", ConfigValue{ConfigType::String}.defaultValue("localhost")},
{"database.cassandra.port", ConfigValue{ConfigType::Integer}.withConstraint(validatePort)},
{"database.cassandra.keyspace", ConfigValue{ConfigType::String}.defaultValue("clio")},
{"database.cassandra.replication_factor", ConfigValue{ConfigType::Integer}.defaultValue(3u)},
{"database.cassandra.table_prefix", ConfigValue{ConfigType::String}.defaultValue("table_prefix")},
{"database.cassandra.max_write_requests_outstanding",
ConfigValue{ConfigType::Integer}.defaultValue(10'000).withConstraint(validateUint32)},
{"database.cassandra.max_read_requests_outstanding",
ConfigValue{ConfigType::Integer}.defaultValue(100'000).withConstraint(validateUint32)},
{"database.cassandra.threads",
ConfigValue{ConfigType::Integer}
.defaultValue(static_cast<uint32_t>(std::thread::hardware_concurrency()))
.withConstraint(validateUint32)},
{"database.cassandra.core_connections_per_host",
ConfigValue{ConfigType::Integer}.defaultValue(1).withConstraint(validateUint16)},
{"database.cassandra.queue_size_io", ConfigValue{ConfigType::Integer}.optional().withConstraint(validateUint16)},
{"database.cassandra.write_batch_size",
ConfigValue{ConfigType::Integer}.defaultValue(20).withConstraint(validateUint16)},
{"etl_source.[].ip", Array{ConfigValue{ConfigType::String}.withConstraint(validateIP)}},
{"etl_source.[].ws_port", Array{ConfigValue{ConfigType::String}.withConstraint(validatePort)}},
{"etl_source.[].grpc_port", Array{ConfigValue{ConfigType::String}.withConstraint(validatePort)}},
{"forwarding.cache_timeout",
ConfigValue{ConfigType::Double}.defaultValue(0.0).withConstraint(validatePositiveDouble)},
{"forwarding.request_timeout",
ConfigValue{ConfigType::Double}.defaultValue(10.0).withConstraint(validatePositiveDouble)},
{"dos_guard.whitelist.[]", Array{ConfigValue{ConfigType::String}}},
{"dos_guard.max_fetches", ConfigValue{ConfigType::Integer}.defaultValue(1000'000).withConstraint(validateUint32)},
{"dos_guard.max_connections", ConfigValue{ConfigType::Integer}.defaultValue(20).withConstraint(validateUint32)},
{"dos_guard.max_requests", ConfigValue{ConfigType::Integer}.defaultValue(20).withConstraint(validateUint32)},
{"dos_guard.sweep_interval",
ConfigValue{ConfigType::Double}.defaultValue(1.0).withConstraint(validatePositiveDouble)},
{"cache.peers.[].ip", Array{ConfigValue{ConfigType::String}.withConstraint(validateIP)}},
{"cache.peers.[].port", Array{ConfigValue{ConfigType::String}.withConstraint(validatePort)}},
{"server.ip", ConfigValue{ConfigType::String}.withConstraint(validateIP)},
{"server.port", ConfigValue{ConfigType::Integer}.withConstraint(validatePort)},
{"server.workers", ConfigValue{ConfigType::Integer}.withConstraint(validateUint32)},
{"server.max_queue_size", ConfigValue{ConfigType::Integer}.defaultValue(0).withConstraint(validateUint32)},
{"server.local_admin", ConfigValue{ConfigType::Boolean}.optional()},
{"server.admin_password", ConfigValue{ConfigType::String}.optional()},
{"prometheus.enabled", ConfigValue{ConfigType::Boolean}.defaultValue(true)},
{"prometheus.compress_reply", ConfigValue{ConfigType::Boolean}.defaultValue(true)},
{"io_threads", ConfigValue{ConfigType::Integer}.defaultValue(2).withConstraint(validateUint16)},
{"cache.num_diffs", ConfigValue{ConfigType::Integer}.defaultValue(32).withConstraint(validateUint16)},
{"cache.num_markers", ConfigValue{ConfigType::Integer}.defaultValue(48).withConstraint(validateUint16)},
{"cache.num_cursors_from_diff", ConfigValue{ConfigType::Integer}.defaultValue(0).withConstraint(validateUint16)},
{"cache.num_cursors_from_account", ConfigValue{ConfigType::Integer}.defaultValue(0).withConstraint(validateUint16)
},
{"cache.page_fetch_size", ConfigValue{ConfigType::Integer}.defaultValue(512).withConstraint(validateUint16)},
{"cache.load", ConfigValue{ConfigType::String}.defaultValue("async").withConstraint(validateLoadMode)},
{"log_channels.[].channel", Array{ConfigValue{ConfigType::String}.optional().withConstraint(validateChannelName)}},
{"log_channels.[].log_level",
Array{ConfigValue{ConfigType::String}.optional().withConstraint(validateLogLevelName)}},
{"log_level", ConfigValue{ConfigType::String}.defaultValue("info").withConstraint(validateLogLevelName)},
{"log_format",
ConfigValue{ConfigType::String}.defaultValue(
R"(%TimeStamp% (%SourceLocation%) [%ThreadID%] %Channel%:%Severity% %Message%)"
)},
{"log_to_console", ConfigValue{ConfigType::Boolean}.defaultValue(false)},
{"log_directory", ConfigValue{ConfigType::String}.optional()},
{"log_rotation_size", ConfigValue{ConfigType::Integer}.defaultValue(2048u).withConstraint(validateUint32)},
{"log_directory_max_size",
ConfigValue{ConfigType::Integer}.defaultValue(50u * 1024u).withConstraint(validateUint32)},
{"log_rotation_hour_interval", ConfigValue{ConfigType::Integer}.defaultValue(12).withConstraint(validateUint32)},
{"log_tag_style", ConfigValue{ConfigType::String}.defaultValue("uint").withConstraint(validateLogTag)},
{"extractor_threads", ConfigValue{ConfigType::Integer}.defaultValue(2u).withConstraint(validateUint32)},
{"read_only", ConfigValue{ConfigType::Boolean}.defaultValue(false)},
{"txn_threshold", ConfigValue{ConfigType::Integer}.defaultValue(0).withConstraint(validateUint16)},
{"start_sequence", ConfigValue{ConfigType::Integer}.optional().withConstraint(validateUint32)},
{"finish_sequence", ConfigValue{ConfigType::Integer}.optional().withConstraint(validateUint32)},
{"ssl_cert_file", ConfigValue{ConfigType::String}.optional()},
{"ssl_key_file", ConfigValue{ConfigType::String}.optional()},
{"api_version.min", ConfigValue{ConfigType::Integer}},
{"api_version.max", ConfigValue{ConfigType::Integer}}}
};
ClioConfigDefinition::ClioConfigDefinition(std::initializer_list<KeyValuePair> pair)
{
@@ -154,7 +73,7 @@ ClioConfigDefinition::getObject(std::string_view prefix, std::optional<std::size
if (hasPrefix && !idx.has_value() && !mapKey.contains(prefixWithDot + "[]"))
return ObjectView{prefix, *this};
}
ASSERT(false, "Key {} is not found in config", prefixWithDot);
ASSERT(false, "Key {} is not found in config", prefix);
std::unreachable();
}
@@ -188,7 +107,7 @@ ClioConfigDefinition::hasItemsWithPrefix(std::string_view key) const
}
ValueView
ClioConfigDefinition::getValue(std::string_view fullKey) const
ClioConfigDefinition::getValueView(std::string_view fullKey) const
{
ASSERT(map_.contains(fullKey), "key {} does not exist in config", fullKey);
if (std::holds_alternative<ConfigValue>(map_.at(fullKey))) {
@@ -198,6 +117,13 @@ ClioConfigDefinition::getValue(std::string_view fullKey) const
std::unreachable();
}
std::chrono::milliseconds
ClioConfigDefinition::toMilliseconds(float value)
{
ASSERT(value >= 0.0f, "Floating point value of seconds must be non-negative, got: {}", value);
return std::chrono::milliseconds{std::lroundf(value * static_cast<float>(util::MILLISECONDS_PER_SECOND))};
}
ValueView
ClioConfigDefinition::getValueInArray(std::string_view fullKey, std::size_t index) const
{

View File

@@ -19,25 +19,33 @@
#pragma once
#include "rpc/common/APIVersion.hpp"
#include "util/Assert.hpp"
#include "util/newconfig/Array.hpp"
#include "util/newconfig/ConfigConstraints.hpp"
#include "util/newconfig/ConfigDescription.hpp"
#include "util/newconfig/ConfigFileInterface.hpp"
#include "util/newconfig/ConfigValue.hpp"
#include "util/newconfig/Error.hpp"
#include "util/newconfig/ObjectView.hpp"
#include "util/newconfig/Types.hpp"
#include "util/newconfig/ValueView.hpp"
#include <boost/json/value.hpp>
#include <boost/json/value_to.hpp>
#include <fmt/core.h>
#include <algorithm>
#include <cassert>
#include <chrono>
#include <cstddef>
#include <cstdint>
#include <expected>
#include <initializer_list>
#include <optional>
#include <string>
#include <string_view>
#include <thread>
#include <unordered_map>
#include <utility>
#include <variant>
@@ -113,7 +121,26 @@ public:
* @return ValueView associated with the given key
*/
[[nodiscard]] ValueView
getValue(std::string_view fullKey) const;
getValueView(std::string_view fullKey) const;
/**
* @brief Returns the specified value of given string if value exists
*
* @tparam T The type T to return
* @param fullKey The config key to search for
* @return Value of key of type T
*/
template <typename T>
T
get(std::string_view fullKey) const
{
ASSERT(map_.contains(fullKey), "key {} does not exist in config", fullKey);
auto const val = map_.at(fullKey);
if (std::holds_alternative<ConfigValue>(val)) {
return ValueView{std::get<ConfigValue>(val)}.getValueImpl<T>();
}
std::unreachable();
}
/**
* @brief Returns the specified ValueView object in an array with a given index
@@ -170,6 +197,29 @@ public:
[[nodiscard]] std::size_t
arraySize(std::string_view prefix) const;
/**
* @brief Method to convert a float seconds value to milliseconds.
*
* @param value The value to convert
* @return The value in milliseconds
*/
static std::chrono::milliseconds
toMilliseconds(float value);
/**
* @brief Returns the specified value of given string of type T if type and value exists
*
* @tparam T The type T to return
* @param fullKey The config key to search for
* @return The value of type T if it exists, std::nullopt otherwise.
*/
template <typename T>
std::optional<T>
maybeValue(std::string_view fullKey) const
{
return getValueView(fullKey).asOptional<T>();
}
/**
* @brief Returns an iterator to the beginning of the configuration map.
*
@@ -229,4 +279,108 @@ private:
std::unordered_map<std::string_view, std::variant<ConfigValue, Array>> map_;
};
/**
* @brief Full Clio Configuration definition.
*
* Specifies which keys are valid in Clio Config and provides default values if user's do not specify one. Those
* without default values must be present in the user's config file.
*/
static ClioConfigDefinition ClioConfig = ClioConfigDefinition{
{{"database.type", ConfigValue{ConfigType::String}.defaultValue("cassandra").withConstraint(validateCassandraName)},
{"database.cassandra.contact_points", ConfigValue{ConfigType::String}.defaultValue("localhost")},
{"database.cassandra.secure_connect_bundle", ConfigValue{ConfigType::String}.optional()},
{"database.cassandra.port", ConfigValue{ConfigType::Integer}.withConstraint(validatePort).optional()},
{"database.cassandra.keyspace", ConfigValue{ConfigType::String}.defaultValue("clio")},
{"database.cassandra.replication_factor",
ConfigValue{ConfigType::Integer}.defaultValue(3u).withConstraint(validateUint16)},
{"database.cassandra.table_prefix", ConfigValue{ConfigType::String}.optional()},
{"database.cassandra.max_write_requests_outstanding",
ConfigValue{ConfigType::Integer}.defaultValue(10'000).withConstraint(validateUint32)},
{"database.cassandra.max_read_requests_outstanding",
ConfigValue{ConfigType::Integer}.defaultValue(100'000).withConstraint(validateUint32)},
{"database.cassandra.threads",
ConfigValue{ConfigType::Integer}
.defaultValue(static_cast<uint32_t>(std::thread::hardware_concurrency()))
.withConstraint(validateUint32)},
{"database.cassandra.core_connections_per_host",
ConfigValue{ConfigType::Integer}.defaultValue(1).withConstraint(validateUint16)},
{"database.cassandra.queue_size_io", ConfigValue{ConfigType::Integer}.optional().withConstraint(validateUint16)},
{"database.cassandra.write_batch_size",
ConfigValue{ConfigType::Integer}.defaultValue(20).withConstraint(validateUint16)},
{"database.cassandra.connect_timeout", ConfigValue{ConfigType::Integer}.optional().withConstraint(validateUint32)},
{"database.cassandra.request_timeout", ConfigValue{ConfigType::Integer}.optional().withConstraint(validateUint32)},
{"database.cassandra.username", ConfigValue{ConfigType::String}.optional()},
{"database.cassandra.password", ConfigValue{ConfigType::String}.optional()},
{"database.cassandra.certfile", ConfigValue{ConfigType::String}.optional()},
{"allow_no_etl", ConfigValue{ConfigType::Boolean}.defaultValue(false)},
{"etl_sources.[].ip", Array{ConfigValue{ConfigType::String}.optional().withConstraint(validateIP)}},
{"etl_sources.[].ws_port", Array{ConfigValue{ConfigType::String}.optional().withConstraint(validatePort)}},
{"etl_sources.[].grpc_port", Array{ConfigValue{ConfigType::String}.optional().withConstraint(validatePort)}},
{"forwarding.cache_timeout",
ConfigValue{ConfigType::Double}.defaultValue(0.0).withConstraint(validatePositiveDouble)},
{"forwarding.request_timeout",
ConfigValue{ConfigType::Double}.defaultValue(10.0).withConstraint(validatePositiveDouble)},
{"rpc.cache_timeout", ConfigValue{ConfigType::Double}.defaultValue(0.0).withConstraint(validatePositiveDouble)},
{"num_markers", ConfigValue{ConfigType::Integer}.optional().withConstraint(validateNumMarkers)},
{"dos_guard.whitelist.[]", Array{ConfigValue{ConfigType::String}}},
{"dos_guard.max_fetches", ConfigValue{ConfigType::Integer}.defaultValue(1000'000u).withConstraint(validateUint32)},
{"dos_guard.max_connections", ConfigValue{ConfigType::Integer}.defaultValue(20u).withConstraint(validateUint32)},
{"dos_guard.max_requests", ConfigValue{ConfigType::Integer}.defaultValue(20u).withConstraint(validateUint32)},
{"dos_guard.sweep_interval",
ConfigValue{ConfigType::Double}.defaultValue(1.0).withConstraint(validatePositiveDouble)},
{"workers",
ConfigValue{ConfigType::Integer}.defaultValue(std::thread::hardware_concurrency()).withConstraint(validateUint32)
},
{"server.ip", ConfigValue{ConfigType::String}.withConstraint(validateIP)},
{"server.port", ConfigValue{ConfigType::Integer}.withConstraint(validatePort)},
{"server.max_queue_size", ConfigValue{ConfigType::Integer}.defaultValue(0).withConstraint(validateUint32)},
{"server.local_admin", ConfigValue{ConfigType::Boolean}.optional()},
{"server.admin_password", ConfigValue{ConfigType::String}.optional()},
{"server.processing_policy",
ConfigValue{ConfigType::String}.defaultValue("parallel").withConstraint(validateProcessingPolicy)},
{"server.parallel_requests_limit", ConfigValue{ConfigType::Integer}.optional()},
{"server.ws_max_sending_queue_size", ConfigValue{ConfigType::Integer}.defaultValue(1500)},
{"server.__ng_web_server", ConfigValue{ConfigType::Boolean}.defaultValue(false)},
{"prometheus.enabled", ConfigValue{ConfigType::Boolean}.defaultValue(true)},
{"prometheus.compress_reply", ConfigValue{ConfigType::Boolean}.defaultValue(true)},
{"io_threads", ConfigValue{ConfigType::Integer}.defaultValue(2).withConstraint(validateIOThreads)},
{"subscription_workers", ConfigValue{ConfigType::Integer}.defaultValue(1).withConstraint(validateUint32)},
{"graceful_period", ConfigValue{ConfigType::Double}.defaultValue(10.0).withConstraint(validatePositiveDouble)},
{"cache.num_diffs", ConfigValue{ConfigType::Integer}.defaultValue(32).withConstraint(validateUint16)},
{"cache.num_markers", ConfigValue{ConfigType::Integer}.defaultValue(48).withConstraint(validateUint16)},
{"cache.num_cursors_from_diff", ConfigValue{ConfigType::Integer}.defaultValue(0).withConstraint(validateUint16)},
{"cache.num_cursors_from_account", ConfigValue{ConfigType::Integer}.defaultValue(0).withConstraint(validateUint16)
},
{"cache.page_fetch_size", ConfigValue{ConfigType::Integer}.defaultValue(512).withConstraint(validateUint16)},
{"cache.load", ConfigValue{ConfigType::String}.defaultValue("async").withConstraint(validateLoadMode)},
{"log_channels.[].channel", Array{ConfigValue{ConfigType::String}.optional().withConstraint(validateChannelName)}},
{"log_channels.[].log_level",
Array{ConfigValue{ConfigType::String}.optional().withConstraint(validateLogLevelName)}},
{"log_level", ConfigValue{ConfigType::String}.defaultValue("info").withConstraint(validateLogLevelName)},
{"log_format",
ConfigValue{ConfigType::String}.defaultValue(
R"(%TimeStamp% (%SourceLocation%) [%ThreadID%] %Channel%:%Severity% %Message%)"
)},
{"log_to_console", ConfigValue{ConfigType::Boolean}.defaultValue(false)},
{"log_directory", ConfigValue{ConfigType::String}.optional()},
{"log_rotation_size", ConfigValue{ConfigType::Integer}.defaultValue(2048).withConstraint(validateUint32)},
{"log_directory_max_size", ConfigValue{ConfigType::Integer}.defaultValue(50 * 1024).withConstraint(validateUint32)
},
{"log_rotation_hour_interval", ConfigValue{ConfigType::Integer}.defaultValue(12).withConstraint(validateUint32)},
{"log_tag_style", ConfigValue{ConfigType::String}.defaultValue("none").withConstraint(validateLogTag)},
{"extractor_threads", ConfigValue{ConfigType::Integer}.defaultValue(1u).withConstraint(validateUint32)},
{"read_only", ConfigValue{ConfigType::Boolean}.defaultValue(false)},
{"txn_threshold", ConfigValue{ConfigType::Integer}.defaultValue(0).withConstraint(validateUint16)},
{"start_sequence", ConfigValue{ConfigType::Integer}.optional().withConstraint(validateUint32)},
{"finish_sequence", ConfigValue{ConfigType::Integer}.optional().withConstraint(validateUint32)},
{"ssl_cert_file", ConfigValue{ConfigType::String}.optional()},
{"ssl_key_file", ConfigValue{ConfigType::String}.optional()},
{"api_version.default",
ConfigValue{ConfigType::Integer}.defaultValue(rpc::API_VERSION_DEFAULT).withConstraint(validateApiVersion)},
{"api_version.min",
ConfigValue{ConfigType::Integer}.defaultValue(rpc::API_VERSION_MIN).withConstraint(validateApiVersion)},
{"api_version.max",
ConfigValue{ConfigType::Integer}.defaultValue(rpc::API_VERSION_MAX).withConstraint(validateApiVersion)}}
};
} // namespace util::config

View File

@@ -63,9 +63,15 @@ public:
private:
static constexpr auto configDescription = std::array{
KV{.key = "database.type", .value = "Type of database to use."},
KV{.key = "database.type", .value = "Type of database to use. Default is Scylladb."},
KV{.key = "database.cassandra.contact_points",
.value = "Comma-separated list of contact points for Cassandra nodes."},
.value =
"A list of IP addresses or hostnames of the initial nodes (Cassandra/Scylladb cluster nodes) that the "
"client will connect to when establishing a connection with the database."},
KV{.key = "database.cassandra.secure_connect_bundle",
.value = "Configuration file that contains the necessary security credentials and connection details for "
"securely "
"connecting to a Cassandra database cluster."},
KV{.key = "database.cassandra.port", .value = "Port number to connect to Cassandra."},
KV{.key = "database.cassandra.keyspace", .value = "Keyspace to use in Cassandra."},
KV{.key = "database.cassandra.replication_factor", .value = "Number of replicated nodes for Scylladb."},
@@ -79,30 +85,59 @@ private:
.value = "Number of core connections per host for Cassandra."},
KV{.key = "database.cassandra.queue_size_io", .value = "Queue size for I/O operations in Cassandra."},
KV{.key = "database.cassandra.write_batch_size", .value = "Batch size for write operations in Cassandra."},
KV{.key = "etl_source.[].ip", .value = "IP address of the ETL source."},
KV{.key = "etl_source.[].ws_port", .value = "WebSocket port of the ETL source."},
KV{.key = "etl_source.[].grpc_port", .value = "gRPC port of the ETL source."},
KV{.key = "database.cassandra.connect_timeout",
.value = "The maximum amount of time in seconds the system will wait for a connection to be successfully "
"established "
"with the database."},
KV{.key = "database.cassandra.request_timeout",
.value =
"The maximum amount of time in seconds the system will wait for a request to be fetched from database."},
KV{.key = "database.cassandra.username", .value = "The username used for authenticating with the database."},
KV{.key = "database.cassandra.password", .value = "The password used for authenticating with the database."},
KV{.key = "database.cassandra.certfile",
.value = "The path to the SSL/TLS certificate file used to establish a secure connection between the client "
"and the "
"Cassandra database."},
KV{.key = "allow_no_etl", .value = "If True, no ETL nodes will run with Clio."},
KV{.key = "etl_sources.[].ip", .value = "IP address of the ETL source."},
KV{.key = "etl_sources.[].ws_port", .value = "WebSocket port of the ETL source."},
KV{.key = "etl_sources.[].grpc_port", .value = "gRPC port of the ETL source."},
KV{.key = "forwarding.cache_timeout",
.value = "Timeout duration for the forwarding cache used in Rippled communication."},
KV{.key = "forwarding.request_timeout",
.value = "Timeout duration for the forwarding request used in Rippled communication."},
KV{.key = "rpc.cache_timeout", .value = "Timeout duration for the rpc request."},
KV{.key = "num_markers",
.value = "The number of markers is the number of coroutines to load the cache concurrently."},
KV{.key = "dos_guard.[].whitelist", .value = "List of IP addresses to whitelist for DOS protection."},
KV{.key = "dos_guard.max_fetches", .value = "Maximum number of fetch operations allowed by DOS guard."},
KV{.key = "dos_guard.max_connections", .value = "Maximum number of concurrent connections allowed by DOS guard."
},
KV{.key = "dos_guard.max_requests", .value = "Maximum number of requests allowed by DOS guard."},
KV{.key = "dos_guard.sweep_interval", .value = "Interval in seconds for DOS guard to sweep/clear its state."},
KV{.key = "cache.peers.[].ip", .value = "IP address of peer nodes to cache."},
KV{.key = "cache.peers.[].port", .value = "Port number of peer nodes to cache."},
KV{.key = "workers", .value = "Number of threads to process RPC requests."},
KV{.key = "server.ip", .value = "IP address of the Clio HTTP server."},
KV{.key = "server.port", .value = "Port number of the Clio HTTP server."},
KV{.key = "server.max_queue_size", .value = "Maximum size of the server's request queue."},
KV{.key = "server.workers", .value = "Maximum number of threads for server to run with."},
KV{.key = "server.max_queue_size",
.value = "Maximum size of the server's request queue. Value of 0 is no limit."},
KV{.key = "server.local_admin", .value = "Indicates if the server should run with admin privileges."},
KV{.key = "server.admin_password", .value = "Password for Clio admin-only APIs."},
KV{.key = "server.processing_policy",
.value = R"(Could be "sequent" or "parallel". For the sequent policy, requests from a single client
connection are processed one by one, with the next request read only after the previous one is processed. For the parallel policy, Clio will accept
all requests and process them in parallel, sending a reply for each request as soon as it is ready.)"},
KV{.key = "server.parallel_requests_limit",
.value = R"(Optional parameter, used only if "processing_strategy" is
"parallel". It limits the number of requests for a single client connection that are processed in parallel. If not specified, the limit is infinite.)"
},
KV{.key = "server.ws_max_sending_queue_size", .value = "Maximum size of the websocket sending queue."},
KV{.key = "prometheus.enabled", .value = "Enable or disable Prometheus metrics."},
KV{.key = "prometheus.compress_reply", .value = "Enable or disable compression of Prometheus responses."},
KV{.key = "io_threads", .value = "Number of I/O threads."},
KV{.key = "io_threads", .value = "Number of I/O threads. Value must be greater than 1"},
KV{.key = "subscription_workers",
.value = "The number of worker threads or processes that are responsible for managing and processing "
"subscription-based tasks."},
KV{.key = "graceful_period", .value = "Number of milliseconds server will wait to shutdown gracefully."},
KV{.key = "cache.num_diffs", .value = "Number of diffs to cache."},
KV{.key = "cache.num_markers", .value = "Number of markers to cache."},
KV{.key = "cache.num_cursors_from_diff", .value = "Num of cursors that are different."},
@@ -126,6 +161,7 @@ private:
KV{.key = "finish_sequence", .value = "Ending ledger index."},
KV{.key = "ssl_cert_file", .value = "Path to the SSL certificate file."},
KV{.key = "ssl_key_file", .value = "Path to the SSL key file."},
KV{.key = "api_version.default", .value = "Default API version Clio will run on."},
KV{.key = "api_version.min", .value = "Minimum API version."},
KV{.key = "api_version.max", .value = "Maximum API version."}
};

View File

@@ -32,6 +32,7 @@
#include <fmt/core.h>
#include <cstddef>
#include <cstdint>
#include <exception>
#include <fstream>
#include <ios>
@@ -58,6 +59,9 @@ extractJsonValue(boost::json::value const& jsonValue)
if (jsonValue.is_int64()) {
return jsonValue.as_int64();
}
if (jsonValue.is_uint64()) {
return static_cast<int64_t>(jsonValue.as_uint64());
}
if (jsonValue.is_string()) {
return jsonValue.as_string().c_str();
}
@@ -67,7 +71,7 @@ extractJsonValue(boost::json::value const& jsonValue)
if (jsonValue.is_double()) {
return jsonValue.as_double();
}
ASSERT(false, "Json is not of type int, string, bool or double");
ASSERT(false, "Json is not of type int, uint, string, bool or double");
std::unreachable();
}
} // namespace

View File

@@ -219,7 +219,9 @@ private:
if (type == ConfigType::Boolean && !std::holds_alternative<bool>(value)) {
return Error{"value does not match type boolean"};
}
if (type == ConfigType::Double && !std::holds_alternative<double>(value)) {
if (type == ConfigType::Double && (!std::holds_alternative<double>(value))) {
if (std::holds_alternative<int64_t>(value))
return std::nullopt;
return Error{"value does not match type double"};
}
if (type == ConfigType::Integer && !std::holds_alternative<int64_t>(value)) {

View File

@@ -51,13 +51,13 @@ ObjectView::containsKey(std::string_view key) const
}
ValueView
ObjectView::getValue(std::string_view key) const
ObjectView::getValueView(std::string_view key) const
{
auto const fullKey = getFullKey(key);
if (arrayIndex_.has_value()) {
return clioConfig_.get().getArray(fullKey).valueAt(arrayIndex_.value());
}
return clioConfig_.get().getValue(fullKey);
return clioConfig_.get().getValueView(fullKey);
}
ObjectView

View File

@@ -19,7 +19,6 @@
#pragma once
#include "util/newconfig/ConfigValue.hpp"
#include "util/newconfig/ValueView.hpp"
#include <cstddef>
@@ -73,7 +72,35 @@ public:
* @return A ValueView object representing the value associated with the key
*/
[[nodiscard]] ValueView
getValue(std::string_view key) const;
getValueView(std::string_view key) const;
/**
* @brief Returns the specified value of given string if value exists
*
* @tparam T The type T to return
* @param key The config key to add to prefix and then search for
* @return Value of key of type T
*/
template <typename T>
T
get(std::string_view key) const
{
return getValueView(key).getValueImpl<T>();
}
/**
* @brief Returns the specified value of given string of type T if type and value exists
*
* @tparam T The type T to return
* @param key The config key to add to prefix and then search for
* @return The value of type T if it exists, std::nullopt otherwise.
*/
template <typename T>
std::optional<T>
maybeValue(std::string_view key) const
{
return getValueView(key).asOptional<T>();
}
/**
* @brief Retrieves an ObjectView in ClioConfigDefinition with key that starts with prefix_.key. The view must be of

View File

@@ -25,8 +25,8 @@
#include <cstdint>
#include <string>
#include <string_view>
#include <utility>
#include <variant>
namespace util::config {
@@ -34,7 +34,7 @@ ValueView::ValueView(ConfigValue const& configVal) : configVal_{configVal}
{
}
std::string_view
std::string
ValueView::asString() const
{
if (this->type() == ConfigType::String && configVal_.get().hasValue())
@@ -56,9 +56,14 @@ double
ValueView::asDouble() const
{
if (configVal_.get().hasValue()) {
if (type() == ConfigType::Double)
return std::get<double>(configVal_.get().getValue());
auto const& val = configVal_.get().getValue();
if (type() == ConfigType::Double) {
if (std::holds_alternative<int64_t>(val)) {
return static_cast<double>(std::get<int64_t>(val));
}
return static_cast<double>(std::get<double>(val));
}
if (type() == ConfigType::Integer)
return static_cast<double>(std::get<int64_t>(configVal_.get().getValue()));
}
@@ -70,11 +75,16 @@ float
ValueView::asFloat() const
{
if (configVal_.get().hasValue()) {
auto const& val = configVal_.get().getValue();
if (type() == ConfigType::Double) {
return static_cast<float>(std::get<double>(configVal_.get().getValue()));
if (std::holds_alternative<int64_t>(val)) {
return static_cast<float>(std::get<int64_t>(val));
}
return static_cast<float>(std::get<double>(val));
}
if (type() == ConfigType::Integer)
return static_cast<float>(std::get<int64_t>(configVal_.get().getValue()));
return static_cast<float>(std::get<int64_t>(val));
}
ASSERT(false, "Value view is not of Float type");
std::unreachable();

View File

@@ -29,7 +29,10 @@
#include <cstddef>
#include <cstdint>
#include <functional>
#include <string_view>
#include <optional>
#include <string>
#include <type_traits>
#include <utility>
namespace util::config {
@@ -53,7 +56,7 @@ public:
* @return The value as a string
* @throws std::bad_variant_access if the value is not a string
*/
[[nodiscard]] std::string_view
[[nodiscard]] std::string
asString() const;
/**
@@ -140,6 +143,51 @@ public:
return configVal_.get().isOptional();
}
/**
* @brief Retrieves the stored value as the specified type T
*
* @tparam T The type to cast the stored value to
* @return The value cast to the specified type T
*/
template <typename T>
T
getValueImpl() const
{
ASSERT(configVal_.get().hasValue(), "ConfigValue does not have a value");
if constexpr (std::is_same_v<T, bool>) {
ASSERT(type() == ConfigType::Boolean, "Value type is not a bool");
return asBool();
} else if constexpr (std::is_integral_v<T>) {
ASSERT(type() == ConfigType::Integer, "Value type is not an int");
return asIntType<T>();
} else if constexpr (std::is_same_v<T, std::string>) {
ASSERT(type() == ConfigType::String, "Value type is not a string");
return asString();
} else if constexpr (std::is_floating_point_v<T>) {
ASSERT(type() == ConfigType::Double || type() == ConfigType::Integer, "Value type is not a floating point");
return asDouble();
}
std::unreachable();
}
/**
* @brief Returns an optional value of the specified type T if valid
*
* @tparam T The type of value to retrieve (must be compatible with internal type)
* @return Returns the value as an optional<T> value exists, or std::nullopt if not
*/
template <typename T>
std::optional<T>
asOptional() const
{
ASSERT(isOptional(), "A Config Value is not an optional value");
if (!hasValue())
return std::nullopt;
return std::make_optional(getValueImpl<T>());
}
private:
std::reference_wrapper<ConfigValue const> configVal_;
};

View File

@@ -20,7 +20,7 @@
#include "util/prometheus/Prometheus.hpp"
#include "util/Assert.hpp"
#include "util/config/Config.hpp"
#include "util/newconfig/ConfigDefinition.hpp"
#include "util/prometheus/Bool.hpp"
#include "util/prometheus/Counter.hpp"
#include "util/prometheus/Gauge.hpp"
@@ -176,10 +176,10 @@ PrometheusImpl::getMetric(
} // namespace util::prometheus
void
PrometheusService::init(util::Config const& config)
PrometheusService::init(util::config::ClioConfigDefinition const& config)
{
bool const enabled = config.valueOr("prometheus.enabled", true);
bool const compressReply = config.valueOr("prometheus.compress_reply", true);
bool const enabled = config.get<bool>("prometheus.enabled");
bool const compressReply = config.get<bool>("prometheus.compress_reply");
instance_ = std::make_unique<util::prometheus::PrometheusImpl>(enabled, compressReply);
}

View File

@@ -19,7 +19,7 @@
#pragma once
#include "util/config/Config.hpp"
#include "util/log/Logger.hpp"
#include "util/prometheus/Bool.hpp"
#include "util/prometheus/Counter.hpp"
#include "util/prometheus/Gauge.hpp"
@@ -28,6 +28,7 @@
#include "util/prometheus/MetricBase.hpp"
#include "util/prometheus/MetricsFamily.hpp"
#include <concepts>
#include <cstdint>
#include <memory>
#include <optional>
@@ -256,7 +257,7 @@ public:
*
* @param config The configuration to use
*/
void static init(util::Config const& config = util::Config{});
void static init(util::config::ClioConfigDefinition const& config);
/**
* @brief Get a bool based metric. It will be created if it doesn't exist

View File

@@ -20,7 +20,7 @@
#include "web/AdminVerificationStrategy.hpp"
#include "util/JsonUtils.hpp"
#include "util/config/Config.hpp"
#include "util/newconfig/ConfigDefinition.hpp"
#include <boost/beast/http/field.hpp>
#include <xrpl/basics/base_uint.h>
@@ -81,7 +81,7 @@ make_AdminVerificationStrategy(std::optional<std::string> password)
}
std::expected<std::shared_ptr<AdminVerificationStrategy>, std::string>
make_AdminVerificationStrategy(util::Config const& config)
make_AdminVerificationStrategy(util::config::ClioConfigDefinition const& config)
{
auto adminPassword = config.maybeValue<std::string>("server.admin_password");
auto const localAdmin = config.maybeValue<bool>("server.local_admin");

View File

@@ -19,7 +19,7 @@
#pragma once
#include "util/config/Config.hpp"
#include "util/newconfig/ConfigDefinition.hpp"
#include <boost/beast/http.hpp>
#include <boost/beast/http/message.hpp>
@@ -116,6 +116,6 @@ make_AdminVerificationStrategy(std::optional<std::string> password);
* @return Admin verification strategy according to the config or an error message.
*/
std::expected<std::shared_ptr<AdminVerificationStrategy>, std::string>
make_AdminVerificationStrategy(util::Config const& serverConfig);
make_AdminVerificationStrategy(util::config::ClioConfigDefinition const& serverConfig);
} // namespace web

View File

@@ -28,8 +28,8 @@
#include "util/JsonUtils.hpp"
#include "util/Profiler.hpp"
#include "util/Taggable.hpp"
#include "util/config/Config.hpp"
#include "util/log/Logger.hpp"
#include "util/newconfig/ConfigDefinition.hpp"
#include "web/impl/ErrorHandling.hpp"
#include "web/interface/ConnectionBase.hpp"
@@ -79,7 +79,7 @@ public:
* @param etl The ETL to use
*/
RPCServerHandler(
util::Config const& config,
util::config::ClioConfigDefinition const& config,
std::shared_ptr<BackendInterface const> const& backend,
std::shared_ptr<RPCEngineType> const& rpcEngine,
std::shared_ptr<ETLType const> const& etl
@@ -88,7 +88,7 @@ public:
, rpcEngine_(rpcEngine)
, etl_(etl)
, tagFactory_(config)
, apiVersionParser_(config.sectionOr("api_version", {}))
, apiVersionParser_(config.getObject("api_version"))
{
}

View File

@@ -336,7 +336,7 @@ using HttpServer = Server<HttpSession, SslHttpSession, HandlerType>;
template <typename HandlerType>
static std::shared_ptr<HttpServer<HandlerType>>
make_HttpServer(
util::Config const& config,
util::config::ClioConfigDefinition const& config,
boost::asio::io_context& ioc,
dosguard::DOSGuardInterface& dosGuard,
std::shared_ptr<HandlerType> const& handler
@@ -350,12 +350,9 @@ make_HttpServer(
return nullptr;
}
if (!config.contains("server"))
return nullptr;
auto const serverConfig = config.section("server");
auto const address = boost::asio::ip::make_address(serverConfig.value<std::string>("ip"));
auto const port = serverConfig.value<unsigned short>("port");
auto const serverConfig = config.getObject("server");
auto const address = boost::asio::ip::make_address(serverConfig.get<std::string>("ip"));
auto const port = serverConfig.get<unsigned short>("port");
auto expectedAdminVerification = make_AdminVerificationStrategy(config);
if (not expectedAdminVerification.has_value()) {
@@ -365,7 +362,7 @@ make_HttpServer(
// If the transactions number is 200 per ledger, A client which subscribes everything will send 400+ feeds for
// each ledger. we allow user delay 3 ledgers by default
auto const maxWsSendingQueueSize = serverConfig.valueOr("ws_max_sending_queue_size", 1500);
auto const maxWsSendingQueueSize = serverConfig.get<uint32_t>("ws_max_sending_queue_size");
auto server = std::make_shared<HttpServer<HandlerType>>(
ioc,

View File

@@ -20,8 +20,10 @@
#include "web/dosguard/DOSGuard.hpp"
#include "util/Assert.hpp"
#include "util/config/Config.hpp"
#include "util/log/Logger.hpp"
#include "util/newconfig/ArrayView.hpp"
#include "util/newconfig/ConfigDefinition.hpp"
#include "util/newconfig/ValueView.hpp"
#include "web/dosguard/WhitelistHandlerInterface.hpp"
#include <boost/iterator/transform_iterator.hpp>
@@ -33,13 +35,15 @@
#include <string_view>
#include <unordered_set>
using namespace util::config;
namespace web::dosguard {
DOSGuard::DOSGuard(util::Config const& config, WhitelistHandlerInterface const& whitelistHandler)
DOSGuard::DOSGuard(ClioConfigDefinition const& config, WhitelistHandlerInterface const& whitelistHandler)
: whitelistHandler_{std::cref(whitelistHandler)}
, maxFetches_{config.valueOr("dos_guard.max_fetches", DEFAULT_MAX_FETCHES)}
, maxConnCount_{config.valueOr("dos_guard.max_connections", DEFAULT_MAX_CONNECTIONS)}
, maxRequestCount_{config.valueOr("dos_guard.max_requests", DEFAULT_MAX_REQUESTS)}
, maxFetches_{config.get<uint32_t>("dos_guard.max_fetches")}
, maxConnCount_{config.get<uint32_t>("dos_guard.max_connections")}
, maxRequestCount_{config.get<uint32_t>("dos_guard.max_requests")}
{
}
@@ -134,15 +138,16 @@ DOSGuard::clear() noexcept
}
[[nodiscard]] std::unordered_set<std::string>
DOSGuard::getWhitelist(util::Config const& config)
DOSGuard::getWhitelist(ClioConfigDefinition const& config)
{
using T = std::unordered_set<std::string> const;
auto whitelist = config.arrayOr("dos_guard.whitelist", {});
auto const transform = [](auto const& elem) { return elem.template value<std::string>(); };
return T{
boost::transform_iterator(std::begin(whitelist), transform),
boost::transform_iterator(std::end(whitelist), transform)
};
std::unordered_set<std::string> ips;
auto const whitelist = config.getArray("dos_guard.whitelist");
for (auto it = whitelist.begin<ValueView>(); it != whitelist.end<ValueView>(); ++it) {
ips.insert((*it).asString());
}
return ips;
}
} // namespace web::dosguard

View File

@@ -19,8 +19,8 @@
#pragma once
#include "util/config/Config.hpp"
#include "util/log/Logger.hpp"
#include "util/newconfig/ConfigDefinition.hpp"
#include "web/dosguard/DOSGuardInterface.hpp"
#include "web/dosguard/WhitelistHandlerInterface.hpp"
@@ -63,17 +63,13 @@ class DOSGuard : public DOSGuardInterface {
util::Logger log_{"RPC"};
public:
static constexpr std::uint32_t DEFAULT_MAX_FETCHES = 1000'000u; /**< Default maximum fetches per sweep */
static constexpr std::uint32_t DEFAULT_MAX_CONNECTIONS = 20u; /**< Default maximum concurrent connections */
static constexpr std::uint32_t DEFAULT_MAX_REQUESTS = 20u; /**< Default maximum requests per sweep */
/**
* @brief Constructs a new DOS guard.
*
* @param config Clio config
* @param whitelistHandler Whitelist handler that checks whitelist for IP addresses
*/
DOSGuard(util::Config const& config, WhitelistHandlerInterface const& whitelistHandler);
DOSGuard(util::config::ClioConfigDefinition const& config, WhitelistHandlerInterface const& whitelistHandler);
/**
* @brief Check whether an ip address is in the whitelist or not.
@@ -148,7 +144,7 @@ public:
private:
[[nodiscard]] static std::unordered_set<std::string>
getWhitelist(util::Config const& config);
getWhitelist(util::config::ClioConfigDefinition const& config);
};
} // namespace web::dosguard

View File

@@ -19,7 +19,7 @@
#include "web/dosguard/IntervalSweepHandler.hpp"
#include "util/config/Config.hpp"
#include "util/newconfig/ConfigDefinition.hpp"
#include "web/dosguard/DOSGuardInterface.hpp"
#include <boost/asio/io_context.hpp>
@@ -32,14 +32,15 @@
namespace web::dosguard {
IntervalSweepHandler::IntervalSweepHandler(
util::Config const& config,
util::config::ClioConfigDefinition const& config,
boost::asio::io_context& ctx,
BaseDOSGuard& dosGuard
)
: repeat_{std::ref(ctx)}
{
auto const sweepInterval{std::max(
std::chrono::milliseconds{1u}, util::Config::toMilliseconds(config.valueOr("dos_guard.sweep_interval", 1.0))
std::chrono::milliseconds{1u},
util::config::ClioConfigDefinition::toMilliseconds(config.get<double>("dos_guard.sweep_interval"))
)};
repeat_.start(sweepInterval, [&dosGuard] { dosGuard.clear(); });
}

View File

@@ -20,7 +20,7 @@
#pragma once
#include "util/Repeat.hpp"
#include "util/config/Config.hpp"
#include "util/newconfig/ConfigDefinition.hpp"
#include <boost/asio/io_context.hpp>
@@ -43,7 +43,7 @@ public:
* @param dosGuard The DOS guard to use
*/
IntervalSweepHandler(
util::Config const& config,
util::config::ClioConfigDefinition const& config,
boost::asio::io_context& ctx,
web::dosguard::BaseDOSGuard& dosGuard
);

View File

@@ -19,7 +19,9 @@
#pragma once
#include "util/config/Config.hpp"
#include "util/newconfig/ArrayView.hpp"
#include "util/newconfig/ConfigDefinition.hpp"
#include "util/newconfig/ValueView.hpp"
#include "web/Resolver.hpp"
#include "web/dosguard/WhitelistHandlerInterface.hpp"
@@ -99,7 +101,7 @@ public:
* @param resolver The resolver to use for hostname resolution
*/
template <SomeResolver HostnameResolverType = Resolver>
WhitelistHandler(util::Config const& config, HostnameResolverType&& resolver = {})
WhitelistHandler(util::config::ClioConfigDefinition const& config, HostnameResolverType&& resolver = {})
{
std::unordered_set<std::string> const arr = getWhitelist(config, std::forward<HostnameResolverType>(resolver));
for (auto const& net : arr)
@@ -121,24 +123,23 @@ public:
private:
template <SomeResolver HostnameResolverType>
[[nodiscard]] static std::unordered_set<std::string>
getWhitelist(util::Config const& config, HostnameResolverType&& resolver)
getWhitelist(util::config::ClioConfigDefinition const& config, HostnameResolverType&& resolver)
{
auto whitelist = config.arrayOr("dos_guard.whitelist", {});
auto const transform = [](auto const& elem) { return elem.template value<std::string>(); };
std::unordered_set<std::string> const hostnames{
boost::transform_iterator(std::begin(whitelist), transform),
boost::transform_iterator(std::end(whitelist), transform)
};
auto const whitelist = config.getArray("dos_guard.whitelist");
std::unordered_set<std::string> hostnames{};
// resolve hostnames to ips
std::unordered_set<std::string> ips;
for (auto it = whitelist.begin<util::config::ValueView>(); it != whitelist.end<util::config::ValueView>(); ++it)
hostnames.insert((*it).asString());
for (auto const& hostname : hostnames) {
auto resolvedIps = resolver.resolve(hostname);
for (auto& ip : resolvedIps) {
ips.insert(std::move(ip));
}
};
return ips;
}
};

View File

@@ -30,7 +30,6 @@
#include "util/JsonUtils.hpp"
#include "util/Profiler.hpp"
#include "util/Taggable.hpp"
#include "util/config/Config.hpp"
#include "util/log/Logger.hpp"
#include "web/SubscriptionContextInterface.hpp"
#include "web/ng/Connection.hpp"
@@ -87,7 +86,7 @@ public:
* @param etl The ETL to use
*/
RPCServerHandler(
util::Config const& config,
util::config::ClioConfigDefinition const& config,
std::shared_ptr<BackendInterface const> const& backend,
std::shared_ptr<RPCEngineType> const& rpcEngine,
std::shared_ptr<ETLType const> const& etl
@@ -96,7 +95,7 @@ public:
, rpcEngine_(rpcEngine)
, etl_(etl)
, tagFactory_(config)
, apiVersionParser_(config.sectionOr("api_version", {}))
, apiVersionParser_(config.getObject("api_version"))
{
}

View File

@@ -21,8 +21,9 @@
#include "util/Assert.hpp"
#include "util/Taggable.hpp"
#include "util/config/Config.hpp"
#include "util/log/Logger.hpp"
#include "util/newconfig/ConfigDefinition.hpp"
#include "util/newconfig/ObjectView.hpp"
#include "web/ng/Connection.hpp"
#include "web/ng/MessageHandler.hpp"
#include "web/ng/ProcessingPolicy.hpp"
@@ -56,22 +57,17 @@ namespace web::ng {
namespace {
std::expected<boost::asio::ip::tcp::endpoint, std::string>
makeEndpoint(util::Config const& serverConfig)
makeEndpoint(util::config::ObjectView const& serverConfig)
{
auto const ip = serverConfig.maybeValue<std::string>("ip");
if (not ip.has_value())
return std::unexpected{"Missing 'ip` in server config."};
auto const ip = serverConfig.get<std::string>("ip");
boost::system::error_code error;
auto const address = boost::asio::ip::make_address(*ip, error);
auto const address = boost::asio::ip::make_address(ip, error);
if (error)
return std::unexpected{fmt::format("Error parsing provided IP: {}", error.message())};
auto const port = serverConfig.maybeValue<unsigned short>("port");
if (not port.has_value())
return std::unexpected{"Missing 'port` in server config."};
return boost::asio::ip::tcp::endpoint{address, *port};
auto const port = serverConfig.get<unsigned short>("port");
return boost::asio::ip::tcp::endpoint{address, port};
}
std::expected<boost::asio::ip::tcp::acceptor, std::string>
@@ -305,13 +301,13 @@ Server::handleConnection(boost::asio::ip::tcp::socket socket, boost::asio::yield
std::expected<Server, std::string>
make_Server(
util::Config const& config,
util::config::ClioConfigDefinition const& config,
Server::OnConnectCheck onConnectCheck,
Server::OnDisconnectHook onDisconnectHook,
boost::asio::io_context& context
)
{
auto const serverConfig = config.section("server");
auto const serverConfig = config.getObject("server");
auto endpoint = makeEndpoint(serverConfig);
if (not endpoint.has_value())
@@ -324,7 +320,7 @@ make_Server(
ProcessingPolicy processingPolicy{ProcessingPolicy::Parallel};
std::optional<size_t> parallelRequestLimit;
auto const processingStrategyStr = serverConfig.valueOr<std::string>("processing_policy", "parallel");
auto const processingStrategyStr = serverConfig.get<std::string>("processing_policy");
if (processingStrategyStr == "sequent") {
processingPolicy = ProcessingPolicy::Sequential;
} else if (processingStrategyStr == "parallel") {
@@ -333,7 +329,7 @@ make_Server(
return std::unexpected{fmt::format("Invalid 'server.processing_strategy': {}", processingStrategyStr)};
}
auto const maxSubscriptionSendQueueSize = serverConfig.maybeValue<size_t>("ws_max_sending_queue_size");
auto const maxSubscriptionSendQueueSize = serverConfig.get<size_t>("ws_max_sending_queue_size");
return Server{
context,

View File

@@ -20,8 +20,8 @@
#pragma once
#include "util/Taggable.hpp"
#include "util/config/Config.hpp"
#include "util/log/Logger.hpp"
#include "util/newconfig/ConfigDefinition.hpp"
#include "web/ng/Connection.hpp"
#include "web/ng/MessageHandler.hpp"
#include "web/ng/ProcessingPolicy.hpp"
@@ -170,7 +170,7 @@ private:
*/
std::expected<Server, std::string>
make_Server(
util::Config const& config,
util::config::ClioConfigDefinition const& config,
Server::OnConnectCheck onConnectCheck,
Server::OnDisconnectHook onDisconnectHook,
boost::asio::io_context& context

View File

@@ -19,7 +19,7 @@
#include "web/ng/impl/ServerSslContext.hpp"
#include "util/config/Config.hpp"
#include "util/newconfig/ConfigDefinition.hpp"
#include <boost/asio/buffer.hpp>
#include <boost/asio/ssl/context.hpp>
@@ -52,10 +52,10 @@ readFile(std::string const& path)
} // namespace
std::expected<std::optional<boost::asio::ssl::context>, std::string>
makeServerSslContext(util::Config const& config)
makeServerSslContext(util::config::ClioConfigDefinition const& config)
{
bool const configHasCertFile = config.contains("ssl_cert_file");
bool const configHasKeyFile = config.contains("ssl_key_file");
bool const configHasCertFile = config.getValueView("ssl_cert_file").hasValue();
bool const configHasKeyFile = config.getValueView("ssl_key_file").hasValue();
if (configHasCertFile != configHasKeyFile)
return std::unexpected{"Config entries 'ssl_cert_file' and 'ssl_key_file' must be set or unset together."};
@@ -63,12 +63,12 @@ makeServerSslContext(util::Config const& config)
if (not configHasCertFile)
return std::nullopt;
auto const certFilename = config.value<std::string>("ssl_cert_file");
auto const certFilename = config.get<std::string>("ssl_cert_file");
auto const certContent = readFile(certFilename);
if (!certContent)
return std::unexpected{"Can't read SSL certificate: " + certFilename};
auto const keyFilename = config.value<std::string>("ssl_key_file");
auto const keyFilename = config.get<std::string>("ssl_key_file");
auto const keyContent = readFile(keyFilename);
if (!keyContent)
return std::unexpected{"Can't read SSL key: " + keyFilename};

View File

@@ -19,7 +19,7 @@
#pragma once
#include "util/config/Config.hpp"
#include "util/newconfig/ConfigDefinition.hpp"
#include <boost/asio/ssl/context.hpp>
@@ -30,7 +30,7 @@
namespace web::ng::impl {
std::expected<std::optional<boost::asio::ssl::context>, std::string>
makeServerSslContext(util::Config const& config);
makeServerSslContext(util::config::ClioConfigDefinition const& config);
std::expected<boost::asio::ssl::context, std::string>
makeServerSslContext(std::string const& certData, std::string const& keyData);

View File

@@ -22,7 +22,7 @@
#include "data/BackendInterface.hpp"
#include "data/DBHelpers.hpp"
#include "data/Types.hpp"
#include "util/config/Config.hpp"
#include "util/newconfig/ConfigDefinition.hpp"
#include <boost/asio/spawn.hpp>
#include <boost/json/object.hpp>
@@ -39,7 +39,7 @@
using namespace data;
struct MockBackend : public BackendInterface {
MockBackend(util::Config)
MockBackend(util::config::ClioConfigDefinition)
{
}

View File

@@ -19,13 +19,20 @@
#pragma once
#include "data/BackendInterface.hpp"
#include "util/LoggerFixtures.hpp"
#include "util/MockBackend.hpp"
#include "util/newconfig/ConfigDefinition.hpp"
#include <gmock/gmock.h>
#include <memory>
template <template <typename> typename MockType = ::testing::NiceMock>
struct MockBackendTestBase : virtual public NoLoggerFixture {
class BackendProxy {
std::shared_ptr<MockType<MockBackend>> backend = std::make_shared<MockType<MockBackend>>(util::Config{});
std::shared_ptr<MockType<MockBackend>> backend =
std::make_shared<MockType<MockBackend>>(util::config::ClioConfigDefinition{{}});
public:
auto

View File

@@ -20,7 +20,10 @@
#pragma once
#include "util/Assert.hpp"
#include "util/config/Config.hpp"
#include "util/Concepts.hpp"
#include "util/newconfig/ConfigDefinition.hpp"
#include "util/newconfig/ConfigValue.hpp"
#include "util/newconfig/Types.hpp"
#include "util/prometheus/Bool.hpp"
#include "util/prometheus/Counter.hpp"
#include "util/prometheus/Gauge.hpp"
@@ -205,7 +208,11 @@ struct WithMockPrometheus : virtual ::testing::Test {
}
std::cerr << "\n";
}
PrometheusService::init();
config::ClioConfigDefinition config{
{"prometheus.compress_reply", config::ConfigValue{config::ConfigType::Boolean}.defaultValue(true)},
{"prometheus.enabled", config::ConfigValue{config::ConfigType::Boolean}.defaultValue(true)}
};
PrometheusService::init(config);
}
static MockPrometheusImpl&
@@ -252,8 +259,11 @@ struct WithMockPrometheus : virtual ::testing::Test {
struct WithPrometheus : virtual ::testing::Test {
WithPrometheus()
{
boost::json::value const config{{"prometheus", boost::json::object{{"compress_reply", false}}}};
PrometheusService::init(Config{config});
config::ClioConfigDefinition config{
{"prometheus.compress_reply", config::ConfigValue{config::ConfigType::Boolean}.defaultValue(false)},
{"prometheus.enabled", config::ConfigValue{config::ConfigType::Boolean}.defaultValue(true)}
};
PrometheusService::init(config);
}
~WithPrometheus() override

View File

@@ -23,7 +23,8 @@
#include "etl/Source.hpp"
#include "feed/SubscriptionManagerInterface.hpp"
#include "rpc/Errors.hpp"
#include "util/config/Config.hpp"
#include "util/newconfig/ConfigDefinition.hpp"
#include "util/newconfig/ObjectView.hpp"
#include <boost/asio/io_context.hpp>
#include <boost/asio/spawn.hpp>
@@ -165,7 +166,7 @@ public:
ON_CALL(*this, makeSource)
.WillByDefault([this](
util::Config const&,
util::config::ObjectView const&,
boost::asio::io_context&,
std::shared_ptr<BackendInterface>,
std::shared_ptr<feed::SubscriptionManagerInterface>,
@@ -205,7 +206,7 @@ public:
MOCK_METHOD(
etl::SourcePtr,
makeSource,
(util::Config const&,
(util::config::ObjectView const&,
boost::asio::io_context&,
std::shared_ptr<BackendInterface>,
std::shared_ptr<feed::SubscriptionManagerInterface>,

View File

@@ -20,7 +20,9 @@
#pragma once
#include "util/Taggable.hpp"
#include "util/config/Config.hpp"
#include "util/newconfig/ConfigDefinition.hpp"
#include "util/newconfig/ConfigValue.hpp"
#include "util/newconfig/Types.hpp"
#include "web/SubscriptionContextInterface.hpp"
#include "web/interface/ConnectionBase.hpp"
@@ -37,7 +39,9 @@ struct MockSession : public web::SubscriptionContextInterface {
MOCK_METHOD(void, setApiSubversion, (uint32_t), (override));
MOCK_METHOD(uint32_t, apiSubversion, (), (const, override));
util::TagDecoratorFactory tagDecoratorFactory{util::Config{}};
util::TagDecoratorFactory tagDecoratorFactory{util::config::ClioConfigDefinition{
{"log_tag_style", util::config::ConfigValue{util::config::ConfigType::String}.defaultValue("none")}
}};
MockSession() : web::SubscriptionContextInterface(tagDecoratorFactory)
{

View File

@@ -21,7 +21,9 @@
#include "data/cassandra/Handle.hpp"
#include "util/AsioContextTestFixture.hpp"
#include "util/MockPrometheus.hpp"
#include "util/config/Config.hpp"
#include "util/newconfig/ConfigDefinition.hpp"
#include "util/newconfig/ConfigValue.hpp"
#include "util/newconfig/Types.hpp"
#include <TestGlobals.hpp>
#include <boost/json/parse.hpp>
@@ -31,6 +33,8 @@
#include <stdexcept>
#include <string>
using namespace util::config;
namespace {
constexpr auto keyspace = "factory_test";
} // namespace
@@ -71,55 +75,31 @@ protected:
TEST_F(BackendCassandraFactoryTest, NoSuchBackend)
{
util::Config const cfg{boost::json::parse(
R"({
"database":
{
"type":"unknown"
}
})"
)};
ClioConfigDefinition cfg{{"database.type", ConfigValue{ConfigType::String}.defaultValue("unknown")}};
EXPECT_THROW(data::make_Backend(cfg), std::runtime_error);
}
TEST_F(BackendCassandraFactoryTest, CreateCassandraBackendDBDisconnect)
{
util::Config const cfg{boost::json::parse(fmt::format(
R"({{
"database":
{{
"type" : "cassandra",
"cassandra" : {{
"contact_points": "{}",
"keyspace": "{}",
"replication_factor": 1,
"connect_timeout": 2
}}
}}
}})",
"127.0.0.2",
keyspace
))};
ClioConfigDefinition cfg{
{"database.type", ConfigValue{ConfigType::String}.defaultValue("cassandra")},
{"database.cassandra.contact_points", ConfigValue{ConfigType::String}.defaultValue("127.0.0.2")},
{"database.cassandra.keyspace", ConfigValue{ConfigType::String}.defaultValue(keyspace)},
{"database.cassandra.replication_factor", ConfigValue{ConfigType::Integer}.defaultValue(1)},
{"database.cassandra.connect_timeout", ConfigValue{ConfigType::Integer}.defaultValue(2)}
};
EXPECT_THROW(data::make_Backend(cfg), std::runtime_error);
}
TEST_F(BackendCassandraFactoryTestWithDB, CreateCassandraBackend)
{
util::Config const cfg{boost::json::parse(fmt::format(
R"({{
"database":
{{
"type": "cassandra",
"cassandra": {{
"contact_points": "{}",
"keyspace": "{}",
"replication_factor": 1
}}
}}
}})",
TestGlobals::instance().backendHost,
keyspace
))};
ClioConfigDefinition cfg{
{"database.type", ConfigValue{ConfigType::String}.defaultValue("cassandra")},
{"database.cassandra.contact_points",
ConfigValue{ConfigType::String}.defaultValue(TestGlobals::instance().backendHost)},
{"database.cassandra.keyspace", ConfigValue{ConfigType::String}.defaultValue(keyspace)},
{"database.cassandra.replication_factor", ConfigValue{ConfigType::Integer}.defaultValue(1)}
};
{
auto backend = data::make_Backend(cfg);
@@ -147,60 +127,37 @@ TEST_F(BackendCassandraFactoryTestWithDB, CreateCassandraBackend)
TEST_F(BackendCassandraFactoryTestWithDB, CreateCassandraBackendReadOnlyWithEmptyDB)
{
util::Config const cfg{boost::json::parse(fmt::format(
R"({{
"read_only": true,
"database":
{{
"type" : "cassandra",
"cassandra" : {{
"contact_points": "{}",
"keyspace": "{}",
"replication_factor": 1
}}
}}
}})",
TestGlobals::instance().backendHost,
keyspace
))};
ClioConfigDefinition cfg{
{"read_only", ConfigValue{ConfigType::Boolean}.defaultValue(true)},
{"database.type", ConfigValue{ConfigType::String}.defaultValue("cassandra")},
{"database.cassandra.contact_points",
ConfigValue{ConfigType::String}.defaultValue(TestGlobals::instance().backendHost)},
{"database.cassandra.keyspace", ConfigValue{ConfigType::String}.defaultValue(keyspace)},
{"database.cassandra.replication_factor", ConfigValue{ConfigType::Integer}.defaultValue(1)}
};
EXPECT_THROW(data::make_Backend(cfg), std::runtime_error);
}
TEST_F(BackendCassandraFactoryTestWithDB, CreateCassandraBackendReadOnlyWithDBReady)
{
util::Config const cfgReadOnly{boost::json::parse(fmt::format(
R"({{
"read_only": true,
"database":
{{
"type" : "cassandra",
"cassandra" : {{
"contact_points": "{}",
"keyspace": "{}",
"replication_factor": 1
}}
}}
}})",
TestGlobals::instance().backendHost,
keyspace
))};
ClioConfigDefinition cfgReadOnly{
{"read_only", ConfigValue{ConfigType::Boolean}.defaultValue(true)},
{"database.type", ConfigValue{ConfigType::String}.defaultValue("cassandra")},
{"database.cassandra.contact_points",
ConfigValue{ConfigType::String}.defaultValue(TestGlobals::instance().backendHost)},
{"database.cassandra.keyspace", ConfigValue{ConfigType::String}.defaultValue(keyspace)},
{"database.cassandra.replication_factor", ConfigValue{ConfigType::Integer}.defaultValue(1)}
};
util::Config const cfgWrite{boost::json::parse(fmt::format(
R"({{
"read_only": false,
"database":
{{
"type" : "cassandra",
"cassandra" : {{
"contact_points": "{}",
"keyspace": "{}",
"replication_factor": 1
}}
}}
}})",
TestGlobals::instance().backendHost,
keyspace
))};
ClioConfigDefinition cfgWrite{
{"read_only", ConfigValue{ConfigType::Boolean}.defaultValue(false)},
{"database.type", ConfigValue{ConfigType::String}.defaultValue("cassandra")},
{"database.cassandra.contact_points",
ConfigValue{ConfigType::String}.defaultValue(TestGlobals::instance().backendHost)},
{"database.cassandra.keyspace", ConfigValue{ConfigType::String}.defaultValue(keyspace)},
{"database.cassandra.replication_factor", ConfigValue{ConfigType::Integer}.defaultValue(1)}
};
EXPECT_TRUE(data::make_Backend(cfgWrite));
EXPECT_TRUE(data::make_Backend(cfgReadOnly));

View File

@@ -30,7 +30,9 @@
#include "util/MockPrometheus.hpp"
#include "util/Random.hpp"
#include "util/StringUtils.hpp"
#include "util/config/Config.hpp"
#include "util/newconfig/ConfigValue.hpp"
#include "util/newconfig/ObjectView.hpp"
#include "util/newconfig/Types.hpp"
#include <TestGlobals.hpp>
#include <boost/asio/impl/spawn.hpp>
@@ -64,25 +66,25 @@
#include <vector>
using namespace util;
using namespace util::config;
using namespace std;
using namespace rpc;
using namespace prometheus;
namespace json = boost::json;
using namespace data::cassandra;
class BackendCassandraTest : public SyncAsioContextTest, public WithPrometheus {
protected:
Config cfg{json::parse(fmt::format(
R"JSON({{
"contact_points": "{}",
"keyspace": "{}",
"replication_factor": 1
}})JSON",
TestGlobals::instance().backendHost,
TestGlobals::instance().backendKeyspace
))};
SettingsProvider settingsProvider{cfg};
ClioConfigDefinition cfg{
{{"database.cassandra.contact_points",
ConfigValue{ConfigType::String}.defaultValue(TestGlobals::instance().backendHost)},
{"database.cassandra.keyspace",
ConfigValue{ConfigType::String}.defaultValue(TestGlobals::instance().backendKeyspace)},
{"database.cassandra.replication_factor", ConfigValue{ConfigType::Integer}.defaultValue(1)}}
};
ObjectView obj = cfg.getObject("database.cassandra");
SettingsProvider settingsProvider{obj};
// recreated for each test
std::unique_ptr<BackendInterface> backend;

View File

@@ -162,6 +162,7 @@ target_sources(
util/newconfig/ConfigValueTests.cpp
util/newconfig/ObjectViewTests.cpp
util/newconfig/JsonConfigFileTests.cpp
util/newconfig/JsonFileTests.cpp
util/newconfig/ValueViewTests.cpp
)
@@ -171,7 +172,6 @@ gtest_discover_tests(clio_tests DISCOVERY_TIMEOUT 90)
# Fix for dwarf5 bug on ci
target_compile_options(clio_options INTERFACE -gdwarf-4)
target_compile_definitions(clio_tests PUBLIC UNITTEST_BUILD)
target_include_directories(clio_tests PRIVATE .)
target_link_libraries(clio_tests PUBLIC clio_testing_common)
set_target_properties(clio_tests PROPERTIES RUNTIME_OUTPUT_DIRECTORY ${CMAKE_BINARY_DIR})

View File

@@ -20,6 +20,7 @@
#include "util/Assert.hpp"
#include "util/LoggerFixtures.hpp"
#include "util/config/Config.hpp"
#include "util/log/Logger.hpp"
#include <boost/filesystem/operations.hpp>
#include <boost/json/conversion.hpp>
@@ -31,7 +32,6 @@
#include <cstdint>
#include <cstdio>
#include <fstream>
#include <optional>
#include <stdexcept>
#include <string>
#include <vector>

View File

@@ -22,7 +22,9 @@
#include "util/LoggerFixtures.hpp"
#include "util/MockPrometheus.hpp"
#include "util/Taggable.hpp"
#include "util/config/Config.hpp"
#include "util/newconfig/ConfigDefinition.hpp"
#include "util/newconfig/ConfigValue.hpp"
#include "util/newconfig/Types.hpp"
#include "web/AdminVerificationStrategy.hpp"
#include "web/SubscriptionContextInterface.hpp"
#include "web/dosguard/DOSGuardMock.hpp"
@@ -49,10 +51,13 @@
using namespace app;
namespace http = boost::beast::http;
using namespace util::config;
struct WebHandlersTest : virtual NoLoggerFixture {
DOSGuardStrictMock dosGuardMock_;
util::TagDecoratorFactory tagFactory_{util::Config{}};
util::TagDecoratorFactory const tagFactory_{
ClioConfigDefinition{{"log_tag_style", ConfigValue{ConfigType::String}.defaultValue("uint")}}
};
std::string const ip_ = "some ip";
StrictMockConnection connectionMock_{ip_, boost::beast::flat_buffer{}, tagFactory_};

View File

@@ -21,9 +21,15 @@
#include "data/cassandra/Types.hpp"
#include "util/LoggerFixtures.hpp"
#include "util/TmpFile.hpp"
#include "util/config/Config.hpp"
#include "util/log/Logger.hpp"
#include "util/newconfig/ConfigDefinition.hpp"
#include "util/newconfig/ConfigFileJson.hpp"
#include "util/newconfig/ConfigValue.hpp"
#include "util/newconfig/ObjectView.hpp"
#include "util/newconfig/Types.hpp"
#include <boost/json/parse.hpp>
#include <boost/json/value.hpp>
#include <fmt/core.h>
#include <gtest/gtest.h>
@@ -33,17 +39,50 @@
#include <variant>
using namespace util;
using namespace util::config;
using namespace std;
namespace json = boost::json;
using namespace data::cassandra;
inline ClioConfigDefinition
getParseSettingsConfig(boost::json::value val)
{
ConfigFileJson const jsonVal{val.as_object()};
auto config = ClioConfigDefinition{
{"database.cassandra.threads",
ConfigValue{ConfigType::Integer}.defaultValue(std::thread::hardware_concurrency())},
{"database.cassandra.contact_points", ConfigValue{ConfigType::String}.defaultValue("127.0.0.1")},
{"database.cassandra.max_write_requests_outstanding", ConfigValue{ConfigType::Integer}.defaultValue(10000)},
{"database.cassandra.max_read_requests_outstanding", ConfigValue{ConfigType::Integer}.defaultValue(100000)},
{"database.cassandra.core_connections_per_host", ConfigValue{ConfigType::Integer}.defaultValue(1)},
{"database.cassandra.certificate", ConfigValue{ConfigType::String}.optional()},
{"database.cassandra.username", ConfigValue{ConfigType::String}.optional()},
{"database.cassandra.password", ConfigValue{ConfigType::String}.optional()},
{"database.cassandra.queue_size_io", ConfigValue{ConfigType::Integer}.optional()},
{"database.cassandra.write_batch_size", ConfigValue{ConfigType::Integer}.defaultValue(20)},
{"database.cassandra.connect_timeout", ConfigValue{ConfigType::Integer}.optional()},
{"database.cassandra.certfile", ConfigValue{ConfigType::String}.optional()},
{"database.cassandra.request_timeout", ConfigValue{ConfigType::Integer}.defaultValue(0)},
{"database.cassandra.secure_connect_bundle", ConfigValue{ConfigType::String}.optional()},
{"database.cassandra.username", ConfigValue{ConfigType::String}.optional()},
{"database.cassandra.password", ConfigValue{ConfigType::String}.optional()},
{"database.cassandra.keyspace", ConfigValue{ConfigType::String}.defaultValue("clio")},
{"database.cassandra.port", ConfigValue{ConfigType::Integer}.optional()},
{"database.cassandra.replication_factor", ConfigValue{ConfigType::Integer}.defaultValue(3)},
{"database.cassandra.table_prefix", ConfigValue{ConfigType::String}.optional()},
};
auto const errors = config.parse(jsonVal);
[&]() { ASSERT_FALSE(errors.has_value()); }();
return config;
};
class SettingsProviderTest : public NoLoggerFixture {};
TEST_F(SettingsProviderTest, Defaults)
{
Config const cfg{json::parse(R"({"contact_points": "127.0.0.1"})")};
SettingsProvider const provider{cfg};
auto const cfg = getParseSettingsConfig(json::parse(R"({"contact_points": "127.0.0.1"})"));
SettingsProvider const provider{cfg.getObject("database.cassandra")};
auto const settings = provider.getSettings();
EXPECT_EQ(settings.threads, std::thread::hardware_concurrency());
@@ -71,15 +110,15 @@ TEST_F(SettingsProviderTest, Defaults)
TEST_F(SettingsProviderTest, SimpleConfig)
{
Config const cfg{json::parse(R"({
"contact_points": "123.123.123.123",
"port": 1234,
"keyspace": "test",
"replication_factor": 42,
"table_prefix": "prefix",
"threads": 24
})")};
SettingsProvider const provider{cfg};
auto const cfg = getParseSettingsConfig(json::parse(R"({
"database.cassandra.contact_points": "123.123.123.123",
"database.cassandra.port": 1234,
"database.cassandra.keyspace": "test",
"database.cassandra.replication_factor": 42,
"database.cassandra.table_prefix": "prefix",
"database.cassandra.threads": 24
})"));
SettingsProvider const provider{cfg.getObject("database.cassandra")};
auto const settings = provider.getSettings();
EXPECT_EQ(settings.threads, 24);
@@ -96,11 +135,11 @@ TEST_F(SettingsProviderTest, SimpleConfig)
TEST_F(SettingsProviderTest, DriverOptionalOptionsSpecified)
{
Config const cfg{json::parse(R"({
"contact_points": "123.123.123.123",
"queue_size_io": 2
})")};
SettingsProvider const provider{cfg};
auto const cfg = getParseSettingsConfig(json::parse(R"({
"database.cassandra.contact_points": "123.123.123.123",
"database.cassandra.queue_size_io": 2
})"));
SettingsProvider const provider{cfg.getObject("database.cassandra")};
auto const settings = provider.getSettings();
EXPECT_EQ(settings.queueSizeIO, 2);
@@ -108,8 +147,9 @@ TEST_F(SettingsProviderTest, DriverOptionalOptionsSpecified)
TEST_F(SettingsProviderTest, SecureBundleConfig)
{
Config const cfg{json::parse(R"({"secure_connect_bundle": "bundleData"})")};
SettingsProvider const provider{cfg};
auto const cfg =
getParseSettingsConfig(json::parse(R"({"database.cassandra.secure_connect_bundle": "bundleData"})"));
SettingsProvider const provider{cfg.getObject("database.cassandra")};
auto const settings = provider.getSettings();
auto const* sb = std::get_if<Settings::SecureConnectionBundle>(&settings.connectionInfo);
@@ -120,14 +160,14 @@ TEST_F(SettingsProviderTest, SecureBundleConfig)
TEST_F(SettingsProviderTest, CertificateConfig)
{
TmpFile const file{"certificateData"};
Config const cfg{json::parse(fmt::format(
auto const cfg = getParseSettingsConfig(json::parse(fmt::format(
R"({{
"contact_points": "127.0.0.1",
"certfile": "{}"
"database.cassandra.contact_points": "127.0.0.1",
"database.cassandra.certfile": "{}"
}})",
file.path
))};
SettingsProvider const provider{cfg};
)));
SettingsProvider const provider{cfg.getObject("database.cassandra")};
auto const settings = provider.getSettings();
EXPECT_EQ(settings.certificate, "certificateData");

View File

@@ -18,20 +18,49 @@
//==============================================================================
#include "etl/CacheLoaderSettings.hpp"
#include "util/config/Config.hpp"
#include "util/newconfig/ConfigDefinition.hpp"
#include "util/newconfig/ConfigFileJson.hpp"
#include "util/newconfig/ConfigValue.hpp"
#include "util/newconfig/Types.hpp"
#include <boost/json/parse.hpp>
#include <boost/json/value.hpp>
#include <gtest/gtest.h>
namespace json = boost::json;
using namespace etl;
using namespace testing;
using namespace util::config;
inline ClioConfigDefinition
generateDefaultCacheConfig()
{
return ClioConfigDefinition{
{{"io_threads", ConfigValue{ConfigType::Integer}.defaultValue(2)},
{"cache.num_diffs", ConfigValue{ConfigType::Integer}.defaultValue(32)},
{"cache.num_markers", ConfigValue{ConfigType::Integer}.defaultValue(48)},
{"cache.num_cursors_from_diff", ConfigValue{ConfigType::Integer}.defaultValue(0)},
{"cache.num_cursors_from_account", ConfigValue{ConfigType::Integer}.defaultValue(0)},
{"cache.page_fetch_size", ConfigValue{ConfigType::Integer}.defaultValue(512)},
{"cache.load", ConfigValue{ConfigType::String}.defaultValue("async")}}
};
}
inline ClioConfigDefinition
getParseCacheConfig(boost::json::value val)
{
ConfigFileJson const jsonVal{val.as_object()};
auto config = generateDefaultCacheConfig();
auto const errors = config.parse(jsonVal);
[&]() { ASSERT_FALSE(errors.has_value()); }();
return config;
}
struct CacheLoaderSettingsTest : Test {};
TEST_F(CacheLoaderSettingsTest, DefaultSettingsParsedCorrectly)
{
auto const cfg = util::Config{json::parse(R"({})")};
auto const cfg = generateDefaultCacheConfig();
auto const settings = make_CacheLoaderSettings(cfg);
auto const defaults = CacheLoaderSettings{};
@@ -40,7 +69,7 @@ TEST_F(CacheLoaderSettingsTest, DefaultSettingsParsedCorrectly)
TEST_F(CacheLoaderSettingsTest, NumThreadsCorrectlyPropagatedThroughConfig)
{
auto const cfg = util::Config{json::parse(R"({"io_threads": 42})")};
auto const cfg = getParseCacheConfig(json::parse(R"({"io_threads": 42})"));
auto const settings = make_CacheLoaderSettings(cfg);
EXPECT_EQ(settings.numThreads, 42);
@@ -48,7 +77,7 @@ TEST_F(CacheLoaderSettingsTest, NumThreadsCorrectlyPropagatedThroughConfig)
TEST_F(CacheLoaderSettingsTest, NumDiffsCorrectlyPropagatedThroughConfig)
{
auto const cfg = util::Config{json::parse(R"({"cache": {"num_diffs": 42}})")};
auto const cfg = getParseCacheConfig(json::parse(R"({"cache": {"num_diffs": 42}})"));
auto const settings = make_CacheLoaderSettings(cfg);
EXPECT_EQ(settings.numCacheDiffs, 42);
@@ -56,7 +85,7 @@ TEST_F(CacheLoaderSettingsTest, NumDiffsCorrectlyPropagatedThroughConfig)
TEST_F(CacheLoaderSettingsTest, NumMarkersCorrectlyPropagatedThroughConfig)
{
auto const cfg = util::Config{json::parse(R"({"cache": {"num_markers": 42}})")};
auto const cfg = getParseCacheConfig(json::parse(R"({"cache": {"num_markers": 42}})"));
auto const settings = make_CacheLoaderSettings(cfg);
EXPECT_EQ(settings.numCacheMarkers, 42);
@@ -64,7 +93,7 @@ TEST_F(CacheLoaderSettingsTest, NumMarkersCorrectlyPropagatedThroughConfig)
TEST_F(CacheLoaderSettingsTest, PageFetchSizeCorrectlyPropagatedThroughConfig)
{
auto const cfg = util::Config{json::parse(R"({"cache": {"page_fetch_size": 42}})")};
auto const cfg = getParseCacheConfig(json::parse(R"({"cache": {"page_fetch_size": 42}})"));
auto const settings = make_CacheLoaderSettings(cfg);
EXPECT_EQ(settings.cachePageFetchSize, 42);
@@ -72,7 +101,7 @@ TEST_F(CacheLoaderSettingsTest, PageFetchSizeCorrectlyPropagatedThroughConfig)
TEST_F(CacheLoaderSettingsTest, SyncLoadStyleCorrectlyPropagatedThroughConfig)
{
auto const cfg = util::Config{json::parse(R"({"cache": {"load": "sYNC"}})")};
auto const cfg = getParseCacheConfig(json::parse(R"({"cache": {"load": "sYNC"}})"));
auto const settings = make_CacheLoaderSettings(cfg);
EXPECT_EQ(settings.loadStyle, CacheLoaderSettings::LoadStyle::SYNC);
@@ -81,7 +110,7 @@ TEST_F(CacheLoaderSettingsTest, SyncLoadStyleCorrectlyPropagatedThroughConfig)
TEST_F(CacheLoaderSettingsTest, AsyncLoadStyleCorrectlyPropagatedThroughConfig)
{
auto const cfg = util::Config{json::parse(R"({"cache": {"load": "aSynC"}})")};
auto const cfg = getParseCacheConfig(json::parse(R"({"cache": {"load": "aSynC"}})"));
auto const settings = make_CacheLoaderSettings(cfg);
EXPECT_EQ(settings.loadStyle, CacheLoaderSettings::LoadStyle::ASYNC);
@@ -91,14 +120,14 @@ TEST_F(CacheLoaderSettingsTest, AsyncLoadStyleCorrectlyPropagatedThroughConfig)
TEST_F(CacheLoaderSettingsTest, NoLoadStyleCorrectlyPropagatedThroughConfig)
{
{
auto const cfg = util::Config{json::parse(R"({"cache": {"load": "nONe"}})")};
auto const cfg = getParseCacheConfig(json::parse(R"({"cache": {"load": "nONe"}})"));
auto const settings = make_CacheLoaderSettings(cfg);
EXPECT_EQ(settings.loadStyle, CacheLoaderSettings::LoadStyle::NONE);
EXPECT_TRUE(settings.isDisabled());
}
{
auto const cfg = util::Config{json::parse(R"({"cache": {"load": "nO"}})")};
auto const cfg = getParseCacheConfig(json::parse(R"({"cache": {"load": "nO"}})"));
auto const settings = make_CacheLoaderSettings(cfg);
EXPECT_EQ(settings.loadStyle, CacheLoaderSettings::LoadStyle::NONE);

View File

@@ -22,13 +22,18 @@
#include "etl/CacheLoaderSettings.hpp"
#include "etl/FakeDiffProvider.hpp"
#include "etl/impl/CacheLoader.hpp"
#include "util/Assert.hpp"
#include "util/MockBackendTestFixture.hpp"
#include "util/MockCache.hpp"
#include "util/MockPrometheus.hpp"
#include "util/async/context/BasicExecutionContext.hpp"
#include "util/config/Config.hpp"
#include "util/newconfig/ConfigDefinition.hpp"
#include "util/newconfig/ConfigFileJson.hpp"
#include "util/newconfig/ConfigValue.hpp"
#include "util/newconfig/Types.hpp"
#include <boost/json/parse.hpp>
#include <boost/json/value.hpp>
#include <gmock/gmock.h>
#include <gtest/gtest.h>
@@ -39,9 +44,34 @@ using namespace etl;
using namespace util;
using namespace data;
using namespace testing;
using namespace util::config;
namespace {
inline ClioConfigDefinition
generateDefaultCacheConfig()
{
return ClioConfigDefinition{
{{"io_threads", ConfigValue{ConfigType::Integer}.defaultValue(2)},
{"cache.num_diffs", ConfigValue{ConfigType::Integer}.defaultValue(32)},
{"cache.num_markers", ConfigValue{ConfigType::Integer}.defaultValue(48)},
{"cache.num_cursors_from_diff", ConfigValue{ConfigType::Integer}.defaultValue(0)},
{"cache.num_cursors_from_account", ConfigValue{ConfigType::Integer}.defaultValue(0)},
{"cache.page_fetch_size", ConfigValue{ConfigType::Integer}.defaultValue(512)},
{"cache.load", ConfigValue{ConfigType::String}.defaultValue("async")}}
};
}
inline ClioConfigDefinition
getParseCacheConfig(boost::json::value val)
{
ConfigFileJson const jsonVal{val.as_object()};
auto config = generateDefaultCacheConfig();
auto const errors = config.parse(jsonVal);
[&]() { ASSERT_FALSE(errors.has_value()); }();
return config;
}
constexpr auto SEQ = 30;
struct CacheLoaderTest : util::prometheus::WithPrometheus, MockBackendTest {
@@ -178,7 +208,7 @@ TEST_P(ParametrizedCacheLoaderTest, CacheDisabledLeadsToCancellation)
//
TEST_F(CacheLoaderTest, SyncCacheLoaderWaitsTillFullyLoaded)
{
auto const cfg = util::Config(json::parse(R"({"cache": {"load": "sync"}})"));
auto const cfg = getParseCacheConfig(json::parse(R"({"cache": {"load": "sync"}})"));
CacheLoader loader{cfg, backend, cache};
auto const diffs = diffProvider.getLatestDiff();
@@ -204,7 +234,7 @@ TEST_F(CacheLoaderTest, SyncCacheLoaderWaitsTillFullyLoaded)
TEST_F(CacheLoaderTest, AsyncCacheLoaderCanBeStopped)
{
auto const cfg = util::Config(json::parse(R"({"cache": {"load": "async"}})"));
auto const cfg = getParseCacheConfig(json::parse(R"({"cache": {"load": "async"}})"));
CacheLoader loader{cfg, backend, cache};
auto const diffs = diffProvider.getLatestDiff();
@@ -232,7 +262,7 @@ TEST_F(CacheLoaderTest, AsyncCacheLoaderCanBeStopped)
TEST_F(CacheLoaderTest, DisabledCacheLoaderDoesNotLoadCache)
{
auto cfg = util::Config(json::parse(R"({"cache": {"load": "none"}})"));
auto const cfg = getParseCacheConfig(json::parse(R"({"cache": {"load": "none"}})"));
CacheLoader loader{cfg, backend, cache};
EXPECT_CALL(cache, updateImp).Times(0);

View File

@@ -23,7 +23,7 @@
#include "util/MockPrometheus.hpp"
#include "util/MockXrpLedgerAPIService.hpp"
#include "util/TestObject.hpp"
#include "util/config/Config.hpp"
#include "util/newconfig/ConfigDefinition.hpp"
#include <gmock/gmock.h>
#include <grpcpp/server_context.h>
@@ -39,11 +39,12 @@
#include <vector>
using namespace etl::impl;
using namespace util::config;
struct GrpcSourceTests : NoLoggerFixture, util::prometheus::WithPrometheus, tests::util::WithMockXrpLedgerAPIService {
GrpcSourceTests()
: WithMockXrpLedgerAPIService("localhost:0")
, mockBackend_(std::make_shared<testing::StrictMock<MockBackend>>(util::Config{}))
, mockBackend_(std::make_shared<testing::StrictMock<MockBackend>>(ClioConfigDefinition{}))
, grpcSource_("localhost", std::to_string(getXRPLMockPort()), mockBackend_)
{
}

View File

@@ -27,7 +27,7 @@
#include "util/MockPrometheus.hpp"
#include "util/MockSubscriptionManager.hpp"
#include "util/TestObject.hpp"
#include "util/config/Config.hpp"
#include "util/newconfig/ConfigDefinition.hpp"
#include <boost/json/parse.hpp>
#include <fmt/core.h>
@@ -42,7 +42,6 @@
using namespace testing;
using namespace etl;
namespace json = boost::json;
using namespace std::chrono;
static auto constexpr ACCOUNT = "rf1BiGeXwwQoi8Z2ueFYTEXSwuJYfV2Jpn";
@@ -63,8 +62,7 @@ struct ETLLedgerPublisherTest : util::prometheus::WithPrometheus, MockBackendTes
{
SyncAsioContextTest::TearDown();
}
util::Config cfg{json::parse("{}")};
util::config::ClioConfigDefinition cfg{{}};
MockCache mockCache;
StrictMockSubscriptionManagerSharedPtr mockSubscriptionManagerPtr;
};

View File

@@ -28,7 +28,12 @@
#include "util/MockSubscriptionManager.hpp"
#include "util/NameGenerator.hpp"
#include "util/Random.hpp"
#include "util/config/Config.hpp"
#include "util/newconfig/Array.hpp"
#include "util/newconfig/ConfigConstraints.hpp"
#include "util/newconfig/ConfigDefinition.hpp"
#include "util/newconfig/ConfigFileJson.hpp"
#include "util/newconfig/ConfigValue.hpp"
#include "util/newconfig/Types.hpp"
#include <boost/asio/io_context.hpp>
#include <boost/asio/spawn.hpp>
@@ -52,20 +57,78 @@
#include <vector>
using namespace etl;
using namespace util::config;
using testing::Return;
constexpr static auto const TwoSourcesLedgerResponse = R"({
"etl_sources": [
{
"ip": "127.0.0.1",
"ws_port": "5005",
"grpc_port": "source1"
},
{
"ip": "127.0.0.1",
"ws_port": "5005",
"grpc_port": "source2"
}
]
})";
constexpr static auto const ThreeSourcesLedgerResponse = R"({
"etl_sources": [
{
"ip": "127.0.0.1",
"ws_port": "5005",
"grpc_port": "source1"
},
{
"ip": "127.0.0.1",
"ws_port": "5005",
"grpc_port": "source2"
},
{
"ip": "127.0.0.1",
"ws_port": "5005",
"grpc_port": "source3"
}
]
})";
inline ClioConfigDefinition
getParseLoadBalancerConfig(boost::json::value val)
{
ClioConfigDefinition config{
{{"forwarding.cache_timeout",
ConfigValue{ConfigType::Double}.defaultValue(0.0).withConstraint(validatePositiveDouble)},
{"forwarding.request_timeout",
ConfigValue{ConfigType::Double}.defaultValue(10.0).withConstraint(validatePositiveDouble)},
{"allow_no_etl", ConfigValue{ConfigType::Boolean}.defaultValue(false)},
{"etl_sources.[].ip", Array{ConfigValue{ConfigType::String}.optional().withConstraint(validateIP)}},
{"etl_sources.[].ws_port", Array{ConfigValue{ConfigType::String}.optional().withConstraint(validatePort)}},
{"etl_sources.[].grpc_port", Array{ConfigValue{ConfigType::String}.optional()}},
{"num_markers", ConfigValue{ConfigType::Integer}.optional().withConstraint(validateNumMarkers)}}
};
auto const errors = config.parse(ConfigFileJson{val.as_object()});
[&]() { ASSERT_FALSE(errors.has_value()); }();
return config;
}
struct LoadBalancerConstructorTests : util::prometheus::WithPrometheus, MockBackendTestStrict {
StrictMockSubscriptionManagerSharedPtr subscriptionManager_;
StrictMockNetworkValidatedLedgersPtr networkManager_;
StrictMockSourceFactory sourceFactory_{2};
boost::asio::io_context ioContext_;
boost::json::value configJson_{{"etl_sources", {"source1", "source2"}}};
boost::json::value configJson_ = boost::json::parse(TwoSourcesLedgerResponse);
std::unique_ptr<LoadBalancer>
makeLoadBalancer()
{
auto const cfg = getParseLoadBalancerConfig(configJson_);
return std::make_unique<LoadBalancer>(
util::Config{configJson_},
cfg,
ioContext_,
backend,
subscriptionManager_,
@@ -191,16 +254,6 @@ TEST_F(LoadBalancerConstructorTests, fetchETLState_DifferentNetworkIDButAllowNoE
makeLoadBalancer();
}
struct LoadBalancerConstructorDeathTest : LoadBalancerConstructorTests {};
TEST_F(LoadBalancerConstructorDeathTest, numMarkersSpecifiedInConfigIsInvalid)
{
uint32_t const numMarkers = 257;
configJson_.as_object()["num_markers"] = numMarkers;
testing::Mock::AllowLeak(&sourceFactory_);
EXPECT_DEATH({ makeLoadBalancer(); }, ".*");
}
struct LoadBalancerOnConnectHookTests : LoadBalancerConstructorTests {
LoadBalancerOnConnectHookTests()
{
@@ -327,7 +380,8 @@ struct LoadBalancer3SourcesTests : LoadBalancerConstructorTests {
LoadBalancer3SourcesTests()
{
sourceFactory_.setSourcesNumber(3);
configJson_.as_object()["etl_sources"] = {"source1", "source2", "source3"};
configJson_ = boost::json::parse(ThreeSourcesLedgerResponse);
EXPECT_CALL(sourceFactory_, makeSource).Times(3);
EXPECT_CALL(sourceFactory_.sourceAt(0), forwardToRippled).WillOnce(Return(boost::json::object{}));
EXPECT_CALL(sourceFactory_.sourceAt(0), run);

View File

@@ -19,7 +19,9 @@
#include "rpc/common/impl/APIVersionParser.hpp"
#include "util/LoggerFixtures.hpp"
#include "util/config/Config.hpp"
#include "util/newconfig/ConfigDefinition.hpp"
#include "util/newconfig/ConfigValue.hpp"
#include "util/newconfig/Types.hpp"
#include <boost/json/parse.hpp>
#include <fmt/core.h>
@@ -29,6 +31,7 @@ constexpr static auto DEFAULT_API_VERSION = 5u;
constexpr static auto MIN_API_VERSION = 2u;
constexpr static auto MAX_API_VERSION = 10u;
using namespace util::config;
using namespace rpc::impl;
namespace json = boost::json;
@@ -93,18 +96,13 @@ TEST_F(RPCAPIVersionTest, ReturnsParsedVersionIfAllPreconditionsAreMet)
TEST_F(RPCAPIVersionTest, GetsValuesFromConfigCorrectly)
{
util::Config const cfg{json::parse(fmt::format(
R"({{
"min": {},
"max": {},
"default": {}
}})",
MIN_API_VERSION,
MAX_API_VERSION,
DEFAULT_API_VERSION
))};
ClioConfigDefinition cfg{
{"api_version.min", ConfigValue{ConfigType::Integer}.defaultValue(MIN_API_VERSION)},
{"api_version.max", ConfigValue{ConfigType::Integer}.defaultValue(MAX_API_VERSION)},
{"api_version.default", ConfigValue{ConfigType::Integer}.defaultValue(DEFAULT_API_VERSION)}
};
ProductionAPIVersionParser const configuredParser{cfg};
ProductionAPIVersionParser const configuredParser{cfg.getObject("api_version")};
{
auto ver = configuredParser.parse(json::parse(R"({"api_version": 2})").as_object());

View File

@@ -25,7 +25,9 @@
#include "util/MockLoadBalancer.hpp"
#include "util/NameGenerator.hpp"
#include "util/Taggable.hpp"
#include "util/config/Config.hpp"
#include "util/newconfig/ConfigDefinition.hpp"
#include "util/newconfig/ConfigValue.hpp"
#include "util/newconfig/Types.hpp"
#include "web/Context.hpp"
#include <boost/json/object.hpp>
@@ -42,6 +44,7 @@
using namespace rpc;
using namespace testing;
using namespace util::config;
namespace json = boost::json;
constexpr static auto CLIENT_IP = "127.0.0.1";
@@ -52,7 +55,7 @@ protected:
std::shared_ptr<MockHandlerProvider> handlerProvider = std::make_shared<MockHandlerProvider>();
MockCounters counters;
util::Config config;
ClioConfigDefinition const config{{"log_tag_style", ConfigValue{ConfigType::String}.defaultValue("none")}};
util::TagDecoratorFactory tagFactory{config};
rpc::impl::ForwardingProxy<MockLoadBalancer, MockCounters, MockHandlerProvider> proxy{

View File

@@ -18,7 +18,6 @@
//==============================================================================
#include "data/BackendInterface.hpp"
#include "data/Types.hpp"
#include "rpc/Errors.hpp"
#include "rpc/FakesAndMocks.hpp"
#include "rpc/RPCEngine.hpp"
@@ -34,8 +33,12 @@
#include "util/MockPrometheus.hpp"
#include "util/NameGenerator.hpp"
#include "util/Taggable.hpp"
#include "util/config/Config.hpp"
#include "web/Context.hpp"
#include "util/newconfig/Array.hpp"
#include "util/newconfig/ConfigConstraints.hpp"
#include "util/newconfig/ConfigDefinition.hpp"
#include "util/newconfig/ConfigFileJson.hpp"
#include "util/newconfig/ConfigValue.hpp"
#include "util/newconfig/Types.hpp"
#include "web/dosguard/DOSGuard.hpp"
#include "web/dosguard/WhitelistHandler.hpp"
@@ -55,6 +58,7 @@ using namespace rpc;
using namespace util;
namespace json = boost::json;
using namespace testing;
using namespace util::config;
namespace {
constexpr auto FORWARD_REPLY = R"JSON({
@@ -66,15 +70,30 @@ constexpr auto FORWARD_REPLY = R"JSON({
})JSON";
} // namespace
inline ClioConfigDefinition
generateDefaultRPCEngineConfig()
{
return ClioConfigDefinition{
{"server.max_queue_size", ConfigValue{ConfigType::Integer}.defaultValue(2)},
{"workers", ConfigValue{ConfigType::Integer}.defaultValue(4).withConstraint(validateUint16)},
{"rpc.cache_timeout", ConfigValue{ConfigType::Double}.defaultValue(0.0).withConstraint(validatePositiveDouble)},
{"log_tag_style", ConfigValue{ConfigType::String}.defaultValue("uint")},
{"dos_guard.whitelist.[]", Array{ConfigValue{ConfigType::String}.optional()}},
{"dos_guard.max_fetches",
ConfigValue{ConfigType::Integer}.defaultValue(1000'000u).withConstraint(validateUint32)},
{"dos_guard.max_connections", ConfigValue{ConfigType::Integer}.defaultValue(20u).withConstraint(validateUint32)
},
{"dos_guard.max_requests", ConfigValue{ConfigType::Integer}.defaultValue(20u).withConstraint(validateUint32)}
};
}
struct RPCEngineTest : util::prometheus::WithPrometheus,
MockBackendTest,
MockCountersTest,
MockLoadBalancerTest,
SyncAsioContextTest {
Config cfg = Config{json::parse(R"JSON({
"server": {"max_queue_size": 2},
"workers": 4
})JSON")};
ClioConfigDefinition cfg = generateDefaultRPCEngineConfig();
util::TagDecoratorFactory tagFactory{cfg};
WorkQueue queue = WorkQueue::make_WorkQueue(cfg);
web::dosguard::WhitelistHandler whitelistHandler{cfg};
@@ -179,7 +198,13 @@ TEST_P(RPCEngineFlowParameterTest, Test)
std::shared_ptr<RPCEngine<MockLoadBalancer, MockCounters>> engine =
RPCEngine<MockLoadBalancer, MockCounters>::make_RPCEngine(
Config{}, backend, mockLoadBalancerPtr, dosGuard, queue, *mockCountersPtr, handlerProvider
generateDefaultRPCEngineConfig(),
backend,
mockLoadBalancerPtr,
dosGuard,
queue,
*mockCountersPtr,
handlerProvider
);
if (testBundle.forwarded) {
@@ -321,7 +346,7 @@ generateCacheTestValuesForParametersTest()
{
return std::vector<RPCEngineCacheTestCaseBundle>{
{.testName = "CacheEnabled",
.config = R"JSON({
.config = R"JSON({
"server": {"max_queue_size": 2},
"workers": 4,
"rpc":
@@ -334,7 +359,7 @@ generateCacheTestValuesForParametersTest()
.config = R"JSON({
"server": {"max_queue_size": 2},
"workers": 4,
"rpc": {}
"rpc": {"cache_timeout": 0}
})JSON",
.method = "server_info",
.isAdmin = false,
@@ -343,7 +368,7 @@ generateCacheTestValuesForParametersTest()
.config = R"JSON({
"server": {"max_queue_size": 2},
"workers": 4,
"rpc": {}
"rpc": {"cache_timeout": 0}
})JSON",
.method = "server_info",
.isAdmin = false,
@@ -388,7 +413,11 @@ INSTANTIATE_TEST_CASE_P(
TEST_P(RPCEngineCacheParameterTest, Test)
{
auto const& testParam = GetParam();
auto const cfgCache = Config{json::parse(testParam.config)};
auto const json = ConfigFileJson{json::parse(testParam.config).as_object()};
auto cfgCache{generateDefaultRPCEngineConfig()};
auto const errors = cfgCache.parse(json);
EXPECT_TRUE(!errors.has_value());
auto const admin = testParam.isAdmin;
auto const method = testParam.method;
@@ -432,11 +461,11 @@ TEST_P(RPCEngineCacheParameterTest, Test)
TEST_F(RPCEngineTest, NotCacheIfErrorHappen)
{
auto const cfgCache = Config{json::parse(R"JSON({
"server": {"max_queue_size": 2},
"workers": 4,
"rpc": {"cache_timeout": 10}
})JSON")};
auto const cfgCache = ClioConfigDefinition{
{"server.max_queue_size", ConfigValue{ConfigType::Integer}.defaultValue(2)},
{"workers", ConfigValue{ConfigType::Integer}.defaultValue(4).withConstraint(validateUint16)},
{"rpc.cache_timeout", ConfigValue{ConfigType::Double}.defaultValue(10.0).withConstraint(validatePositiveDouble)}
};
auto const notAdmin = false;
auto const method = "server_info";

View File

@@ -20,7 +20,9 @@
#include "rpc/WorkQueue.hpp"
#include "util/LoggerFixtures.hpp"
#include "util/MockPrometheus.hpp"
#include "util/config/Config.hpp"
#include "util/newconfig/ConfigDefinition.hpp"
#include "util/newconfig/ConfigValue.hpp"
#include "util/newconfig/Types.hpp"
#include "util/prometheus/Counter.hpp"
#include "util/prometheus/Gauge.hpp"
@@ -34,22 +36,20 @@
#include <semaphore>
using namespace util;
using namespace util::config;
using namespace rpc;
using namespace util::prometheus;
namespace {
constexpr auto JSONConfig = R"JSON({
"server": { "max_queue_size" : 2 },
"workers": 4
})JSON";
} // namespace
struct RPCWorkQueueTestBase : NoLoggerFixture {
ClioConfigDefinition cfg = {
{"server.max_queue_size", ConfigValue{ConfigType::Integer}.defaultValue(2)},
{"workers", ConfigValue{ConfigType::Integer}.defaultValue(4)}
};
struct WorkQueueTestBase : NoLoggerFixture {
Config cfg = Config{boost::json::parse(JSONConfig)};
WorkQueue queue = WorkQueue::make_WorkQueue(cfg);
};
struct WorkQueueTest : WithPrometheus, WorkQueueTestBase {};
struct WorkQueueTest : WithPrometheus, RPCWorkQueueTestBase {};
TEST_F(WorkQueueTest, WhitelistedExecutionCountAddsUp)
{
@@ -158,7 +158,7 @@ TEST_F(WorkQueueStopTest, CallsOnTasksCompleteWhenStoppingOnLastTask)
queue.join();
}
struct WorkQueueMockPrometheusTest : WithMockPrometheus, WorkQueueTestBase {};
struct WorkQueueMockPrometheusTest : WithMockPrometheus, RPCWorkQueueTestBase {};
TEST_F(WorkQueueMockPrometheusTest, postCoroCouhters)
{

View File

@@ -21,7 +21,10 @@
#include "rpc/common/Types.hpp"
#include "rpc/handlers/VersionHandler.hpp"
#include "util/HandlerBaseTestFixture.hpp"
#include "util/config/Config.hpp"
#include "util/log/Logger.hpp"
#include "util/newconfig/ConfigDefinition.hpp"
#include "util/newconfig/ConfigValue.hpp"
#include "util/newconfig/Types.hpp"
#include <boost/json/parse.hpp>
#include <boost/json/value.hpp>
@@ -33,26 +36,32 @@ constexpr static auto MIN_API_VERSION = 2u;
constexpr static auto MAX_API_VERSION = 10u;
using namespace rpc;
namespace json = boost::json;
using namespace util::config;
class RPCVersionHandlerTest : public HandlerBaseTest {};
TEST_F(RPCVersionHandlerTest, Default)
{
util::Config cfg{json::parse(fmt::format(
ClioConfigDefinition cfg{
{"api_version.min", ConfigValue{ConfigType::Integer}.defaultValue(MIN_API_VERSION)},
{"api_version.max", ConfigValue{ConfigType::Integer}.defaultValue(MAX_API_VERSION)},
{"api_version.default", ConfigValue{ConfigType::Integer}.defaultValue(DEFAULT_API_VERSION)}
};
boost::json::value jsonData = boost::json::parse(fmt::format(
R"({{
"min": {},
"max": {},
"default": {}
"api_version.min": {},
"api_version.max": {},
"api_version.default": {}
}})",
MIN_API_VERSION,
MAX_API_VERSION,
DEFAULT_API_VERSION
))};
));
runSpawn([&](auto yield) {
auto const handler = AnyHandler{VersionHandler{cfg}};
auto const output = handler.process(static_cast<json::value>(cfg), Context{yield});
auto const output = handler.process(jsonData, Context{yield});
ASSERT_TRUE(output);
// check all against all the correct values

View File

@@ -19,7 +19,9 @@
#include "util/LoggerFixtures.hpp"
#include "util/SignalsHandler.hpp"
#include "util/config/Config.hpp"
#include "util/newconfig/ConfigDefinition.hpp"
#include "util/newconfig/ConfigValue.hpp"
#include "util/newconfig/Types.hpp"
#include <boost/json/value.hpp>
#include <gmock/gmock.h>
@@ -33,6 +35,7 @@
#include <thread>
using namespace util;
using namespace util::config;
using testing::MockFunction;
using testing::StrictMock;
@@ -63,14 +66,18 @@ struct SignalsHandlerTestsBase : NoLoggerFixture {
TEST(SignalsHandlerDeathTest, CantCreateTwoSignalsHandlers)
{
auto makeHandler = []() { return SignalsHandler{Config{}, []() {}}; };
auto makeHandler = []() {
return SignalsHandler{
ClioConfigDefinition{{"graceful_period", ConfigValue{ConfigType::Double}.defaultValue(10.f)}}, []() {}
};
};
auto const handler = makeHandler();
EXPECT_DEATH({ makeHandler(); }, ".*");
}
struct SignalsHandlerTests : SignalsHandlerTestsBase {
SignalsHandler handler_{
util::Config{boost::json::value{{"graceful_period", 3.0}}},
ClioConfigDefinition{{"graceful_period", ConfigValue{ConfigType::Double}.defaultValue(3.0)}},
forceExitHandler_.AsStdFunction()
};
};
@@ -94,7 +101,7 @@ TEST_F(SignalsHandlerTests, OneSignal)
struct SignalsHandlerTimeoutTests : SignalsHandlerTestsBase {
SignalsHandler handler_{
util::Config{boost::json::value{{"graceful_period", 0.001}}},
ClioConfigDefinition{{"graceful_period", ConfigValue{ConfigType::Double}.defaultValue(0.001)}},
forceExitHandler_.AsStdFunction()
};
};

View File

@@ -29,6 +29,7 @@
#include <gtest/gtest.h>
#include <cstddef>
#include <string>
using namespace util::config;
@@ -99,8 +100,8 @@ TEST_F(ArrayViewTest, ArrayWithObj)
auto const obj1 = arrVals.objectAt(0);
auto const obj2 = arrValAlt.objectAt(0);
EXPECT_NEAR(obj1.getValue("sub").asDouble(), obj2.getValue("sub").asDouble(), precision);
EXPECT_NEAR(obj1.getValue("sub").asDouble(), 111.11, precision);
EXPECT_NEAR(obj1.get<double>("sub"), obj2.get<double>("sub"), precision);
EXPECT_NEAR(obj1.get<double>("sub"), 111.11, precision);
}
TEST_F(ArrayViewTest, IterateArray)
@@ -134,14 +135,14 @@ TEST_F(ArrayViewTest, IterateObject)
EXPECT_EQ(3, arr.size());
auto it = arr.begin<ObjectView>();
EXPECT_EQ(111.11, (*it).getValue("sub").asDouble());
EXPECT_EQ("subCategory", (*it++).getValue("sub2").asString());
EXPECT_EQ(111.11, (*it).get<double>("sub"));
EXPECT_EQ("subCategory", (*it++).get<std::string>("sub2"));
EXPECT_EQ(4321.55, (*it).getValue("sub").asDouble());
EXPECT_EQ("temporary", (*it++).getValue("sub2").asString());
EXPECT_EQ(4321.55, (*it).get<double>("sub"));
EXPECT_EQ("temporary", (*it++).get<std::string>("sub2"));
EXPECT_EQ(5555.44, (*it).getValue("sub").asDouble());
EXPECT_EQ("london", (*it++).getValue("sub2").asString());
EXPECT_EQ(5555.44, (*it).get<double>("sub"));
EXPECT_EQ("london", (*it++).get<std::string>("sub2"));
EXPECT_EQ(it, arr.end<ObjectView>());
}

View File

@@ -32,6 +32,7 @@
#include <algorithm>
#include <cstdint>
#include <optional>
#include <string>
#include <string_view>
#include <unordered_set>
@@ -45,14 +46,29 @@ struct NewConfigTest : testing::Test {
TEST_F(NewConfigTest, fetchValues)
{
auto const v = configData.getValue("header.port");
auto const v = configData.getValueView("header.port");
EXPECT_EQ(v.type(), ConfigType::Integer);
EXPECT_EQ("value", configData.getValue("header.text1").asString());
EXPECT_EQ(123, configData.getValue("header.port").asIntType<int>());
EXPECT_EQ(true, configData.getValue("header.admin").asBool());
EXPECT_EQ("TSM", configData.getValue("header.sub.sub2Value").asString());
EXPECT_EQ(444.22, configData.getValue("ip").asDouble());
EXPECT_EQ("value", configData.getValueView("header.text1").asString());
EXPECT_EQ(123, configData.getValueView("header.port").asIntType<int>());
EXPECT_EQ(true, configData.getValueView("header.admin").asBool());
EXPECT_EQ("TSM", configData.getValueView("header.sub.sub2Value").asString());
EXPECT_EQ(444.22, configData.getValueView("ip").asDouble());
}
TEST_F(NewConfigTest, fetchValuesByTemplate)
{
EXPECT_EQ("value", configData.get<std::string>("header.text1"));
EXPECT_EQ(123, configData.get<int>("header.port"));
EXPECT_EQ(true, configData.get<bool>("header.admin"));
EXPECT_EQ("TSM", configData.get<std::string>("header.sub.sub2Value"));
EXPECT_EQ(444.22, configData.get<double>("ip"));
}
TEST_F(NewConfigTest, fetchOptionalValues)
{
EXPECT_EQ(std::nullopt, configData.maybeValue<double>("optional.withNoDefault"));
EXPECT_EQ(0.0, configData.maybeValue<double>("optional.withDefault"));
}
TEST_F(NewConfigTest, fetchObjectDirectly)
@@ -62,7 +78,7 @@ TEST_F(NewConfigTest, fetchObjectDirectly)
auto const obj2 = obj.getObject("sub");
EXPECT_TRUE(obj2.containsKey("sub2Value"));
EXPECT_EQ(obj2.getValue("sub2Value").asString(), "TSM");
EXPECT_EQ(obj2.getValueView("sub2Value").asString(), "TSM");
}
TEST_F(NewConfigTest, CheckKeys)
@@ -112,45 +128,45 @@ struct NewConfigDeathTest : NewConfigTest {};
TEST_F(NewConfigDeathTest, GetNonExistentKeys)
{
EXPECT_DEATH({ [[maybe_unused]] auto a_ = configData.getValue("head."); }, ".*");
EXPECT_DEATH({ [[maybe_unused]] auto a_ = configData.getValue("asdf"); }, ".*");
EXPECT_DEATH({ [[maybe_unused]] auto unused = configData.getValueView("head."); }, ".*");
EXPECT_DEATH({ [[maybe_unused]] auto unused = configData.getValueView("asdf"); }, ".*");
}
TEST_F(NewConfigDeathTest, GetValueButIsArray)
{
EXPECT_DEATH({ [[maybe_unused]] auto a_ = configData.getValue("dosguard.whitelist"); }, ".*");
EXPECT_DEATH({ [[maybe_unused]] auto a_ = configData.getValue("dosguard.whitelist.[]"); }, ".*");
EXPECT_DEATH({ [[maybe_unused]] auto unused = configData.getValueView("dosguard.whitelist"); }, ".*");
EXPECT_DEATH({ [[maybe_unused]] auto unused = configData.getValueView("dosguard.whitelist.[]"); }, ".*");
}
TEST_F(NewConfigDeathTest, GetNonExistentObjectKey)
{
ASSERT_FALSE(configData.contains("head"));
EXPECT_DEATH({ [[maybe_unused]] auto a_ = configData.getObject("head"); }, ".*");
EXPECT_DEATH({ [[maybe_unused]] auto a_ = configData.getObject("doesNotExist"); }, ".*");
EXPECT_DEATH({ [[maybe_unused]] auto unused = configData.getObject("head"); }, ".*");
EXPECT_DEATH({ [[maybe_unused]] auto unused = configData.getObject("doesNotExist"); }, ".*");
}
TEST_F(NewConfigDeathTest, GetObjectButIsArray)
{
EXPECT_DEATH({ [[maybe_unused]] auto a_ = configData.getObject("array"); }, ".*");
EXPECT_DEATH({ [[maybe_unused]] auto a_ = configData.getObject("array", 2); }, ".*");
EXPECT_DEATH({ [[maybe_unused]] auto unused = configData.getObject("array"); }, ".*");
EXPECT_DEATH({ [[maybe_unused]] auto unused = configData.getObject("array", 2); }, ".*");
}
TEST_F(NewConfigDeathTest, GetArrayButIsValue)
{
EXPECT_DEATH({ [[maybe_unused]] auto a_ = configData.getArray("header.text1"); }, ".*");
EXPECT_DEATH({ [[maybe_unused]] auto unused = configData.getArray("header.text1"); }, ".*");
}
TEST_F(NewConfigDeathTest, GetNonExistentArrayKey)
{
EXPECT_DEATH({ [[maybe_unused]] auto a_ = configData.getArray("asdf"); }, ".*");
EXPECT_DEATH({ [[maybe_unused]] auto unused = configData.getArray("asdf"); }, ".*");
}
TEST(ConfigDescription, GetValues)
{
ClioConfigDescription const definition{};
EXPECT_EQ(definition.get("database.type"), "Type of database to use.");
EXPECT_EQ(definition.get("etl_source.[].ip"), "IP address of the ETL source.");
EXPECT_EQ(definition.get("database.type"), "Type of database to use. Default is Scylladb.");
EXPECT_EQ(definition.get("etl_sources.[].ip"), "IP address of the ETL source.");
EXPECT_EQ(definition.get("prometheus.enabled"), "Enable or disable Prometheus metrics.");
}
@@ -159,7 +175,7 @@ TEST(ConfigDescriptionAssertDeathTest, NonExistingKeyTest)
ClioConfigDescription const definition{};
EXPECT_DEATH({ [[maybe_unused]] auto a = definition.get("data"); }, ".*");
EXPECT_DEATH({ [[maybe_unused]] auto a = definition.get("etl_source.[]"); }, ".*");
EXPECT_DEATH({ [[maybe_unused]] auto a = definition.get("etl_sources.[]"); }, ".*");
}
/** @brief Testing override the default values with the ones in Json */
@@ -177,39 +193,39 @@ TEST_F(OverrideConfigVals, ValidateValuesStrings)
{
// make sure the values in configData are overriden
EXPECT_TRUE(configData.contains("header.text1"));
EXPECT_EQ(configData.getValue("header.text1").asString(), "value");
EXPECT_EQ(configData.getValueView("header.text1").asString(), "value");
EXPECT_FALSE(configData.contains("header.sub"));
EXPECT_TRUE(configData.contains("header.sub.sub2Value"));
EXPECT_EQ(configData.getValue("header.sub.sub2Value").asString(), "TSM");
EXPECT_EQ(configData.getValueView("header.sub.sub2Value").asString(), "TSM");
EXPECT_TRUE(configData.contains("requireValue"));
EXPECT_EQ(configData.getValue("requireValue").asString(), "required");
EXPECT_EQ(configData.getValueView("requireValue").asString(), "required");
}
TEST_F(OverrideConfigVals, ValidateValuesDouble)
{
EXPECT_TRUE(configData.contains("optional.withDefault"));
EXPECT_EQ(configData.getValue("optional.withDefault").asDouble(), 0.0);
EXPECT_EQ(configData.getValueView("optional.withDefault").asDouble(), 0.0);
// make sure the values not overwritten, (default values) are there too
EXPECT_TRUE(configData.contains("ip"));
EXPECT_EQ(configData.getValue("ip").asDouble(), 444.22);
EXPECT_EQ(configData.getValueView("ip").asDouble(), 444.22);
}
TEST_F(OverrideConfigVals, ValidateValuesInteger)
{
EXPECT_TRUE(configData.contains("dosguard.port"));
EXPECT_EQ(configData.getValue("dosguard.port").asIntType<int>(), 44444);
EXPECT_EQ(configData.getValueView("dosguard.port").asIntType<int>(), 44444);
EXPECT_TRUE(configData.contains("header.port"));
EXPECT_EQ(configData.getValue("header.port").asIntType<int64_t>(), 321);
EXPECT_EQ(configData.getValueView("header.port").asIntType<int64_t>(), 321);
}
TEST_F(OverrideConfigVals, ValidateValuesBool)
{
EXPECT_TRUE(configData.contains("header.admin"));
EXPECT_EQ(configData.getValue("header.admin").asBool(), false);
EXPECT_EQ(configData.getValueView("header.admin").asBool(), false);
}
TEST_F(OverrideConfigVals, ValidateIntegerValuesInArrays)
@@ -266,12 +282,12 @@ TEST_F(OverrideConfigVals, FetchObjectByArray)
auto const obj2InArr = configData.getObject("array", 1);
auto const obj3InArr = configData.getObject("array", 2);
EXPECT_EQ(objInArr.getValue("sub").asDouble(), 111.11);
EXPECT_EQ(objInArr.getValue("sub2").asString(), "subCategory");
EXPECT_EQ(obj2InArr.getValue("sub").asDouble(), 4321.55);
EXPECT_EQ(obj2InArr.getValue("sub2").asString(), "temporary");
EXPECT_EQ(obj3InArr.getValue("sub").asDouble(), 5555.44);
EXPECT_EQ(obj3InArr.getValue("sub2").asString(), "london");
EXPECT_EQ(objInArr.getValueView("sub").asDouble(), 111.11);
EXPECT_EQ(objInArr.getValueView("sub2").asString(), "subCategory");
EXPECT_EQ(obj2InArr.getValueView("sub").asDouble(), 4321.55);
EXPECT_EQ(obj2InArr.getValueView("sub2").asString(), "temporary");
EXPECT_EQ(obj3InArr.getValueView("sub").asDouble(), 5555.44);
EXPECT_EQ(obj3InArr.getValueView("sub2").asString(), "london");
}
struct IncorrectOverrideValues : testing::Test {

View File

@@ -65,7 +65,6 @@ TEST(ConfigValue, SetValuesOnPortConstraint)
EXPECT_TRUE(cvPort.setValue(33.33).has_value());
EXPECT_TRUE(cvPort.setValue(33.33).value().error == "value does not match type integer");
EXPECT_FALSE(cvPort.setValue(1).has_value());
auto cvPort2 = ConfigValue{ConfigType::String}.defaultValue("4444").withConstraint(validatePort);
auto const strPortError = cvPort2.setValue("100000");
EXPECT_TRUE(strPortError.has_value());
@@ -150,9 +149,10 @@ TEST(ConfigValue, PositiveDoubleConstraint)
auto const doubleCons{PositiveDouble{}};
EXPECT_FALSE(doubleCons.checkConstraint(0.2));
EXPECT_FALSE(doubleCons.checkConstraint(5.54));
EXPECT_FALSE(doubleCons.checkConstraint(3));
EXPECT_TRUE(doubleCons.checkConstraint("-5"));
EXPECT_EQ(doubleCons.checkConstraint("-5")->error, "Double number must be of type int or double");
EXPECT_EQ(doubleCons.checkConstraint(-5.6)->error, "Double number must be greater than 0");
EXPECT_EQ(doubleCons.checkConstraint(-5.6)->error, "Double number must be greater than or equal to 0");
EXPECT_FALSE(doubleCons.checkConstraint(12.1));
}

View File

@@ -0,0 +1,99 @@
//------------------------------------------------------------------------------
/*
This file is part of clio: https://github.com/XRPLF/clio
Copyright (c) 2024, 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/TmpFile.hpp"
#include "util/newconfig/ConfigFileJson.hpp"
#include "util/newconfig/FakeConfigData.hpp"
#include <boost/json/parse.hpp>
#include <gtest/gtest.h>
#include <algorithm>
#include <cstdint>
#include <string>
#include <variant>
#include <vector>
struct JsonFromTempFile : testing::Test {
JsonFromTempFile() : jsonFileObj{util::config::ConfigFileJson::make_ConfigFileJson(TmpFile(JSONData).path).value()}
{
}
ConfigFileJson jsonFileObj;
};
TEST_F(JsonFromTempFile, validateKeys)
{
EXPECT_TRUE(jsonFileObj.containsKey("header.text1"));
EXPECT_TRUE(jsonFileObj.containsKey("header.sub.sub2Value"));
EXPECT_TRUE(jsonFileObj.containsKey("dosguard.port"));
EXPECT_FALSE(jsonFileObj.containsKey("idk"));
EXPECT_FALSE(jsonFileObj.containsKey("optional.withNoDefault"));
}
TEST_F(JsonFromTempFile, validateValues)
{
EXPECT_EQ(std::get<std::string>(jsonFileObj.getValue("header.text1")), "value");
EXPECT_EQ(std::get<std::string>(jsonFileObj.getValue("header.sub.sub2Value")), "TSM");
EXPECT_EQ(std::get<int64_t>(jsonFileObj.getValue("dosguard.port")), 44444);
}
TEST_F(JsonFromTempFile, validateArrayValue)
{
// validate array.[].sub matches expected values
EXPECT_TRUE(jsonFileObj.containsKey("array.[].sub"));
auto const arrSub = jsonFileObj.getArray("array.[].sub");
EXPECT_EQ(arrSub.size(), 3);
std::vector<double> expectedArrSubVal{111.11, 4321.55, 5555.44};
std::vector<double> actualArrSubVal{};
for (auto it = arrSub.begin(); it != arrSub.end(); ++it) {
ASSERT_TRUE(std::holds_alternative<double>(*it));
actualArrSubVal.emplace_back(std::get<double>(*it));
}
EXPECT_TRUE(std::ranges::equal(expectedArrSubVal, actualArrSubVal));
// validate array.[].sub2 matches expected values
EXPECT_TRUE(jsonFileObj.containsKey("array.[].sub2"));
auto const arrSub2 = jsonFileObj.getArray("array.[].sub2");
EXPECT_EQ(arrSub2.size(), 3);
std::vector<std::string> expectedArrSub2Val{"subCategory", "temporary", "london"};
std::vector<std::string> actualArrSub2Val{};
for (auto it = arrSub2.begin(); it != arrSub2.end(); ++it) {
ASSERT_TRUE(std::holds_alternative<std::string>(*it));
actualArrSub2Val.emplace_back(std::get<std::string>(*it));
}
EXPECT_TRUE(std::ranges::equal(expectedArrSub2Val, actualArrSub2Val));
EXPECT_TRUE(jsonFileObj.containsKey("dosguard.whitelist.[]"));
auto const whitelistArr = jsonFileObj.getArray("dosguard.whitelist.[]");
EXPECT_EQ(whitelistArr.size(), 2);
EXPECT_EQ("125.5.5.1", std::get<std::string>(whitelistArr.at(0)));
EXPECT_EQ("204.2.2.1", std::get<std::string>(whitelistArr.at(1)));
}
struct JsonValueDeathTest : JsonFromTempFile {};
TEST_F(JsonValueDeathTest, invalidGetValues)
{
// not possible for json value to call a value that doesn't exist
EXPECT_DEATH([[maybe_unused]] auto a = jsonFileObj.getArray("header.text1"), ".*");
}

View File

@@ -26,6 +26,9 @@
#include <boost/json/parse.hpp>
#include <gtest/gtest.h>
#include <optional>
#include <string>
using namespace util::config;
struct ObjectViewTest : testing::Test {
@@ -38,17 +41,36 @@ struct ObjectViewTest : testing::Test {
ClioConfigDefinition configData = generateConfig();
};
TEST_F(ObjectViewTest, ObjectValueTest)
TEST_F(ObjectViewTest, ObjectContainsKeyTest)
{
auto const headerObj = configData.getObject("header");
EXPECT_FALSE(headerObj.containsKey("header"));
EXPECT_TRUE(headerObj.containsKey("text1"));
EXPECT_TRUE(headerObj.containsKey("port"));
EXPECT_TRUE(headerObj.containsKey("admin"));
}
EXPECT_EQ("value", headerObj.getValue("text1").asString());
EXPECT_EQ(321, headerObj.getValue("port").asIntType<int>());
EXPECT_EQ(false, headerObj.getValue("admin").asBool());
TEST_F(ObjectViewTest, ObjectValueTest)
{
auto const headerObj = configData.getObject("header");
EXPECT_EQ("value", headerObj.getValueView("text1").asString());
EXPECT_EQ(321, headerObj.getValueView("port").asIntType<int>());
EXPECT_EQ(false, headerObj.getValueView("admin").asBool());
}
TEST_F(ObjectViewTest, ObjectGetValueByTemplateTest)
{
auto const headerObj = configData.getObject("header");
EXPECT_EQ("value", headerObj.get<std::string>("text1"));
EXPECT_EQ(321, headerObj.get<int>("port"));
EXPECT_EQ(false, headerObj.get<bool>("admin"));
}
TEST_F(ObjectViewTest, GetOptionalValue)
{
auto const optionalObj = configData.getObject("optional");
EXPECT_EQ(std::nullopt, optionalObj.maybeValue<double>("withNoDefault"));
EXPECT_EQ(0.0, optionalObj.maybeValue<double>("withDefault"));
}
TEST_F(ObjectViewTest, ObjectValuesInArray)
@@ -63,11 +85,11 @@ TEST_F(ObjectViewTest, ObjectValuesInArray)
// object's key is only "sub" and "sub2"
EXPECT_FALSE(firstObj.containsKey("array.[].sub"));
EXPECT_EQ(firstObj.getValue("sub").asDouble(), 111.11);
EXPECT_EQ(firstObj.getValue("sub2").asString(), "subCategory");
EXPECT_EQ(firstObj.getValueView("sub").asDouble(), 111.11);
EXPECT_EQ(firstObj.getValueView("sub2").asString(), "subCategory");
EXPECT_EQ(secondObj.getValue("sub").asDouble(), 4321.55);
EXPECT_EQ(secondObj.getValue("sub2").asString(), "temporary");
EXPECT_EQ(secondObj.getValueView("sub").asDouble(), 4321.55);
EXPECT_EQ(secondObj.getValueView("sub2").asString(), "temporary");
}
TEST_F(ObjectViewTest, GetObjectsInDifferentWays)
@@ -79,15 +101,15 @@ TEST_F(ObjectViewTest, GetObjectsInDifferentWays)
// this returns the 1st object inside "low"
ObjectView const sameObjFromConfigData = configData.getObject("higher.[].low", 0);
EXPECT_EQ(sameObjFromConfigData.getValue("admin").asBool(), firstObj.getValue("low.admin").asBool());
EXPECT_EQ(sameObjFromConfigData.getValueView("admin").asBool(), firstObj.getValueView("low.admin").asBool());
EXPECT_FALSE(firstObj.containsKey("low"));
EXPECT_TRUE(firstObj.containsKey("low.admin"));
ObjectView const objLow = firstObj.getObject("low");
EXPECT_TRUE(objLow.containsKey("section"));
EXPECT_TRUE(objLow.containsKey("admin"));
EXPECT_EQ(objLow.getValue("section").asString(), "WebServer");
EXPECT_EQ(objLow.getValue("admin").asBool(), false);
EXPECT_EQ(objLow.getValueView("section").asString(), "WebServer");
EXPECT_EQ(objLow.getValueView("admin").asBool(), false);
}
TEST_F(ObjectViewTest, getArrayInObject)
@@ -120,3 +142,9 @@ TEST_F(ObjectViewDeathTest, KeyisArrayView)
// dies because only 1 object in higher.[].low
EXPECT_DEATH({ [[maybe_unused]] auto _ = configData.getObject("higher.[].low", 1); }, ".*");
}
TEST_F(ObjectViewDeathTest, KeyisNotOptional)
{
// dies because not an optional
EXPECT_DEATH({ [[maybe_unused]] auto _ = configData.getObject("header").maybeValue<std::string>("text1"); }, ".*");
}

View File

@@ -17,6 +17,7 @@
*/
//==============================================================================
#include "util/newconfig/ConfigConstraints.hpp"
#include "util/newconfig/ConfigDefinition.hpp"
#include "util/newconfig/ConfigValue.hpp"
#include "util/newconfig/FakeConfigData.hpp"
@@ -26,6 +27,8 @@
#include <gtest/gtest.h>
#include <cstdint>
#include <optional>
#include <string>
using namespace util::config;
@@ -45,7 +48,7 @@ TEST_F(ValueViewTest, ValueView)
TEST_F(ValueViewTest, DifferentIntegerTest)
{
auto const vv = configData.getValue("header.port");
auto const vv = configData.getValueView("header.port");
auto const uint32 = vv.asIntType<uint32_t>();
auto const uint64 = vv.asIntType<uint64_t>();
auto const int32 = vv.asIntType<int32_t>();
@@ -65,27 +68,58 @@ TEST_F(ValueViewTest, DifferentIntegerTest)
EXPECT_NEAR(doubleVal, sameDouble, precision);
EXPECT_NEAR(floatVal, sameFloat, precision);
auto const ipVal = configData.getValue("ip");
auto const ipVal = configData.getValueView("ip");
auto const ipDouble = ipVal.asDouble();
auto const ipFloat = ipVal.asFloat();
EXPECT_NEAR(ipDouble, 444.22, precision);
EXPECT_NEAR(ipFloat, 444.22f, precision);
}
TEST_F(ValueViewTest, IntegerAsDoubleTypeValue)
{
auto const cv = ConfigValue{ConfigType::Double}.defaultValue(432).withConstraint(validatePositiveDouble);
ValueView const vv{cv};
auto const doubleVal = vv.asFloat();
auto const floatVal = vv.asDouble();
auto const precision = 1e-9;
EXPECT_NEAR(doubleVal, 432, precision);
EXPECT_NEAR(floatVal, 432, precision);
}
TEST_F(ValueViewTest, OptionalValues)
{
auto const cv = ConfigValue{ConfigType::Integer}.defaultValue(432).optional();
auto const cv2 = ConfigValue{ConfigType::Double}.optional();
auto const cv3 = ConfigValue{ConfigType::String}.optional();
auto const cv4 = ConfigValue{ConfigType::String}.defaultValue("hello").optional();
ValueView const vv{cv};
ValueView const vv2{cv2};
ValueView const vv3{cv3};
ValueView const vv4{cv4};
EXPECT_EQ(vv.asOptional<uint32_t>().value(), 432);
EXPECT_EQ(vv.asOptional<uint64_t>().value(), 432);
EXPECT_EQ(vv2.asOptional<uint64_t>(), std::nullopt);
EXPECT_EQ(vv3.asOptional<std::string>(), std::nullopt);
EXPECT_EQ(vv4.asOptional<std::string>(), "hello");
}
struct ValueDeathTest : ValueViewTest {};
TEST_F(ValueDeathTest, WrongTypes)
{
auto const vv = configData.getValue("header.port");
EXPECT_DEATH({ [[maybe_unused]] auto a_ = vv.asBool(); }, ".*");
EXPECT_DEATH({ [[maybe_unused]] auto a_ = vv.asString(); }, ".*");
auto const vv = configData.getValueView("header.port");
EXPECT_DEATH({ [[maybe_unused]] auto unused = vv.asBool(); }, ".*");
EXPECT_DEATH({ [[maybe_unused]] auto unused = vv.asString(); }, ".*");
auto const cv = ConfigValue{ConfigType::Integer}.defaultValue(-5);
auto const vv2 = ValueView(cv);
EXPECT_DEATH({ [[maybe_unused]] auto a_ = vv2.asIntType<uint32_t>(); }, ".*");
EXPECT_DEATH({ [[maybe_unused]] auto unused = vv2.asIntType<uint32_t>(); }, ".*");
auto const cv2 = ConfigValue{ConfigType::String}.defaultValue("asdf");
auto const vv3 = ValueView(cv2);
EXPECT_DEATH({ [[maybe_unused]] auto a_ = vv3.asDouble(); }, ".*");
EXPECT_DEATH({ [[maybe_unused]] auto a_ = vv3.asFloat(); }, ".*");
EXPECT_DEATH({ [[maybe_unused]] auto unused = vv3.asDouble(); }, ".*");
EXPECT_DEATH({ [[maybe_unused]] auto unused = vv3.asFloat(); }, ".*");
}

View File

@@ -18,7 +18,9 @@
//==============================================================================
#include "util/MockPrometheus.hpp"
#include "util/NameGenerator.hpp"
#include "util/config/Config.hpp"
#include "util/newconfig/ConfigDefinition.hpp"
#include "util/newconfig/ConfigValue.hpp"
#include "util/newconfig/Types.hpp"
#include "util/prometheus/Http.hpp"
#include "util/prometheus/Label.hpp"
#include "util/prometheus/Prometheus.hpp"
@@ -36,6 +38,7 @@
#include <string>
using namespace util::prometheus;
using namespace util::config;
namespace http = boost::beast::http;
struct PrometheusCheckRequestTestsParams {
@@ -51,8 +54,11 @@ struct PrometheusCheckRequestTests : public ::testing::TestWithParam<PrometheusC
TEST_P(PrometheusCheckRequestTests, isPrometheusRequest)
{
boost::json::value const configJson{{"prometheus", boost::json::object{{"enabled", GetParam().prometheusEnabled}}}};
PrometheusService::init(util::Config{configJson});
ClioConfigDefinition config{
{"prometheus.enabled", ConfigValue{ConfigType::Boolean}.defaultValue(GetParam().prometheusEnabled)},
{"prometheus.compress_reply", ConfigValue{ConfigType::Boolean}.defaultValue(true)}
};
PrometheusService::init(config);
boost::beast::http::request<boost::beast::http::string_body> req;
req.method(GetParam().method);
req.target(GetParam().target);
@@ -122,8 +128,11 @@ TEST_F(PrometheusHandleRequestTests, emptyResponse)
TEST_F(PrometheusHandleRequestTests, prometheusDisabled)
{
boost::json::value const configJson({{"prometheus", boost::json::object{{"enabled", false}}}});
PrometheusService::init(util::Config(configJson));
ClioConfigDefinition config{
{"prometheus.enabled", ConfigValue{ConfigType::Boolean}.defaultValue(false)},
{"prometheus.compress_reply", ConfigValue{ConfigType::Boolean}.defaultValue(true)}
};
PrometheusService::init(config);
auto response = handlePrometheusRequest(req, true);
ASSERT_TRUE(response.has_value());
EXPECT_EQ(response->result(), http::status::forbidden);
@@ -221,9 +230,10 @@ TEST_F(PrometheusHandleRequestTests, responseWithCounterAndGauge)
TEST_F(PrometheusHandleRequestTests, compressReply)
{
PrometheusService::init(
util::Config(boost::json::value{{"prometheus", boost::json::object{{"compress_reply", true}}}})
);
PrometheusService::init(ClioConfigDefinition{
{"prometheus.compress_reply", ConfigValue{ConfigType::Boolean}.defaultValue(true)},
{"prometheus.enabled", ConfigValue{ConfigType::Boolean}.defaultValue(true)},
});
auto& gauge = PrometheusService::gaugeInt("test_gauge", Labels{});
++gauge;

View File

@@ -19,19 +19,26 @@
#include "util/LoggerFixtures.hpp"
#include "util/NameGenerator.hpp"
#include "util/config/Config.hpp"
#include "util/newconfig/ConfigDefinition.hpp"
#include "util/newconfig/ConfigFileJson.hpp"
#include "util/newconfig/ConfigValue.hpp"
#include "util/newconfig/Types.hpp"
#include "web/AdminVerificationStrategy.hpp"
#include <boost/beast/http/field.hpp>
#include <boost/beast/http/message.hpp>
#include <boost/beast/http/string_body.hpp>
#include <boost/json/parse.hpp>
#include <boost/json/value.hpp>
#include <gtest/gtest.h>
#include <iostream>
#include <optional>
#include <ostream>
#include <string>
namespace http = boost::beast::http;
using namespace util::config;
class IPAdminVerificationStrategyTest : public NoLoggerFixture {
protected:
@@ -134,9 +141,21 @@ struct MakeAdminVerificationStrategyFromConfigTestParams {
struct MakeAdminVerificationStrategyFromConfigTest
: public testing::TestWithParam<MakeAdminVerificationStrategyFromConfigTestParams> {};
inline ClioConfigDefinition
generateDefaultAdminConfig()
{
return ClioConfigDefinition{
{{"server.local_admin", ConfigValue{ConfigType::Boolean}.optional()},
{"server.admin_password", ConfigValue{ConfigType::String}.optional()}}
};
}
TEST_P(MakeAdminVerificationStrategyFromConfigTest, ChecksConfig)
{
util::Config const serverConfig{boost::json::parse(GetParam().config)};
ConfigFileJson const js{boost::json::parse(GetParam().config).as_object()};
ClioConfigDefinition serverConfig{generateDefaultAdminConfig()};
auto const errors = serverConfig.parse(js);
ASSERT_TRUE(!errors.has_value());
auto const result = web::make_AdminVerificationStrategy(serverConfig);
EXPECT_EQ(not result.has_value(), GetParam().expectedError);
}

View File

@@ -26,7 +26,9 @@
#include "util/MockRPCEngine.hpp"
#include "util/NameGenerator.hpp"
#include "util/Taggable.hpp"
#include "util/config/Config.hpp"
#include "util/newconfig/ConfigDefinition.hpp"
#include "util/newconfig/ConfigValue.hpp"
#include "util/newconfig/Types.hpp"
#include "web/RPCServerHandler.hpp"
#include "web/SubscriptionContextInterface.hpp"
#include "web/interface/ConnectionBase.hpp"
@@ -44,6 +46,7 @@
#include <vector>
using namespace web;
using namespace util::config;
namespace {
@@ -81,7 +84,12 @@ struct MockWsBase : public web::ConnectionBase {
};
struct WebRPCServerHandlerTest : util::prometheus::WithPrometheus, MockBackendTest, SyncAsioContextTest {
util::Config cfg;
util::config::ClioConfigDefinition cfg{
{"log_tag_style", ConfigValue{ConfigType::String}.defaultValue("none")},
{"api_version.default", ConfigValue{ConfigType::Integer}.defaultValue(rpc::API_VERSION_DEFAULT)},
{"api_version.min", ConfigValue{ConfigType::Integer}.defaultValue(rpc::API_VERSION_MIN)},
{"api_version.max", ConfigValue{ConfigType::Integer}.defaultValue(rpc::API_VERSION_MAX)}
};
std::shared_ptr<MockAsyncRPCEngine> rpcEngine = std::make_shared<MockAsyncRPCEngine>();
std::shared_ptr<MockETLService> etl = std::make_shared<MockETLService>();
std::shared_ptr<util::TagDecoratorFactory> tagFactory = std::make_shared<util::TagDecoratorFactory>(cfg);

View File

@@ -23,7 +23,12 @@
#include "util/TestHttpClient.hpp"
#include "util/TestWebSocketClient.hpp"
#include "util/TmpFile.hpp"
#include "util/config/Config.hpp"
#include "util/newconfig/Array.hpp"
#include "util/newconfig/ConfigDefinition.hpp"
#include "util/newconfig/ConfigFileJson.hpp"
#include "util/newconfig/ConfigValue.hpp"
#include "util/newconfig/Types.hpp"
#include "util/prometheus/Gauge.hpp"
#include "util/prometheus/Label.hpp"
#include "util/prometheus/Prometheus.hpp"
#include "web/AdminVerificationStrategy.hpp"
@@ -61,6 +66,7 @@
#include <vector>
using namespace util;
using namespace util::config;
using namespace web::impl;
using namespace web;
@@ -105,6 +111,30 @@ generateJSONDataOverload(std::string_view port)
));
}
inline ClioConfigDefinition
getParseServerConfig(boost::json::value val)
{
ConfigFileJson const jsonVal{val.as_object()};
auto config = ClioConfigDefinition{
{"server.ip", ConfigValue{ConfigType::String}},
{"server.port", ConfigValue{ConfigType::Integer}},
{"server.admin_password", ConfigValue{ConfigType::String}.optional()},
{"server.local_admin", ConfigValue{ConfigType::Boolean}.optional()},
{"server.ws_max_sending_queue_size", ConfigValue{ConfigType::Integer}.defaultValue(1500)},
{"log_tag_style", ConfigValue{ConfigType::String}.defaultValue("uint")},
{"dos_guard.max_fetches", ConfigValue{ConfigType::Integer}},
{"dos_guard.sweep_interval", ConfigValue{ConfigType::Integer}},
{"dos_guard.max_connections", ConfigValue{ConfigType::Integer}},
{"dos_guard.max_requests", ConfigValue{ConfigType::Integer}},
{"dos_guard.whitelist.[]", Array{ConfigValue{ConfigType::String}.optional()}},
{"ssl_key_file", ConfigValue{ConfigType::String}.optional()},
{"ssl_cert_file", ConfigValue{ConfigType::String}.optional()},
};
auto const errors = config.parse(jsonVal);
[&]() { ASSERT_FALSE(errors.has_value()); }();
return config;
};
struct WebServerTest : NoLoggerFixture {
~WebServerTest() override
{
@@ -131,12 +161,12 @@ struct WebServerTest : NoLoggerFixture {
// this ctx is for dos timer
boost::asio::io_context ctxSync;
std::string const port = std::to_string(tests::util::generateFreePort());
Config cfg{generateJSONWithDynamicPort(port)};
ClioConfigDefinition cfg{getParseServerConfig(generateJSONWithDynamicPort(port))};
dosguard::WhitelistHandler whitelistHandler{cfg};
dosguard::DOSGuard dosGuard{cfg, whitelistHandler};
dosguard::IntervalSweepHandler sweepHandler{cfg, ctxSync, dosGuard};
Config cfgOverload{generateJSONDataOverload(port)};
ClioConfigDefinition cfgOverload{getParseServerConfig(generateJSONDataOverload(port))};
dosguard::WhitelistHandler whitelistHandlerOverload{cfgOverload};
dosguard::DOSGuard dosGuardOverload{cfgOverload, whitelistHandlerOverload};
dosguard::IntervalSweepHandler sweepHandlerOverload{cfgOverload, ctxSync, dosGuardOverload};
@@ -184,7 +214,7 @@ namespace {
template <class Executor>
std::shared_ptr<web::HttpServer<Executor>>
makeServerSync(
util::Config const& config,
util::config::ClioConfigDefinition const& config,
boost::asio::io_context& ioc,
web::dosguard::DOSGuardInterface& dosGuard,
std::shared_ptr<Executor> const& handler
@@ -213,7 +243,7 @@ makeServerSync(
TEST_F(WebServerTest, Http)
{
auto e = std::make_shared<EchoExecutor>();
auto const e = std::make_shared<EchoExecutor>();
auto const server = makeServerSync(cfg, ctx, dosGuard, e);
auto const [status, res] = HttpSyncClient::post("localhost", port, R"({"Hello":1})");
EXPECT_EQ(res, R"({"Hello":1})");
@@ -233,7 +263,7 @@ TEST_F(WebServerTest, Ws)
TEST_F(WebServerTest, HttpInternalError)
{
auto e = std::make_shared<ExceptionExecutor>();
auto const e = std::make_shared<ExceptionExecutor>();
auto const server = makeServerSync(cfg, ctx, dosGuard, e);
auto const [status, res] = HttpSyncClient::post("localhost", port, R"({})");
EXPECT_EQ(
@@ -273,31 +303,31 @@ TEST_F(WebServerTest, WsInternalErrorNotJson)
TEST_F(WebServerTest, IncompleteSslConfig)
{
auto e = std::make_shared<EchoExecutor>();
auto const e = std::make_shared<EchoExecutor>();
auto jsonConfig = generateJSONWithDynamicPort(port);
jsonConfig.as_object()["ssl_key_file"] = sslKeyFile.path;
auto const server = makeServerSync(Config{jsonConfig}, ctx, dosGuard, e);
auto const server = makeServerSync(getParseServerConfig(jsonConfig), ctx, dosGuard, e);
EXPECT_EQ(server, nullptr);
}
TEST_F(WebServerTest, WrongSslConfig)
{
auto e = std::make_shared<EchoExecutor>();
auto const e = std::make_shared<EchoExecutor>();
auto jsonConfig = generateJSONWithDynamicPort(port);
jsonConfig.as_object()["ssl_key_file"] = sslKeyFile.path;
jsonConfig.as_object()["ssl_cert_file"] = "wrong_path";
auto const server = makeServerSync(Config{jsonConfig}, ctx, dosGuard, e);
auto const server = makeServerSync(getParseServerConfig(jsonConfig), ctx, dosGuard, e);
EXPECT_EQ(server, nullptr);
}
TEST_F(WebServerTest, Https)
{
auto e = std::make_shared<EchoExecutor>();
cfg = Config{addSslConfig(generateJSONWithDynamicPort(port))};
auto const e = std::make_shared<EchoExecutor>();
cfg = getParseServerConfig(addSslConfig(generateJSONWithDynamicPort(port)));
auto const server = makeServerSync(cfg, ctx, dosGuard, e);
auto const res = HttpsSyncClient::syncPost("localhost", port, R"({"Hello":1})");
EXPECT_EQ(res, R"({"Hello":1})");
@@ -306,7 +336,7 @@ TEST_F(WebServerTest, Https)
TEST_F(WebServerTest, Wss)
{
auto e = std::make_shared<EchoExecutor>();
cfg = Config{addSslConfig(generateJSONWithDynamicPort(port))};
cfg = getParseServerConfig(addSslConfig(generateJSONWithDynamicPort(port)));
auto server = makeServerSync(cfg, ctx, dosGuard, e);
WebServerSslSyncClient wsClient;
wsClient.connect("localhost", port);
@@ -317,7 +347,7 @@ TEST_F(WebServerTest, Wss)
TEST_F(WebServerTest, HttpRequestOverload)
{
auto e = std::make_shared<EchoExecutor>();
auto const e = std::make_shared<EchoExecutor>();
auto const server = makeServerSync(cfg, ctx, dosGuardOverload, e);
auto [status, res] = HttpSyncClient::post("localhost", port, R"({})");
EXPECT_EQ(res, "{}");
@@ -353,7 +383,7 @@ TEST_F(WebServerTest, WsRequestOverload)
TEST_F(WebServerTest, HttpPayloadOverload)
{
std::string const s100(100, 'a');
auto e = std::make_shared<EchoExecutor>();
auto const e = std::make_shared<EchoExecutor>();
auto server = makeServerSync(cfg, ctx, dosGuardOverload, e);
auto const [status, res] = HttpSyncClient::post("localhost", port, fmt::format(R"({{"payload":"{}"}})", s100));
EXPECT_EQ(
@@ -366,7 +396,7 @@ TEST_F(WebServerTest, HttpPayloadOverload)
TEST_F(WebServerTest, WsPayloadOverload)
{
std::string const s100(100, 'a');
auto e = std::make_shared<EchoExecutor>();
auto const e = std::make_shared<EchoExecutor>();
auto server = makeServerSync(cfg, ctx, dosGuardOverload, e);
WebSocketSyncClient wsClient;
wsClient.connect("localhost", port);
@@ -380,7 +410,7 @@ TEST_F(WebServerTest, WsPayloadOverload)
TEST_F(WebServerTest, WsTooManyConnection)
{
auto e = std::make_shared<EchoExecutor>();
auto const e = std::make_shared<EchoExecutor>();
auto server = makeServerSync(cfg, ctx, dosGuardOverload, e);
// max connection is 2, exception should happen when the third connection is made
WebSocketSyncClient wsClient1;
@@ -504,15 +534,38 @@ struct WebServerAdminTestParams {
std::string expectedResponse;
};
inline ClioConfigDefinition
getParseAdminServerConfig(boost::json::value val)
{
ConfigFileJson const jsonVal{val.as_object()};
auto config = ClioConfigDefinition{
{"server.ip", ConfigValue{ConfigType::String}},
{"server.port", ConfigValue{ConfigType::Integer}},
{"server.admin_password", ConfigValue{ConfigType::String}.optional()},
{"server.local_admin", ConfigValue{ConfigType::Boolean}.optional()},
{"server.processing_policy", ConfigValue{ConfigType::String}.defaultValue("parallel")},
{"server.parallel_requests_limit", ConfigValue{ConfigType::Integer}.optional()},
{"server.ws_max_sending_queue_size", ConfigValue{ConfigType::Integer}.defaultValue(1500)},
{"ssl_cert_file", ConfigValue{ConfigType::String}.optional()},
{"ssl_key_file", ConfigValue{ConfigType::String}.optional()},
{"prometheus.enabled", ConfigValue{ConfigType::Boolean}.defaultValue(true)},
{"prometheus.compress_reply", ConfigValue{ConfigType::Boolean}.defaultValue(true)},
{"log_tag_style", ConfigValue{ConfigType::String}.defaultValue("uint")}
};
auto const errors = config.parse(jsonVal);
[&]() { ASSERT_FALSE(errors.has_value()); }();
return config;
};
class WebServerAdminTest : public WebServerTest, public ::testing::WithParamInterface<WebServerAdminTestParams> {};
TEST_P(WebServerAdminTest, WsAdminCheck)
{
auto e = std::make_shared<AdminCheckExecutor>();
Config const serverConfig{boost::json::parse(GetParam().config)};
ClioConfigDefinition const serverConfig{getParseAdminServerConfig(boost::json::parse(GetParam().config))};
auto server = makeServerSync(serverConfig, ctx, dosGuardOverload, e);
WebSocketSyncClient wsClient;
uint32_t const webServerPort = serverConfig.value<uint32_t>("server.port");
uint32_t const webServerPort = serverConfig.get<uint32_t>("server.port");
wsClient.connect("localhost", std::to_string(webServerPort), GetParam().headers);
std::string const request = "Why hello";
auto const res = wsClient.syncPost(request);
@@ -522,11 +575,11 @@ TEST_P(WebServerAdminTest, WsAdminCheck)
TEST_P(WebServerAdminTest, HttpAdminCheck)
{
auto e = std::make_shared<AdminCheckExecutor>();
Config const serverConfig{boost::json::parse(GetParam().config)};
auto const e = std::make_shared<AdminCheckExecutor>();
ClioConfigDefinition const serverConfig{getParseAdminServerConfig(boost::json::parse(GetParam().config))};
auto server = makeServerSync(serverConfig, ctx, dosGuardOverload, e);
std::string const request = "Why hello";
uint32_t const webServerPort = serverConfig.value<uint32_t>("server.port");
uint32_t const webServerPort = serverConfig.get<uint32_t>("server.port");
auto const [status, res] =
HttpSyncClient::post("localhost", std::to_string(webServerPort), request, GetParam().headers);
@@ -616,8 +669,10 @@ TEST_F(WebServerTest, AdminErrorCfgTestBothAdminPasswordAndLocalAdminSet)
webServerPort
);
auto e = std::make_shared<AdminCheckExecutor>();
Config const serverConfig{boost::json::parse(JSONServerConfigWithBothAdminPasswordAndLocalAdmin)};
auto const e = std::make_shared<AdminCheckExecutor>();
ClioConfigDefinition const serverConfig{
getParseAdminServerConfig(boost::json::parse(JSONServerConfigWithBothAdminPasswordAndLocalAdmin))
};
EXPECT_THROW(web::make_HttpServer(serverConfig, ctx, dosGuardOverload, e), std::logic_error);
}
@@ -635,8 +690,10 @@ TEST_F(WebServerTest, AdminErrorCfgTestBothAdminPasswordAndLocalAdminFalse)
webServerPort
);
auto e = std::make_shared<AdminCheckExecutor>();
Config const serverConfig{boost::json::parse(JSONServerConfigWithNoAdminPasswordAndLocalAdminFalse)};
auto const e = std::make_shared<AdminCheckExecutor>();
ClioConfigDefinition const serverConfig{
getParseAdminServerConfig(boost::json::parse(JSONServerConfigWithNoAdminPasswordAndLocalAdminFalse))
};
EXPECT_THROW(web::make_HttpServer(serverConfig, ctx, dosGuardOverload, e), std::logic_error);
}
@@ -644,9 +701,11 @@ struct WebServerPrometheusTest : util::prometheus::WithPrometheus, WebServerTest
TEST_F(WebServerPrometheusTest, rejectedWithoutAdminPassword)
{
auto e = std::make_shared<EchoExecutor>();
auto const e = std::make_shared<EchoExecutor>();
uint32_t const webServerPort = tests::util::generateFreePort();
Config const serverConfig{boost::json::parse(JSONServerConfigWithAdminPassword(webServerPort))};
ClioConfigDefinition const serverConfig{
getParseAdminServerConfig(boost::json::parse(JSONServerConfigWithAdminPassword(webServerPort)))
};
auto server = makeServerSync(serverConfig, ctx, dosGuard, e);
auto const [status, res] = HttpSyncClient::get("localhost", std::to_string(webServerPort), "", "/metrics");
@@ -662,15 +721,18 @@ TEST_F(WebServerPrometheusTest, rejectedIfPrometheusIsDisabled)
"server":{{
"ip": "0.0.0.0",
"port": {},
"admin_password": "secret"
"admin_password": "secret",
"ws_max_sending_queue_size": 1500
}},
"prometheus": {{ "enabled": false }}
}})JSON",
webServerPort
);
auto e = std::make_shared<EchoExecutor>();
Config const serverConfig{boost::json::parse(JSONServerConfigWithDisabledPrometheus)};
auto const e = std::make_shared<EchoExecutor>();
ClioConfigDefinition const serverConfig{
getParseAdminServerConfig(boost::json::parse(JSONServerConfigWithDisabledPrometheus))
};
PrometheusService::init(serverConfig);
auto server = makeServerSync(serverConfig, ctx, dosGuard, e);
auto const [status, res] = HttpSyncClient::get(
@@ -692,8 +754,10 @@ TEST_F(WebServerPrometheusTest, validResponse)
uint32_t const webServerPort = tests::util::generateFreePort();
auto& testCounter = PrometheusService::counterInt("test_counter", util::prometheus::Labels());
++testCounter;
auto e = std::make_shared<EchoExecutor>();
Config const serverConfig{boost::json::parse(JSONServerConfigWithAdminPassword(webServerPort))};
auto const e = std::make_shared<EchoExecutor>();
ClioConfigDefinition const serverConfig{
getParseAdminServerConfig(boost::json::parse(JSONServerConfigWithAdminPassword(webServerPort)))
};
auto server = makeServerSync(serverConfig, ctx, dosGuard, e);
auto const [status, res] = HttpSyncClient::get(
"localhost",

View File

@@ -20,6 +20,9 @@
#include "util/LoggerFixtures.hpp"
#include "util/Taggable.hpp"
#include "util/config/Config.hpp"
#include "util/newconfig/ConfigDefinition.hpp"
#include "util/newconfig/ConfigValue.hpp"
#include "util/newconfig/Types.hpp"
#include "web/SubscriptionContext.hpp"
#include "web/SubscriptionContextInterface.hpp"
#include "web/interface/ConnectionBaseMock.hpp"
@@ -31,9 +34,12 @@
#include <string>
using namespace web;
using namespace util::config;
struct SubscriptionContextTests : NoLoggerFixture {
util::TagDecoratorFactory tagFactory_{util::Config{}};
util::TagDecoratorFactory tagFactory_{ClioConfigDefinition{
{"log_tag_style", ConfigValue{ConfigType::String}.defaultValue("uint")},
}};
ConnectionBaseStrictMockPtr connection_ =
std::make_shared<testing::StrictMock<ConnectionBaseMock>>(tagFactory_, "some ip");

View File

@@ -18,7 +18,10 @@
//==============================================================================
#include "util/LoggerFixtures.hpp"
#include "util/config/Config.hpp"
#include "util/newconfig/Array.hpp"
#include "util/newconfig/ConfigDefinition.hpp"
#include "util/newconfig/ConfigValue.hpp"
#include "util/newconfig/Types.hpp"
#include "web/dosguard/DOSGuard.hpp"
#include "web/dosguard/WhitelistHandlerInterface.hpp"
@@ -31,8 +34,8 @@
using namespace testing;
using namespace util;
using namespace std;
using namespace util::config;
using namespace web::dosguard;
namespace json = boost::json;
struct DOSGuardTest : NoLoggerFixture {
static constexpr auto JSONData = R"JSON(
@@ -54,7 +57,12 @@ struct DOSGuardTest : NoLoggerFixture {
MOCK_METHOD(bool, isWhiteListed, (std::string_view ip), (const));
};
Config cfg{json::parse(JSONData)};
ClioConfigDefinition cfg{
{{"dos_guard.max_fetches", ConfigValue{ConfigType::Integer}.defaultValue(100)},
{"dos_guard.max_connections", ConfigValue{ConfigType::Integer}.defaultValue(2)},
{"dos_guard.max_requests", ConfigValue{ConfigType::Integer}.defaultValue(3)},
{"dos_guard.whitelist", Array{ConfigValue{ConfigType::String}}}}
};
NiceMock<MockWhitelistHandler> whitelistHandler;
DOSGuard guard{cfg, whitelistHandler};
};

View File

@@ -18,7 +18,9 @@
//==============================================================================
#include "util/AsioContextTestFixture.hpp"
#include "util/config/Config.hpp"
#include "util/newconfig/ConfigDefinition.hpp"
#include "util/newconfig/ConfigValue.hpp"
#include "util/newconfig/Types.hpp"
#include "web/dosguard/DOSGuardMock.hpp"
#include "web/dosguard/IntervalSweepHandler.hpp"
@@ -29,6 +31,7 @@
#include <chrono>
using namespace web::dosguard;
using namespace util::config;
struct IntervalSweepHandlerTest : SyncAsioContextTest {
protected:
@@ -42,7 +45,7 @@ protected:
DOSGuardStrictMock guardMock;
util::Config cfg{boost::json::parse(JSONData)};
ClioConfigDefinition cfg{{"dos_guard.sweep_interval", ConfigValue{ConfigType::Integer}.defaultValue(0)}};
IntervalSweepHandler sweepHandler{cfg, ctx, guardMock};
};

View File

@@ -16,11 +16,18 @@
OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.
*/
//==============================================================================
#include "util/LoggerFixtures.hpp"
#include "util/config/Config.hpp"
#include "util/log/Logger.hpp"
#include "util/newconfig/Array.hpp"
#include "util/newconfig/ConfigDefinition.hpp"
#include "util/newconfig/ConfigFileJson.hpp"
#include "util/newconfig/ConfigValue.hpp"
#include "util/newconfig/Types.hpp"
#include "web/dosguard/WhitelistHandler.hpp"
#include <boost/json/parse.hpp>
#include <boost/json/value.hpp>
#include <gmock/gmock.h>
#include <gtest/gtest.h>
@@ -29,10 +36,21 @@
#include <vector>
using namespace util;
using namespace util::config;
using namespace web::dosguard;
struct WhitelistHandlerTest : NoLoggerFixture {};
inline ClioConfigDefinition
getParseWhitelistHandlerConfig(boost::json::value val)
{
ConfigFileJson const jsonVal{val.as_object()};
auto config = ClioConfigDefinition{{"dos_guard.whitelist.[]", Array{ConfigValue{ConfigType::String}}}};
auto const errors = config.parse(jsonVal);
[&]() { ASSERT_FALSE(errors.has_value()); }();
return config;
}
TEST_F(WhitelistHandlerTest, TestWhiteListIPV4)
{
struct MockResolver {
@@ -58,7 +76,7 @@ TEST_F(WhitelistHandlerTest, TestWhiteListIPV4)
.Times(3)
.WillRepeatedly([](auto hostname) -> std::vector<std::string> { return {std::string{hostname}}; });
Config const cfg{boost::json::parse(JSONDataIPV4)};
ClioConfigDefinition const cfg{getParseWhitelistHandlerConfig(boost::json::parse(JSONDataIPV4))};
WhitelistHandler const whitelistHandler{cfg, mockResolver};
EXPECT_TRUE(whitelistHandler.isWhiteListed("192.168.1.10"));
@@ -80,7 +98,7 @@ TEST_F(WhitelistHandlerTest, TestWhiteListResolvesHostname)
}
)JSON";
Config const cfg{boost::json::parse(JSONDataIPV4)};
ClioConfigDefinition const cfg{getParseWhitelistHandlerConfig(boost::json::parse(JSONDataIPV4))};
WhitelistHandler const whitelistHandler{cfg};
EXPECT_TRUE(whitelistHandler.isWhiteListed("127.0.0.1"));
@@ -102,7 +120,7 @@ TEST_F(WhitelistHandlerTest, TestWhiteListIPV6)
}
)JSON";
Config const cfg{boost::json::parse(JSONDataIPV6)};
ClioConfigDefinition const cfg{getParseWhitelistHandlerConfig(boost::json::parse(JSONDataIPV6))};
WhitelistHandler const whitelistHandler{cfg};
EXPECT_TRUE(whitelistHandler.isWhiteListed("2002:1dd8:85a7:0000:0000:8a6e:0000:1111"));

View File

@@ -21,7 +21,9 @@
#include "util/LoggerFixtures.hpp"
#include "util/NameGenerator.hpp"
#include "util/Taggable.hpp"
#include "util/config/Config.hpp"
#include "util/newconfig/ConfigDefinition.hpp"
#include "util/newconfig/ConfigValue.hpp"
#include "util/newconfig/Types.hpp"
#include "web/impl/ErrorHandling.hpp"
#include "web/interface/ConnectionBaseMock.hpp"
@@ -37,9 +39,12 @@
using namespace web::impl;
using namespace web;
using namespace util::config;
struct ErrorHandlingTests : NoLoggerFixture {
util::TagDecoratorFactory tagFactory_{util::Config{}};
util::TagDecoratorFactory tagFactory_{ClioConfigDefinition{
{"log_tag_style", ConfigValue{ConfigType::String}.defaultValue("uint")},
}};
std::string const clientIp_ = "some ip";
ConnectionBaseStrictMockPtr connection_ =
std::make_shared<testing::StrictMock<ConnectionBaseMock>>(tagFactory_, clientIp_);

View File

@@ -26,6 +26,9 @@
#include "util/MockRPCEngine.hpp"
#include "util/Taggable.hpp"
#include "util/config/Config.hpp"
#include "util/newconfig/ConfigDefinition.hpp"
#include "util/newconfig/ConfigValue.hpp"
#include "util/newconfig/Types.hpp"
#include "web/SubscriptionContextInterface.hpp"
#include "web/ng/MockConnection.hpp"
#include "web/ng/RPCServerHandler.hpp"
@@ -54,16 +57,23 @@
using namespace web::ng;
using testing::Return;
using testing::StrictMock;
using namespace util::config;
namespace http = boost::beast::http;
struct ng_RPCServerHandlerTest : util::prometheus::WithPrometheus, MockBackendTestStrict, SyncAsioContextTest {
ClioConfigDefinition config{ClioConfigDefinition{
{"log_tag_style", ConfigValue{ConfigType::String}.defaultValue("uint")},
{"api_version.min", ConfigValue{ConfigType::Integer}.defaultValue(1)},
{"api_version.max", ConfigValue{ConfigType::Integer}.defaultValue(2)},
{"api_version.default", ConfigValue{ConfigType::Integer}.defaultValue(1)}
}};
std::shared_ptr<testing::StrictMock<MockRPCEngine>> rpcEngine_ =
std::make_shared<testing::StrictMock<MockRPCEngine>>();
std::shared_ptr<StrictMock<MockETLService>> etl_ = std::make_shared<StrictMock<MockETLService>>();
RPCServerHandler<MockRPCEngine, MockETLService> rpcServerHandler_{util::Config{}, backend, rpcEngine_, etl_};
RPCServerHandler<MockRPCEngine, MockETLService> rpcServerHandler_{config, backend, rpcEngine_, etl_};
util::TagDecoratorFactory tagFactory_{util::Config{}};
util::TagDecoratorFactory tagFactory_{config};
StrictMockConnectionMetadata connectionMetadata_{"some ip", tagFactory_};
static Request

View File

@@ -20,6 +20,9 @@
#include "util/Taggable.hpp"
#include "util/build/Build.hpp"
#include "util/config/Config.hpp"
#include "util/newconfig/ConfigDefinition.hpp"
#include "util/newconfig/ConfigValue.hpp"
#include "util/newconfig/Types.hpp"
#include "web/ng/MockConnection.hpp"
#include "web/ng/Request.hpp"
#include "web/ng/Response.hpp"
@@ -41,6 +44,7 @@
using namespace web::ng;
namespace http = boost::beast::http;
using namespace util::config;
struct ResponseDeathTest : testing::Test {};
@@ -133,7 +137,9 @@ TEST_F(ResponseTest, asConstBufferJson)
TEST_F(ResponseTest, createFromStringAndConnection)
{
util::TagDecoratorFactory const tagDecoratorFactory{util::Config{}};
util::TagDecoratorFactory tagDecoratorFactory{
ClioConfigDefinition{{"log_tag_style", ConfigValue{ConfigType::String}.defaultValue("uint")}}
};
StrictMockConnection const connection{"some ip", boost::beast::flat_buffer{}, tagDecoratorFactory};
std::string const responseMessage = "response message";
@@ -150,7 +156,9 @@ TEST_F(ResponseTest, createFromStringAndConnection)
TEST_F(ResponseTest, createFromJsonAndConnection)
{
util::TagDecoratorFactory const tagDecoratorFactory{util::Config{}};
util::TagDecoratorFactory tagDecoratorFactory{
ClioConfigDefinition{{"log_tag_style", ConfigValue{ConfigType::String}.defaultValue("uint")}}
};
StrictMockConnection const connection{"some ip", boost::beast::flat_buffer{}, tagDecoratorFactory};
boost::json::object const responseMessage{{"key", "value"}};

View File

@@ -24,7 +24,11 @@
#include "util/Taggable.hpp"
#include "util/TestHttpClient.hpp"
#include "util/TestWebSocketClient.hpp"
#include "util/config/Config.hpp"
#include "util/newconfig/ConfigConstraints.hpp"
#include "util/newconfig/ConfigDefinition.hpp"
#include "util/newconfig/ConfigFileJson.hpp"
#include "util/newconfig/ConfigValue.hpp"
#include "util/newconfig/Types.hpp"
#include "web/SubscriptionContextInterface.hpp"
#include "web/ng/Connection.hpp"
#include "web/ng/ProcessingPolicy.hpp"
@@ -53,6 +57,7 @@
#include <string>
using namespace web::ng;
using namespace util::config;
namespace http = boost::beast::http;
@@ -68,7 +73,22 @@ struct MakeServerTest : NoLoggerFixture, testing::WithParamInterface<MakeServerT
TEST_P(MakeServerTest, Make)
{
util::Config const config{boost::json::parse(GetParam().configJson)};
ConfigFileJson const json{boost::json::parse(GetParam().configJson).as_object()};
util::config::ClioConfigDefinition config{
{"server.ip", ConfigValue{ConfigType::String}.optional()},
{"server.port", ConfigValue{ConfigType::Integer}.optional()},
{"server.processing_policy", ConfigValue{ConfigType::String}.defaultValue("parallel")},
{"server.parallel_requests_limit", ConfigValue{ConfigType::Integer}.optional()},
{"server.ws_max_sending_queue_size", ConfigValue{ConfigType::Integer}.defaultValue(1500)},
{"log_tag_style", ConfigValue{ConfigType::String}.defaultValue("uint")},
{"ssl_cert_file", ConfigValue{ConfigType::String}.optional()},
{"ssl_key_file", ConfigValue{ConfigType::String}.optional()}
};
auto const errors = config.parse(json);
ASSERT_TRUE(!errors.has_value());
auto const expectedServer =
make_Server(config, [](auto&&) -> std::expected<void, Response> { return {}; }, [](auto&&) {}, ioContext_);
EXPECT_EQ(expectedServer.has_value(), GetParam().expectSuccess);
@@ -78,15 +98,6 @@ INSTANTIATE_TEST_CASE_P(
MakeServerTests,
MakeServerTest,
testing::Values(
MakeServerTestBundle{
"NoIp",
R"json(
{
"server": {"port": 12345}
}
)json",
false
},
MakeServerTestBundle{
"BadEndpoint",
R"json(
@@ -96,21 +107,12 @@ INSTANTIATE_TEST_CASE_P(
)json",
false
},
MakeServerTestBundle{
"PortMissing",
R"json(
{
"server": {"ip": "127.0.0.1"}
}
)json",
false
},
MakeServerTestBundle{
"BadSslConfig",
R"json(
{
"server": {"ip": "127.0.0.1", "port": 12345},
"ssl_cert_file": "somг_file"
"ssl_cert_file": "some_file"
}
)json",
false
@@ -157,8 +159,17 @@ struct ServerTest : SyncAsioContextTest {
uint32_t const serverPort_ = tests::util::generateFreePort();
util::Config const config_{
boost::json::object{{"server", boost::json::object{{"ip", "127.0.0.1"}, {"port", serverPort_}}}}
ClioConfigDefinition const config_{
{"server.ip", ConfigValue{ConfigType::String}.defaultValue("127.0.0.1").withConstraint(validateIP)},
{"server.port", ConfigValue{ConfigType::Integer}.defaultValue(serverPort_).withConstraint(validatePort)},
{"server.processing_policy", ConfigValue{ConfigType::String}.defaultValue("parallel")},
{"server.admin_password", ConfigValue{ConfigType::String}.optional()},
{"server.local_admin", ConfigValue{ConfigType::Boolean}.optional()},
{"server.parallel_requests_limit", ConfigValue{ConfigType::Integer}.optional()},
{"server.ws_max_sending_queue_size", ConfigValue{ConfigType::Integer}.defaultValue(1500)},
{"log_tag_style", ConfigValue{ConfigType::String}.defaultValue("uint")},
{"ssl_key_file", ConfigValue{ConfigType::String}.optional()},
{"ssl_cert_file", ConfigValue{ConfigType::String}.optional()}
};
Server::OnConnectCheck emptyOnConnectCheck_ = [](auto&&) -> std::expected<void, Response> { return {}; };
@@ -183,7 +194,9 @@ struct ServerTest : SyncAsioContextTest {
TEST_F(ServerTest, BadEndpoint)
{
boost::asio::ip::tcp::endpoint const endpoint{boost::asio::ip::address_v4::from_string("1.2.3.4"), 0};
util::TagDecoratorFactory const tagDecoratorFactory{util::Config{boost::json::value{}}};
util::TagDecoratorFactory const tagDecoratorFactory{
ClioConfigDefinition{{"log_tag_style", ConfigValue{ConfigType::String}.defaultValue("uint")}}
};
Server server{
ctx,
endpoint,
@@ -241,7 +254,9 @@ TEST_F(ServerHttpTest, OnConnectCheck)
{
auto const serverPort = tests::util::generateFreePort();
boost::asio::ip::tcp::endpoint const endpoint{boost::asio::ip::address_v4::from_string("0.0.0.0"), serverPort};
util::TagDecoratorFactory const tagDecoratorFactory{util::Config{boost::json::value{}}};
util::TagDecoratorFactory const tagDecoratorFactory{
ClioConfigDefinition{{"log_tag_style", ConfigValue{ConfigType::String}.defaultValue("uint")}}
};
testing::StrictMock<testing::MockFunction<std::expected<void, Response>(Connection const&)>> onConnectCheck;
@@ -298,7 +313,9 @@ TEST_F(ServerHttpTest, OnConnectCheckFailed)
{
auto const serverPort = tests::util::generateFreePort();
boost::asio::ip::tcp::endpoint const endpoint{boost::asio::ip::address_v4::from_string("0.0.0.0"), serverPort};
util::TagDecoratorFactory const tagDecoratorFactory{util::Config{boost::json::value{}}};
util::TagDecoratorFactory const tagDecoratorFactory{
ClioConfigDefinition{{"log_tag_style", ConfigValue{ConfigType::String}.defaultValue("uint")}}
};
testing::StrictMock<testing::MockFunction<std::expected<void, Response>(Connection const&)>> onConnectCheck;
@@ -354,7 +371,9 @@ TEST_F(ServerHttpTest, OnDisconnectHook)
{
auto const serverPort = tests::util::generateFreePort();
boost::asio::ip::tcp::endpoint const endpoint{boost::asio::ip::address_v4::from_string("0.0.0.0"), serverPort};
util::TagDecoratorFactory const tagDecoratorFactory{util::Config{boost::json::value{}}};
util::TagDecoratorFactory const tagDecoratorFactory{
ClioConfigDefinition{{"log_tag_style", ConfigValue{ConfigType::String}.defaultValue("uint")}}
};
testing::StrictMock<testing::MockFunction<void(Connection const&)>> OnDisconnectHookMock;

View File

@@ -19,7 +19,9 @@
#include "util/AsioContextTestFixture.hpp"
#include "util/Taggable.hpp"
#include "util/config/Config.hpp"
#include "util/newconfig/ConfigDefinition.hpp"
#include "util/newconfig/ConfigValue.hpp"
#include "util/newconfig/Types.hpp"
#include "web/SubscriptionContextInterface.hpp"
#include "web/ng/Connection.hpp"
#include "web/ng/Error.hpp"
@@ -41,11 +43,14 @@
#include <string>
using namespace web::ng;
using namespace util::config;
struct ng_SubscriptionContextTests : SyncAsioContextTest {
util::TagDecoratorFactory tagFactory_{util::Config{}};
util::TagDecoratorFactory tagFactory_{ClioConfigDefinition{
{"log_tag_style", ConfigValue{ConfigType::String}.defaultValue("uint")},
}};
MockWsConnectionImpl connection_{"some ip", boost::beast::flat_buffer{}, tagFactory_};
testing::StrictMock<testing::MockFunction<bool(Error const&, Connection const&)>> errorHandler_;
testing::StrictMock<testing::MockFunction<bool(web::ng::Error const&, Connection const&)>> errorHandler_;
SubscriptionContext
makeSubscriptionContext(boost::asio::yield_context yield, std::optional<size_t> maxSendQueueSize = std::nullopt)

View File

@@ -20,7 +20,9 @@
#include "util/AsioContextTestFixture.hpp"
#include "util/Taggable.hpp"
#include "util/UnsupportedType.hpp"
#include "util/config/Config.hpp"
#include "util/newconfig/ConfigDefinition.hpp"
#include "util/newconfig/ConfigValue.hpp"
#include "util/newconfig/Types.hpp"
#include "web/SubscriptionContextInterface.hpp"
#include "web/ng/Connection.hpp"
#include "web/ng/Error.hpp"
@@ -57,6 +59,7 @@
using namespace web::ng::impl;
using namespace web::ng;
using namespace util;
using testing::Return;
namespace beast = boost::beast;
namespace http = boost::beast::http;
@@ -64,7 +67,9 @@ namespace websocket = boost::beast::websocket;
struct ConnectionHandlerTest : SyncAsioContextTest {
ConnectionHandlerTest(ProcessingPolicy policy, std::optional<size_t> maxParallelConnections)
: tagFactory_{util::Config{}}
: tagFactory_{util::config::ClioConfigDefinition{
{"log_tag_style", config::ConfigValue{config::ConfigType::String}.defaultValue("uint")}
}}
, connectionHandler_{
policy,
maxParallelConnections,
@@ -104,7 +109,9 @@ struct ConnectionHandlerTest : SyncAsioContextTest {
util::TagDecoratorFactory tagFactory_;
ConnectionHandler connectionHandler_;
util::TagDecoratorFactory tagDecoratorFactory_{util::Config(boost::json::object{{"log_tag_style", "uint"}})};
util::TagDecoratorFactory tagDecoratorFactory_{config::ClioConfigDefinition{
{"log_tag_style", config::ConfigValue{config::ConfigType::String}.defaultValue("uint")}
}};
StrictMockHttpConnectionPtr mockHttpConnection_ =
std::make_unique<StrictMockHttpConnection>("1.2.3.4", beast::flat_buffer{}, tagDecoratorFactory_);
StrictMockWsConnectionPtr mockWsConnection_ =

Some files were not shown because too many files have changed in this diff Show More