mirror of
https://github.com/Xahau/xahaud.git
synced 2025-12-05 08:47:53 +00:00
Implement node-to-shard RPC control
This commit is contained in:
@@ -598,7 +598,7 @@ target_sources (rippled PRIVATE
|
|||||||
src/ripple/rpc/handlers/LogLevel.cpp
|
src/ripple/rpc/handlers/LogLevel.cpp
|
||||||
src/ripple/rpc/handlers/LogRotate.cpp
|
src/ripple/rpc/handlers/LogRotate.cpp
|
||||||
src/ripple/rpc/handlers/Manifest.cpp
|
src/ripple/rpc/handlers/Manifest.cpp
|
||||||
src/ripple/rpc/handlers/NodeToShardStatus.cpp
|
src/ripple/rpc/handlers/NodeToShard.cpp
|
||||||
src/ripple/rpc/handlers/NoRippleCheck.cpp
|
src/ripple/rpc/handlers/NoRippleCheck.cpp
|
||||||
src/ripple/rpc/handlers/OwnerInfo.cpp
|
src/ripple/rpc/handlers/OwnerInfo.cpp
|
||||||
src/ripple/rpc/handlers/PathFind.cpp
|
src/ripple/rpc/handlers/PathFind.cpp
|
||||||
@@ -934,6 +934,7 @@ if (tests)
|
|||||||
src/test/rpc/LedgerRPC_test.cpp
|
src/test/rpc/LedgerRPC_test.cpp
|
||||||
src/test/rpc/LedgerRequestRPC_test.cpp
|
src/test/rpc/LedgerRequestRPC_test.cpp
|
||||||
src/test/rpc/ManifestRPC_test.cpp
|
src/test/rpc/ManifestRPC_test.cpp
|
||||||
|
src/test/rpc/NodeToShardRPC_test.cpp
|
||||||
src/test/rpc/NoRippleCheck_test.cpp
|
src/test/rpc/NoRippleCheck_test.cpp
|
||||||
src/test/rpc/NoRipple_test.cpp
|
src/test/rpc/NoRipple_test.cpp
|
||||||
src/test/rpc/OwnerInfo_test.cpp
|
src/test/rpc/OwnerInfo_test.cpp
|
||||||
|
|||||||
@@ -159,7 +159,7 @@ printHelp(const po::options_description& desc)
|
|||||||
" ledger_request <ledger>\n"
|
" ledger_request <ledger>\n"
|
||||||
" log_level [[<partition>] <severity>]\n"
|
" log_level [[<partition>] <severity>]\n"
|
||||||
" logrotate\n"
|
" logrotate\n"
|
||||||
" nodetoshard_status\n"
|
" node_to_shard [status|start|stop]\n"
|
||||||
" peers\n"
|
" peers\n"
|
||||||
" ping\n"
|
" ping\n"
|
||||||
" random\n"
|
" random\n"
|
||||||
|
|||||||
@@ -29,6 +29,9 @@ NodeStoreScheduler::NodeStoreScheduler(JobQueue& jobQueue) : jobQueue_(jobQueue)
|
|||||||
void
|
void
|
||||||
NodeStoreScheduler::scheduleTask(NodeStore::Task& task)
|
NodeStoreScheduler::scheduleTask(NodeStore::Task& task)
|
||||||
{
|
{
|
||||||
|
if (jobQueue_.isStopped())
|
||||||
|
return;
|
||||||
|
|
||||||
if (!jobQueue_.addJob(jtWRITE, "NodeObject::store", [&task](Job&) {
|
if (!jobQueue_.addJob(jtWRITE, "NodeObject::store", [&task](Job&) {
|
||||||
task.performScheduledTask();
|
task.performScheduledTask();
|
||||||
}))
|
}))
|
||||||
@@ -42,6 +45,9 @@ NodeStoreScheduler::scheduleTask(NodeStore::Task& task)
|
|||||||
void
|
void
|
||||||
NodeStoreScheduler::onFetch(NodeStore::FetchReport const& report)
|
NodeStoreScheduler::onFetch(NodeStore::FetchReport const& report)
|
||||||
{
|
{
|
||||||
|
if (jobQueue_.isStopped())
|
||||||
|
return;
|
||||||
|
|
||||||
jobQueue_.addLoadEvents(
|
jobQueue_.addLoadEvents(
|
||||||
report.fetchType == NodeStore::FetchType::async ? jtNS_ASYNC_READ
|
report.fetchType == NodeStore::FetchType::async ? jtNS_ASYNC_READ
|
||||||
: jtNS_SYNC_READ,
|
: jtNS_SYNC_READ,
|
||||||
@@ -52,6 +58,9 @@ NodeStoreScheduler::onFetch(NodeStore::FetchReport const& report)
|
|||||||
void
|
void
|
||||||
NodeStoreScheduler::onBatchWrite(NodeStore::BatchWriteReport const& report)
|
NodeStoreScheduler::onBatchWrite(NodeStore::BatchWriteReport const& report)
|
||||||
{
|
{
|
||||||
|
if (jobQueue_.isStopped())
|
||||||
|
return;
|
||||||
|
|
||||||
jobQueue_.addLoadEvents(jtNS_WRITE, report.writeCount, report.elapsed);
|
jobQueue_.addLoadEvents(jtNS_WRITE, report.writeCount, report.elapsed);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|||||||
@@ -936,6 +936,15 @@ private:
|
|||||||
return jvRequest;
|
return jvRequest;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
Json::Value
|
||||||
|
parseNodeToShard(Json::Value const& jvParams)
|
||||||
|
{
|
||||||
|
Json::Value jvRequest;
|
||||||
|
jvRequest[jss::action] = jvParams[0u].asString();
|
||||||
|
|
||||||
|
return jvRequest;
|
||||||
|
}
|
||||||
|
|
||||||
// peer_reservations_add <public_key> [<name>]
|
// peer_reservations_add <public_key> [<name>]
|
||||||
Json::Value
|
Json::Value
|
||||||
parsePeerReservationsAdd(Json::Value const& jvParams)
|
parsePeerReservationsAdd(Json::Value const& jvParams)
|
||||||
@@ -1257,7 +1266,7 @@ public:
|
|||||||
{"log_level", &RPCParser::parseLogLevel, 0, 2},
|
{"log_level", &RPCParser::parseLogLevel, 0, 2},
|
||||||
{"logrotate", &RPCParser::parseAsIs, 0, 0},
|
{"logrotate", &RPCParser::parseAsIs, 0, 0},
|
||||||
{"manifest", &RPCParser::parseManifest, 1, 1},
|
{"manifest", &RPCParser::parseManifest, 1, 1},
|
||||||
{"nodetoshard_status", &RPCParser::parseAsIs, 0, 0},
|
{"node_to_shard", &RPCParser::parseNodeToShard, 1, 1},
|
||||||
{"owner_info", &RPCParser::parseAccountItems, 1, 3},
|
{"owner_info", &RPCParser::parseAccountItems, 1, 3},
|
||||||
{"peers", &RPCParser::parseAsIs, 0, 0},
|
{"peers", &RPCParser::parseAsIs, 0, 0},
|
||||||
{"ping", &RPCParser::parseAsIs, 0, 0},
|
{"ping", &RPCParser::parseAsIs, 0, 0},
|
||||||
|
|||||||
@@ -252,9 +252,25 @@ public:
|
|||||||
[[nodiscard]] virtual boost::filesystem::path const&
|
[[nodiscard]] virtual boost::filesystem::path const&
|
||||||
getRootDir() const = 0;
|
getRootDir() const = 0;
|
||||||
|
|
||||||
|
/** Returns a JSON object detailing the status of an ongoing
|
||||||
|
database import if one is running, otherwise an error
|
||||||
|
object.
|
||||||
|
*/
|
||||||
virtual Json::Value
|
virtual Json::Value
|
||||||
getDatabaseImportStatus() const = 0;
|
getDatabaseImportStatus() const = 0;
|
||||||
|
|
||||||
|
/** Initiates a NodeStore to ShardStore import and returns
|
||||||
|
the result in a JSON object.
|
||||||
|
*/
|
||||||
|
virtual Json::Value
|
||||||
|
startNodeToShard() = 0;
|
||||||
|
|
||||||
|
/** Terminates a NodeStore to ShardStore import and returns
|
||||||
|
the result in a JSON object.
|
||||||
|
*/
|
||||||
|
virtual Json::Value
|
||||||
|
stopNodeToShard() = 0;
|
||||||
|
|
||||||
/** Returns the first ledger sequence of the shard currently being imported
|
/** Returns the first ledger sequence of the shard currently being imported
|
||||||
from the NodeStore
|
from the NodeStore
|
||||||
|
|
||||||
|
|||||||
@@ -722,6 +722,8 @@ DatabaseShardImp::stop()
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
std::unique_lock lock(mutex_);
|
||||||
|
|
||||||
// Notify the shard being imported
|
// Notify the shard being imported
|
||||||
// from the node store to stop
|
// from the node store to stop
|
||||||
if (databaseImportStatus_)
|
if (databaseImportStatus_)
|
||||||
@@ -735,7 +737,28 @@ DatabaseShardImp::stop()
|
|||||||
// Wait for the node store import thread
|
// Wait for the node store import thread
|
||||||
// if necessary
|
// if necessary
|
||||||
if (databaseImporter_.joinable())
|
if (databaseImporter_.joinable())
|
||||||
databaseImporter_.join();
|
{
|
||||||
|
// Tells the import function to halt
|
||||||
|
haltDatabaseImport_ = true;
|
||||||
|
|
||||||
|
// Wait for the function to exit
|
||||||
|
while (databaseImportStatus_)
|
||||||
|
{
|
||||||
|
// Unlock just in case the import
|
||||||
|
// function is waiting on the mutex
|
||||||
|
lock.unlock();
|
||||||
|
|
||||||
|
std::this_thread::sleep_for(std::chrono::milliseconds(100));
|
||||||
|
lock.lock();
|
||||||
|
}
|
||||||
|
|
||||||
|
// Calling join while holding the mutex_ without
|
||||||
|
// first making sure that doImportDatabase has
|
||||||
|
// exited could lead to deadlock via the mutex
|
||||||
|
// acquisition that occurs in that function
|
||||||
|
if (databaseImporter_.joinable())
|
||||||
|
databaseImporter_.join();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
void
|
void
|
||||||
@@ -754,15 +777,19 @@ DatabaseShardImp::importDatabase(Database& source)
|
|||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
// Run the lengthy node store import process in the background
|
startDatabaseImportThread(lock);
|
||||||
// on a dedicated thread.
|
|
||||||
databaseImporter_ = std::thread([this] { doImportDatabase(); });
|
|
||||||
}
|
}
|
||||||
|
|
||||||
void
|
void
|
||||||
DatabaseShardImp::doImportDatabase()
|
DatabaseShardImp::doImportDatabase()
|
||||||
{
|
{
|
||||||
if (isStopping())
|
auto shouldHalt = [this] {
|
||||||
|
bool expected = true;
|
||||||
|
return haltDatabaseImport_.compare_exchange_strong(expected, false) ||
|
||||||
|
isStopping();
|
||||||
|
};
|
||||||
|
|
||||||
|
if (shouldHalt())
|
||||||
return;
|
return;
|
||||||
|
|
||||||
auto loadLedger =
|
auto loadLedger =
|
||||||
@@ -848,7 +875,7 @@ DatabaseShardImp::doImportDatabase()
|
|||||||
for (std::uint32_t shardIndex = earliestIndex; shardIndex <= latestIndex;
|
for (std::uint32_t shardIndex = earliestIndex; shardIndex <= latestIndex;
|
||||||
++shardIndex)
|
++shardIndex)
|
||||||
{
|
{
|
||||||
if (isStopping())
|
if (shouldHalt())
|
||||||
return;
|
return;
|
||||||
|
|
||||||
auto const pathDesignation = [this, shardIndex] {
|
auto const pathDesignation = [this, shardIndex] {
|
||||||
@@ -920,7 +947,7 @@ DatabaseShardImp::doImportDatabase()
|
|||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
|
|
||||||
if (isStopping())
|
if (shouldHalt())
|
||||||
return;
|
return;
|
||||||
|
|
||||||
bool const needsHistoricalPath =
|
bool const needsHistoricalPath =
|
||||||
@@ -938,7 +965,7 @@ DatabaseShardImp::doImportDatabase()
|
|||||||
{
|
{
|
||||||
std::lock_guard lock(mutex_);
|
std::lock_guard lock(mutex_);
|
||||||
|
|
||||||
if (isStopping())
|
if (shouldHalt())
|
||||||
return;
|
return;
|
||||||
|
|
||||||
databaseImportStatus_->currentIndex = shardIndex;
|
databaseImportStatus_->currentIndex = shardIndex;
|
||||||
@@ -967,7 +994,7 @@ DatabaseShardImp::doImportDatabase()
|
|||||||
|
|
||||||
while (auto const ledgerSeq = shard->prepare())
|
while (auto const ledgerSeq = shard->prepare())
|
||||||
{
|
{
|
||||||
if (isStopping())
|
if (shouldHalt())
|
||||||
return;
|
return;
|
||||||
|
|
||||||
// Not const so it may be moved later
|
// Not const so it may be moved later
|
||||||
@@ -989,7 +1016,7 @@ DatabaseShardImp::doImportDatabase()
|
|||||||
recentStored = std::move(ledger);
|
recentStored = std::move(ledger);
|
||||||
}
|
}
|
||||||
|
|
||||||
if (isStopping())
|
if (shouldHalt())
|
||||||
return;
|
return;
|
||||||
|
|
||||||
using namespace boost::filesystem;
|
using namespace boost::filesystem;
|
||||||
@@ -1044,17 +1071,14 @@ DatabaseShardImp::doImportDatabase()
|
|||||||
{
|
{
|
||||||
JLOG(j_.error()) << "shard " << shardIndex
|
JLOG(j_.error()) << "shard " << shardIndex
|
||||||
<< " failed to import from the NodeStore";
|
<< " failed to import from the NodeStore";
|
||||||
shard->removeOnDestroy();
|
|
||||||
|
if (shard)
|
||||||
|
shard->removeOnDestroy();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
{
|
if (shouldHalt())
|
||||||
std::lock_guard lock(mutex_);
|
return;
|
||||||
if (isStopping())
|
|
||||||
return;
|
|
||||||
|
|
||||||
databaseImportStatus_.reset();
|
|
||||||
}
|
|
||||||
|
|
||||||
updateFileStats();
|
updateFileStats();
|
||||||
}
|
}
|
||||||
@@ -1200,10 +1224,10 @@ DatabaseShardImp::sweep()
|
|||||||
Json::Value
|
Json::Value
|
||||||
DatabaseShardImp::getDatabaseImportStatus() const
|
DatabaseShardImp::getDatabaseImportStatus() const
|
||||||
{
|
{
|
||||||
Json::Value ret(Json::objectValue);
|
|
||||||
|
|
||||||
if (std::lock_guard lock(mutex_); databaseImportStatus_)
|
if (std::lock_guard lock(mutex_); databaseImportStatus_)
|
||||||
{
|
{
|
||||||
|
Json::Value ret(Json::objectValue);
|
||||||
|
|
||||||
ret[jss::firstShardIndex] = databaseImportStatus_->earliestIndex;
|
ret[jss::firstShardIndex] = databaseImportStatus_->earliestIndex;
|
||||||
ret[jss::lastShardIndex] = databaseImportStatus_->latestIndex;
|
ret[jss::lastShardIndex] = databaseImportStatus_->latestIndex;
|
||||||
ret[jss::currentShardIndex] = databaseImportStatus_->currentIndex;
|
ret[jss::currentShardIndex] = databaseImportStatus_->currentIndex;
|
||||||
@@ -1216,11 +1240,59 @@ DatabaseShardImp::getDatabaseImportStatus() const
|
|||||||
currentShard[jss::storedSeqs] = shard->getStoredSeqs();
|
currentShard[jss::storedSeqs] = shard->getStoredSeqs();
|
||||||
|
|
||||||
ret[jss::currentShard] = currentShard;
|
ret[jss::currentShard] = currentShard;
|
||||||
}
|
|
||||||
else
|
|
||||||
ret = "Database import not running";
|
|
||||||
|
|
||||||
return ret;
|
if (haltDatabaseImport_)
|
||||||
|
ret[jss::message] = "Database import halt initiated...";
|
||||||
|
|
||||||
|
return ret;
|
||||||
|
}
|
||||||
|
|
||||||
|
return RPC::make_error(rpcINTERNAL, "Database import not running");
|
||||||
|
}
|
||||||
|
|
||||||
|
Json::Value
|
||||||
|
DatabaseShardImp::startNodeToShard()
|
||||||
|
{
|
||||||
|
std::lock_guard lock(mutex_);
|
||||||
|
|
||||||
|
if (!init_)
|
||||||
|
return RPC::make_error(rpcINTERNAL, "Shard store not initialized");
|
||||||
|
|
||||||
|
if (databaseImporter_.joinable())
|
||||||
|
return RPC::make_error(
|
||||||
|
rpcINTERNAL, "Database import already in progress");
|
||||||
|
|
||||||
|
if (isStopping())
|
||||||
|
return RPC::make_error(rpcINTERNAL, "Node is shutting down");
|
||||||
|
|
||||||
|
startDatabaseImportThread(lock);
|
||||||
|
|
||||||
|
Json::Value result(Json::objectValue);
|
||||||
|
result[jss::message] = "Database import initiated...";
|
||||||
|
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
|
||||||
|
Json::Value
|
||||||
|
DatabaseShardImp::stopNodeToShard()
|
||||||
|
{
|
||||||
|
std::lock_guard lock(mutex_);
|
||||||
|
|
||||||
|
if (!init_)
|
||||||
|
return RPC::make_error(rpcINTERNAL, "Shard store not initialized");
|
||||||
|
|
||||||
|
if (!databaseImporter_.joinable())
|
||||||
|
return RPC::make_error(rpcINTERNAL, "Database import not running");
|
||||||
|
|
||||||
|
if (isStopping())
|
||||||
|
return RPC::make_error(rpcINTERNAL, "Node is shutting down");
|
||||||
|
|
||||||
|
haltDatabaseImport_ = true;
|
||||||
|
|
||||||
|
Json::Value result(Json::objectValue);
|
||||||
|
result[jss::message] = "Database import halt initiated...";
|
||||||
|
|
||||||
|
return result;
|
||||||
}
|
}
|
||||||
|
|
||||||
std::optional<std::uint32_t>
|
std::optional<std::uint32_t>
|
||||||
@@ -2131,6 +2203,27 @@ DatabaseShardImp::updatePeers(std::lock_guard<std::mutex> const& lock) const
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
void
|
||||||
|
DatabaseShardImp::startDatabaseImportThread(std::lock_guard<std::mutex> const&)
|
||||||
|
{
|
||||||
|
// Run the lengthy node store import process in the background
|
||||||
|
// on a dedicated thread.
|
||||||
|
databaseImporter_ = std::thread([this] {
|
||||||
|
doImportDatabase();
|
||||||
|
|
||||||
|
std::lock_guard lock(mutex_);
|
||||||
|
|
||||||
|
// Make sure to clear this in case the import
|
||||||
|
// exited early.
|
||||||
|
databaseImportStatus_.reset();
|
||||||
|
|
||||||
|
// Detach the thread so subsequent attempts
|
||||||
|
// to start the import won't get held up by
|
||||||
|
// the old thread of execution
|
||||||
|
databaseImporter_.detach();
|
||||||
|
});
|
||||||
|
}
|
||||||
|
|
||||||
//------------------------------------------------------------------------------
|
//------------------------------------------------------------------------------
|
||||||
|
|
||||||
std::unique_ptr<DatabaseShard>
|
std::unique_ptr<DatabaseShard>
|
||||||
|
|||||||
@@ -135,6 +135,12 @@ public:
|
|||||||
Json::Value
|
Json::Value
|
||||||
getDatabaseImportStatus() const override;
|
getDatabaseImportStatus() const override;
|
||||||
|
|
||||||
|
Json::Value
|
||||||
|
startNodeToShard() override;
|
||||||
|
|
||||||
|
Json::Value
|
||||||
|
stopNodeToShard() override;
|
||||||
|
|
||||||
std::optional<std::uint32_t>
|
std::optional<std::uint32_t>
|
||||||
getDatabaseImportSequence() const override;
|
getDatabaseImportSequence() const override;
|
||||||
|
|
||||||
@@ -285,6 +291,9 @@ private:
|
|||||||
// Thread for running node store import
|
// Thread for running node store import
|
||||||
std::thread databaseImporter_;
|
std::thread databaseImporter_;
|
||||||
|
|
||||||
|
// Indicates whether the import should stop
|
||||||
|
std::atomic_bool haltDatabaseImport_{false};
|
||||||
|
|
||||||
// Initialize settings from the configuration file
|
// Initialize settings from the configuration file
|
||||||
// Lock must be held
|
// Lock must be held
|
||||||
bool
|
bool
|
||||||
@@ -407,6 +416,10 @@ private:
|
|||||||
// Update peers with the status of every complete and incomplete shard
|
// Update peers with the status of every complete and incomplete shard
|
||||||
void
|
void
|
||||||
updatePeers(std::lock_guard<std::mutex> const& lock) const;
|
updatePeers(std::lock_guard<std::mutex> const& lock) const;
|
||||||
|
|
||||||
|
// Start the node store import process
|
||||||
|
void
|
||||||
|
startDatabaseImportThread(std::lock_guard<std::mutex> const&);
|
||||||
};
|
};
|
||||||
|
|
||||||
} // namespace NodeStore
|
} // namespace NodeStore
|
||||||
|
|||||||
@@ -89,7 +89,7 @@ doLogRotate(RPC::JsonContext&);
|
|||||||
Json::Value
|
Json::Value
|
||||||
doManifest(RPC::JsonContext&);
|
doManifest(RPC::JsonContext&);
|
||||||
Json::Value
|
Json::Value
|
||||||
doNodeToShardStatus(RPC::JsonContext&);
|
doNodeToShard(RPC::JsonContext&);
|
||||||
Json::Value
|
Json::Value
|
||||||
doNoRippleCheck(RPC::JsonContext&);
|
doNoRippleCheck(RPC::JsonContext&);
|
||||||
Json::Value
|
Json::Value
|
||||||
|
|||||||
86
src/ripple/rpc/handlers/NodeToShard.cpp
Normal file
86
src/ripple/rpc/handlers/NodeToShard.cpp
Normal file
@@ -0,0 +1,86 @@
|
|||||||
|
//------------------------------------------------------------------------------
|
||||||
|
/*
|
||||||
|
This file is part of rippled: https://github.com/ripple/rippled
|
||||||
|
Copyright (c) 2021 Ripple Labs Inc.
|
||||||
|
|
||||||
|
Permission to use, copy, modify, and/or 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 <ripple/app/main/Application.h>
|
||||||
|
#include <ripple/app/misc/NetworkOPs.h>
|
||||||
|
#include <ripple/json/json_value.h>
|
||||||
|
#include <ripple/net/RPCErr.h>
|
||||||
|
#include <ripple/nodestore/DatabaseShard.h>
|
||||||
|
#include <ripple/protocol/ErrorCodes.h>
|
||||||
|
#include <ripple/protocol/jss.h>
|
||||||
|
#include <ripple/rpc/Context.h>
|
||||||
|
|
||||||
|
namespace ripple {
|
||||||
|
|
||||||
|
// node_to_shard [status|start|stop]
|
||||||
|
Json::Value
|
||||||
|
doNodeToShard(RPC::JsonContext& context)
|
||||||
|
{
|
||||||
|
if (context.app.config().reporting())
|
||||||
|
return rpcError(rpcREPORTING_UNSUPPORTED);
|
||||||
|
|
||||||
|
// Shard store must be enabled
|
||||||
|
auto const shardStore = context.app.getShardStore();
|
||||||
|
if (!shardStore)
|
||||||
|
return rpcError(rpcINTERNAL, "No shard store");
|
||||||
|
|
||||||
|
if (!context.params.isMember(jss::action))
|
||||||
|
return RPC::missing_field_error(jss::action);
|
||||||
|
|
||||||
|
// Obtain and normalize the action to perform
|
||||||
|
auto const action = [&context] {
|
||||||
|
auto value = context.params[jss::action].asString();
|
||||||
|
boost::to_lower(value);
|
||||||
|
|
||||||
|
return value;
|
||||||
|
}();
|
||||||
|
|
||||||
|
// Vector of allowed actions
|
||||||
|
std::vector<std::string> const allowedActions = {"status", "start", "stop"};
|
||||||
|
|
||||||
|
// Validate the action
|
||||||
|
if (std::find(allowedActions.begin(), allowedActions.end(), action) ==
|
||||||
|
allowedActions.end())
|
||||||
|
return RPC::invalid_field_error(jss::action);
|
||||||
|
|
||||||
|
// Perform the action
|
||||||
|
if (action == "status")
|
||||||
|
{
|
||||||
|
// Get the status of the database import
|
||||||
|
return shardStore->getDatabaseImportStatus();
|
||||||
|
}
|
||||||
|
else if (action == "start")
|
||||||
|
{
|
||||||
|
// Kick off an import
|
||||||
|
return shardStore->startNodeToShard();
|
||||||
|
}
|
||||||
|
else if (action == "stop")
|
||||||
|
{
|
||||||
|
// Halt an import
|
||||||
|
return shardStore->stopNodeToShard();
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
// Shouldn't happen
|
||||||
|
assert(false);
|
||||||
|
return rpcError(rpcINTERNAL);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
} // namespace ripple
|
||||||
@@ -1,43 +0,0 @@
|
|||||||
//------------------------------------------------------------------------------
|
|
||||||
/*
|
|
||||||
This file is part of rippled: https://github.com/ripple/rippled
|
|
||||||
Copyright (c) 2021 Ripple Labs Inc.
|
|
||||||
|
|
||||||
Permission to use, copy, modify, and/or 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 <ripple/app/main/Application.h>
|
|
||||||
#include <ripple/app/misc/NetworkOPs.h>
|
|
||||||
#include <ripple/json/json_value.h>
|
|
||||||
#include <ripple/nodestore/DatabaseShard.h>
|
|
||||||
#include <ripple/protocol/ErrorCodes.h>
|
|
||||||
#include <ripple/protocol/jss.h>
|
|
||||||
#include <ripple/rpc/Context.h>
|
|
||||||
|
|
||||||
namespace ripple {
|
|
||||||
|
|
||||||
Json::Value
|
|
||||||
doNodeToShardStatus(RPC::JsonContext& context)
|
|
||||||
{
|
|
||||||
Json::Value ret(Json::objectValue);
|
|
||||||
|
|
||||||
if (auto const shardStore = context.app.getShardStore())
|
|
||||||
ret[jss::info] = shardStore->getDatabaseImportStatus();
|
|
||||||
else
|
|
||||||
ret = RPC::make_error(rpcINTERNAL, "No shard store");
|
|
||||||
|
|
||||||
return ret;
|
|
||||||
}
|
|
||||||
|
|
||||||
} // namespace ripple
|
|
||||||
|
|||||||
@@ -107,10 +107,7 @@ Handler const handlerArray[]{
|
|||||||
{"log_level", byRef(&doLogLevel), Role::ADMIN, NO_CONDITION},
|
{"log_level", byRef(&doLogLevel), Role::ADMIN, NO_CONDITION},
|
||||||
{"logrotate", byRef(&doLogRotate), Role::ADMIN, NO_CONDITION},
|
{"logrotate", byRef(&doLogRotate), Role::ADMIN, NO_CONDITION},
|
||||||
{"manifest", byRef(&doManifest), Role::USER, NO_CONDITION},
|
{"manifest", byRef(&doManifest), Role::USER, NO_CONDITION},
|
||||||
{"nodetoshard_status",
|
{"node_to_shard", byRef(&doNodeToShard), Role::ADMIN, NO_CONDITION},
|
||||||
byRef(&doNodeToShardStatus),
|
|
||||||
Role::USER,
|
|
||||||
NO_CONDITION},
|
|
||||||
{"noripple_check", byRef(&doNoRippleCheck), Role::USER, NO_CONDITION},
|
{"noripple_check", byRef(&doNoRippleCheck), Role::USER, NO_CONDITION},
|
||||||
{"owner_info", byRef(&doOwnerInfo), Role::USER, NEEDS_CURRENT_LEDGER},
|
{"owner_info", byRef(&doOwnerInfo), Role::USER, NEEDS_CURRENT_LEDGER},
|
||||||
{"peers", byRef(&doPeers), Role::ADMIN, NO_CONDITION},
|
{"peers", byRef(&doPeers), Role::ADMIN, NO_CONDITION},
|
||||||
|
|||||||
@@ -19,6 +19,7 @@
|
|||||||
|
|
||||||
#include <ripple/app/ledger/LedgerMaster.h>
|
#include <ripple/app/ledger/LedgerMaster.h>
|
||||||
#include <ripple/app/ledger/LedgerToJson.h>
|
#include <ripple/app/ledger/LedgerToJson.h>
|
||||||
|
#include <ripple/app/misc/LoadFeeTrack.h>
|
||||||
#include <ripple/app/misc/SHAMapStore.h>
|
#include <ripple/app/misc/SHAMapStore.h>
|
||||||
#include <ripple/app/rdb/backend/RelationalDBInterfaceSqlite.h>
|
#include <ripple/app/rdb/backend/RelationalDBInterfaceSqlite.h>
|
||||||
#include <ripple/basics/Slice.h>
|
#include <ripple/basics/Slice.h>
|
||||||
@@ -263,6 +264,10 @@ class DatabaseShard_test : public TestBase
|
|||||||
{
|
{
|
||||||
using namespace test::jtx;
|
using namespace test::jtx;
|
||||||
|
|
||||||
|
// The local fee may go up, especially in the online delete tests
|
||||||
|
while (env_.app().getFeeTrack().lowerLocalFee())
|
||||||
|
std::this_thread::sleep_for(std::chrono::milliseconds(1));
|
||||||
|
|
||||||
if (isNewAccounts(seq))
|
if (isNewAccounts(seq))
|
||||||
env_.fund(XRP(iniAmount), accounts_[nAccounts_[seq] - 1]);
|
env_.fund(XRP(iniAmount), accounts_[nAccounts_[seq] - 1]);
|
||||||
|
|
||||||
@@ -1249,14 +1254,22 @@ class DatabaseShard_test : public TestBase
|
|||||||
Database& ndb = env.app().getNodeStore();
|
Database& ndb = env.app().getNodeStore();
|
||||||
BEAST_EXPECT(db);
|
BEAST_EXPECT(db);
|
||||||
|
|
||||||
|
auto& store = env.app().getSHAMapStore();
|
||||||
|
|
||||||
|
// Allow online delete to delete the startup ledgers
|
||||||
|
// so that it will take some time for the import to
|
||||||
|
// catch up to the point of the next rotation
|
||||||
|
store.setCanDelete(10);
|
||||||
|
|
||||||
// Create some ledgers for the shard store to import
|
// Create some ledgers for the shard store to import
|
||||||
auto const shardCount = 5;
|
auto const shardCount = 5;
|
||||||
TestData data(seedValue, 4, shardCount);
|
TestData data(seedValue, 4, shardCount);
|
||||||
if (!BEAST_EXPECT(data.makeLedgers(env)))
|
if (!BEAST_EXPECT(data.makeLedgers(env)))
|
||||||
return;
|
return;
|
||||||
|
|
||||||
auto& store = env.app().getSHAMapStore();
|
store.rendezvous();
|
||||||
auto lastRotated = store.getLastRotated();
|
auto const lastRotated = store.getLastRotated();
|
||||||
|
BEAST_EXPECT(lastRotated >= 553 && lastRotated < 1103);
|
||||||
|
|
||||||
// Start the import
|
// Start the import
|
||||||
db->importDatabase(ndb);
|
db->importDatabase(ndb);
|
||||||
@@ -1267,37 +1280,45 @@ class DatabaseShard_test : public TestBase
|
|||||||
std::this_thread::sleep_for(std::chrono::milliseconds(1));
|
std::this_thread::sleep_for(std::chrono::milliseconds(1));
|
||||||
}
|
}
|
||||||
|
|
||||||
// Enable online deletion now that the import has started
|
// Enable unimpeded online deletion now that the import has started
|
||||||
store.setCanDelete(std::numeric_limits<std::uint32_t>::max());
|
store.setCanDelete(std::numeric_limits<std::uint32_t>::max());
|
||||||
|
|
||||||
auto pauseVerifier = std::thread([lastRotated, &store, db, this] {
|
auto pauseVerifier = std::thread([lastRotated, &store, db, this] {
|
||||||
while (true)
|
// The import should still be running when this thread starts
|
||||||
|
BEAST_EXPECT(db->getDatabaseImportSequence());
|
||||||
|
auto rotationProgress = lastRotated;
|
||||||
|
while (auto const ledgerSeq = db->getDatabaseImportSequence())
|
||||||
{
|
{
|
||||||
// Make sure database rotations dont interfere
|
// Make sure database rotations dont interfere
|
||||||
// with the import
|
// with the import
|
||||||
|
|
||||||
if (store.getLastRotated() != lastRotated)
|
auto const last = store.getLastRotated();
|
||||||
|
if (last != rotationProgress)
|
||||||
{
|
{
|
||||||
// A rotation occurred during shard import. Not
|
// A rotation occurred during shard import. Not
|
||||||
// necessarily an error
|
// necessarily an error
|
||||||
|
|
||||||
auto const ledgerSeq = db->getDatabaseImportSequence();
|
BEAST_EXPECT(
|
||||||
BEAST_EXPECT(!ledgerSeq || ledgerSeq >= lastRotated);
|
!ledgerSeq || ledgerSeq >= rotationProgress);
|
||||||
|
rotationProgress = last;
|
||||||
break;
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
|
|
||||||
|
auto join = [&pauseVerifier]() {
|
||||||
|
if (pauseVerifier.joinable())
|
||||||
|
pauseVerifier.join();
|
||||||
|
};
|
||||||
|
|
||||||
// Create more ledgers to trigger online deletion
|
// Create more ledgers to trigger online deletion
|
||||||
data = TestData(seedValue * 2);
|
data = TestData(seedValue * 2);
|
||||||
if (!BEAST_EXPECT(data.makeLedgers(env, shardCount)))
|
if (!BEAST_EXPECT(data.makeLedgers(env, shardCount)))
|
||||||
{
|
{
|
||||||
pauseVerifier.join();
|
join();
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
pauseVerifier.join();
|
join();
|
||||||
BEAST_EXPECT(store.getLastRotated() != lastRotated);
|
BEAST_EXPECT(store.getLastRotated() != lastRotated);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|||||||
331
src/test/rpc/NodeToShardRPC_test.cpp
Normal file
331
src/test/rpc/NodeToShardRPC_test.cpp
Normal file
@@ -0,0 +1,331 @@
|
|||||||
|
//------------------------------------------------------------------------------
|
||||||
|
/*
|
||||||
|
This file is part of rippled: https://github.com/ripple/rippled
|
||||||
|
Copyright (c) 2021 Ripple Labs Inc.
|
||||||
|
|
||||||
|
Permission to use, copy, modify, and/or 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 <ripple/beast/unit_test.h>
|
||||||
|
#include <ripple/beast/utility/temp_dir.h>
|
||||||
|
#include <ripple/core/ConfigSections.h>
|
||||||
|
#include <ripple/nodestore/DatabaseShard.h>
|
||||||
|
#include <ripple/protocol/jss.h>
|
||||||
|
#include <test/jtx/Env.h>
|
||||||
|
|
||||||
|
namespace ripple {
|
||||||
|
namespace test {
|
||||||
|
|
||||||
|
class NodeToShardRPC_test : public beast::unit_test::suite
|
||||||
|
{
|
||||||
|
bool
|
||||||
|
importCompleted(
|
||||||
|
NodeStore::DatabaseShard* shardStore,
|
||||||
|
std::uint8_t const numberOfShards,
|
||||||
|
Json::Value const& result)
|
||||||
|
{
|
||||||
|
auto const info = shardStore->getShardInfo();
|
||||||
|
|
||||||
|
// Assume completed if the import isn't running
|
||||||
|
auto const completed =
|
||||||
|
result[jss::error_message] == "Database import not running";
|
||||||
|
|
||||||
|
if (completed)
|
||||||
|
{
|
||||||
|
BEAST_EXPECT(
|
||||||
|
info->incomplete().size() + info->finalized().size() ==
|
||||||
|
numberOfShards);
|
||||||
|
}
|
||||||
|
|
||||||
|
return completed;
|
||||||
|
}
|
||||||
|
|
||||||
|
public:
|
||||||
|
void
|
||||||
|
testStart()
|
||||||
|
{
|
||||||
|
testcase("Start");
|
||||||
|
|
||||||
|
beast::temp_dir tempDir;
|
||||||
|
|
||||||
|
jtx::Env env = [&] {
|
||||||
|
auto c = jtx::envconfig();
|
||||||
|
auto& section = c->section(ConfigSection::shardDatabase());
|
||||||
|
section.set("path", tempDir.path());
|
||||||
|
section.set("max_historical_shards", "20");
|
||||||
|
section.set("ledgers_per_shard", "256");
|
||||||
|
section.set("earliest_seq", "257");
|
||||||
|
auto& sectionNode = c->section(ConfigSection::nodeDatabase());
|
||||||
|
sectionNode.set("earliest_seq", "257");
|
||||||
|
sectionNode.set("ledgers_per_shard", "256");
|
||||||
|
c->setupControl(true, true, true);
|
||||||
|
|
||||||
|
return jtx::Env(*this, std::move(c));
|
||||||
|
}();
|
||||||
|
|
||||||
|
std::uint8_t const numberOfShards = 10;
|
||||||
|
|
||||||
|
// Create some ledgers so that we can initiate a
|
||||||
|
// shard store database import.
|
||||||
|
for (int i = 0; i < env.app().getShardStore()->ledgersPerShard() *
|
||||||
|
(numberOfShards + 1);
|
||||||
|
++i)
|
||||||
|
{
|
||||||
|
env.close();
|
||||||
|
}
|
||||||
|
|
||||||
|
auto shardStore = env.app().getShardStore();
|
||||||
|
if (!BEAST_EXPECT(shardStore))
|
||||||
|
return;
|
||||||
|
|
||||||
|
{
|
||||||
|
// Initiate a shard store import via the RPC
|
||||||
|
// interface.
|
||||||
|
|
||||||
|
Json::Value jvParams;
|
||||||
|
jvParams[jss::action] = "start";
|
||||||
|
|
||||||
|
auto const result = env.rpc(
|
||||||
|
"json", "node_to_shard", to_string(jvParams))[jss::result];
|
||||||
|
|
||||||
|
BEAST_EXPECT(
|
||||||
|
result[jss::message] == "Database import initiated...");
|
||||||
|
}
|
||||||
|
|
||||||
|
while (!shardStore->getDatabaseImportSequence())
|
||||||
|
{
|
||||||
|
// Wait until the import starts
|
||||||
|
std::this_thread::sleep_for(std::chrono::milliseconds(1));
|
||||||
|
}
|
||||||
|
|
||||||
|
{
|
||||||
|
// Verify that the import is in progress with
|
||||||
|
// the node_to_shard status RPC command
|
||||||
|
|
||||||
|
Json::Value jvParams;
|
||||||
|
jvParams[jss::action] = "status";
|
||||||
|
|
||||||
|
auto const result = env.rpc(
|
||||||
|
"json", "node_to_shard", to_string(jvParams))[jss::result];
|
||||||
|
|
||||||
|
BEAST_EXPECT(
|
||||||
|
result[jss::status] == "success" ||
|
||||||
|
importCompleted(shardStore, numberOfShards, result));
|
||||||
|
|
||||||
|
std::chrono::seconds const maxWait{60};
|
||||||
|
auto const start = std::chrono::system_clock::now();
|
||||||
|
|
||||||
|
while (true)
|
||||||
|
{
|
||||||
|
// Verify that the status object accurately
|
||||||
|
// reflects import progress.
|
||||||
|
|
||||||
|
auto const completeShards =
|
||||||
|
shardStore->getShardInfo()->finalized();
|
||||||
|
|
||||||
|
if (!completeShards.empty())
|
||||||
|
{
|
||||||
|
auto const result = env.rpc(
|
||||||
|
"json",
|
||||||
|
"node_to_shard",
|
||||||
|
to_string(jvParams))[jss::result];
|
||||||
|
|
||||||
|
if (!importCompleted(shardStore, numberOfShards, result))
|
||||||
|
{
|
||||||
|
BEAST_EXPECT(result[jss::firstShardIndex] == 1);
|
||||||
|
BEAST_EXPECT(result[jss::lastShardIndex] == 10);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if (boost::icl::contains(completeShards, 1))
|
||||||
|
{
|
||||||
|
auto const result = env.rpc(
|
||||||
|
"json",
|
||||||
|
"node_to_shard",
|
||||||
|
to_string(jvParams))[jss::result];
|
||||||
|
|
||||||
|
BEAST_EXPECT(
|
||||||
|
result[jss::currentShardIndex] >= 1 ||
|
||||||
|
importCompleted(shardStore, numberOfShards, result));
|
||||||
|
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (std::this_thread::sleep_for(std::chrono::milliseconds{100});
|
||||||
|
std::chrono::system_clock::now() - start > maxWait)
|
||||||
|
{
|
||||||
|
BEAST_EXPECTS(
|
||||||
|
false, "Import timeout: could just be a slow machine.");
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// Wait for the import to complete
|
||||||
|
while (!boost::icl::contains(
|
||||||
|
shardStore->getShardInfo()->finalized(), 10))
|
||||||
|
{
|
||||||
|
if (std::this_thread::sleep_for(std::chrono::milliseconds{100});
|
||||||
|
std::chrono::system_clock::now() - start > maxWait)
|
||||||
|
{
|
||||||
|
BEAST_EXPECT(
|
||||||
|
importCompleted(shardStore, numberOfShards, result));
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
void
|
||||||
|
testStop()
|
||||||
|
{
|
||||||
|
testcase("Stop");
|
||||||
|
|
||||||
|
beast::temp_dir tempDir;
|
||||||
|
|
||||||
|
jtx::Env env = [&] {
|
||||||
|
auto c = jtx::envconfig();
|
||||||
|
auto& section = c->section(ConfigSection::shardDatabase());
|
||||||
|
section.set("path", tempDir.path());
|
||||||
|
section.set("max_historical_shards", "20");
|
||||||
|
section.set("ledgers_per_shard", "256");
|
||||||
|
section.set("earliest_seq", "257");
|
||||||
|
auto& sectionNode = c->section(ConfigSection::nodeDatabase());
|
||||||
|
sectionNode.set("earliest_seq", "257");
|
||||||
|
sectionNode.set("ledgers_per_shard", "256");
|
||||||
|
c->setupControl(true, true, true);
|
||||||
|
|
||||||
|
return jtx::Env(*this, std::move(c));
|
||||||
|
}();
|
||||||
|
|
||||||
|
std::uint8_t const numberOfShards = 10;
|
||||||
|
|
||||||
|
// Create some ledgers so that we can initiate a
|
||||||
|
// shard store database import.
|
||||||
|
for (int i = 0; i < env.app().getShardStore()->ledgersPerShard() *
|
||||||
|
(numberOfShards + 1);
|
||||||
|
++i)
|
||||||
|
{
|
||||||
|
env.close();
|
||||||
|
}
|
||||||
|
|
||||||
|
auto shardStore = env.app().getShardStore();
|
||||||
|
if (!BEAST_EXPECT(shardStore))
|
||||||
|
return;
|
||||||
|
|
||||||
|
{
|
||||||
|
// Initiate a shard store import via the RPC
|
||||||
|
// interface.
|
||||||
|
|
||||||
|
Json::Value jvParams;
|
||||||
|
jvParams[jss::action] = "start";
|
||||||
|
|
||||||
|
auto const result = env.rpc(
|
||||||
|
"json", "node_to_shard", to_string(jvParams))[jss::result];
|
||||||
|
|
||||||
|
BEAST_EXPECT(
|
||||||
|
result[jss::message] == "Database import initiated...");
|
||||||
|
}
|
||||||
|
|
||||||
|
{
|
||||||
|
// Verify that the import is in progress with
|
||||||
|
// the node_to_shard status RPC command
|
||||||
|
|
||||||
|
Json::Value jvParams;
|
||||||
|
jvParams[jss::action] = "status";
|
||||||
|
|
||||||
|
auto const result = env.rpc(
|
||||||
|
"json", "node_to_shard", to_string(jvParams))[jss::result];
|
||||||
|
|
||||||
|
BEAST_EXPECT(
|
||||||
|
result[jss::status] == "success" ||
|
||||||
|
importCompleted(shardStore, numberOfShards, result));
|
||||||
|
|
||||||
|
std::chrono::seconds const maxWait{10};
|
||||||
|
auto const start = std::chrono::system_clock::now();
|
||||||
|
|
||||||
|
while (shardStore->getShardInfo()->finalized().empty())
|
||||||
|
{
|
||||||
|
// Wait for at least one shard to complete
|
||||||
|
|
||||||
|
if (std::this_thread::sleep_for(std::chrono::milliseconds{100});
|
||||||
|
std::chrono::system_clock::now() - start > maxWait)
|
||||||
|
{
|
||||||
|
BEAST_EXPECTS(
|
||||||
|
false, "Import timeout: could just be a slow machine.");
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
{
|
||||||
|
Json::Value jvParams;
|
||||||
|
jvParams[jss::action] = "stop";
|
||||||
|
|
||||||
|
auto const result = env.rpc(
|
||||||
|
"json", "node_to_shard", to_string(jvParams))[jss::result];
|
||||||
|
|
||||||
|
BEAST_EXPECT(
|
||||||
|
result[jss::message] == "Database import halt initiated..." ||
|
||||||
|
importCompleted(shardStore, numberOfShards, result));
|
||||||
|
}
|
||||||
|
|
||||||
|
std::chrono::seconds const maxWait{10};
|
||||||
|
auto const start = std::chrono::system_clock::now();
|
||||||
|
|
||||||
|
while (true)
|
||||||
|
{
|
||||||
|
// Wait until we can verify that the import has
|
||||||
|
// stopped
|
||||||
|
|
||||||
|
Json::Value jvParams;
|
||||||
|
jvParams[jss::action] = "status";
|
||||||
|
|
||||||
|
auto const result = env.rpc(
|
||||||
|
"json", "node_to_shard", to_string(jvParams))[jss::result];
|
||||||
|
|
||||||
|
// When the import has stopped, polling the
|
||||||
|
// status returns an error
|
||||||
|
if (result.isMember(jss::error))
|
||||||
|
{
|
||||||
|
if (BEAST_EXPECT(result.isMember(jss::error_message)))
|
||||||
|
{
|
||||||
|
BEAST_EXPECT(
|
||||||
|
result[jss::error_message] ==
|
||||||
|
"Database import not running");
|
||||||
|
}
|
||||||
|
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (std::this_thread::sleep_for(std::chrono::milliseconds{100});
|
||||||
|
std::chrono::system_clock::now() - start > maxWait)
|
||||||
|
{
|
||||||
|
BEAST_EXPECTS(
|
||||||
|
false, "Import timeout: could just be a slow machine.");
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
void
|
||||||
|
run() override
|
||||||
|
{
|
||||||
|
testStart();
|
||||||
|
testStop();
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
BEAST_DEFINE_TESTSUITE(NodeToShardRPC, rpc, ripple);
|
||||||
|
} // namespace test
|
||||||
|
} // namespace ripple
|
||||||
@@ -4301,28 +4301,53 @@ static RPCCallTestData const rpcCallTestArray[] = {
|
|||||||
]
|
]
|
||||||
})"},
|
})"},
|
||||||
|
|
||||||
// nodetoshard_status
|
// node_to_shard
|
||||||
// -------------------------------------------------------------------
|
// -------------------------------------------------------------------
|
||||||
{"nodetoshard_status: minimal.",
|
{"node_to_shard: status.",
|
||||||
__LINE__,
|
__LINE__,
|
||||||
{
|
{"node_to_shard", "status"},
|
||||||
"nodetoshard_status",
|
|
||||||
},
|
|
||||||
RPCCallTestData::no_exception,
|
RPCCallTestData::no_exception,
|
||||||
R"({
|
R"({
|
||||||
"method" : "nodetoshard_status",
|
"method" : "node_to_shard",
|
||||||
"params" : [
|
"params" : [
|
||||||
{
|
{
|
||||||
"api_version" : %MAX_API_VER%,
|
"api_version" : %MAX_API_VER%,
|
||||||
|
"action" : "status"
|
||||||
}
|
}
|
||||||
]
|
]
|
||||||
})"},
|
})"},
|
||||||
{"nodetoshard_status: too many arguments.",
|
{"node_to_shard: start.",
|
||||||
__LINE__,
|
__LINE__,
|
||||||
{"nodetoshard_status", "extra"},
|
{"node_to_shard", "start"},
|
||||||
RPCCallTestData::no_exception,
|
RPCCallTestData::no_exception,
|
||||||
R"({
|
R"({
|
||||||
"method" : "nodetoshard_status",
|
"method" : "node_to_shard",
|
||||||
|
"params" : [
|
||||||
|
{
|
||||||
|
"api_version" : %MAX_API_VER%,
|
||||||
|
"action" : "start"
|
||||||
|
}
|
||||||
|
]
|
||||||
|
})"},
|
||||||
|
{"node_to_shard: stop.",
|
||||||
|
__LINE__,
|
||||||
|
{"node_to_shard", "stop"},
|
||||||
|
RPCCallTestData::no_exception,
|
||||||
|
R"({
|
||||||
|
"method" : "node_to_shard",
|
||||||
|
"params" : [
|
||||||
|
{
|
||||||
|
"api_version" : %MAX_API_VER%,
|
||||||
|
"action" : "stop"
|
||||||
|
}
|
||||||
|
]
|
||||||
|
})"},
|
||||||
|
{"node_to_shard: too many arguments.",
|
||||||
|
__LINE__,
|
||||||
|
{"node_to_shard", "start", "stop"},
|
||||||
|
RPCCallTestData::no_exception,
|
||||||
|
R"({
|
||||||
|
"method" : "node_to_shard",
|
||||||
"params" : [
|
"params" : [
|
||||||
{
|
{
|
||||||
"error" : "badSyntax",
|
"error" : "badSyntax",
|
||||||
@@ -4331,6 +4356,19 @@ static RPCCallTestData const rpcCallTestArray[] = {
|
|||||||
}
|
}
|
||||||
]
|
]
|
||||||
})"},
|
})"},
|
||||||
|
{"node_to_shard: invalid argument.",
|
||||||
|
__LINE__,
|
||||||
|
{"node_to_shard", "invalid"},
|
||||||
|
RPCCallTestData::no_exception,
|
||||||
|
R"({
|
||||||
|
"method" : "node_to_shard",
|
||||||
|
"params" : [
|
||||||
|
{
|
||||||
|
"api_version" : %MAX_API_VER%,
|
||||||
|
"action" : "invalid"
|
||||||
|
}
|
||||||
|
]
|
||||||
|
})"},
|
||||||
|
|
||||||
// owner_info
|
// owner_info
|
||||||
// ------------------------------------------------------------------
|
// ------------------------------------------------------------------
|
||||||
|
|||||||
Reference in New Issue
Block a user