1
0
Fork 0

Missing post failure on readlock in SynchronizeShard (#7860)

* when post cannot be delivered for readlock waiting for put meningless

* devel pull

* we can rely on POST alone

* clean up

* wrong errorCode tested

* fixed nullptr

* correct timeout

* fixed nullptr

* merge devel

* log level

* maintenace default log level raised to INFO, shard synchronisation only logs start/success all WARN/ERR demoted to DEBUG

* message unification

* devel merge

* fixed query registry

* tobias remarks

* mchacki part one

* static string for rebootid

* working tests

* use the reboot tracker from clusterinfo

* use the reboot tracker from clusterinfo

* rebootid be rebootid

* fixed serverid and all working

* static strings

* callguard out of reboottracker

* call callbacks, when query is ditched

* clear priv

* change log

* my oh my

* simon attention

* merge seems fine

* typo

* fix headers
This commit is contained in:
Kaveh Vahedipour 2019-11-26 17:46:26 +01:00 committed by KVS85
parent b9b9a0c613
commit d11c0954e8
11 changed files with 331 additions and 175 deletions

View File

@ -1,6 +1,14 @@
devel
-----
<<<<<<< HEAD
* Shard synchronisation readlock aware of rebootId
* Bugfix: In an AQL cluster query, when gathering unsorted data in combination
with a LIMIT with non-zero offset, if this offset exactly matches the number
of documents in the first shards consumed, the rest of the documents was not
returned.
=======
* Fixed GET _api/gharial to also include the name property in every returned graph.
This is a consistency fix within the API as all other APIs include the name.
As a work around the returned _key can be used, which is identical to the name.
@ -152,6 +160,7 @@ devel
* Added "SmartJoins for Views" to the ArangoDB Enterprise Edition that allows running
cluster joins between two certain sharded collections or views with performance close
to that of a local join operation.
>>>>>>> a50f07e928e0e50168aac0039a5b9f2333a0520e
* Allow collection names to be at most 256 characters long, instead of 64 characters
in previous versions.

View File

@ -35,6 +35,7 @@
using namespace arangodb;
using namespace arangodb::aql;
using namespace arangodb::cluster;
QueryRegistry::~QueryRegistry() {
std::vector<std::pair<std::string, QueryId>> toDelete;
@ -70,7 +71,8 @@ QueryRegistry::~QueryRegistry() {
/// @brief insert
void QueryRegistry::insert(QueryId id, Query* query, double ttl,
bool isPrepared, bool keepLease) {
bool isPrepared, bool keepLease,
std::unique_ptr<CallbackGuard>&& rGuard) {
TRI_ASSERT(query != nullptr);
TRI_ASSERT(query->trx() != nullptr);
LOG_TOPIC("77778", DEBUG, arangodb::Logger::AQL)
@ -83,7 +85,7 @@ void QueryRegistry::insert(QueryId id, Query* query, double ttl,
}
// create the query info object outside of the lock
auto p = std::make_unique<QueryInfo>(id, query, ttl, isPrepared);
auto p = std::make_unique<QueryInfo>(id, query, ttl, isPrepared, std::move(rGuard));
p->_isOpen = keepLease;
// now insert into table of running queries
@ -226,6 +228,10 @@ void QueryRegistry::destroy(std::string const& vocbase, QueryId id,
TRI_ERROR_BAD_PARAMETER, "query with given vocbase and id not found");
}
if (q->second->_rebootGuard != nullptr) {
q->second->_rebootGuard->callAndClear();
}
if (q->second->_isOpen && !ignoreOpened) {
// query in use by another thread/request
q->second->_query->kill();
@ -408,13 +414,15 @@ void QueryRegistry::disallowInserts() {
// from here on, there shouldn't be any more inserts into the registry
}
QueryRegistry::QueryInfo::QueryInfo(QueryId id, Query* query, double ttl, bool isPrepared)
QueryRegistry::QueryInfo::QueryInfo(QueryId id, Query* query, double ttl, bool isPrepared,
std::unique_ptr<arangodb::cluster::CallbackGuard>&& rebootGuard)
: _vocbase(&(query->vocbase())),
_id(id),
_query(query),
_isOpen(false),
_isPrepared(isPrepared),
_timeToLive(ttl),
_expires(TRI_microtime() + ttl) {}
_expires(TRI_microtime() + ttl),
_rebootGuard(std::move(rebootGuard)) {}
QueryRegistry::QueryInfo::~QueryInfo() { delete _query; }

View File

@ -27,11 +27,13 @@
#include "Aql/types.h"
#include "Basics/Common.h"
#include "Basics/ReadWriteLock.h"
#include "Cluster/CallbackGuard.h"
#include "Cluster/ResultT.h"
struct TRI_vocbase_t;
namespace arangodb {
namespace aql {
class ExecutionEngine;
class Query;
@ -55,7 +57,9 @@ class QueryRegistry {
/// With keepLease == true the query will be kept open and it is guaranteed
/// that the caller can continue to use it exclusively.
/// This is identical to an atomic sequence of insert();open();
TEST_VIRTUAL void insert(QueryId id, Query* query, double ttl, bool isPrepare, bool keepLease);
TEST_VIRTUAL void insert(
QueryId id, Query* query, double ttl, bool isPrepare, bool keepLease,
std::unique_ptr<arangodb::cluster::CallbackGuard>&& = nullptr);
/// @brief open, find a query in the registry, if none is found, a nullptr
/// is returned, otherwise, ownership of the query is transferred to the
@ -120,7 +124,8 @@ class QueryRegistry {
QueryInfo(QueryInfo const&) = delete;
QueryInfo& operator=(QueryInfo const&) = delete;
QueryInfo(QueryId id, Query* query, double ttl, bool isPrepared);
QueryInfo(QueryId id, Query* query, double ttl, bool isPrepared,
std::unique_ptr<arangodb::cluster::CallbackGuard>&& rebootGuard = nullptr);
~QueryInfo();
TRI_vocbase_t* _vocbase; // the vocbase
@ -131,6 +136,8 @@ class QueryRegistry {
bool _isPrepared;
double _timeToLive; // in seconds
double _expires; // UNIX UTC timestamp of expiration
std::unique_ptr<arangodb::cluster::CallbackGuard> _rebootGuard;
// Callback to remove query, when rebootId changes
};
/// @brief _queries, the actual map of maps for the registry

View File

@ -148,6 +148,7 @@ endif ()
set(LIB_ARANGO_GRAPH_SOURCES
Aql/PruneExpressionEvaluator.cpp
Cluster/CallbackGuard.cpp
Cluster/ClusterEdgeCursor.cpp
Cluster/ClusterTraverser.cpp
Cluster/TraverserEngine.cpp

View File

@ -0,0 +1,58 @@
////////////////////////////////////////////////////////////////////////////////
/// DISCLAIMER
///
/// Copyright 2019 ArangoDB GmbH, Cologne, Germany
///
/// Licensed under the Apache License, Version 2.0 (the "License");
/// you may not use this file except in compliance with the License.
/// You may obtain a copy of the License at
///
/// http://www.apache.org/licenses/LICENSE-2.0
///
/// Unless required by applicable law or agreed to in writing, software
/// distributed under the License is distributed on an "AS IS" BASIS,
/// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
/// See the License for the specific language governing permissions and
/// limitations under the License.
///
/// Copyright holder is ArangoDB GmbH, Cologne, Germany
///
/// @author Tobias Gödderz
////////////////////////////////////////////////////////////////////////////////
#include "CallbackGuard.h"
using namespace arangodb;
using namespace arangodb::cluster;
CallbackGuard::CallbackGuard() : _callback(nullptr) {}
CallbackGuard::CallbackGuard(std::function<void(void)> callback)
: _callback(std::move(callback)) {}
// NOLINTNEXTLINE(hicpp-noexcept-move,performance-noexcept-move-constructor)
CallbackGuard::CallbackGuard(CallbackGuard&& other)
: _callback(std::move(other._callback)) {
other._callback = nullptr;
}
// NOLINTNEXTLINE(hicpp-noexcept-move,performance-noexcept-move-constructor)
CallbackGuard& CallbackGuard::operator=(CallbackGuard&& other) {
call();
_callback = std::move(other._callback);
other._callback = nullptr;
return *this;
}
CallbackGuard::~CallbackGuard() { call(); }
void CallbackGuard::callAndClear() {
call();
_callback = nullptr;
}
void CallbackGuard::call() {
if (_callback) {
_callback();
}
}

View File

@ -0,0 +1,71 @@
////////////////////////////////////////////////////////////////////////////////
/// DISCLAIMER
///
/// Copyright 2019 ArangoDB GmbH, Cologne, Germany
///
/// Licensed under the Apache License, Version 2.0 (the "License");
/// you may not use this file except in compliance with the License.
/// You may obtain a copy of the License at
///
/// http://www.apache.org/licenses/LICENSE-2.0
///
/// Unless required by applicable law or agreed to in writing, software
/// distributed under the License is distributed on an "AS IS" BASIS,
/// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
/// See the License for the specific language governing permissions and
/// limitations under the License.
///
/// Copyright holder is ArangoDB GmbH, Cologne, Germany
///
/// @author Tobias Gödderz
////////////////////////////////////////////////////////////////////////////////
#ifndef ARANGOD_CLUSTER_CALLBACKGUARD_H
#define ARANGOD_CLUSTER_CALLBACKGUARD_H
#include <functional>
#include "Cluster/ClusterTypes.h"
namespace arangodb {
namespace cluster {
/// @brief If constructed with a callback, the given callback will be called
/// exactly once: Either during destruction, or when the object is overwritten
/// (via operator=()), or when it's explicitly cleared. It's not copyable,
/// but movable.
class CallbackGuard {
public:
// Calls the callback given callback upon destruction.
// Allows only move semantics and no copy semantics.
CallbackGuard();
// IMPORTANT NOTE:
// The passed callback should not throw exceptions, they will not be caught
// here, but thrown by the destructor!
explicit CallbackGuard(std::function<void(void)> callback);
~CallbackGuard();
// Note that the move constructor of std::function is not noexcept until
// C++20. Thus we cannot mark the constructors here noexcept.
// NOLINTNEXTLINE(hicpp-noexcept-move,performance-noexcept-move-constructor)
CallbackGuard(CallbackGuard&& other);
// operator= additionally calls the _callback, and this can also throw.
// NOLINTNEXTLINE(hicpp-noexcept-move,performance-noexcept-move-constructor)
CallbackGuard& operator=(CallbackGuard&&);
CallbackGuard(CallbackGuard const&) = delete;
CallbackGuard& operator=(CallbackGuard const&) = delete;
/// @brief Call the contained callback, then delete it.
void callAndClear();
private:
void call();
std::function<void(void)> _callback;
};
}}
#endif

View File

@ -23,7 +23,7 @@
#ifndef ARANGOD_CLUSTER_REBOOTTRACKER_H
#define ARANGOD_CLUSTER_REBOOTTRACKER_H
#include "Cluster/ClusterTypes.h"
#include "Cluster/CallbackGuard.h"
#include "Basics/Mutex.h"
#include "Scheduler/Scheduler.h"
#include "Scheduler/SchedulerFeature.h"
@ -36,42 +36,6 @@
namespace arangodb {
namespace cluster {
/// @brief If constructed with a callback, the given callback will be called
/// exactly once: Either during destruction, or when the object is overwritten
/// (via operator=()), or when it's explicitly cleared. It's not copyable,
/// but movable.
class CallbackGuard {
public:
// Calls the callback given callback upon destruction.
// Allows only move semantics and no copy semantics.
CallbackGuard();
// IMPORTANT NOTE:
// The passed callback should not throw exceptions, they will not be caught
// here, but thrown by the destructor!
explicit CallbackGuard(std::function<void(void)> callback);
~CallbackGuard();
// Note that the move constructor of std::function is not noexcept until
// C++20. Thus we cannot mark the constructors here noexcept.
// NOLINTNEXTLINE(hicpp-noexcept-move,performance-noexcept-move-constructor)
CallbackGuard(CallbackGuard&& other);
// operator= additionally calls the _callback, and this can also throw.
// NOLINTNEXTLINE(hicpp-noexcept-move,performance-noexcept-move-constructor)
CallbackGuard& operator=(CallbackGuard&&);
CallbackGuard(CallbackGuard const&) = delete;
CallbackGuard& operator=(CallbackGuard const&) = delete;
/// @brief Call the contained callback, then delete it.
void callAndClear();
private:
void call();
std::function<void(void)> _callback;
};
// Note:
// Instances of this class must be destructed during shutdown before the
// scheduler is destroyed.

View File

@ -87,7 +87,7 @@ std::string const TTL("ttl");
using namespace std::chrono;
SynchronizeShard::SynchronizeShard(MaintenanceFeature& feature, ActionDescription const& desc)
: ActionBase(feature, desc) {
: ActionBase(feature, desc) {
std::stringstream error;
if (!desc.has(COLLECTION)) {
@ -125,8 +125,8 @@ SynchronizeShard::SynchronizeShard(MaintenanceFeature& feature, ActionDescriptio
SynchronizeShard::~SynchronizeShard() = default;
static std::stringstream& AppendShardInformationToMessage(
std::string const& database, std::string const& shard, std::string const& planId,
std::chrono::system_clock::time_point const& startTime, std::stringstream& msg) {
std::string const& database, std::string const& shard, std::string const& planId,
std::chrono::system_clock::time_point const& startTime, std::stringstream& msg) {
auto const endTime = system_clock::now();
msg << "local shard: '" << database << "/" << shard << "', "
<< "for central: '" << database << "/" << planId << "', "
@ -188,7 +188,7 @@ static arangodb::Result collectionCount(std::shared_ptr<arangodb::LogicalCollect
}
OperationResult opResult =
trx.count(collectionName, arangodb::transaction::CountType::Normal);
trx.count(collectionName, arangodb::transaction::CountType::Normal);
res = trx.finish(opResult.result);
if (res.fail()) {
@ -225,7 +225,7 @@ static arangodb::Result addShardFollower(
auto collection = vocbase->lookupCollection(shard);
if (collection == nullptr) {
std::string errorMsg(
"SynchronizeShard::addShardFollower: Failed to lookup collection ");
"SynchronizeShard::addShardFollower: Failed to lookup collection ");
errorMsg += shard;
LOG_TOPIC("4a8db", ERR, Logger::MAINTENANCE) << errorMsg;
return arangodb::Result(TRI_ERROR_ARANGO_DATA_SOURCE_NOT_FOUND, errorMsg);
@ -269,7 +269,7 @@ static arangodb::Result addShardFollower(
}
}
}
network::RequestOptions options;
options.database = database;
options.timeout = network::Timeout(timeout);
@ -297,7 +297,7 @@ static arangodb::Result addShardFollower(
return arangodb::Result();
} catch (std::exception const& e) {
std::string errorMsg(
"SynchronizeShard::addShardFollower: Failed to lookup database ");
"SynchronizeShard::addShardFollower: Failed to lookup database ");
errorMsg += database;
errorMsg += " exception: ";
errorMsg += e.what();
@ -414,92 +414,74 @@ static inline bool isStopping() {
return server.isStopping();
}
arangodb::Result SynchronizeShard::getReadLock(network::ConnectionPool* pool,
std::string const& endpoint,
std::string const& database,
std::string const& collection,
std::string const& clientId, uint64_t rlid,
bool soft, double timeout) {
if (pool == nullptr) { // nullptr only happens during controlled shutdown
arangodb::Result SynchronizeShard::getReadLock(
network::ConnectionPool* pool,
std::string const& endpoint, std::string const& database,
std::string const& collection, std::string const& clientId,
uint64_t rlid, bool soft, double timeout) {
// This function can be implemented in a more robust manner for server
// versions > 3.4. Starting with 3.4 the POST requests to the read lock API
// terminates the server side thread as soon as the lock request comes in.
// The POST request thus is answered immediately back to the caller.
// The servers (<=3.3) with lower versions hold the POST request for as long
// as the corresponding DELETE_REQ has not been successfully submitted.
using namespace std::chrono;
auto const start = steady_clock::now();
// nullptr only happens during controlled shutdown
if (pool == nullptr) {
return arangodb::Result(TRI_ERROR_SHUTTING_DOWN,
"startReadLockOnLeader: Shutting down");
"cancelReadLockOnLeader: Shutting down");
}
VPackBuilder bodyBuilder;
{
VPackObjectBuilder o(&bodyBuilder);
bodyBuilder.add(ID, VPackValue(std::to_string(rlid)));
bodyBuilder.add(COLLECTION, VPackValue(collection));
bodyBuilder.add(TTL, VPackValue(timeout));
bodyBuilder.add(StaticStrings::ReplicationSoftLockOnly, VPackValue(soft));
}
auto body = bodyBuilder.steal();
VPackBuilder body;
{ VPackObjectBuilder o(&body);
body.add(ID, VPackValue(std::to_string(rlid)));
body.add(COLLECTION, VPackValue(collection));
body.add(TTL, VPackValue(timeout));
body.add("serverId", VPackValue(arangodb::ServerState::instance()->getId()));
body.add(StaticStrings::RebootId, VPackValue(ServerState::instance()->getRebootId()));
body.add(StaticStrings::ReplicationSoftLockOnly, VPackValue(soft)); }
auto buf = body.steal();
auto const url = DB + database + REPL_HOLD_READ_LOCK;
// Try to POST the lock body. If POST fails, we should just exit and retry
// SynchroShard anew.
network::RequestOptions options;
options.database = database;
options.timeout = network::Timeout(timeout);
options.skipScheduler = true; // hack to speed up future.get()
options.timeout = network::Timeout(timeout);
auto res = network::sendRequest(
pool, endpoint, fuerte::RestVerb::Post,
url, *buf, options).get();
if (!res.fail() && res.response->statusCode() == fuerte::StatusOK) {
// Habemus clausum, we have a lock
return arangodb::Result();
}
LOG_TOPIC("cba32", DEBUG, Logger::MAINTENANCE)
<< "startReadLockOnLeader: couldn't POST lock body, "
<< network::fuerteToArangoErrorMessage(res) << ", giving up.";
// We MUSTN'T exit without trying to clean up a lock that was maybe acquired
if (res.error == fuerte::Error::CouldNotConnect) {
return arangodb::Result(
TRI_ERROR_INTERNAL,
"startReadLockOnLeader: couldn't POST lock body, giving up.");
}
auto dummy = network::sendRequest(pool, endpoint, fuerte::RestVerb::Post, REPL_HOLD_READ_LOCK,
*body, options);
// Intentionally do not look at the outcome, even in case of an error
// we must make sure that the read lock on the leader is not active!
// This is done automatically below.
size_t const maxTries = 9; // 511s max
double sleepTime = 1.0;
size_t count = 0;
while (++count < maxTries) { // wait for some time until read lock established:
if (isStopping()) {
return arangodb::Result(TRI_ERROR_SHUTTING_DOWN);
}
// Now check that we hold the read lock:
auto res = network::sendRequest(pool, endpoint, fuerte::RestVerb::Put, REPL_HOLD_READ_LOCK,
*body, options)
.get();
if (res.ok() && res.response->statusCode() == fuerte::StatusOK) {
auto const slice = res.response->slice();
TRI_ASSERT(slice.isObject());
VPackSlice lockHeld = slice.get("lockHeld");
if (lockHeld.isBoolean() && lockHeld.getBool()) {
return arangodb::Result();
}
LOG_TOPIC("b681f", DEBUG, Logger::MAINTENANCE)
<< "startReadLockOnLeader: Lock not yet acquired...";
} else {
if (res.ok() && res.response->statusCode() == fuerte::StatusNotFound) {
auto const slice = res.response->slice();
if (slice.isObject()) {
VPackSlice s = slice.get(StaticStrings::ErrorNum);
if (s.isNumber()) {
int errorNum = s.getNumber<int>();
if (errorNum == TRI_ERROR_ARANGO_DATABASE_NOT_FOUND) {
// database is gone. we can now give up
break;
}
}
}
// fall-through to other cases intentional here
}
std::string message = network::fuerteToArangoErrorMessage(res);
LOG_TOPIC("a82bc", DEBUG, Logger::MAINTENANCE)
<< "startReadLockOnLeader: Do not see read lock yet:" << message;
}
std::this_thread::sleep_for(duration<double>(sleepTime));
sleepTime *= 2.0;
double timeLeft =
double(timeout) - duration<double>(steady_clock::now()-start).count() ;
if (timeLeft < 60.0) {
timeLeft = 60.0;
}
LOG_TOPIC("75e2b", ERR, Logger::MAINTENANCE) << "startReadLockOnLeader: giving up";
// Ambiguous POST, we'll try to DELETE a potentially acquired lock
try {
auto r = network::sendRequest(pool, endpoint, fuerte::RestVerb::Delete, REPL_HOLD_READ_LOCK,
*body, options)
auto r = network::sendRequest(pool, endpoint, fuerte::RestVerb::Delete, url,
*buf, options)
.get();
if (r.fail() || r.response->statusCode() != fuerte::StatusOK) {
std::string addendum = network::fuerteToArangoErrorMessage(r);
@ -509,16 +491,15 @@ arangodb::Result SynchronizeShard::getReadLock(network::ConnectionPool* pool,
}
} catch (std::exception const& e) {
LOG_TOPIC("7fcc9", ERR, Logger::MAINTENANCE)
<< "startReadLockOnLeader: exception in cancel: " << e.what();
<< "startReadLockOnLeader: exception in cancel: " << e.what();
}
return arangodb::Result(TRI_ERROR_CLUSTER_TIMEOUT,
"startReadLockOnLeader: giving up");
return arangodb::Result(
TRI_ERROR_CLUSTER_TIMEOUT, "startReadLockOnLeader: giving up");
}
arangodb::Result SynchronizeShard::startReadLockOnLeader(
std::string const& endpoint, std::string const& database, std::string const& collection,
std::string const& clientId, uint64_t& rlid, bool soft, double timeout) {
std::string const& endpoint, std::string const& database, std::string const& collection,
std::string const& clientId, uint64_t& rlid, bool soft, double timeout) {
// Read lock id
rlid = 0;
NetworkFeature& nf = _feature.server().getFeature<NetworkFeature>();
@ -542,6 +523,7 @@ static arangodb::ResultT<SyncerId> replicationSynchronize(
std::shared_ptr<arangodb::LogicalCollection> const& col,
VPackSlice const& config, std::string const& clientInfoString,
ApplierType applierType, std::shared_ptr<VPackBuilder> sy) {
auto& vocbase = col->vocbase();
auto database = vocbase.name();
@ -607,17 +589,17 @@ static arangodb::ResultT<SyncerId> replicationSynchronize(
} catch (arangodb::basics::Exception const& ex) {
std::string s("cannot sync from remote endpoint: ");
s += ex.what() + std::string(". last progress message was '") +
syncer->progress() + "'";
syncer->progress() + "'";
return Result(ex.code(), s);
} catch (std::exception const& ex) {
std::string s("cannot sync from remote endpoint: ");
s += ex.what() + std::string(". last progress message was '") +
syncer->progress() + "'";
syncer->progress() + "'";
return Result(TRI_ERROR_INTERNAL, s);
} catch (...) {
std::string s(
"cannot sync from remote endpoint: unknown exception. last progress "
"message was '");
"cannot sync from remote endpoint: unknown exception. last progress "
"message was '");
s += syncer->progress() + "'";
return Result(TRI_ERROR_INTERNAL, s);
}
@ -833,7 +815,7 @@ bool SynchronizeShard::first() {
if (asResult.ok()) {
if (Logger::isEnabled(LogLevel::DEBUG, Logger::MAINTENANCE)) {
std::stringstream msg;
msg << "synchronizeOneShard: shortcut worked, done, ";
msg << "SynchronizeShard: shortcut worked, done, ";
AppendShardInformationToMessage(database, shard, planId, startTime, msg);
LOG_TOPIC("f4a5b", DEBUG, Logger::MAINTENANCE) << msg.str();
}
@ -861,7 +843,7 @@ bool SynchronizeShard::first() {
if (isStopping()) {
std::string errorMessage(
"synchronizeOneShard: synchronization failed for shard ");
"SynchronizeShard: synchronization failed for shard ");
errorMessage += shard + ": shutdown in progress, giving up";
LOG_TOPIC("a0f9a", INFO, Logger::MAINTENANCE) << errorMessage;
_result.reset(TRI_ERROR_SHUTTING_DOWN, errorMessage);
@ -933,9 +915,7 @@ bool SynchronizeShard::first() {
if (collections.length() == 0 || collections[0].get("name").copyString() != shard) {
std::stringstream error;
error
<< "shard " << shard
<< " seems to be gone from leader, this "
error << "shard " << shard << " seems to be gone from leader, this "
"can happen if a collection was dropped during synchronization!";
LOG_TOPIC("664ae", WARN, Logger::MAINTENANCE) << "SynchronizeOneShard: " << error.str();
_result.reset(TRI_ERROR_INTERNAL, error.str());
@ -943,13 +923,13 @@ bool SynchronizeShard::first() {
}
auto lastTick =
arangodb::basics::VelocyPackHelper::getNumericValue<TRI_voc_tick_t>(sy, LAST_LOG_TICK,
0);
arangodb::basics::VelocyPackHelper::getNumericValue<TRI_voc_tick_t>(
sy, LAST_LOG_TICK, 0);
VPackBuilder builder;
ResultT<TRI_voc_tick_t> tickResult =
catchupWithReadLock(ep, database, *collection, clientId, shard,
leader, lastTick, builder);
catchupWithReadLock(ep, database, *collection, clientId, shard,
leader, lastTick, builder);
if (!tickResult.ok()) {
LOG_TOPIC("0a4d4", INFO, Logger::MAINTENANCE) << syncRes.errorMessage();
_result.reset(tickResult.result());
@ -997,9 +977,9 @@ bool SynchronizeShard::first() {
}
ResultT<TRI_voc_tick_t> SynchronizeShard::catchupWithReadLock(
std::string const& ep, std::string const& database, LogicalCollection const& collection,
std::string const& clientId, std::string const& shard,
std::string const& leader, TRI_voc_tick_t lastLogTick, VPackBuilder& builder) {
std::string const& ep, std::string const& database, LogicalCollection const& collection,
std::string const& clientId, std::string const& shard,
std::string const& leader, TRI_voc_tick_t lastLogTick, VPackBuilder& builder) {
bool didTimeout = true;
int tries = 0;
double timeout = 300.0;
@ -1008,7 +988,7 @@ ResultT<TRI_voc_tick_t> SynchronizeShard::catchupWithReadLock(
if (isStopping()) {
std::string errorMessage =
"synchronizeOneShard: startReadLockOnLeader (soft): shutting down";
"SynchronizeShard: startReadLockOnLeader (soft): shutting down";
return ResultT<TRI_voc_tick_t>::error(TRI_ERROR_SHUTTING_DOWN, errorMessage);
}
@ -1023,7 +1003,7 @@ ResultT<TRI_voc_tick_t> SynchronizeShard::catchupWithReadLock(
clientId, lockJobId, true, timeout);
if (!res.ok()) {
std::string errorMessage =
"synchronizeOneShard: error in startReadLockOnLeader (soft):" + res.errorMessage();
"SynchronizeShard: error in startReadLockOnLeader (soft):" + res.errorMessage();
return ResultT<TRI_voc_tick_t>::error(TRI_ERROR_INTERNAL, errorMessage);
}
@ -1036,8 +1016,8 @@ ResultT<TRI_voc_tick_t> SynchronizeShard::catchupWithReadLock(
if (!res.ok()) {
LOG_TOPIC("b15ee", INFO, Logger::MAINTENANCE)
<< "Could not cancel soft read lock on leader: " << res.errorMessage();
}
});
}
});
LOG_TOPIC("5eb37", DEBUG, Logger::MAINTENANCE) << "lockJobId: " << lockJobId;
@ -1066,7 +1046,7 @@ ResultT<TRI_voc_tick_t> SynchronizeShard::catchupWithReadLock(
if (!res.ok()) {
std::string errorMessage(
"synchronizeOneshard: error in syncCollectionCatchup: ");
"synchronizeOneshard: error in syncCollectionCatchup: ");
errorMessage += res.errorMessage();
return ResultT<TRI_voc_tick_t>::error(TRI_ERROR_INTERNAL, errorMessage);
}
@ -1087,7 +1067,7 @@ ResultT<TRI_voc_tick_t> SynchronizeShard::catchupWithReadLock(
lastLogTick = tickReached;
if (didTimeout) {
LOG_TOPIC("e516e", INFO, Logger::MAINTENANCE)
<< "Renewing softLock for " << shard << " on leader: " << leader;
<< "Renewing softLock for " << shard << " on leader: " << leader;
}
}
if (didTimeout) {
@ -1110,7 +1090,7 @@ Result SynchronizeShard::catchupWithExclusiveLock(
lockJobId, false);
if (!res.ok()) {
std::string errorMessage =
"synchronizeOneShard: error in startReadLockOnLeader (hard):" + res.errorMessage();
"SynchronizeShard: error in startReadLockOnLeader (hard):" + res.errorMessage();
return {TRI_ERROR_INTERNAL, errorMessage};
}
auto readLockGuard = arangodb::scopeGuard([&, this]() {
@ -1122,8 +1102,8 @@ Result SynchronizeShard::catchupWithExclusiveLock(
if (!res.ok()) {
LOG_TOPIC("067a8", INFO, Logger::MAINTENANCE)
<< "Could not cancel hard read lock on leader: " << res.errorMessage();
}
});
}
});
LOG_TOPIC("d76cb", DEBUG, Logger::MAINTENANCE) << "lockJobId: " << lockJobId;
@ -1143,7 +1123,7 @@ Result SynchronizeShard::catchupWithExclusiveLock(
if (!res.ok()) {
std::string errorMessage(
"synchronizeOneshard: error in syncCollectionFinalize: ");
"synchronizeOneshard: error in syncCollectionFinalize: ");
errorMessage += res.errorMessage();
return {TRI_ERROR_INTERNAL, errorMessage};
}
@ -1155,7 +1135,7 @@ Result SynchronizeShard::catchupWithExclusiveLock(
if (!res.ok()) {
std::string errorMessage(
"synchronizeOneshard: error in addShardFollower: ");
"synchronizeOneshard: error in addShardFollower: ");
errorMessage += res.errorMessage();
return {TRI_ERROR_INTERNAL, errorMessage};
}
@ -1169,9 +1149,12 @@ Result SynchronizeShard::catchupWithExclusiveLock(
void SynchronizeShard::setState(ActionState state) {
if ((COMPLETE == state || FAILED == state) && _state != state) {
TRI_ASSERT(_description.has("shard"));
_feature.incShardVersion(_description.get("shard"));
auto const& shard = _description.get("shard");
if (COMPLETE == state) {
LOG_TOPIC("50827", INFO, Logger::MAINTENANCE)
<< "SynchronizeShard: synchronization completed for shard " << shard;
}
_feature.incShardVersion(shard);
}
ActionBase::setState(state);
}

View File

@ -36,6 +36,7 @@
#include "Cluster/ClusterHelpers.h"
#include "Cluster/ClusterMethods.h"
#include "Cluster/FollowerInfo.h"
#include "Cluster/RebootTracker.h"
#include "Cluster/ResignShardLeadership.h"
#include "GeneralServer/AuthenticationFeature.h"
#include "Indexes/Index.h"
@ -73,6 +74,7 @@
using namespace arangodb;
using namespace arangodb::basics;
using namespace arangodb::rest;
using namespace arangodb::cluster;
namespace {
std::string const dataString("data");
@ -2529,16 +2531,36 @@ void RestReplicationHandler::handleCommandHoldReadLockCollection() {
return;
}
RebootId rebootId(0);
std::string serverId;
if (!body.isObject()) {
generateError(rest::ResponseCode::BAD, TRI_ERROR_HTTP_BAD_PARAMETER,
"body needs to be an object with attributes 'collection', "
"'ttl' and 'id'");
return;
}
VPackSlice collection = body.get("collection");
VPackSlice ttlSlice = body.get("ttl");
VPackSlice idSlice = body.get("id");
VPackSlice const collection = body.get("collection");
VPackSlice const ttlSlice = body.get("ttl");
VPackSlice const idSlice = body.get("id");
if (body.hasKey(StaticStrings::RebootId)) {
if (body.get(StaticStrings::RebootId).isInteger()) {
if (body.hasKey("serverId") && body.get("serverId").isString()) {
rebootId = RebootId(body.get(StaticStrings::RebootId).getNumber<uint64_t>());
serverId = body.get("serverId").copyString();
} else {
generateError(rest::ResponseCode::BAD, TRI_ERROR_HTTP_BAD_PARAMETER,
"'rebootId' must be accompanied by string attribute 'serverId'");
return;
}
} else {
generateError(rest::ResponseCode::BAD, TRI_ERROR_HTTP_BAD_PARAMETER,
"'rebootId' must be an integer attribute");
return;
}
}
if (!collection.isString() || !ttlSlice.isNumber() || !idSlice.isString()) {
generateError(rest::ResponseCode::BAD, TRI_ERROR_HTTP_BAD_PARAMETER,
@ -2584,7 +2606,7 @@ void RestReplicationHandler::handleCommandHoldReadLockCollection() {
LOG_TOPIC("4fac2", DEBUG, Logger::REPLICATION)
<< "Attempt to create a Lock: " << id << " for shard: " << _vocbase.name()
<< "/" << col->name() << " of type: " << (doSoftLock ? "soft" : "hard");
Result res = createBlockingTransaction(id, *col, ttl, lockType);
Result res = createBlockingTransaction(id, *col, ttl, lockType, rebootId, serverId);
if (!res.ok()) {
generateError(res);
return;
@ -2918,7 +2940,9 @@ ReplicationApplier* RestReplicationHandler::getApplier(bool& global) {
Result RestReplicationHandler::createBlockingTransaction(aql::QueryId id,
LogicalCollection& col, double ttl,
AccessMode::Type access) const {
AccessMode::Type access,
RebootId const& rebootId,
std::string const& serverId) {
// This is a constant JSON structure for Queries.
// we actually do not need a plan, as we only want the query registry to have
// a hold of our transaction
@ -2940,6 +2964,7 @@ Result RestReplicationHandler::createBlockingTransaction(aql::QueryId id,
{
auto ctx = transaction::StandaloneContext::Create(_vocbase);
auto trx = std::make_shared<SingleCollectionTransaction>(ctx, col, access);
query->setTransactionContext(ctx);
// Inject will take over responsiblilty of transaction, even on error case.
query->injectTransaction(std::move(trx));
@ -2950,8 +2975,29 @@ Result RestReplicationHandler::createBlockingTransaction(aql::QueryId id,
TRI_ASSERT(isLockHeld(id).is(TRI_ERROR_HTTP_NOT_FOUND));
ClusterInfo& ci = server().getFeature<ClusterFeature>().clusterInfo();
std::string vn = _vocbase.name();
try {
queryRegistry->insert(id, query.get(), ttl, true, true);
std::function<void(void)> f =
[=]() {
try {
// Code does not matter, read only access, so we can roll back.
QueryRegistryFeature::registry()->destroy(vn, id, TRI_ERROR_QUERY_KILLED, false);
} catch (...) {
// All errors that show up here can only be
// triggered if the query is destroyed in between.
}
};
std::string comment = std::string("SynchronizeShard from ") + serverId +
" for " + col.name() + " access mode " + AccessMode::typeString(access);
auto rGuard = std::make_unique<CallbackGuard>(
ci.rebootTracker().callMeOnChange(
RebootTracker::PeerState(serverId, rebootId), f, comment));
queryRegistry->insert(id, query.get(), ttl, true, true, std::move(rGuard));
} catch (...) {
// For compatibility we only return this error
return {TRI_ERROR_TRANSACTION_INTERNAL, "cannot begin read transaction"};

View File

@ -29,6 +29,7 @@
#include "Basics/Common.h"
#include "Basics/Result.h"
#include "Cluster/ResultT.h"
#include "Cluster/ClusterTypes.h"
#include "Replication/Syncer.h"
#include "Replication/common-defines.h"
#include "RestHandler/RestVocbaseBaseHandler.h"
@ -476,7 +477,9 @@ class RestReplicationHandler : public RestVocbaseBaseHandler {
/// the given time to live.
//////////////////////////////////////////////////////////////////////////////
Result createBlockingTransaction(aql::QueryId id, LogicalCollection& col,
double ttl, AccessMode::Type access) const;
double ttl, AccessMode::Type access,
RebootId const& rebootId,
std::string const& serverId);
//////////////////////////////////////////////////////////////////////////////
/// @brief Test if we already have the read-lock

View File

@ -37,6 +37,7 @@
#include "Aql/ExecutionNode.h"
#include "Aql/Query.h"
#include "Aql/QueryRegistry.h"
#include "Cluster/RebootTracker.h"
#include "Transaction/Methods.h"
using namespace arangodb;
@ -273,7 +274,8 @@ TEST(EngineInfoContainerTest,
// Mock the Registry
fakeit::When(Method(mockRegistry, insert))
.Do([&](QueryId id, Query* query, double timeout, bool isPrepared, bool keepLease) {
.Do([&](QueryId id, Query* query, double timeout, bool isPrepared, bool keepLease,
std::unique_ptr<arangodb::cluster::CallbackGuard>&) {
ASSERT_NE(id, 0);
ASSERT_NE(query, nullptr);
ASSERT_TRUE(isPrepared);
@ -497,7 +499,8 @@ TEST(EngineInfoContainerTest, snippets_are_a_stack_insert_node_always_into_top_s
// handled first. With same fakeit magic we could make this ordering
// independent which is is fine as well for the production code.
fakeit::When(Method(mockRegistry, insert))
.Do([&](QueryId id, Query* query, double timeout, bool isPrepared, bool keepLease) {
.Do([&](QueryId id, Query* query, double timeout, bool isPrepared, bool keepLease,
std::unique_ptr<arangodb::cluster::CallbackGuard>&) {
ASSERT_NE(id, 0);
ASSERT_NE(query, nullptr);
ASSERT_TRUE(isPrepared);
@ -506,7 +509,8 @@ TEST(EngineInfoContainerTest, snippets_are_a_stack_insert_node_always_into_top_s
ASSERT_EQ(query, &queryClone);
secondId = id;
})
.Do([&](QueryId id, Query* query, double timeout, bool isPrepared, bool keepLease) {
.Do([&](QueryId id, Query* query, double timeout, bool isPrepared, bool keepLease,
std::unique_ptr<arangodb::cluster::CallbackGuard>&) {
ASSERT_NE(id, 0);
ASSERT_NE(query, nullptr);
ASSERT_EQ(timeout, 600.0);
@ -683,7 +687,8 @@ TEST(EngineInfoContainerTest, error_cases_cloning_of_a_query_fails_throws_an_err
// Mock the Registry
fakeit::When(Method(mockRegistry, insert))
.Do([&](QueryId id, Query* query, double timeout, bool isPrepared, bool keepLease) {
.Do([&](QueryId id, Query* query, double timeout, bool isPrepared, bool keepLease,
std::unique_ptr<arangodb::cluster::CallbackGuard>&) {
ASSERT_NE(id, 0);
ASSERT_NE(query, nullptr);
ASSERT_EQ(timeout, 600.0);
@ -847,7 +852,8 @@ TEST(EngineInfoContainerTest, error_cases_cloning_of_a_query_fails_returns_a_nul
// Mock the Registry
fakeit::When(Method(mockRegistry, insert))
.Do([&](QueryId id, Query* query, double timeout, bool isPrepared, bool keepLease) {
.Do([&](QueryId id, Query* query, double timeout, bool isPrepared, bool keepLease,
std::unique_ptr<arangodb::cluster::CallbackGuard>&) {
ASSERT_NE(id, 0);
ASSERT_NE(query, nullptr);
ASSERT_EQ(timeout, 600.0);