Implement node-to-shard RPC control

This commit is contained in:
Devon White
2021-04-07 12:03:33 -04:00
committed by manojsdoshi
parent 0320d2169e
commit 00a4c3a478
14 changed files with 666 additions and 95 deletions

View File

@@ -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

View File

@@ -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"

View File

@@ -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);
} }

View File

@@ -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},

View File

@@ -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

View File

@@ -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>

View File

@@ -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

View File

@@ -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

View 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

View File

@@ -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

View File

@@ -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},

View File

@@ -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);
} }

View 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

View File

@@ -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
// ------------------------------------------------------------------ // ------------------------------------------------------------------