/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you 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 (C) 2015 ScyllaDB
*
* Modified by ScyllaDB
*/
/*
* This file is part of Scylla.
*
* Scylla is free software: you can redistribute it and/or modify
* it under the terms of the GNU Affero General Public License as published by
* the Free Software Foundation, either version 3 of the License, or
* (at your option) any later version.
*
* Scylla is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
* GNU General Public License for more details.
*
* You should have received a copy of the GNU General Public License
* along with Scylla. If not, see .
*/
#include "partition_range_compat.hh"
#include "db/consistency_level.hh"
#include "db/commitlog/commitlog.hh"
#include "storage_proxy.hh"
#include "unimplemented.hh"
#include "mutation.hh"
#include "frozen_mutation.hh"
#include "supervisor.hh"
#include "query_result_merger.hh"
#include
#include "message/messaging_service.hh"
#include "gms/failure_detector.hh"
#include "gms/gossiper.hh"
#include "storage_service.hh"
#include
#include "db/read_repair_decision.hh"
#include "db/config.hh"
#include "db/batchlog_manager.hh"
#include "db/hints/manager.hh"
#include "db/system_keyspace.hh"
#include "exceptions/exceptions.hh"
#include
#include
#include
#include
#include
#include
#include
#include
#include
#include
#include
#include
#include
#include
#include
#include
#include
#include "utils/latency.hh"
#include "schema.hh"
#include "schema_registry.hh"
#include "utils/joinpoint.hh"
#include
#include
#include
#include "db/timeout_clock.hh"
#include "multishard_mutation_query.hh"
#include "database.hh"
#include "db/consistency_level_validations.hh"
#include "cdc/log.hh"
#include "cdc/stats.hh"
#include "cdc/cdc_options.hh"
#include "utils/histogram_metrics_helper.hh"
#include "service/paxos/prepare_summary.hh"
#include "service/paxos/proposal.hh"
#include "locator/token_metadata.hh"
#include "seastar/core/coroutine.hh"
namespace bi = boost::intrusive;
namespace service {
static logging::logger slogger("storage_proxy");
static logging::logger qlogger("query_result");
static logging::logger mlogger("mutation_data");
namespace storage_proxy_stats {
static const sstring COORDINATOR_STATS_CATEGORY("storage_proxy_coordinator");
static const sstring REPLICA_STATS_CATEGORY("storage_proxy_replica");
static const seastar::metrics::label op_type_label("op_type");
static const seastar::metrics::label scheduling_group_label("scheduling_group_name");
seastar::metrics::label_instance current_scheduling_group_label() {
return scheduling_group_label(current_scheduling_group().name());
}
}
thread_local uint64_t paxos_response_handler::next_id = 0;
distributed _the_storage_proxy;
using namespace exceptions;
using fbu = utils::fb_utilities;
static inline
query::digest_algorithm digest_algorithm(service::storage_proxy& proxy) {
return proxy.features().cluster_supports_digest_for_null_values()
? query::digest_algorithm::xxHash
: query::digest_algorithm::legacy_xxHash_without_null_digest;
}
static inline
const dht::token& start_token(const dht::partition_range& r) {
static const dht::token min_token = dht::minimum_token();
return r.start() ? r.start()->value().token() : min_token;
}
static inline
const dht::token& end_token(const dht::partition_range& r) {
static const dht::token max_token = dht::maximum_token();
return r.end() ? r.end()->value().token() : max_token;
}
static inline
sstring get_dc(gms::inet_address ep) {
auto& snitch_ptr = locator::i_endpoint_snitch::get_local_snitch_ptr();
return snitch_ptr->get_datacenter(ep);
}
static inline
sstring get_local_dc() {
auto local_addr = utils::fb_utilities::get_broadcast_address();
return get_dc(local_addr);
}
unsigned storage_proxy::cas_shard(const schema& s, dht::token token) {
return dht::shard_of(s, token);
}
class mutation_holder {
protected:
size_t _size = 0;
schema_ptr _schema;
public:
virtual ~mutation_holder() {}
virtual bool store_hint(db::hints::manager& hm, gms::inet_address ep, tracing::trace_state_ptr tr_state) = 0;
virtual future<> apply_locally(storage_proxy& sp, storage_proxy::clock_type::time_point timeout,
tracing::trace_state_ptr tr_state) = 0;
virtual future<> apply_remotely(storage_proxy& sp, gms::inet_address ep, std::vector&& forward,
storage_proxy::response_id_type response_id, storage_proxy::clock_type::time_point timeout,
tracing::trace_state_ptr tr_state) = 0;
virtual bool is_shared() = 0;
size_t size() const {
return _size;
}
const schema_ptr& schema() {
return _schema;
}
// called only when all replicas replied
virtual void release_mutation() = 0;
// called when reply is received
// alllows mutation holder to have its own accounting
virtual void reply(gms::inet_address ep) {};
};
// different mutation for each destination (for read repairs)
class per_destination_mutation : public mutation_holder {
std::unordered_map> _mutations;
dht::token _token;
public:
per_destination_mutation(const std::unordered_map>& mutations) {
for (auto&& m : mutations) {
lw_shared_ptr fm;
if (m.second) {
_schema = m.second.value().schema();
_token = m.second.value().token();
fm = make_lw_shared(freeze(m.second.value()));
_size += fm->representation().size();
}
_mutations.emplace(m.first, std::move(fm));
}
}
virtual bool store_hint(db::hints::manager& hm, gms::inet_address ep, tracing::trace_state_ptr tr_state) override {
auto m = _mutations[ep];
if (m) {
return hm.store_hint(ep, _schema, std::move(m), tr_state);
} else {
return false;
}
}
virtual future<> apply_locally(storage_proxy& sp, storage_proxy::clock_type::time_point timeout,
tracing::trace_state_ptr tr_state) override {
auto m = _mutations[utils::fb_utilities::get_broadcast_address()];
if (m) {
tracing::trace(tr_state, "Executing a mutation locally");
return sp.mutate_locally(_schema, *m, std::move(tr_state), db::commitlog::force_sync::no, timeout);
}
return make_ready_future<>();
}
virtual future<> apply_remotely(storage_proxy& sp, gms::inet_address ep, std::vector&& forward,
storage_proxy::response_id_type response_id, storage_proxy::clock_type::time_point timeout,
tracing::trace_state_ptr tr_state) override {
auto m = _mutations[ep];
if (m) {
tracing::trace(tr_state, "Sending a mutation to /{}", ep);
return sp._messaging.send_mutation(netw::messaging_service::msg_addr{ep, 0}, timeout, *m,
std::move(forward), utils::fb_utilities::get_broadcast_address(), this_shard_id(),
response_id, tracing::make_trace_info(tr_state));
}
sp.got_response(response_id, ep, std::nullopt);
return make_ready_future<>();
}
virtual bool is_shared() override {
return false;
}
virtual void release_mutation() override {
for (auto&& m : _mutations) {
if (m.second) {
m.second.release();
}
}
}
dht::token& token() {
return _token;
}
};
// same mutation for each destination
class shared_mutation : public mutation_holder {
protected:
lw_shared_ptr _mutation;
public:
explicit shared_mutation(frozen_mutation_and_schema&& fm_a_s)
: _mutation(make_lw_shared(std::move(fm_a_s.fm))) {
_size = _mutation->representation().size();
_schema = std::move(fm_a_s.s);
}
explicit shared_mutation(const mutation& m) : shared_mutation(frozen_mutation_and_schema{freeze(m), m.schema()}) {
}
virtual bool store_hint(db::hints::manager& hm, gms::inet_address ep, tracing::trace_state_ptr tr_state) override {
return hm.store_hint(ep, _schema, _mutation, tr_state);
}
virtual future<> apply_locally(storage_proxy& sp, storage_proxy::clock_type::time_point timeout,
tracing::trace_state_ptr tr_state) override {
tracing::trace(tr_state, "Executing a mutation locally");
return sp.mutate_locally(_schema, *_mutation, std::move(tr_state), db::commitlog::force_sync::no, timeout);
}
virtual future<> apply_remotely(storage_proxy& sp, gms::inet_address ep, std::vector&& forward,
storage_proxy::response_id_type response_id, storage_proxy::clock_type::time_point timeout,
tracing::trace_state_ptr tr_state) override {
tracing::trace(tr_state, "Sending a mutation to /{}", ep);
return sp._messaging.send_mutation(netw::messaging_service::msg_addr{ep, 0}, timeout, *_mutation,
std::move(forward), utils::fb_utilities::get_broadcast_address(), this_shard_id(),
response_id, tracing::make_trace_info(tr_state));
}
virtual bool is_shared() override {
return true;
}
virtual void release_mutation() override {
_mutation.release();
}
};
// shared mutation, but gets sent as a hint
class hint_mutation : public shared_mutation {
public:
using shared_mutation::shared_mutation;
virtual bool store_hint(db::hints::manager& hm, gms::inet_address ep, tracing::trace_state_ptr tr_state) override {
throw std::runtime_error("Attempted to store a hint for a hint");
}
virtual future<> apply_locally(storage_proxy& sp, storage_proxy::clock_type::time_point timeout,
tracing::trace_state_ptr tr_state) override {
// A hint will be sent to all relevant endpoints when the endpoint it was originally intended for
// becomes unavailable - this might include the current node
return sp.mutate_hint(_schema, *_mutation, std::move(tr_state), timeout);
}
virtual future<> apply_remotely(storage_proxy& sp, gms::inet_address ep, std::vector&& forward,
storage_proxy::response_id_type response_id, storage_proxy::clock_type::time_point timeout,
tracing::trace_state_ptr tr_state) override {
tracing::trace(tr_state, "Sending a hint to /{}", ep);
return sp._messaging.send_hint_mutation(netw::messaging_service::msg_addr{ep, 0}, timeout, *_mutation,
std::move(forward), utils::fb_utilities::get_broadcast_address(), this_shard_id(),
response_id, tracing::make_trace_info(tr_state));
}
};
class cas_mutation : public mutation_holder {
lw_shared_ptr _proposal;
shared_ptr _handler;
public:
explicit cas_mutation(lw_shared_ptr proposal, schema_ptr s, shared_ptr handler)
: _proposal(std::move(proposal)), _handler(std::move(handler)) {
_size = _proposal->update.representation().size();
_schema = std::move(s);
}
virtual bool store_hint(db::hints::manager& hm, gms::inet_address ep, tracing::trace_state_ptr tr_state) override {
return false; // CAS does not save hints yet
}
virtual future<> apply_locally(storage_proxy& sp, storage_proxy::clock_type::time_point timeout,
tracing::trace_state_ptr tr_state) override {
tracing::trace(tr_state, "Executing a learn locally");
return paxos::paxos_state::learn(_schema, *_proposal, timeout, tr_state);
}
virtual future<> apply_remotely(storage_proxy& sp, gms::inet_address ep, std::vector&& forward,
storage_proxy::response_id_type response_id, storage_proxy::clock_type::time_point timeout,
tracing::trace_state_ptr tr_state) override {
tracing::trace(tr_state, "Sending a learn to /{}", ep);
return sp._messaging.send_paxos_learn(netw::messaging_service::msg_addr{ep, 0}, timeout,
*_proposal, std::move(forward), utils::fb_utilities::get_broadcast_address(),
this_shard_id(), response_id, tracing::make_trace_info(tr_state));
}
virtual bool is_shared() override {
return true;
}
virtual void release_mutation() override {
_proposal.release();
}
virtual void reply(gms::inet_address ep) override {
// The handler will be set for "learn", but not for PAXOS repair
// since repair may not include all replicas
if (_handler) {
if (_handler->learned(ep)) {
// It's OK to start PRUNE while LEARN is still in progress: LEARN
// doesn't read any data from system.paxos, and PRUNE tombstone
// will cover LEARNed value even if it arrives out of order.
_handler->prune(_proposal->ballot);
}
}
};
};
class abstract_write_response_handler : public seastar::enable_shared_from_this {
protected:
using error = storage_proxy::error;
storage_proxy::response_id_type _id;
promise<> _ready; // available when cl is achieved
shared_ptr _proxy;
tracing::trace_state_ptr _trace_state;
db::consistency_level _cl;
size_t _total_block_for = 0;
db::write_type _type;
std::unique_ptr _mutation_holder;
std::unordered_set _targets; // who we sent this mutation to
// added dead_endpoints as a memeber here as well. This to be able to carry the info across
// calls in helper methods in a convinient way. Since we hope this will be empty most of the time
// it should not be a huge burden. (flw)
std::vector _dead_endpoints;
size_t _cl_acks = 0;
bool _cl_achieved = false;
bool _throttled = false;
error _error = error::NONE;
size_t _failed = 0; // only failures that may impact consistency
size_t _all_failures = 0; // total amount of failures
size_t _total_endpoints = 0;
storage_proxy::write_stats& _stats;
lw_shared_ptr _cdc_operation_result_tracker;
timer _expire_timer;
service_permit _permit; // holds admission permit until operation completes
protected:
virtual bool waited_for(gms::inet_address from) = 0;
void signal(gms::inet_address from) {
_mutation_holder->reply(from);
if (waited_for(from)) {
signal();
}
}
public:
abstract_write_response_handler(shared_ptr p, keyspace& ks, db::consistency_level cl, db::write_type type,
std::unique_ptr mh, std::unordered_set targets, tracing::trace_state_ptr trace_state,
storage_proxy::write_stats& stats, service_permit permit, size_t pending_endpoints = 0, std::vector dead_endpoints = {})
: _id(p->get_next_response_id()), _proxy(std::move(p)), _trace_state(trace_state), _cl(cl), _type(type), _mutation_holder(std::move(mh)), _targets(std::move(targets)),
_dead_endpoints(std::move(dead_endpoints)), _stats(stats), _expire_timer([this] { timeout_cb(); }), _permit(std::move(permit)) {
// original comment from cassandra:
// during bootstrap, include pending endpoints in the count
// or we may fail the consistency level guarantees (see #833, #8058)
_total_block_for = db::block_for(ks, _cl) + pending_endpoints;
++_stats.writes;
}
virtual ~abstract_write_response_handler() {
--_stats.writes;
if (_cl_achieved) {
if (_throttled) {
_ready.set_value();
} else {
_stats.background_writes--;
_proxy->_global_stats.background_write_bytes -= _mutation_holder->size();
_proxy->unthrottle();
}
} else {
if (_error == error::TIMEOUT) {
_ready.set_exception(mutation_write_timeout_exception(get_schema()->ks_name(), get_schema()->cf_name(), _cl, _cl_acks, _total_block_for, _type));
} else if (_error == error::FAILURE) {
_ready.set_exception(mutation_write_failure_exception(get_schema()->ks_name(), get_schema()->cf_name(), _cl, _cl_acks, _failed, _total_block_for, _type));
}
if (_cdc_operation_result_tracker) {
_cdc_operation_result_tracker->on_mutation_failed();
}
}
}
bool is_counter() const {
return _type == db::write_type::COUNTER;
}
void set_cdc_operation_result_tracker(lw_shared_ptr tracker) {
_cdc_operation_result_tracker = std::move(tracker);
}
// While delayed, a request is not throttled.
void unthrottle() {
_stats.background_writes++;
_proxy->_global_stats.background_write_bytes += _mutation_holder->size();
_throttled = false;
_ready.set_value();
}
void signal(size_t nr = 1) {
_cl_acks += nr;
if (!_cl_achieved && _cl_acks >= _total_block_for) {
_cl_achieved = true;
delay(get_trace_state(), [] (abstract_write_response_handler* self) {
if (self->_proxy->need_throttle_writes()) {
self->_throttled = true;
self->_proxy->_throttled_writes.push_back(self->_id);
++self->_stats.throttled_writes;
} else {
self->unthrottle();
}
});
}
}
virtual bool failure(gms::inet_address from, size_t count, error err) {
if (waited_for(from)) {
_failed += count;
if (_total_block_for + _failed > _total_endpoints) {
_error = err;
delay(get_trace_state(), [] (abstract_write_response_handler*) { });
return true;
}
}
return false;
}
void on_timeout() {
if (_cl_achieved) {
slogger.trace("Write is not acknowledged by {} replicas after achieving CL", get_targets());
}
_error = error::TIMEOUT;
// We don't delay request completion after a timeout, but its possible we are currently delaying.
}
// return true on last ack
bool response(gms::inet_address from) {
auto it = _targets.find(from);
if (it != _targets.end()) {
signal(from);
_targets.erase(it);
} else {
slogger.warn("Receive outdated write ack from {}", from);
}
return _targets.size() == 0;
}
// return true if handler is no longer needed because
// CL cannot be reached
bool failure_response(gms::inet_address from, size_t count, error err) {
if (!_targets.contains(from)) {
// There is a little change we can get outdated reply
// if the coordinator was restarted after sending a request and
// getting reply back. The chance is low though since initial
// request id is initialized to server starting time
slogger.warn("Receive outdated write failure from {}", from);
return false;
}
_all_failures += count;
// we should not fail CL=ANY requests since they may succeed after
// writing hints
return _cl != db::consistency_level::ANY && failure(from, count, err);
}
void check_for_early_completion() {
if (_all_failures == _targets.size()) {
// leftover targets are all reported error, so nothing to wait for any longer
timeout_cb();
}
}
void expire_at(storage_proxy::clock_type::time_point timeout) {
_expire_timer.arm(timeout);
}
void on_released() {
_expire_timer.cancel();
if (_targets.size() == 0) {
_mutation_holder->release_mutation();
}
}
void timeout_cb() {
if (_cl_achieved || _cl == db::consistency_level::ANY) {
// we are here because either cl was achieved, but targets left in the handler are not
// responding, so a hint should be written for them, or cl == any in which case
// hints are counted towards consistency, so we need to write hints and count how much was written
auto hints = _proxy->hint_to_dead_endpoints(_mutation_holder, get_targets(), _type, get_trace_state());
signal(hints);
if (_cl == db::consistency_level::ANY && hints) {
slogger.trace("Wrote hint to satisfy CL.ANY after no replicas acknowledged the write");
}
if (_cl_achieved) { // For CL=ANY this can still be false
for (auto&& ep : get_targets()) {
++stats().background_replica_writes_failed.get_ep_stat(ep);
}
stats().background_writes_failed += int(!_targets.empty());
}
}
on_timeout();
_proxy->remove_response_handler(_id);
}
db::view::update_backlog max_backlog() {
return boost::accumulate(
get_targets() | boost::adaptors::transformed([this] (gms::inet_address ep) {
return _proxy->get_backlog_of(ep);
}),
db::view::update_backlog::no_backlog(),
[] (const db::view::update_backlog& lhs, const db::view::update_backlog& rhs) {
return std::max(lhs, rhs);
});
}
std::chrono::microseconds calculate_delay(db::view::update_backlog backlog) {
constexpr auto delay_limit_us = 1000000;
auto adjust = [] (float x) { return x * x * x; };
auto budget = std::max(storage_proxy::clock_type::duration(0),
_expire_timer.get_timeout() - storage_proxy::clock_type::now());
std::chrono::microseconds ret(uint32_t(adjust(backlog.relative_size()) * delay_limit_us));
// "budget" has millisecond resolution and can potentially be long
// in the future so converting it to microseconds may overflow.
// So to compare buget and ret we need to convert both to the lower
// resolution.
if (std::chrono::duration_cast(ret) < budget) {
return ret;
} else {
// budget is small (< ret) so can be converted to microseconds
return budget;
}
}
// Calculates how much to delay completing the request. The delay adds to the request's inherent latency.
template
void delay(tracing::trace_state_ptr trace, Func&& on_resume) {
auto backlog = max_backlog();
auto delay = calculate_delay(backlog);
stats().last_mv_flow_control_delay = delay;
if (delay.count() == 0) {
tracing::trace(trace, "Delay decision due to throttling: do not delay, resuming now");
on_resume(this);
} else {
++stats().throttled_base_writes;
tracing::trace(trace, "Delaying user write due to view update backlog {}/{} by {}us",
backlog.current, backlog.max, delay.count());
// Waited on indirectly.
(void)sleep_abortable(delay).finally([self = shared_from_this(), on_resume = std::forward(on_resume)] {
--self->stats().throttled_base_writes;
on_resume(self.get());
}).handle_exception_type([] (const seastar::sleep_aborted& ignored) { });
}
}
future<> wait() {
return _ready.get_future();
}
const std::unordered_set& get_targets() const {
return _targets;
}
const std::vector& get_dead_endpoints() const {
return _dead_endpoints;
}
bool store_hint(db::hints::manager& hm, gms::inet_address ep, tracing::trace_state_ptr tr_state) {
return _mutation_holder->store_hint(hm, ep, tr_state);
}
future<> apply_locally(storage_proxy::clock_type::time_point timeout, tracing::trace_state_ptr tr_state) {
return _mutation_holder->apply_locally(*_proxy, timeout, std::move(tr_state));
}
future<> apply_remotely(gms::inet_address ep, std::vector&& forward,
storage_proxy::response_id_type response_id, storage_proxy::clock_type::time_point timeout,
tracing::trace_state_ptr tr_state) {
return _mutation_holder->apply_remotely(*_proxy, ep, std::move(forward), response_id, timeout, std::move(tr_state));
}
const schema_ptr& get_schema() const {
return _mutation_holder->schema();
}
const size_t get_mutation_size() const {
return _mutation_holder->size();
}
storage_proxy::response_id_type id() const {
return _id;
}
bool read_repair_write() {
return !_mutation_holder->is_shared();
}
const tracing::trace_state_ptr& get_trace_state() const {
return _trace_state;
}
storage_proxy::write_stats& stats() {
return _stats;
}
friend storage_proxy;
};
class datacenter_write_response_handler : public abstract_write_response_handler {
bool waited_for(gms::inet_address from) override {
return fbu::is_me(from) || db::is_local(from);
}
public:
datacenter_write_response_handler(shared_ptr p, keyspace& ks, db::consistency_level cl, db::write_type type,
std::unique_ptr mh, std::unordered_set targets,
const std::vector& pending_endpoints, std::vector dead_endpoints, tracing::trace_state_ptr tr_state,
storage_proxy::write_stats& stats, service_permit permit) :
abstract_write_response_handler(std::move(p), ks, cl, type, std::move(mh),
std::move(targets), std::move(tr_state), stats, std::move(permit), db::count_local_endpoints(pending_endpoints), std::move(dead_endpoints)) {
_total_endpoints = db::count_local_endpoints(_targets);
}
};
class write_response_handler : public abstract_write_response_handler {
bool waited_for(gms::inet_address from) override {
return true;
}
public:
write_response_handler(shared_ptr p, keyspace& ks, db::consistency_level cl, db::write_type type,
std::unique_ptr mh, std::unordered_set targets,
const std::vector& pending_endpoints, std::vector dead_endpoints, tracing::trace_state_ptr tr_state,
storage_proxy::write_stats& stats, service_permit permit) :
abstract_write_response_handler(std::move(p), ks, cl, type, std::move(mh),
std::move(targets), std::move(tr_state), stats, std::move(permit), pending_endpoints.size(), std::move(dead_endpoints)) {
_total_endpoints = _targets.size();
}
};
class view_update_write_response_handler : public write_response_handler, public bi::list_base_hook> {
public:
view_update_write_response_handler(shared_ptr p, keyspace& ks, db::consistency_level cl,
std::unique_ptr mh, std::unordered_set targets,
const std::vector& pending_endpoints, std::vector dead_endpoints, tracing::trace_state_ptr tr_state,
storage_proxy::write_stats& stats, service_permit permit):
write_response_handler(p, ks, cl, db::write_type::VIEW, std::move(mh),
std::move(targets), pending_endpoints, std::move(dead_endpoints), std::move(tr_state), stats, std::move(permit)) {
register_in_intrusive_list(*p);
}
~view_update_write_response_handler();
private:
void register_in_intrusive_list(storage_proxy& p);
};
class storage_proxy::view_update_handlers_list : public bi::list, bi::constant_time_size> {
// _live_iterators holds all iterators that point into the bi:list in the base class of this object.
// If we remove a view_update_write_response_handler from the list, and an iterator happens to point
// into it, we advance the iterator so it doesn't point at a removed object. See #4912.
std::vector _live_iterators;
public:
view_update_handlers_list() {
_live_iterators.reserve(10); // We only expect 1.
}
void register_live_iterator(iterator* itp) noexcept { // We don't tolerate failure, so abort instead
_live_iterators.push_back(itp);
}
void unregister_live_iterator(iterator* itp) {
_live_iterators.erase(boost::remove(_live_iterators, itp), _live_iterators.end());
}
void update_live_iterators(view_update_write_response_handler* vuwrh) {
// vuwrh is being removed from the b::list, so if any live iterator points at it,
// move it to the next object (this requires that the list is traversed in the forward
// direction).
for (auto& itp : _live_iterators) {
if (&**itp == vuwrh) {
++*itp;
}
}
}
class iterator_guard {
view_update_handlers_list& _vuhl;
iterator* _itp;
public:
iterator_guard(view_update_handlers_list& vuhl, iterator& it) : _vuhl(vuhl), _itp(&it) {
_vuhl.register_live_iterator(_itp);
}
~iterator_guard() {
_vuhl.unregister_live_iterator(_itp);
}
};
};
void view_update_write_response_handler::register_in_intrusive_list(storage_proxy& p) {
p.get_view_update_handlers_list().push_back(*this);
}
view_update_write_response_handler::~view_update_write_response_handler() {
_proxy->_view_update_handlers_list->update_live_iterators(this);
}
class datacenter_sync_write_response_handler : public abstract_write_response_handler {
struct dc_info {
size_t acks;
size_t total_block_for;
size_t total_endpoints;
size_t failures;
};
std::unordered_map _dc_responses;
bool waited_for(gms::inet_address from) override {
auto& snitch_ptr = locator::i_endpoint_snitch::get_local_snitch_ptr();
sstring data_center = snitch_ptr->get_datacenter(from);
auto dc_resp = _dc_responses.find(data_center);
if (dc_resp->second.acks < dc_resp->second.total_block_for) {
++dc_resp->second.acks;
return true;
}
return false;
}
public:
datacenter_sync_write_response_handler(shared_ptr p, keyspace& ks, db::consistency_level cl, db::write_type type,
std::unique_ptr mh, std::unordered_set targets, const std::vector& pending_endpoints,
std::vector dead_endpoints, tracing::trace_state_ptr tr_state, storage_proxy::write_stats& stats, service_permit permit) :
abstract_write_response_handler(std::move(p), ks, cl, type, std::move(mh), targets, std::move(tr_state), stats, std::move(permit), 0, dead_endpoints) {
auto& snitch_ptr = locator::i_endpoint_snitch::get_local_snitch_ptr();
for (auto& target : targets) {
auto dc = snitch_ptr->get_datacenter(target);
if (!_dc_responses.contains(dc)) {
auto pending_for_dc = boost::range::count_if(pending_endpoints, [&snitch_ptr, &dc] (const gms::inet_address& ep){
return snitch_ptr->get_datacenter(ep) == dc;
});
size_t total_endpoints_for_dc = boost::range::count_if(targets, [&snitch_ptr, &dc] (const gms::inet_address& ep){
return snitch_ptr->get_datacenter(ep) == dc;
});
_dc_responses.emplace(dc, dc_info{0, db::local_quorum_for(ks, dc) + pending_for_dc, total_endpoints_for_dc, 0});
_total_block_for += pending_for_dc;
}
}
}
bool failure(gms::inet_address from, size_t count, error err) override {
auto& snitch_ptr = locator::i_endpoint_snitch::get_local_snitch_ptr();
const sstring& dc = snitch_ptr->get_datacenter(from);
auto dc_resp = _dc_responses.find(dc);
dc_resp->second.failures += count;
_failed += count;
if (dc_resp->second.total_block_for + dc_resp->second.failures > dc_resp->second.total_endpoints) {
_error = err;
return true;
}
return false;
}
};
static future<> sleep_approx_50ms() {
static thread_local std::default_random_engine re{std::random_device{}()};
static thread_local std::uniform_int_distribution<> dist(0, 100);
return seastar::sleep(std::chrono::milliseconds(dist(re)));
}
static future> sleep_and_restart() {
return sleep_approx_50ms().then([] {
return std::optional(); // continue
});
}
/**
* Begin a Paxos session by sending a prepare request and completing any in-progress requests seen in the replies.
*
* @return the Paxos ballot promised by the replicas if no in-progress requests were seen and a quorum of
* nodes have seen the most recent commit. Otherwise, return null.
*/
future
paxos_response_handler::begin_and_repair_paxos(client_state& cs, unsigned& contentions, bool is_write) {
if (!_proxy->features().cluster_supports_lwt()) {
throw std::runtime_error("The cluster does not support Paxos. Upgrade all the nodes to the version with LWT support.");
}
return do_with(api::timestamp_type(0), shared_from_this(), [this, &cs, &contentions, is_write]
(api::timestamp_type& min_timestamp_micros_to_use, shared_ptr& prh) {
return repeat_until_value([this, &contentions, &cs, &min_timestamp_micros_to_use, is_write] {
if (storage_proxy::clock_type::now() > _cas_timeout) {
return make_exception_future>(
mutation_write_timeout_exception(_schema->ks_name(), _schema->cf_name(), _cl_for_paxos, 0,
_required_participants, db::write_type::CAS)
);
}
// We want a timestamp that is guaranteed to be unique for that node (so that the ballot is
// globally unique), but if we've got a prepare rejected already we also want to make sure
// we pick a timestamp that has a chance to be promised, i.e. one that is greater that the
// most recently known in progress (#5667). Lastly, we don't want to use a timestamp that is
// older than the last one assigned by ClientState or operations may appear out-of-order
// (#7801).
api::timestamp_type ballot_micros = cs.get_timestamp_for_paxos(min_timestamp_micros_to_use);
// Note that ballotMicros is not guaranteed to be unique if two proposal are being handled
// concurrently by the same coordinator. But we still need ballots to be unique for each
// proposal so we have to use getRandomTimeUUIDFromMicros.
utils::UUID ballot = utils::UUID_gen::get_random_time_UUID_from_micros(ballot_micros);
paxos::paxos_state::logger.debug("CAS[{}] Preparing {}", _id, ballot);
tracing::trace(tr_state, "Preparing {}", ballot);
return prepare_ballot(ballot)
.then([this, &contentions, ballot, &min_timestamp_micros_to_use, is_write] (paxos::prepare_summary summary) {
if (!summary.promised) {
paxos::paxos_state::logger.debug("CAS[{}] Some replicas have already promised a higher ballot than ours; aborting", _id);
tracing::trace(tr_state, "Some replicas have already promised a higher ballot than ours; aborting");
contentions++;
return sleep_and_restart();
}
min_timestamp_micros_to_use = utils::UUID_gen::micros_timestamp(summary.most_recent_promised_ballot) + 1;
std::optional in_progress = std::move(summary.most_recent_proposal);
// If we have an in-progress accepted ballot greater than the most recent commit
// we know, then it's an in-progress round that needs to be completed, so do it.
if (in_progress &&
(!summary.most_recent_commit ||
(summary.most_recent_commit &&
in_progress->ballot.timestamp() > summary.most_recent_commit->ballot.timestamp()))) {
paxos::paxos_state::logger.debug("CAS[{}] Finishing incomplete paxos round {}", _id, *in_progress);
tracing::trace(tr_state, "Finishing incomplete paxos round {}", *in_progress);
if (is_write) {
++_proxy->get_stats().cas_write_unfinished_commit;
} else {
++_proxy->get_stats().cas_read_unfinished_commit;
}
auto refreshed_in_progress = make_lw_shared(ballot, std::move(in_progress->update));
return accept_proposal(refreshed_in_progress, false).then([this, &contentions, refreshed_in_progress] (bool is_accepted) mutable {
if (is_accepted) {
return learn_decision(std::move(refreshed_in_progress), false).then([] {
return make_ready_future>(std::optional());
}).handle_exception_type([] (mutation_write_timeout_exception& e) {
e.type = db::write_type::CAS;
// we're still doing preparation for the paxos rounds, so we want to use the CAS (see cASSANDRA-8672)
return make_exception_future>(std::move(e));
});
} else {
paxos::paxos_state::logger.debug("CAS[{}] Some replicas have already promised a higher ballot than ours; aborting", _id);
tracing::trace(tr_state, "Some replicas have already promised a higher ballot than ours; aborting");
// sleep a random amount to give the other proposer a chance to finish
contentions++;
return sleep_and_restart();
}
});
}
// To be able to propose our value on a new round, we need a quorum of replica to have learn
// the previous one. Why is explained at:
// https://issues.apache.org/jira/browse/CASSANDRA-5062?focusedCommentId=13619810&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-13619810)
// Since we waited for quorum nodes, if some of them haven't seen the last commit (which may
// just be a timing issue, but may also mean we lost messages), we pro-actively "repair"
// those nodes, and retry.
auto now_in_sec = utils::UUID_gen::unix_timestamp_in_sec(ballot);
std::unordered_set missing_mrc = summary.replicas_missing_most_recent_commit(_schema, now_in_sec);
if (missing_mrc.size() > 0) {
paxos::paxos_state::logger.debug("CAS[{}] Repairing replicas that missed the most recent commit", _id);
tracing::trace(tr_state, "Repairing replicas that missed the most recent commit");
std::array, schema_ptr, dht::token, std::unordered_set>, 1>
m{std::make_tuple(make_lw_shared(std::move(*summary.most_recent_commit)), _schema, _key.token(), std::move(missing_mrc))};
// create_write_response_handler is overloaded for paxos::proposal and will
// create cas_mutation holder, which consequently will ensure paxos::learn is
// used.
auto f = _proxy->mutate_internal(std::move(m), db::consistency_level::ANY, false, tr_state, _permit, _timeout);
// TODO: provided commits did not invalidate the prepare we just did above (which they
// didn't), we could just wait for all the missing most recent commits to
// acknowledge this decision and then move on with proposing our value.
return f.then_wrapped([prh = shared_from_this()] (future<> f) {
if (f.failed()) {
paxos::paxos_state::logger.debug("CAS[{}] Failure during commit repair {}", prh->_id, f.get_exception());
} else {
f.ignore_ready_future();
}
return std::optional(); // continue
});
}
return make_ready_future>(ballot_and_data{ballot, std::move(summary.data)});
});
});
});
}
template struct dependent_false : std::false_type {};
// This function implement prepare stage of Paxos protocol and collects metadata needed to repair
// previously unfinished round (if there was one).
future paxos_response_handler::prepare_ballot(utils::UUID ballot) {
struct {
size_t errors = 0;
// Whether the value of the requested key received from participating replicas match.
bool digests_match = true;
// Digest corresponding to the value of the requested key received from participating replicas.
std::optional digest;
// the promise can be set before all replies are received at which point
// the optional will be disengaged so further replies are ignored
std::optional> p = promise();
void set_value(paxos::prepare_summary&& s) {
p->set_value(std::move(s));
p.reset();
}
void set_exception(std::exception_ptr&& e) {
p->set_exception(std::move(e));
p.reset();
}
} request_tracker;
auto f = request_tracker.p->get_future();
// We may continue collecting prepare responses in the background after the reply is ready
(void)do_with(paxos::prepare_summary(_live_endpoints.size()), std::move(request_tracker), shared_from_this(),
[this, ballot] (paxos::prepare_summary& summary, auto& request_tracker, shared_ptr& prh) mutable {
paxos::paxos_state::logger.trace("CAS[{}] prepare_ballot: sending ballot {} to {}", _id, ballot, _live_endpoints);
return parallel_for_each(_live_endpoints, [this, &summary, ballot, &request_tracker] (gms::inet_address peer) mutable {
return futurize_invoke([&] {
// To generate less network traffic, only the closest replica (first one in the list of participants)
// sends query result content while other replicas send digests needed to check consistency.
bool only_digest = peer != _live_endpoints[0];
auto da = digest_algorithm(get_local_storage_proxy());
if (fbu::is_me(peer)) {
tracing::trace(tr_state, "prepare_ballot: prepare {} locally", ballot);
return paxos::paxos_state::prepare(tr_state, _schema, *_cmd, _key.key(), ballot, only_digest, da, _timeout);
} else {
tracing::trace(tr_state, "prepare_ballot: sending prepare {} to {}", ballot, peer);
return _proxy->_messaging.send_paxos_prepare(peer, _timeout, *_cmd, _key.key(), ballot, only_digest, da,
tracing::make_trace_info(tr_state));
}
}).then_wrapped([this, &summary, &request_tracker, peer, ballot]
(future response_f) mutable {
if (!request_tracker.p) {
response_f.ignore_ready_future();
return; // ignore the response since a completion was already signaled
}
if (response_f.failed()) {
auto ex = response_f.get_exception();
if (is_timeout_exception(ex)) {
paxos::paxos_state::logger.trace("CAS[{}] prepare_ballot: timeout while sending ballot {} to {}", _id,
ballot, peer);
auto e = std::make_exception_ptr(mutation_write_timeout_exception(_schema->ks_name(), _schema->cf_name(),
_cl_for_paxos, summary.committed_ballots_by_replica.size(), _required_participants,
db::write_type::CAS));
request_tracker.set_exception(std::move(e));
} else {
request_tracker.errors++;
paxos::paxos_state::logger.trace("CAS[{}] prepare_ballot: fail to send ballot {} to {}: {}", _id,
ballot, peer, ex);
if (_required_participants + request_tracker.errors > _live_endpoints.size()) {
auto e = std::make_exception_ptr(mutation_write_failure_exception(_schema->ks_name(),
_schema->cf_name(), _cl_for_paxos, summary.committed_ballots_by_replica.size(),
request_tracker.errors, _required_participants, db::write_type::CAS));
request_tracker.set_exception(std::move(e));
}
}
return;
}
auto on_prepare_response = [&] (auto&& response) {
using T = std::decay_t;
if constexpr (std::is_same_v) {
tracing::trace(tr_state, "prepare_ballot: got more up to date ballot {} from /{}", response, peer);
paxos::paxos_state::logger.trace("CAS[{}] prepare_ballot: got more up to date ballot {} from {}", _id, response, peer);
// We got an UUID that prevented our proposal from succeeding
summary.update_most_recent_promised_ballot(response);
summary.promised = false;
request_tracker.set_value(std::move(summary));
return;
} else if constexpr (std::is_same_v) {
utils::UUID mrc_ballot = utils::UUID_gen::min_time_UUID(0);
paxos::paxos_state::logger.trace("CAS[{}] prepare_ballot: got a response {} from {}", _id, response, peer);
tracing::trace(tr_state, "prepare_ballot: got a response {} from /{}", response, peer);
// Find the newest learned value among all replicas that answered.
// It will be used to "repair" replicas that did not learn this value yet.
if (response.most_recent_commit) {
mrc_ballot = response.most_recent_commit->ballot;
if (!summary.most_recent_commit ||
summary.most_recent_commit->ballot.timestamp() < mrc_ballot.timestamp()) {
summary.most_recent_commit = std::move(response.most_recent_commit);
}
}
// cannot throw since the memory was reserved ahead
summary.committed_ballots_by_replica.emplace(peer, mrc_ballot);
if (response.accepted_proposal) {
summary.update_most_recent_promised_ballot(response.accepted_proposal->ballot);
// If some response has an accepted proposal, then we should replay the proposal with the highest ballot.
// So find the highest accepted proposal here.
if (!summary.most_recent_proposal || response.accepted_proposal > summary.most_recent_proposal) {
summary.most_recent_proposal = std::move(response.accepted_proposal);
}
}
// Check if the query result attached to the promise matches query results received from other participants.
if (request_tracker.digests_match) {
if (response.data_or_digest) {
foreign_ptr> data;
if (std::holds_alternative>>(*response.data_or_digest)) {
data = std::move(std::get>>(*response.data_or_digest));
}
auto& digest = data ? data->digest() : std::get(*response.data_or_digest);
if (request_tracker.digest) {
if (*request_tracker.digest != digest) {
request_tracker.digests_match = false;
}
} else {
request_tracker.digest = digest;
}
if (request_tracker.digests_match && !summary.data && data) {
summary.data = std::move(data);
}
} else {
request_tracker.digests_match = false;
}
if (!request_tracker.digests_match) {
request_tracker.digest.reset();
summary.data.reset();
}
}
if (summary.committed_ballots_by_replica.size() == _required_participants) { // got all replies
tracing::trace(tr_state, "prepare_ballot: got enough replies to proceed");
paxos::paxos_state::logger.trace("CAS[{}] prepare_ballot: got enough replies to proceed", _id);
request_tracker.set_value(std::move(summary));
}
} else {
static_assert(dependent_false::value, "unexpected type!");
}
};
std::visit(on_prepare_response, response_f.get0());
});
});
});
return f;
}
// This function implements accept stage of the Paxos protocol.
future paxos_response_handler::accept_proposal(lw_shared_ptr proposal, bool timeout_if_partially_accepted) {
struct {
// the promise can be set before all replies are received at which point
// the optional will be disengaged so further replies are ignored
std::optional> p = promise();
size_t accepts = 0;
size_t rejects = 0;
size_t errors = 0;
size_t all_replies() const {
return accepts + rejects + errors;
}
size_t non_accept_replies() const {
return rejects + errors;
}
size_t non_error_replies() const {
return accepts + rejects;
}
void set_value(bool v) {
p->set_value(v);
p.reset();
}
void set_exception(std::exception_ptr&& e) {
p->set_exception(std::move(e));
p.reset();
}
} request_tracker;
auto f = request_tracker.p->get_future();
// We may continue collecting propose responses in the background after the reply is ready
(void)do_with(std::move(request_tracker), shared_from_this(), [this, timeout_if_partially_accepted, proposal = std::move(proposal)]
(auto& request_tracker, shared_ptr& prh) {
paxos::paxos_state::logger.trace("CAS[{}] accept_proposal: sending commit {} to {}", _id, *proposal, _live_endpoints);
return parallel_for_each(_live_endpoints, [this, &request_tracker, timeout_if_partially_accepted, proposal = std::move(proposal)] (gms::inet_address peer) mutable {
return futurize_invoke([&] {
if (fbu::is_me(peer)) {
tracing::trace(tr_state, "accept_proposal: accept {} locally", *proposal);
return paxos::paxos_state::accept(tr_state, _schema, proposal->update.decorated_key(*_schema).token(), *proposal, _timeout);
} else {
tracing::trace(tr_state, "accept_proposal: send accept {} to {}", *proposal, peer);
return _proxy->_messaging.send_paxos_accept(peer, _timeout, *proposal, tracing::make_trace_info(tr_state));
}
}).then_wrapped([this, &request_tracker, timeout_if_partially_accepted, proposal, peer] (future accepted_f) {
if (!request_tracker.p) {
accepted_f.ignore_ready_future();
// Ignore the response since a completion was already signaled.
return;
}
bool is_timeout = false;
if (accepted_f.failed()) {
auto ex = accepted_f.get_exception();
if (is_timeout_exception(ex)) {
paxos::paxos_state::logger.trace("CAS[{}] accept_proposal: timeout while sending proposal {} to {}",
_id, *proposal, peer);
is_timeout = true;
} else {
paxos::paxos_state::logger.trace("CAS[{}] accept_proposal: failure while sending proposal {} to {}: {}", _id,
*proposal, peer, ex);
request_tracker.errors++;
}
} else {
bool accepted = accepted_f.get0();
tracing::trace(tr_state, "accept_proposal: got \"{}\" from /{}", accepted ? "accepted" : "rejected", peer);
paxos::paxos_state::logger.trace("CAS[{}] accept_proposal: got \"{}\" from {}", _id,
accepted ? "accepted" : "rejected", peer);
accepted ? request_tracker.accepts++ : request_tracker.rejects++;
}
/**
* The code has two modes of operation, controlled by the timeout_if_partially_accepted parameter.
*
* In timeout_if_partially_accepted is false, we will return a failure as soon as a majority of nodes reject
* the proposal. This is used when replaying a proposal from an earlier leader.
*
* Otherwise, we wait for either all replicas to respond or until we achieve
* the desired quorum. We continue to wait for all replicas even after we know we cannot succeed
* because we need to know if no node at all has accepted our proposal or if at least one has.
* In the former case, a proposer is guaranteed no-one will replay its value; in the
* latter we don't, so we must timeout in case another leader replays it before we
* can; see CASSANDRA-6013.
*/
if (request_tracker.accepts == _required_participants) {
tracing::trace(tr_state, "accept_proposal: got enough accepts to proceed");
paxos::paxos_state::logger.trace("CAS[{}] accept_proposal: got enough accepts to proceed", _id);
request_tracker.set_value(true);
} else if (is_timeout) {
auto e = std::make_exception_ptr(mutation_write_timeout_exception(_schema->ks_name(), _schema->cf_name(),
_cl_for_paxos, request_tracker.non_error_replies(), _required_participants, db::write_type::CAS));
request_tracker.set_exception(std::move(e));
} else if (_required_participants + request_tracker.errors > _live_endpoints.size()) {
// We got one too many errors. The quorum is no longer reachable. We can fail here
// timeout_if_partially_accepted or not because failing is always safe - a client cannot
// assume that the value was not committed.
auto e = std::make_exception_ptr(mutation_write_failure_exception(_schema->ks_name(),
_schema->cf_name(), _cl_for_paxos, request_tracker.non_error_replies(),
request_tracker.errors, _required_participants, db::write_type::CAS));
request_tracker.set_exception(std::move(e));
} else if (_required_participants + request_tracker.non_accept_replies() > _live_endpoints.size() && !timeout_if_partially_accepted) {
// In case there is no need to reply with a timeout if at least one node is accepted
// we can fail the request as soon is we know a quorum is unreachable.
tracing::trace(tr_state, "accept_proposal: got enough rejects to proceed");
paxos::paxos_state::logger.trace("CAS[{}] accept_proposal: got enough rejects to proceed", _id);
request_tracker.set_value(false);
} else if (request_tracker.all_replies() == _live_endpoints.size()) { // wait for all replies
if (request_tracker.accepts == 0 && request_tracker.errors == 0) {
tracing::trace(tr_state, "accept_proposal: proposal is fully rejected");
paxos::paxos_state::logger.trace("CAS[{}] accept_proposal: proposal is fully rejected", _id);
// Return false if fully refused. Consider errors as accepts here since it
// is not possible to know for sure.
request_tracker.set_value(false);
} else {
// We got some rejects, but not all, and there were errors. So we can't know for
// sure that the proposal is fully rejected, and it is obviously not
// accepted, either.
paxos::paxos_state::logger.trace("CAS[{}] accept_proposal: proposal is partially rejected", _id);
tracing::trace(tr_state, "accept_proposal: proposal is partially rejected");
_proxy->get_stats().cas_write_timeout_due_to_uncertainty++;
// TODO: we report write timeout exception to be compatible with Cassandra,
// which uses write_timeout_exception to signal any "unknown" state.
// To be changed in scope of work on https://issues.apache.org/jira/browse/CASSANDRA-15350
auto e = std::make_exception_ptr(mutation_write_timeout_exception(_schema->ks_name(),
_schema->cf_name(), _cl_for_paxos, request_tracker.accepts, _required_participants,
db::write_type::CAS));
request_tracker.set_exception(std::move(e));
}
} // wait for more replies
}); // send_paxos_accept.then_wrapped
}); // parallel_for_each
}); // do_with
return f;
}
// debug output in mutate_internal needs this
std::ostream& operator<<(std::ostream& os, const paxos_response_handler& h) {
os << "paxos_response_handler{" << h.id() << "}";
return os;
}
// This function implements learning stage of Paxos protocol
future<> paxos_response_handler::learn_decision(lw_shared_ptr decision, bool allow_hints) {
tracing::trace(tr_state, "learn_decision: committing {} with cl={}", *decision, _cl_for_learn);
paxos::paxos_state::logger.trace("CAS[{}] learn_decision: committing {} with cl={}", _id, *decision, _cl_for_learn);
// FIXME: allow_hints is ignored. Consider if we should follow it and remove if not.
// Right now we do not store hints for when committing decisions.
// `mutate_internal` behaves differently when its template parameter is a range of mutations and when it's
// a range of (decision, schema, token)-tuples. Both code paths diverge on `create_write_response_handler`.
// We use the first path for CDC mutations (if present) and the latter for "paxos mutations".
// Attempts to send both kinds of mutations in one shot caused an infinite loop.
future<> f_cdc = make_ready_future<>();
if (_schema->cdc_options().enabled()) {
auto update_mut = decision->update.unfreeze(_schema);
const auto base_tbl_id = update_mut.column_family_id();
std::vector update_mut_vec{std::move(update_mut)};
auto cdc = _proxy->get_cdc_service();
if (cdc && cdc->needs_cdc_augmentation(update_mut_vec)) {
f_cdc = cdc->augment_mutation_call(_timeout, std::move(update_mut_vec), tr_state, _cl_for_learn)
.then([this, base_tbl_id, cdc = cdc->shared_from_this()] (std::tuple, lw_shared_ptr>&& t) {
auto mutations = std::move(std::get<0>(t));
auto tracker = std::move(std::get<1>(t));
// Pick only the CDC ("augmenting") mutations
std::erase_if(mutations, [base_tbl_id = std::move(base_tbl_id)] (const mutation& v) {
return v.schema()->id() == base_tbl_id;
});
if (mutations.empty()) {
return make_ready_future<>();
}
return _proxy->mutate_internal(std::move(mutations), _cl_for_learn, false, tr_state, _permit, _timeout, std::move(tracker));
});
}
}
// Path for the "base" mutations
std::array, schema_ptr, shared_ptr, dht::token>, 1> m{std::make_tuple(std::move(decision), _schema, shared_from_this(), _key.token())};
future<> f_lwt = _proxy->mutate_internal(std::move(m), _cl_for_learn, false, tr_state, _permit, _timeout);
return when_all_succeed(std::move(f_cdc), std::move(f_lwt)).discard_result();
}
void paxos_response_handler::prune(utils::UUID ballot) {
if ( _proxy->get_stats().cas_now_pruning >= pruning_limit) {
_proxy->get_stats().cas_coordinator_dropped_prune++;
return;
}
_proxy->get_stats().cas_now_pruning++;
_proxy->get_stats().cas_prune++;
// running in the background, but the amount of the bg job is limited by pruning_limit
// it is waited by holding shared pointer to storage_proxy which guaranties
// that storage_proxy::stop() will wait for this to complete
(void)parallel_for_each(_live_endpoints, [this, ballot] (gms::inet_address peer) mutable {
if (fbu::is_me(peer)) {
tracing::trace(tr_state, "prune: prune {} locally", ballot);
return paxos::paxos_state::prune(_schema, _key.key(), ballot, _timeout, tr_state);
} else {
tracing::trace(tr_state, "prune: send prune of {} to {}", ballot, peer);
return _proxy->_messaging.send_paxos_prune(peer, _timeout, _schema->version(), _key.key(), ballot, tracing::make_trace_info(tr_state));
}
}).then_wrapped([h = shared_from_this()] (future<> f) {
h->_proxy->get_stats().cas_now_pruning--;
try {
f.get();
} catch (rpc::closed_error&) {
// ignore errors due to closed connection
} catch (...) {
paxos::paxos_state::logger.error("CAS[{}] prune: failed {}", h->_id, std::current_exception());
}
});
}
bool paxos_response_handler::learned(gms::inet_address ep) {
if (_learned < _required_participants) {
if (boost::range::find(_live_endpoints, ep) != _live_endpoints.end()) {
_learned++;
return _learned == _required_participants;
}
}
return false;
}
static std::vector
replica_ids_to_endpoints(const locator::token_metadata& tm, const std::vector& replica_ids) {
std::vector endpoints;
endpoints.reserve(replica_ids.size());
for (const auto& replica_id : replica_ids) {
if (auto endpoint_opt = tm.get_endpoint_for_host_id(replica_id)) {
endpoints.push_back(*endpoint_opt);
}
}
return endpoints;
}
static std::vector
endpoints_to_replica_ids(const locator::token_metadata& tm, const std::vector& endpoints) {
std::vector replica_ids;
replica_ids.reserve(endpoints.size());
for (const auto& endpoint : endpoints) {
if (auto replica_id_opt = tm.get_host_id_if_known(endpoint)) {
replica_ids.push_back(*replica_id_opt);
}
}
return replica_ids;
}
query::max_result_size storage_proxy::get_max_result_size(const query::partition_slice& slice) const {
// Unpaged and reverse queries.
if (!slice.options.contains() || slice.options.contains()) {
auto& db = _db.local();
// We only limit user queries.
if (current_scheduling_group() == db.get_statement_scheduling_group()) {
return query::max_result_size(db.get_config().max_memory_for_unlimited_query_soft_limit(), db.get_config().max_memory_for_unlimited_query_hard_limit());
} else {
return query::max_result_size(query::result_memory_limiter::unlimited_result_size);
}
} else {
return query::max_result_size(query::result_memory_limiter::maximum_result_size);
}
}
bool storage_proxy::need_throttle_writes() const {
return get_global_stats().background_write_bytes > _background_write_throttle_threahsold || get_global_stats().queued_write_bytes > 6*1024*1024;
}
void storage_proxy::unthrottle() {
while(!need_throttle_writes() && !_throttled_writes.empty()) {
auto id = _throttled_writes.front();
_throttled_writes.pop_front();
auto it = _response_handlers.find(id);
if (it != _response_handlers.end()) {
it->second->unthrottle();
}
}
}
storage_proxy::response_id_type storage_proxy::register_response_handler(shared_ptr&& h) {
auto id = h->id();
auto e = _response_handlers.emplace(id, std::move(h));
assert(e.second);
return id;
}
void storage_proxy::remove_response_handler(storage_proxy::response_id_type id) {
auto entry = _response_handlers.find(id);
assert(entry != _response_handlers.end());
remove_response_handler_entry(std::move(entry));
}
void storage_proxy::remove_response_handler_entry(response_handlers_map::iterator entry) {
entry->second->on_released();
_response_handlers.erase(std::move(entry));
}
void storage_proxy::got_response(storage_proxy::response_id_type id, gms::inet_address from, std::optional backlog) {
auto it = _response_handlers.find(id);
if (it != _response_handlers.end()) {
tracing::trace(it->second->get_trace_state(), "Got a response from /{}", from);
if (it->second->response(from)) {
remove_response_handler_entry(std::move(it)); // last one, remove entry. Will cancel expiration timer too.
} else {
it->second->check_for_early_completion();
}
}
maybe_update_view_backlog_of(std::move(from), std::move(backlog));
}
void storage_proxy::got_failure_response(storage_proxy::response_id_type id, gms::inet_address from, size_t count, std::optional backlog, error err) {
auto it = _response_handlers.find(id);
if (it != _response_handlers.end()) {
tracing::trace(it->second->get_trace_state(), "Got {} failures from /{}", count, from);
if (it->second->failure_response(from, count, err)) {
remove_response_handler_entry(std::move(it));
} else {
it->second->check_for_early_completion();
}
}
maybe_update_view_backlog_of(std::move(from), std::move(backlog));
}
void storage_proxy::maybe_update_view_backlog_of(gms::inet_address replica, std::optional backlog) {
if (backlog) {
auto now = std::chrono::duration_cast(std::chrono::system_clock::now().time_since_epoch()).count();
_view_update_backlogs[replica] = {std::move(*backlog), now};
}
}
db::view::update_backlog storage_proxy::get_view_update_backlog() const {
return _max_view_update_backlog.add_fetch(this_shard_id(), get_db().local().get_view_update_backlog());
}
db::view::update_backlog storage_proxy::get_backlog_of(gms::inet_address ep) const {
auto it = _view_update_backlogs.find(ep);
if (it == _view_update_backlogs.end()) {
return db::view::update_backlog::no_backlog();
}
return it->second.backlog;
}
future<> storage_proxy::response_wait(storage_proxy::response_id_type id, clock_type::time_point timeout) {
auto& handler = _response_handlers.find(id)->second;
handler->expire_at(timeout);
return handler->wait();
}
::shared_ptr& storage_proxy::get_write_response_handler(storage_proxy::response_id_type id) {
return _response_handlers.find(id)->second;
}
storage_proxy::response_id_type storage_proxy::create_write_response_handler(keyspace& ks, db::consistency_level cl, db::write_type type, std::unique_ptr m,
std::unordered_set targets, const std::vector& pending_endpoints, std::vector dead_endpoints, tracing::trace_state_ptr tr_state,
storage_proxy::write_stats& stats, service_permit permit)
{
shared_ptr h;
auto& rs = ks.get_replication_strategy();
if (db::is_datacenter_local(cl)) {
h = ::make_shared(shared_from_this(), ks, cl, type, std::move(m), std::move(targets), pending_endpoints, std::move(dead_endpoints), std::move(tr_state), stats, std::move(permit));
} else if (cl == db::consistency_level::EACH_QUORUM && rs.get_type() == locator::replication_strategy_type::network_topology){
h = ::make_shared(shared_from_this(), ks, cl, type, std::move(m), std::move(targets), pending_endpoints, std::move(dead_endpoints), std::move(tr_state), stats, std::move(permit));
} else if (type == db::write_type::VIEW) {
h = ::make_shared(shared_from_this(), ks, cl, std::move(m), std::move(targets), pending_endpoints, std::move(dead_endpoints), std::move(tr_state), stats, std::move(permit));
} else {
h = ::make_shared(shared_from_this(), ks, cl, type, std::move(m), std::move(targets), pending_endpoints, std::move(dead_endpoints), std::move(tr_state), stats, std::move(permit));
}
return register_response_handler(std::move(h));
}
seastar::metrics::label storage_proxy_stats::split_stats::datacenter_label("datacenter");
storage_proxy_stats::split_stats::split_stats(const sstring& category, const sstring& short_description_prefix, const sstring& long_description_prefix, const sstring& op_type, bool auto_register_metrics)
: _short_description_prefix(short_description_prefix)
, _long_description_prefix(long_description_prefix)
, _category(category)
, _op_type(op_type)
, _auto_register_metrics(auto_register_metrics) { }
storage_proxy_stats::write_stats::write_stats()
: writes_attempts(COORDINATOR_STATS_CATEGORY, "total_write_attempts", "total number of write requests", "mutation_data")
, writes_errors(COORDINATOR_STATS_CATEGORY, "write_errors", "number of write requests that failed", "mutation_data")
, background_replica_writes_failed(COORDINATOR_STATS_CATEGORY, "background_replica_writes_failed", "number of replica writes that timed out or failed after CL was reached", "mutation_data")
, read_repair_write_attempts(COORDINATOR_STATS_CATEGORY, "read_repair_write_attempts", "number of write operations in a read repair context", "mutation_data") { }
storage_proxy_stats::write_stats::write_stats(const sstring& category, bool auto_register_stats)
: writes_attempts(category, "total_write_attempts", "total number of write requests", "mutation_data", auto_register_stats)
, writes_errors(category, "write_errors", "number of write requests that failed", "mutation_data", auto_register_stats)
, background_replica_writes_failed(category, "background_replica_writes_failed", "number of replica writes that timed out or failed after CL was reached", "mutation_data", auto_register_stats)
, read_repair_write_attempts(category, "read_repair_write_attempts", "number of write operations in a read repair context", "mutation_data", auto_register_stats) { }
void storage_proxy_stats::write_stats::register_split_metrics_local() {
writes_attempts.register_metrics_local();
writes_errors.register_metrics_local();
background_replica_writes_failed.register_metrics_local();
read_repair_write_attempts.register_metrics_local();
}
void storage_proxy_stats::write_stats::register_stats() {
namespace sm = seastar::metrics;
_metrics.add_group(COORDINATOR_STATS_CATEGORY, {
sm::make_histogram("write_latency", sm::description("The general write latency histogram"),
{storage_proxy_stats::current_scheduling_group_label()},
[this]{return to_metrics_histogram(estimated_write);}),
sm::make_queue_length("foreground_writes", [this] { return writes - background_writes; },
sm::description("number of currently pending foreground write requests"),
{storage_proxy_stats::current_scheduling_group_label()}),
sm::make_queue_length("background_writes", background_writes,
sm::description("number of currently pending background write requests"),
{storage_proxy_stats::current_scheduling_group_label()}),
sm::make_queue_length("current_throttled_base_writes", throttled_base_writes,
sm::description("number of currently throttled base replica write requests"),
{storage_proxy_stats::current_scheduling_group_label()}),
sm::make_gauge("last_mv_flow_control_delay", [this] { return std::chrono::duration(last_mv_flow_control_delay).count(); },
sm::description("delay (in seconds) added for MV flow control in the last request"),
{storage_proxy_stats::current_scheduling_group_label()}),
sm::make_total_operations("throttled_writes", throttled_writes,
sm::description("number of throttled write requests"),
{storage_proxy_stats::current_scheduling_group_label()}),
sm::make_total_operations("write_timeouts", write_timeouts._count,
sm::description("number of write request failed due to a timeout"),
{storage_proxy_stats::current_scheduling_group_label()}),
sm::make_total_operations("write_unavailable", write_unavailables._count,
sm::description("number write requests failed due to an \"unavailable\" error"),
{storage_proxy_stats::current_scheduling_group_label()}),
sm::make_total_operations("background_writes_failed", background_writes_failed,
sm::description("number of write requests that failed after CL was reached"),
{storage_proxy_stats::current_scheduling_group_label()}),
sm::make_total_operations("writes_coordinator_outside_replica_set", writes_coordinator_outside_replica_set,
sm::description("number of CQL write requests which arrived to a non-replica and had to be forwarded to a replica"),
{storage_proxy_stats::current_scheduling_group_label()}),
sm::make_total_operations("reads_coordinator_outside_replica_set", reads_coordinator_outside_replica_set,
sm::description("number of CQL read requests which arrived to a non-replica and had to be forwarded to a replica"),
{storage_proxy_stats::current_scheduling_group_label()}),
sm::make_total_operations("writes_failed_due_to_too_many_in_flight_hints", writes_failed_due_to_too_many_in_flight_hints,
sm::description("number of CQL write requests which failed because the hinted handoff mechanism is overloaded "
"and cannot store any more in-flight hints"),
{storage_proxy_stats::current_scheduling_group_label()}),
});
}
storage_proxy_stats::stats::stats()
: write_stats()
, data_read_attempts(COORDINATOR_STATS_CATEGORY, "reads", "number of data read requests", "data")
, data_read_completed(COORDINATOR_STATS_CATEGORY, "completed_reads", "number of data read requests that completed", "data")
, data_read_errors(COORDINATOR_STATS_CATEGORY, "read_errors", "number of data read requests that failed", "data")
, digest_read_attempts(COORDINATOR_STATS_CATEGORY, "reads", "number of digest read requests", "digest")
, digest_read_completed(COORDINATOR_STATS_CATEGORY, "completed_reads", "number of digest read requests that completed", "digest")
, digest_read_errors(COORDINATOR_STATS_CATEGORY, "read_errors", "number of digest read requests that failed", "digest")
, mutation_data_read_attempts(COORDINATOR_STATS_CATEGORY, "reads", "number of mutation data read requests", "mutation_data")
, mutation_data_read_completed(COORDINATOR_STATS_CATEGORY, "completed_reads", "number of mutation data read requests that completed", "mutation_data")
, mutation_data_read_errors(COORDINATOR_STATS_CATEGORY, "read_errors", "number of mutation data read requests that failed", "mutation_data") { }
void storage_proxy_stats::stats::register_split_metrics_local() {
write_stats::register_split_metrics_local();
data_read_attempts.register_metrics_local();
data_read_completed.register_metrics_local();
data_read_errors.register_metrics_local();
digest_read_attempts.register_metrics_local();
digest_read_completed.register_metrics_local();
mutation_data_read_attempts.register_metrics_local();
mutation_data_read_completed.register_metrics_local();
mutation_data_read_errors.register_metrics_local();
}
void storage_proxy_stats::stats::register_stats() {
namespace sm = seastar::metrics;
write_stats::register_stats();
_metrics.add_group(COORDINATOR_STATS_CATEGORY, {
sm::make_histogram("read_latency", sm::description("The general read latency histogram"),
{storage_proxy_stats::current_scheduling_group_label()},
[this]{ return to_metrics_histogram(estimated_read);}),
sm::make_queue_length("foreground_reads", foreground_reads,
sm::description("number of currently pending foreground read requests"),
{storage_proxy_stats::current_scheduling_group_label()}),
sm::make_queue_length("background_reads", [this] { return reads - foreground_reads; },
sm::description("number of currently pending background read requests"),
{storage_proxy_stats::current_scheduling_group_label()}),
sm::make_total_operations("read_retries", read_retries,
sm::description("number of read retry attempts"),
{storage_proxy_stats::current_scheduling_group_label()}),
sm::make_total_operations("canceled_read_repairs", global_read_repairs_canceled_due_to_concurrent_write,
sm::description("number of global read repairs canceled due to a concurrent write"),
{storage_proxy_stats::current_scheduling_group_label()}),
sm::make_total_operations("foreground_read_repairs", read_repair_repaired_blocking,
sm::description("number of foreground read repairs"),
{storage_proxy_stats::current_scheduling_group_label()}),
sm::make_total_operations("background_read_repairs", read_repair_repaired_background,
sm::description("number of background read repairs"),
{storage_proxy_stats::current_scheduling_group_label()}),
sm::make_total_operations("read_timeouts", read_timeouts._count,
sm::description("number of read request failed due to a timeout"),
{storage_proxy_stats::current_scheduling_group_label()}),
sm::make_total_operations("read_unavailable", read_unavailables._count,
sm::description("number read requests failed due to an \"unavailable\" error"),
{storage_proxy_stats::current_scheduling_group_label()}),
sm::make_total_operations("range_timeouts", range_slice_timeouts._count,
sm::description("number of range read operations failed due to a timeout"),
{storage_proxy_stats::current_scheduling_group_label()}),
sm::make_total_operations("range_unavailable", range_slice_unavailables._count,
sm::description("number of range read operations failed due to an \"unavailable\" error"),
{storage_proxy_stats::current_scheduling_group_label()}),
sm::make_total_operations("speculative_digest_reads", speculative_digest_reads,
sm::description("number of speculative digest read requests that were sent"),
{storage_proxy_stats::current_scheduling_group_label()}),
sm::make_total_operations("speculative_data_reads", speculative_data_reads,
sm::description("number of speculative data read requests that were sent"),
{storage_proxy_stats::current_scheduling_group_label()}),
sm::make_histogram("cas_read_latency", sm::description("Transactional read latency histogram"),
{storage_proxy_stats::current_scheduling_group_label()},
[this]{ return to_metrics_histogram(estimated_cas_read);}),
sm::make_histogram("cas_write_latency", sm::description("Transactional write latency histogram"),
{storage_proxy_stats::current_scheduling_group_label()},
[this]{return to_metrics_histogram(estimated_cas_write);}),
sm::make_total_operations("cas_write_timeouts", cas_write_timeouts._count,
sm::description("number of transactional write request failed due to a timeout"),
{storage_proxy_stats::current_scheduling_group_label()}),
sm::make_total_operations("cas_write_unavailable", cas_write_unavailables._count,
sm::description("number of transactional write requests failed due to an \"unavailable\" error"),
{storage_proxy_stats::current_scheduling_group_label()}),
sm::make_total_operations("cas_read_timeouts", cas_read_timeouts._count,
sm::description("number of transactional read request failed due to a timeout"),
{storage_proxy_stats::current_scheduling_group_label()}),
sm::make_total_operations("cas_read_unavailable", cas_read_unavailables._count,
sm::description("number of transactional read requests failed due to an \"unavailable\" error"),
{storage_proxy_stats::current_scheduling_group_label()}),
sm::make_total_operations("cas_read_unfinished_commit", cas_read_unfinished_commit,
sm::description("number of transaction commit attempts that occurred on read"),
{storage_proxy_stats::current_scheduling_group_label()}),
sm::make_total_operations("cas_write_unfinished_commit", cas_write_unfinished_commit,
sm::description("number of transaction commit attempts that occurred on write"),
{storage_proxy_stats::current_scheduling_group_label()}),
sm::make_total_operations("cas_write_condition_not_met", cas_write_condition_not_met,
sm::description("number of transaction preconditions that did not match current values"),
{storage_proxy_stats::current_scheduling_group_label()}),
sm::make_total_operations("cas_write_timeout_due_to_uncertainty", cas_write_timeout_due_to_uncertainty,
sm::description("how many times write timeout was reported because of uncertainty in the result"),
{storage_proxy_stats::current_scheduling_group_label()}),
sm::make_total_operations("cas_failed_read_round_optimization", cas_failed_read_round_optimization,
sm::description("CAS read rounds issued only if previous value is missing on some replica"),
{storage_proxy_stats::current_scheduling_group_label()}),
sm::make_histogram("cas_read_contention", sm::description("how many contended reads were encountered"),
{storage_proxy_stats::current_scheduling_group_label()},
[this]{ return cas_read_contention.get_histogram(1, 8);}),
sm::make_histogram("cas_write_contention", sm::description("how many contended writes were encountered"),
{storage_proxy_stats::current_scheduling_group_label()},
[this]{ return cas_write_contention.get_histogram(1, 8);}),
sm::make_total_operations("cas_prune", cas_prune,
sm::description("how many times paxos prune was done after successful cas operation"),
{storage_proxy_stats::current_scheduling_group_label()}),
sm::make_total_operations("cas_dropped_prune", cas_coordinator_dropped_prune,
sm::description("how many times a coordinator did not perfom prune after cas"),
{storage_proxy_stats::current_scheduling_group_label()}),
sm::make_total_operations("cas_total_operations", cas_total_operations,
sm::description("number of total paxos operations executed (reads and writes)"),
{storage_proxy_stats::current_scheduling_group_label()}),
sm::make_gauge("cas_foreground", cas_foreground,
sm::description("how many paxos operations that did not yet produce a result are running"),
{storage_proxy_stats::current_scheduling_group_label()}),
sm::make_gauge("cas_background", [this] { return cas_total_running - cas_foreground; },
sm::description("how many paxos operations are still running after a result was alredy returned"),
{storage_proxy_stats::current_scheduling_group_label()}),
});
_metrics.add_group(REPLICA_STATS_CATEGORY, {
sm::make_total_operations("received_counter_updates", received_counter_updates,
sm::description("number of counter updates received by this node acting as an update leader"),
{storage_proxy_stats::current_scheduling_group_label()}),
sm::make_total_operations("received_mutations", received_mutations,
sm::description("number of mutations received by a replica Node"),
{storage_proxy_stats::current_scheduling_group_label()}),
sm::make_total_operations("forwarded_mutations", forwarded_mutations,
sm::description("number of mutations forwarded to other replica Nodes"),
{storage_proxy_stats::current_scheduling_group_label()}),
sm::make_total_operations("forwarding_errors", forwarding_errors,
sm::description("number of errors during forwarding mutations to other replica Nodes"),
{storage_proxy_stats::current_scheduling_group_label()}),
sm::make_total_operations("reads", replica_data_reads,
sm::description("number of remote data read requests this Node received"),
{storage_proxy_stats::current_scheduling_group_label(), storage_proxy_stats::op_type_label("data")}),
sm::make_total_operations("reads", replica_mutation_data_reads,
sm::description("number of remote mutation data read requests this Node received"),
{storage_proxy_stats::current_scheduling_group_label(), storage_proxy_stats::op_type_label("mutation_data")}),
sm::make_total_operations("reads", replica_digest_reads,
sm::description("number of remote digest read requests this Node received"),
{storage_proxy_stats::current_scheduling_group_label(), storage_proxy_stats::op_type_label("digest")}),
sm::make_total_operations("cross_shard_ops", replica_cross_shard_ops,
sm::description("number of operations that crossed a shard boundary"),
{storage_proxy_stats::current_scheduling_group_label()}),
sm::make_total_operations("cas_dropped_prune", cas_replica_dropped_prune,
sm::description("how many times a coordinator did not perfom prune after cas"),
{storage_proxy_stats::current_scheduling_group_label()}),
});
}
inline uint64_t& storage_proxy_stats::split_stats::get_ep_stat(gms::inet_address ep) noexcept {
if (fbu::is_me(ep)) {
return _local.val;
}
try {
sstring dc = get_dc(ep);
if (_auto_register_metrics) {
register_metrics_for(ep);
}
return _dc_stats[dc].val;
} catch (...) {
static thread_local uint64_t dummy_stat;
slogger.error("Failed to obtain stats ({}), fall-back to dummy", std::current_exception());
return dummy_stat;
}
}
void storage_proxy_stats::split_stats::register_metrics_local() {
namespace sm = seastar::metrics;
_metrics.add_group(_category, {
sm::make_derive(_short_description_prefix + sstring("_local_node"), [this] { return _local.val; },
sm::description(_long_description_prefix + "on a local Node"), {storage_proxy_stats::current_scheduling_group_label(), op_type_label(_op_type)})
});
}
void storage_proxy_stats::split_stats::register_metrics_for(gms::inet_address ep) {
namespace sm = seastar::metrics;
sstring dc = get_dc(ep);
// if this is the first time we see an endpoint from this DC - add a
// corresponding collectd metric
if (auto [ignored, added] = _dc_stats.try_emplace(dc); added) {
_metrics.add_group(_category, {
sm::make_derive(_short_description_prefix + sstring("_remote_node"), [this, dc] { return _dc_stats[dc].val; },
sm::description(seastar::format("{} when communicating with external Nodes in DC {}", _long_description_prefix, dc)), {storage_proxy_stats::current_scheduling_group_label(), datacenter_label(dc), op_type_label(_op_type)})
});
}
}
void storage_proxy_stats::global_write_stats::register_stats() {
namespace sm = seastar::metrics;
_metrics.add_group(COORDINATOR_STATS_CATEGORY, {
sm::make_current_bytes("queued_write_bytes", queued_write_bytes,
sm::description("number of bytes in pending write requests"),
{storage_proxy_stats::current_scheduling_group_label()}),
sm::make_current_bytes("background_write_bytes", background_write_bytes,
sm::description("number of bytes in pending background write requests"),
{storage_proxy_stats::current_scheduling_group_label()}),
});
}
void storage_proxy_stats::global_stats::register_stats() {
global_write_stats::register_stats();
}
// A helper structure for differentiating hints from mutations in overload resolution
struct hint_wrapper {
mutation mut;
};
inline std::ostream& operator<<(std::ostream& os, const hint_wrapper& h) {
return os << "hint_wrapper{" << h.mut << "}";
}
using namespace std::literals::chrono_literals;
storage_proxy::~storage_proxy() {}
storage_proxy::storage_proxy(distributed& db, storage_proxy::config cfg, db::view::node_update_backlog& max_view_update_backlog,
scheduling_group_key stats_key, gms::feature_service& feat, const locator::shared_token_metadata& stm, netw::messaging_service& ms)
: _db(db)
, _shared_token_metadata(stm)
, _read_smp_service_group(cfg.read_smp_service_group)
, _write_smp_service_group(cfg.write_smp_service_group)
, _hints_write_smp_service_group(cfg.hints_write_smp_service_group)
, _write_ack_smp_service_group(cfg.write_ack_smp_service_group)
, _next_response_id(std::chrono::system_clock::now().time_since_epoch()/1ms)
, _hints_resource_manager(cfg.available_memory / 10)
, _hints_manager(_db.local().get_config().hints_directory(), cfg.hinted_handoff_enabled, _db.local().get_config().max_hint_window_in_ms(), _hints_resource_manager, _db)
, _hints_directory_initializer(std::move(cfg.hints_directory_initializer))
, _hints_for_views_manager(_db.local().get_config().view_hints_directory(), {}, _db.local().get_config().max_hint_window_in_ms(), _hints_resource_manager, _db)
, _stats_key(stats_key)
, _features(feat)
, _messaging(ms)
, _background_write_throttle_threahsold(cfg.available_memory / 10)
, _mutate_stage{"storage_proxy_mutate", &storage_proxy::do_mutate}
, _max_view_update_backlog(max_view_update_backlog)
, _view_update_handlers_list(std::make_unique()) {
namespace sm = seastar::metrics;
_metrics.add_group(storage_proxy_stats::COORDINATOR_STATS_CATEGORY, {
sm::make_queue_length("current_throttled_writes", [this] { return _throttled_writes.size(); },
sm::description("number of currently throttled write requests")),
});
slogger.trace("hinted DCs: {}", cfg.hinted_handoff_enabled.to_configuration_string());
_hints_manager.register_metrics("hints_manager");
_hints_for_views_manager.register_metrics("hints_for_views_manager");
}
storage_proxy::unique_response_handler::unique_response_handler(storage_proxy& p_, response_id_type id_) : id(id_), p(p_) {}
storage_proxy::unique_response_handler::unique_response_handler(unique_response_handler&& x) : id(x.id), p(x.p) { x.id = 0; };
storage_proxy::unique_response_handler::~unique_response_handler() {
if (id) {
p.remove_response_handler(id);
}
}
storage_proxy::response_id_type storage_proxy::unique_response_handler::release() {
auto r = id;
id = 0;
return r;
}
future<>
storage_proxy::mutate_locally(const mutation& m, tracing::trace_state_ptr tr_state, db::commitlog::force_sync sync, clock_type::time_point timeout, smp_service_group smp_grp) {
auto shard = _db.local().shard_of(m);
get_stats().replica_cross_shard_ops += shard != this_shard_id();
return _db.invoke_on(shard, {smp_grp, timeout},
[s = global_schema_ptr(m.schema()),
m = freeze(m),
gtr = tracing::global_trace_state_ptr(std::move(tr_state)),
timeout,
sync] (database& db) mutable -> future<> {
return db.apply(s, m, gtr.get(), sync, timeout);
});
}
future<>
storage_proxy::mutate_locally(const schema_ptr& s, const frozen_mutation& m, tracing::trace_state_ptr tr_state, db::commitlog::force_sync sync, clock_type::time_point timeout,
smp_service_group smp_grp) {
auto shard = _db.local().shard_of(m);
get_stats().replica_cross_shard_ops += shard != this_shard_id();
return _db.invoke_on(shard, {smp_grp, timeout},
[&m, gs = global_schema_ptr(s), gtr = tracing::global_trace_state_ptr(std::move(tr_state)), timeout, sync] (database& db) mutable -> future<> {
return db.apply(gs, m, gtr.get(), sync, timeout);
});
}
future<>
storage_proxy::mutate_locally(std::vector mutations, tracing::trace_state_ptr tr_state, clock_type::time_point timeout, smp_service_group smp_grp) {
return do_with(std::move(mutations), [this, timeout, tr_state = std::move(tr_state), smp_grp] (std::vector& pmut) mutable {
return parallel_for_each(pmut.begin(), pmut.end(), [this, tr_state = std::move(tr_state), timeout, smp_grp] (const mutation& m) mutable {
return mutate_locally(m, tr_state, db::commitlog::force_sync::no, timeout, smp_grp);
});
});
}
future<>
storage_proxy::mutate_locally(std::vector mutation, tracing::trace_state_ptr tr_state, clock_type::time_point timeout) {
return mutate_locally(std::move(mutation), tr_state, timeout, _write_smp_service_group);
}
future<>
storage_proxy::mutate_hint(const schema_ptr& s, const frozen_mutation& m, tracing::trace_state_ptr tr_state, clock_type::time_point timeout) {
auto shard = _db.local().shard_of(m);
get_stats().replica_cross_shard_ops += shard != this_shard_id();
return _db.invoke_on(shard, {_hints_write_smp_service_group, timeout}, [&m, gs = global_schema_ptr(s), tr_state = std::move(tr_state), timeout] (database& db) mutable -> future<> {
return db.apply_hint(gs, m, std::move(tr_state), timeout);
});
}
future<>
storage_proxy::mutate_counters_on_leader(std::vector mutations, db::consistency_level cl, clock_type::time_point timeout,
tracing::trace_state_ptr trace_state, service_permit permit) {
get_stats().received_counter_updates += mutations.size();
return do_with(std::move(mutations), [this, cl, timeout, trace_state = std::move(trace_state), permit = std::move(permit)] (std::vector& update_ms) mutable {
return parallel_for_each(update_ms, [this, cl, timeout, trace_state, permit] (frozen_mutation_and_schema& fm_a_s) {
return mutate_counter_on_leader_and_replicate(fm_a_s.s, std::move(fm_a_s.fm), cl, timeout, trace_state, permit);
});
});
}
future<>
storage_proxy::mutate_counter_on_leader_and_replicate(const schema_ptr& s, frozen_mutation fm, db::consistency_level cl, clock_type::time_point timeout,
tracing::trace_state_ptr trace_state, service_permit permit) {
auto shard = _db.local().shard_of(fm);
bool local = shard == this_shard_id();
get_stats().replica_cross_shard_ops += !local;
return _db.invoke_on(shard, {_write_smp_service_group, timeout}, [gs = global_schema_ptr(s), fm = std::move(fm), cl, timeout, gt = tracing::global_trace_state_ptr(std::move(trace_state)), permit = std::move(permit), local] (database& db) {
auto trace_state = gt.get();
auto p = local ? std::move(permit) : /* FIXME: either obtain a real permit on this shard or hold original one across shard */ empty_service_permit();
return db.apply_counter_update(gs, fm, timeout, trace_state).then([cl, timeout, trace_state, p = std::move(p)] (mutation m) mutable {
return service::get_local_storage_proxy().replicate_counter_from_leader(std::move(m), cl, std::move(trace_state), timeout, std::move(p));
});
});
}
storage_proxy::response_id_type
storage_proxy::create_write_response_handler_helper(schema_ptr s, const dht::token& token, std::unique_ptr mh,
db::consistency_level cl, db::write_type type, tracing::trace_state_ptr tr_state, service_permit permit) {
auto keyspace_name = s->ks_name();
keyspace& ks = _db.local().find_keyspace(keyspace_name);
auto& rs = ks.get_replication_strategy();
std::vector natural_endpoints = rs.get_natural_endpoints_without_node_being_replaced(token);
std::vector pending_endpoints = get_token_metadata_ptr()->pending_endpoints_for(token, keyspace_name);
slogger.trace("creating write handler for token: {} natural: {} pending: {}", token, natural_endpoints, pending_endpoints);
tracing::trace(tr_state, "Creating write handler for token: {} natural: {} pending: {}", token, natural_endpoints ,pending_endpoints);
// Check if this node, which is serving as a coordinator for
// the mutation, is also a replica for the partition being
// changed. Mutations sent by drivers unaware of token
// distribution create a lot of network noise and thus should be
// accounted in the metrics.
if (std::find(natural_endpoints.begin(), natural_endpoints.end(),
utils::fb_utilities::get_broadcast_address()) == natural_endpoints.end()) {
get_stats().writes_coordinator_outside_replica_set++;
}
// filter out natural_endpoints from pending_endpoints if the latter is not yet updated during node join
auto itend = boost::range::remove_if(pending_endpoints, [&natural_endpoints] (gms::inet_address& p) {
return boost::range::find(natural_endpoints, p) != natural_endpoints.end();
});
pending_endpoints.erase(itend, pending_endpoints.end());
auto all = boost::range::join(natural_endpoints, pending_endpoints);
if (cannot_hint(all, type)) {
get_stats().writes_failed_due_to_too_many_in_flight_hints++;
// avoid OOMing due to excess hints. we need to do this check even for "live" nodes, since we can
// still generate hints for those if it's overloaded or simply dead but not yet known-to-be-dead.
// The idea is that if we have over maxHintsInProgress hints in flight, this is probably due to
// a small number of nodes causing problems, so we should avoid shutting down writes completely to
// healthy nodes. Any node with no hintsInProgress is considered healthy.
throw overloaded_exception(_hints_manager.size_of_hints_in_progress());
}
// filter live endpoints from dead ones
std::unordered_set live_endpoints;
std::vector dead_endpoints;
live_endpoints.reserve(all.size());
dead_endpoints.reserve(all.size());
std::partition_copy(all.begin(), all.end(), std::inserter(live_endpoints, live_endpoints.begin()),
std::back_inserter(dead_endpoints), std::bind1st(std::mem_fn(&gms::gossiper::is_alive), &gms::get_local_gossiper()));
slogger.trace("creating write handler with live: {} dead: {}", live_endpoints, dead_endpoints);
tracing::trace(tr_state, "Creating write handler with live: {} dead: {}", live_endpoints, dead_endpoints);
db::assure_sufficient_live_nodes(cl, ks, live_endpoints, pending_endpoints);
return create_write_response_handler(ks, cl, type, std::move(mh), std::move(live_endpoints), pending_endpoints,
std::move(dead_endpoints), std::move(tr_state), get_stats(), std::move(permit));
}
/**
* Helper for create_write_response_handler, shared across mutate/mutate_atomically.
* Both methods do roughly the same thing, with the latter intermixing batch log ops
* in the logic.
* Since ordering is (maybe?) significant, we need to carry some info across from here
* to the hint method below (dead nodes).
*/
storage_proxy::response_id_type
storage_proxy::create_write_response_handler(const mutation& m, db::consistency_level cl, db::write_type type, tracing::trace_state_ptr tr_state, service_permit permit) {
return create_write_response_handler_helper(m.schema(), m.token(), std::make_unique(m), cl, type, tr_state,
std::move(permit));
}
storage_proxy::response_id_type
storage_proxy::create_write_response_handler(const hint_wrapper& h, db::consistency_level cl, db::write_type type, tracing::trace_state_ptr tr_state, service_permit permit) {
return create_write_response_handler_helper(h.mut.schema(), h.mut.token(), std::make_unique(h.mut), cl, type, tr_state,
std::move(permit));
}
storage_proxy::response_id_type
storage_proxy::create_write_response_handler(const std::unordered_map>& m, db::consistency_level cl, db::write_type type, tracing::trace_state_ptr tr_state, service_permit permit) {
std::unordered_set endpoints(m.size());
boost::copy(m | boost::adaptors::map_keys, std::inserter(endpoints, endpoints.begin()));
auto mh = std::make_unique(m);
slogger.trace("creating write handler for read repair token: {} endpoint: {}", mh->token(), endpoints);
tracing::trace(tr_state, "Creating write handler for read repair token: {} endpoint: {}", mh->token(), endpoints);
auto keyspace_name = mh->schema()->ks_name();
keyspace& ks = _db.local().find_keyspace(keyspace_name);
return create_write_response_handler(ks, cl, type, std::move(mh), std::move(endpoints), std::vector(), std::vector(), std::move(tr_state), get_stats(), std::move(permit));
}
storage_proxy::response_id_type
storage_proxy::create_write_response_handler(const std::tuple, schema_ptr, shared_ptr, dht::token>& meta,
db::consistency_level cl, db::write_type type, tracing::trace_state_ptr tr_state, service_permit permit) {
auto& [commit, s, h, t] = meta;
return create_write_response_handler_helper(s, t, std::make_unique(std::move(commit), s, std::move(h)), cl,
db::write_type::CAS, tr_state, std::move(permit));
}
storage_proxy::response_id_type
storage_proxy::create_write_response_handler(const std::tuple, schema_ptr, dht::token, std::unordered_set>& meta,
db::consistency_level cl, db::write_type type, tracing::trace_state_ptr tr_state, service_permit permit) {
auto& [commit, s, token, endpoints] = meta;
slogger.trace("creating write handler for paxos repair token: {} endpoint: {}", token, endpoints);
tracing::trace(tr_state, "Creating write handler for paxos repair token: {} endpoint: {}", token, endpoints);
auto keyspace_name = s->ks_name();
keyspace& ks = _db.local().find_keyspace(keyspace_name);
return create_write_response_handler(ks, cl, db::write_type::CAS, std::make_unique(std::move(commit), s, nullptr), std::move(endpoints),
std::vector(), std::vector(), std::move(tr_state), get_stats(), std::move(permit));
}
void storage_proxy::register_cdc_operation_result_tracker(const std::vector& ids, lw_shared_ptr tracker) {
if (!tracker) {
return;
}
for (auto& id : ids) {
auto& h = get_write_response_handler(id.id);
if (h->get_schema()->cdc_options().enabled()) {
h->set_cdc_operation_result_tracker(tracker);
}
}
}
void
storage_proxy::hint_to_dead_endpoints(response_id_type id, db::consistency_level cl) {
auto& h = *get_write_response_handler(id);
size_t hints = hint_to_dead_endpoints(h._mutation_holder, h.get_dead_endpoints(), h._type, h.get_trace_state());
if (cl == db::consistency_level::ANY) {
// for cl==ANY hints are counted towards consistency
h.signal(hints);
}
}
template
future> storage_proxy::mutate_prepare(Range&& mutations, db::consistency_level cl, db::write_type type, service_permit permit, CreateWriteHandler create_handler) {
// apply is used to convert exceptions to exceptional future
return futurize_invoke([this] (Range&& mutations, db::consistency_level cl, db::write_type type, service_permit permit, CreateWriteHandler create_handler) {
std::vector ids;
ids.reserve(std::distance(std::begin(mutations), std::end(mutations)));
for (auto& m : mutations) {
ids.emplace_back(*this, create_handler(m, cl, type, permit));
}
return make_ready_future>(std::move(ids));
}, std::forward(mutations), cl, type, std::move(permit), std::move(create_handler));
}
template
future> storage_proxy::mutate_prepare(Range&& mutations, db::consistency_level cl, db::write_type type, tracing::trace_state_ptr tr_state, service_permit permit) {
return mutate_prepare<>(std::forward(mutations), cl, type, std::move(permit), [this, tr_state = std::move(tr_state)] (const typename std::decay_t::value_type& m, db::consistency_level cl, db::write_type type, service_permit permit) mutable {
return create_write_response_handler(m, cl, type, tr_state, std::move(permit));
});
}
future<> storage_proxy::mutate_begin(std::vector ids, db::consistency_level cl,
tracing::trace_state_ptr trace_state, std::optional timeout_opt) {
return parallel_for_each(ids, [this, cl, timeout_opt] (unique_response_handler& protected_response) {
auto response_id = protected_response.id;
// This function, mutate_begin(), is called after a preemption point
// so it's possible that other code besides our caller just ran. In
// particular, Scylla may have noticed that a remote node went down,
// called storage_proxy::on_down(), and removed some of the ongoing
// handlers, including this id. If this happens, we need to ignore
// this id - not try to look it up or start a send.
if (!_response_handlers.contains(response_id)) {
protected_response.release(); // Don't try to remove this id again
// Requests that time-out normally below after response_wait()
// result in an exception (see ~abstract_write_response_handler())
// However, here we no longer have the handler or its information
// to put in the exception. The exception is not needed for
// correctness (e.g., hints are written by timeout_cb(), not
// because of an exception here).
slogger.debug("unstarted write cancelled for id {}", response_id);
return make_ready_future<>();
}
// it is better to send first and hint afterwards to reduce latency
// but request may complete before hint_to_dead_endpoints() is called and
// response_id handler will be removed, so we will have to do hint with separate
// frozen_mutation copy, or manage handler live time differently.
hint_to_dead_endpoints(response_id, cl);
auto timeout = timeout_opt.value_or(clock_type::now() + std::chrono::milliseconds(_db.local().get_config().write_request_timeout_in_ms()));
// call before send_to_live_endpoints() for the same reason as above
auto f = response_wait(response_id, timeout);
send_to_live_endpoints(protected_response.release(), timeout); // response is now running and it will either complete or timeout
return f;
});
}
// this function should be called with a future that holds result of mutation attempt (usually
// future returned by mutate_begin()). The future should be ready when function is called.
future<> storage_proxy::mutate_end(future<> mutate_result, utils::latency_counter lc, write_stats& stats, tracing::trace_state_ptr trace_state) {
assert(mutate_result.available());
stats.write.mark(lc.stop().latency());
if (lc.is_start()) {
stats.estimated_write.add(lc.latency());
}
try {
mutate_result.get();
tracing::trace(trace_state, "Mutation successfully completed");
return make_ready_future<>();
} catch (no_such_keyspace& ex) {
tracing::trace(trace_state, "Mutation failed: write to non existing keyspace: {}", ex.what());
slogger.trace("Write to non existing keyspace: {}", ex.what());
return make_exception_future<>(std::current_exception());
} catch(mutation_write_timeout_exception& ex) {
// timeout
tracing::trace(trace_state, "Mutation failed: write timeout; received {:d} of {:d} required replies", ex.received, ex.block_for);
slogger.debug("Write timeout; received {} of {} required replies", ex.received, ex.block_for);
stats.write_timeouts.mark();
return make_exception_future<>(std::current_exception());
} catch (exceptions::unavailable_exception& ex) {
tracing::trace(trace_state, "Mutation failed: unavailable");
stats.write_unavailables.mark();
slogger.trace("Unavailable");
return make_exception_future<>(std::current_exception());
} catch(overloaded_exception& ex) {
tracing::trace(trace_state, "Mutation failed: overloaded");
stats.write_unavailables.mark();
slogger.trace("Overloaded");
return make_exception_future<>(std::current_exception());
} catch (...) {
tracing::trace(trace_state, "Mutation failed: unknown reason");
throw;
}
}
gms::inet_address storage_proxy::find_leader_for_counter_update(const mutation& m, db::consistency_level cl) {
auto& ks = _db.local().find_keyspace(m.schema()->ks_name());
auto live_endpoints = get_live_endpoints(ks, m.token());
if (live_endpoints.empty()) {
throw exceptions::unavailable_exception(cl, block_for(ks, cl), 0);
}
const auto my_address = utils::fb_utilities::get_broadcast_address();
// Early return if coordinator can become the leader (so one extra internode message can be
// avoided). With token-aware drivers this is the expected case, so we are doing it ASAP.
if (boost::algorithm::any_of_equal(live_endpoints, my_address)) {
return my_address;
}
const auto local_endpoints = boost::copy_range>(live_endpoints | boost::adaptors::filtered([&] (auto&& ep) {
return db::is_local(ep);
}));
if (local_endpoints.empty()) {
// FIXME: O(n log n) to get maximum
auto& snitch = locator::i_endpoint_snitch::get_local_snitch_ptr();
snitch->sort_by_proximity(my_address, live_endpoints);
return live_endpoints[0];
} else {
static thread_local std::default_random_engine re{std::random_device{}()};
std::uniform_int_distribution<> dist(0, local_endpoints.size() - 1);
return local_endpoints[dist(re)];
}
}
template
future<> storage_proxy::mutate_counters(Range&& mutations, db::consistency_level cl, tracing::trace_state_ptr tr_state, service_permit permit, clock_type::time_point timeout) {
if (boost::empty(mutations)) {
return make_ready_future<>();
}
slogger.trace("mutate_counters cl={}", cl);
mlogger.trace("counter mutations={}", mutations);
// Choose a leader for each mutation
std::unordered_map> leaders;
for (auto& m : mutations) {
auto leader = find_leader_for_counter_update(m, cl);
leaders[leader].emplace_back(frozen_mutation_and_schema { freeze(m), m.schema() });
// FIXME: check if CL can be reached
}
// Forward mutations to the leaders chosen for them
auto my_address = utils::fb_utilities::get_broadcast_address();
return parallel_for_each(leaders, [this, cl, timeout, tr_state = std::move(tr_state), permit = std::move(permit), my_address] (auto& endpoint_and_mutations) {
auto endpoint = endpoint_and_mutations.first;
// The leader receives a vector of mutations and processes them together,
// so if there is a timeout we don't really know which one is to "blame"
// and what to put in ks and cf fields of write timeout exception.
// Let's just use the schema of the first mutation in a vector.
auto handle_error = [this, sp = this->shared_from_this(), s = endpoint_and_mutations.second[0].s, cl, permit] (std::exception_ptr exp) {
auto& ks = _db.local().find_keyspace(s->ks_name());
try {
std::rethrow_exception(std::move(exp));
} catch (rpc::timeout_error&) {
return make_exception_future<>(mutation_write_timeout_exception(s->ks_name(), s->cf_name(), cl, 0, db::block_for(ks, cl), db::write_type::COUNTER));
} catch (timed_out_error&) {
return make_exception_future<>(mutation_write_timeout_exception(s->ks_name(), s->cf_name(), cl, 0, db::block_for(ks, cl), db::write_type::COUNTER));
} catch (rpc::closed_error&) {
return make_exception_future<>(mutation_write_failure_exception(s->ks_name(), s->cf_name(), cl, 0, 1, db::block_for(ks, cl), db::write_type::COUNTER));
}
};
auto f = make_ready_future<>();
if (endpoint == my_address) {
f = this->mutate_counters_on_leader(std::move(endpoint_and_mutations.second), cl, timeout, tr_state, permit);
} else {
auto& mutations = endpoint_and_mutations.second;
auto fms = boost::copy_range>(mutations | boost::adaptors::transformed([] (auto& m) {
return std::move(m.fm);
}));
// Coordinator is preferred as the leader - if it's not selected we can assume
// that the query was non-token-aware and bump relevant metric.
get_stats().writes_coordinator_outside_replica_set += fms.size();
auto msg_addr = netw::messaging_service::msg_addr{ endpoint_and_mutations.first, 0 };
tracing::trace(tr_state, "Enqueuing counter update to {}", msg_addr);
f = _messaging.send_counter_mutation(msg_addr, timeout, std::move(fms), cl, tracing::make_trace_info(tr_state));
}
return f.handle_exception(std::move(handle_error));
});
}
storage_proxy::paxos_participants
storage_proxy::get_paxos_participants(const sstring& ks_name, const dht::token &token, db::consistency_level cl_for_paxos) {
keyspace& ks = _db.local().find_keyspace(ks_name);
auto& rs = ks.get_replication_strategy();
std::vector natural_endpoints = rs.get_natural_endpoints_without_node_being_replaced(token);
std::vector pending_endpoints = get_token_metadata_ptr()->pending_endpoints_for(token, ks_name);
if (cl_for_paxos == db::consistency_level::LOCAL_SERIAL) {
auto itend = boost::range::remove_if(natural_endpoints, std::not_fn(std::cref(db::is_local)));
natural_endpoints.erase(itend, natural_endpoints.end());
itend = boost::range::remove_if(pending_endpoints, std::not_fn(std::cref(db::is_local)));
pending_endpoints.erase(itend, pending_endpoints.end());
}
// filter out natural_endpoints from pending_endpoints if the latter is not yet updated during node join
// should never happen, but better to be safe
auto itend = boost::range::remove_if(pending_endpoints, [&natural_endpoints] (gms::inet_address& p) {
return boost::range::find(natural_endpoints, p) != natural_endpoints.end();
});
pending_endpoints.erase(itend, pending_endpoints.end());
const size_t participants = pending_endpoints.size() + natural_endpoints.size();
const size_t quorum_size = natural_endpoints.size() / 2 + 1;
const size_t required_participants = quorum_size + pending_endpoints.size();
std::vector live_endpoints;
live_endpoints.reserve(participants);
boost::copy(boost::range::join(natural_endpoints, pending_endpoints) |
boost::adaptors::filtered(std::bind1st(std::mem_fn(&gms::gossiper::is_alive), &gms::get_local_gossiper())), std::back_inserter(live_endpoints));
if (live_endpoints.size() < required_participants) {
throw exceptions::unavailable_exception(cl_for_paxos, required_participants, live_endpoints.size());
}
// We cannot allow CAS operations with 2 or more pending endpoints, see #8346.
// Note that we fake an impossible number of required nodes in the unavailable exception
// to nail home the point that it's an impossible operation no matter how many nodes are live.
if (pending_endpoints.size() > 1) {
throw exceptions::unavailable_exception(fmt::format(
"Cannot perform LWT operation as there is more than one ({}) pending range movement", pending_endpoints.size()),
cl_for_paxos, participants + 1, live_endpoints.size());
}
bool dead = participants != live_endpoints.size();
// Apart from the ballot, paxos_state::prepare() also sends the current value of the requested key.
// If the values received from different replicas match, we skip a separate query stage thus saving
// one network round trip. To generate less traffic, only closest replicas send data, others send
// digests that are used to check consistency. For this optimization to work, we need to sort the
// list of participants by proximity to this instance.
sort_endpoints_by_proximity(live_endpoints);
return paxos_participants{std::move(live_endpoints), required_participants, dead};
}
/**
* Use this method to have these Mutations applied
* across all replicas. This method will take care
* of the possibility of a replica being down and hint
* the data across to some other replica.
*
* @param mutations the mutations to be applied across the replicas
* @param consistency_level the consistency level for the operation
* @param tr_state trace state handle
*/
future<> storage_proxy::mutate(std::vector mutations, db::consistency_level cl, clock_type::time_point timeout, tracing::trace_state_ptr tr_state, service_permit permit, bool raw_counters) {
if (_cdc && _cdc->needs_cdc_augmentation(mutations)) {
return _cdc->augment_mutation_call(timeout, std::move(mutations), tr_state, cl).then([this, cl, timeout, tr_state, permit = std::move(permit), raw_counters, cdc = _cdc->shared_from_this()](std::tuple, lw_shared_ptr>&& t) mutable {
auto mutations = std::move(std::get<0>(t));
auto tracker = std::move(std::get<1>(t));
return _mutate_stage(this, std::move(mutations), cl, timeout, std::move(tr_state), std::move(permit), raw_counters, std::move(tracker));
});
}
return _mutate_stage(this, std::move(mutations), cl, timeout, std::move(tr_state), std::move(permit), raw_counters, nullptr);
}
future<> storage_proxy::do_mutate(std::vector mutations, db::consistency_level cl, clock_type::time_point timeout, tracing::trace_state_ptr tr_state, service_permit permit, bool raw_counters, lw_shared_ptr cdc_tracker) {
auto mid = raw_counters ? mutations.begin() : boost::range::partition(mutations, [] (auto&& m) {
return m.schema()->is_counter();
});
return seastar::when_all_succeed(
mutate_counters(boost::make_iterator_range(mutations.begin(), mid), cl, tr_state, permit, timeout),
mutate_internal(boost::make_iterator_range(mid, mutations.end()), cl, false, tr_state, permit, timeout, std::move(cdc_tracker))
).discard_result();
}
future<> storage_proxy::replicate_counter_from_leader(mutation m, db::consistency_level cl, tracing::trace_state_ptr tr_state,
clock_type::time_point timeout, service_permit permit) {
// FIXME: do not send the mutation to itself, it has already been applied (it is not incorrect to do so, though)
return mutate_internal(std::array{std::move(m)}, cl, true, std::move(tr_state), std::move(permit), timeout);
}
/*
* Range template parameter can either be range of 'mutation' or a range of 'std::unordered_map'.
* create_write_response_handler() has specialization for both types. The one for the former uses keyspace to figure out
* endpoints to send mutation to, the one for the late uses enpoints that are used as keys for the map.
*/
template
future<>
storage_proxy::mutate_internal(Range mutations, db::consistency_level cl, bool counters, tracing::trace_state_ptr tr_state, service_permit permit,
std::optional timeout_opt, lw_shared_ptr cdc_tracker) {
if (boost::empty(mutations)) {
return make_ready_future<>();
}
slogger.trace("mutate cl={}", cl);
mlogger.trace("mutations={}", mutations);
// If counters is set it means that we are replicating counter shards. There
// is no need for special handling anymore, since the leader has already
// done its job, but we need to return correct db::write_type in case of
// a timeout so that client doesn't attempt to retry the request.
auto type = counters ? db::write_type::COUNTER
: (std::next(std::begin(mutations)) == std::end(mutations) ? db::write_type::SIMPLE : db::write_type::UNLOGGED_BATCH);
utils::latency_counter lc;
lc.start();
return mutate_prepare(mutations, cl, type, tr_state, std::move(permit)).then([this, cl, timeout_opt, tracker = std::move(cdc_tracker),
tr_state] (std::vector ids) mutable {
register_cdc_operation_result_tracker(ids, tracker);
return mutate_begin(std::move(ids), cl, tr_state, timeout_opt);
}).then_wrapped([this, p = shared_from_this(), lc, tr_state] (future<> f) mutable {
return p->mutate_end(std::move(f), lc, get_stats(), std::move(tr_state));
});
}
future<>
storage_proxy::mutate_with_triggers(std::vector mutations, db::consistency_level cl,
clock_type::time_point timeout,
bool should_mutate_atomically, tracing::trace_state_ptr tr_state, service_permit permit, bool raw_counters) {
warn(unimplemented::cause::TRIGGERS);
if (should_mutate_atomically) {
assert(!raw_counters);
return mutate_atomically(std::move(mutations), cl, timeout, std::move(tr_state), std::move(permit));
}
return mutate(std::move(mutations), cl, timeout, std::move(tr_state), std::move(permit), raw_counters);
}
/**
* See mutate. Adds additional steps before and after writing a batch.
* Before writing the batch (but after doing availability check against the FD for the row replicas):
* write the entire batch to a batchlog elsewhere in the cluster.
* After: remove the batchlog entry (after writing hints for the batch rows, if necessary).
*
* @param mutations the Mutations to be applied across the replicas
* @param consistency_level the consistency level for the operation
*/
future<>
storage_proxy::mutate_atomically(std::vector mutations, db::consistency_level cl, clock_type::time_point timeout, tracing::trace_state_ptr tr_state, service_permit permit) {
utils::latency_counter lc;
lc.start();
class context {
storage_proxy& _p;
const locator::token_metadata_ptr _tmptr;
std::vector _mutations;
lw_shared_ptr _cdc_tracker;
db::consistency_level _cl;
clock_type::time_point _timeout;
tracing::trace_state_ptr _trace_state;
storage_proxy::stats& _stats;
service_permit _permit;
const utils::UUID _batch_uuid;
const std::unordered_set _batchlog_endpoints;
public:
context(storage_proxy & p, std::vector&& mutations, lw_shared_ptr&& cdc_tracker, db::consistency_level cl, clock_type::time_point timeout, tracing::trace_state_ptr tr_state, service_permit permit)
: _p(p)
, _tmptr(p.get_token_metadata_ptr())
, _mutations(std::move(mutations))
, _cdc_tracker(std::move(cdc_tracker))
, _cl(cl)
, _timeout(timeout)
, _trace_state(std::move(tr_state))
, _stats(p.get_stats())
, _permit(std::move(permit))
, _batch_uuid(utils::UUID_gen::get_time_UUID())
, _batchlog_endpoints(
[this]() -> std::unordered_set {
auto local_addr = utils::fb_utilities::get_broadcast_address();
auto& topology = _tmptr->get_topology();
auto& local_endpoints = topology.get_datacenter_racks().at(get_local_dc());
auto local_rack = locator::i_endpoint_snitch::get_local_snitch_ptr()->get_rack(local_addr);
auto chosen_endpoints = db::get_batchlog_manager().local().endpoint_filter(local_rack, local_endpoints);
if (chosen_endpoints.empty()) {
if (_cl == db::consistency_level::ANY) {
return {local_addr};
}
throw exceptions::unavailable_exception(db::consistency_level::ONE, 1, 0);
}
return chosen_endpoints;
}()) {
tracing::trace(_trace_state, "Created a batch context");
tracing::set_batchlog_endpoints(_trace_state, _batchlog_endpoints);
}
future<> send_batchlog_mutation(mutation m, db::consistency_level cl = db::consistency_level::ONE) {
return _p.mutate_prepare<>(std::array{std::move(m)}, cl, db::write_type::BATCH_LOG, _permit, [this] (const mutation& m, db::consistency_level cl, db::write_type type, service_permit permit) {
auto& ks = _p._db.local().find_keyspace(m.schema()->ks_name());
return _p.create_write_response_handler(ks, cl, type, std::make_unique(m), _batchlog_endpoints, {}, {}, _trace_state, _stats, std::move(permit));
}).then([this, cl] (std::vector ids) {
_p.register_cdc_operation_result_tracker(ids, _cdc_tracker);
return _p.mutate_begin(std::move(ids), cl, _trace_state, _timeout);
});
}
future<> sync_write_to_batchlog() {
auto m = db::get_batchlog_manager().local().get_batch_log_mutation_for(_mutations, _batch_uuid, netw::messaging_service::current_version);
tracing::trace(_trace_state, "Sending a batchlog write mutation");
return send_batchlog_mutation(std::move(m));
};
future<> async_remove_from_batchlog() {
// delete batch
auto schema = _p._db.local().find_schema(db::system_keyspace::NAME, db::system_keyspace::BATCHLOG);
auto key = partition_key::from_exploded(*schema, {uuid_type->decompose(_batch_uuid)});
auto now = service::client_state(service::client_state::internal_tag()).get_timestamp();
mutation m(schema, key);
m.partition().apply_delete(*schema, clustering_key_prefix::make_empty(), tombstone(now, gc_clock::now()));
tracing::trace(_trace_state, "Sending a batchlog remove mutation");
return send_batchlog_mutation(std::move(m), db::consistency_level::ANY).handle_exception([] (std::exception_ptr eptr) {
slogger.error("Failed to remove mutations from batchlog: {}", eptr);
});
};
future<> run() {
return _p.mutate_prepare(_mutations, _cl, db::write_type::BATCH, _trace_state, _permit).then([this] (std::vector ids) {
return sync_write_to_batchlog().then([this, ids = std::move(ids)] () mutable {
tracing::trace(_trace_state, "Sending batch mutations");
_p.register_cdc_operation_result_tracker(ids, _cdc_tracker);
return _p.mutate_begin(std::move(ids), _cl, _trace_state, _timeout);
}).then(std::bind(&context::async_remove_from_batchlog, this));
});
}
};
auto mk_ctxt = [this, tr_state, timeout, permit = std::move(permit), cl] (std::vector mutations, lw_shared_ptr tracker) mutable {
try {
return make_ready_future>(make_lw_shared(*this, std::move(mutations), std::move(tracker), cl, timeout, std::move(tr_state), std::move(permit)));
} catch(...) {
return make_exception_future>(std::current_exception());
}
};
auto cleanup = [p = shared_from_this(), lc, tr_state] (future<> f) mutable {
return p->mutate_end(std::move(f), lc, p->get_stats(), std::move(tr_state));
};
if (_cdc && _cdc->needs_cdc_augmentation(mutations)) {
return _cdc->augment_mutation_call(timeout, std::move(mutations), std::move(tr_state), cl).then([this, mk_ctxt = std::move(mk_ctxt), cleanup = std::move(cleanup), cdc = _cdc->shared_from_this()](std::tuple, lw_shared_ptr>&& t) mutable {
auto mutations = std::move(std::get<0>(t));
auto tracker = std::move(std::get<1>(t));
return std::move(mk_ctxt)(std::move(mutations), std::move(tracker)).then([this] (lw_shared_ptr ctxt) {
return ctxt->run().finally([ctxt]{});
}).then_wrapped(std::move(cleanup));
});
}
return mk_ctxt(std::move(mutations), nullptr).then([this] (lw_shared_ptr ctxt) {
return ctxt->run().finally([ctxt]{});
}).then_wrapped(std::move(cleanup));
}
template
bool storage_proxy::cannot_hint(const Range& targets, db::write_type type) const {
// if hints are disabled we "can always hint" since there's going to be no hint generated in this case
return hints_enabled(type) && boost::algorithm::any_of(targets, std::bind(&db::hints::manager::too_many_in_flight_hints_for, &_hints_manager, std::placeholders::_1));
}
future<> storage_proxy::send_to_endpoint(
std::unique_ptr m,
gms::inet_address target,
std::vector pending_endpoints,
db::write_type type,
tracing::trace_state_ptr tr_state,
write_stats& stats,
allow_hints allow_hints) {
utils::latency_counter lc;
lc.start();
std::optional timeout;
db::consistency_level cl = allow_hints ? db::consistency_level::ANY : db::consistency_level::ONE;
if (type == db::write_type::VIEW) {
// View updates have a near-infinite timeout to avoid incurring the extra work of writting hints
// and to apply backpressure.
timeout = clock_type::now() + 5min;
}
return mutate_prepare(std::array{std::move(m)}, cl, type, /* does view building should hold a real permit */ empty_service_permit(),
[this, tr_state, target = std::array{target}, pending_endpoints = std::move(pending_endpoints), &stats] (
std::unique_ptr& m,
db::consistency_level cl,
db::write_type type, service_permit permit) mutable {
std::unordered_set targets;
targets.reserve(pending_endpoints.size() + 1);
std::vector dead_endpoints;
boost::algorithm::partition_copy(
boost::range::join(pending_endpoints, target),
std::inserter(targets, targets.begin()),
std::back_inserter(dead_endpoints),
[] (gms::inet_address ep) { return gms::get_local_gossiper().is_alive(ep); });
auto& ks = _db.local().find_keyspace(m->schema()->ks_name());
slogger.trace("Creating write handler with live: {}; dead: {}", targets, dead_endpoints);
db::assure_sufficient_live_nodes(cl, ks, targets, pending_endpoints);
return create_write_response_handler(
ks,
cl,
type,
std::move(m),
std::move(targets),
pending_endpoints,
std::move(dead_endpoints),
tr_state,
stats,
std::move(permit));
}).then([this, cl, tr_state = std::move(tr_state), timeout = std::move(timeout)] (std::vector ids) mutable {
return mutate_begin(std::move(ids), cl, std::move(tr_state), std::move(timeout));
}).then_wrapped([p = shared_from_this(), lc, &stats] (future<>&& f) {
return p->mutate_end(std::move(f), lc, stats, nullptr);
});
}
future<> storage_proxy::send_to_endpoint(
frozen_mutation_and_schema fm_a_s,
gms::inet_address target,
std::vector pending_endpoints,
db::write_type type,
tracing::trace_state_ptr tr_state,
allow_hints allow_hints) {
return send_to_endpoint(
std::make_unique(std::move(fm_a_s)),
std::move(target),
std::move(pending_endpoints),
type,
std::move(tr_state),
get_stats(),
allow_hints);
}
future<> storage_proxy::send_to_endpoint(
frozen_mutation_and_schema fm_a_s,
gms::inet_address target,
std::vector pending_endpoints,
db::write_type type,
tracing::trace_state_ptr tr_state,
write_stats& stats,
allow_hints allow_hints) {
return send_to_endpoint(
std::make_unique(std::move(fm_a_s)),
std::move(target),
std::move(pending_endpoints),
type,
std::move(tr_state),
stats,
allow_hints);
}
future<> storage_proxy::send_hint_to_endpoint(frozen_mutation_and_schema fm_a_s, gms::inet_address target) {
if (!_features.cluster_supports_hinted_handoff_separate_connection()) {
return send_to_endpoint(
std::make_unique(std::move(fm_a_s)),
std::move(target),
{ },
db::write_type::SIMPLE,
tracing::trace_state_ptr(),
get_stats(),
allow_hints::no);
}
return send_to_endpoint(
std::make_unique(std::move(fm_a_s)),
std::move(target),
{ },
db::write_type::SIMPLE,
tracing::trace_state_ptr(),
get_stats(),
allow_hints::no);
}
future<> storage_proxy::send_hint_to_all_replicas(frozen_mutation_and_schema fm_a_s) {
if (!_features.cluster_supports_hinted_handoff_separate_connection()) {
std::array ms{fm_a_s.fm.unfreeze(fm_a_s.s)};
return mutate_internal(std::move(ms), db::consistency_level::ALL, false, nullptr, empty_service_permit());
}
std::array ms{hint_wrapper { std::move(fm_a_s.fm.unfreeze(fm_a_s.s)) }};
return mutate_internal(std::move(ms), db::consistency_level::ALL, false, nullptr, empty_service_permit());
}
/**
* Send the mutations to the right targets, write it locally if it corresponds or writes a hint when the node
* is not available.
*
* Note about hints:
*
* | Hinted Handoff | Consist. Level |
* | on | >=1 | --> wait for hints. We DO NOT notify the handler with handler.response() for hints;
* | on | ANY | --> wait for hints. Responses count towards consistency.
* | off | >=1 | --> DO NOT fire hints. And DO NOT wait for them to complete.
* | off | ANY | --> DO NOT fire hints. And DO NOT wait for them to complete.
*
* @throws OverloadedException if the hints cannot be written/enqueued
*/
// returned future is ready when sent is complete, not when mutation is executed on all (or any) targets!
void storage_proxy::send_to_live_endpoints(storage_proxy::response_id_type response_id, clock_type::time_point timeout)
{
// extra-datacenter replicas, grouped by dc
std::unordered_map> dc_groups;
std::vector>> local;
local.reserve(3);
auto handler_ptr = get_write_response_handler(response_id);
auto& stats = handler_ptr->stats();
auto& handler = *handler_ptr;
auto& global_stats = handler._proxy->_global_stats;
for(auto dest: handler.get_targets()) {
sstring dc = get_dc(dest);
// read repair writes do not go through coordinator since mutations are per destination
if (handler.read_repair_write() || dc == get_local_dc()) {
local.emplace_back("", std::vector({dest}));
} else {
dc_groups[dc].push_back(dest);
}
}
auto all = boost::range::join(local, dc_groups);
auto my_address = utils::fb_utilities::get_broadcast_address();
// lambda for applying mutation locally
auto lmutate = [handler_ptr, response_id, this, my_address, timeout] () mutable {
return handler_ptr->apply_locally(timeout, handler_ptr->get_trace_state())
.then([response_id, this, my_address, h = std::move(handler_ptr), p = shared_from_this()] {
// make mutation alive until it is processed locally, otherwise it
// may disappear if write timeouts before this future is ready
got_response(response_id, my_address, get_view_update_backlog());
});
};
// lambda for applying mutation remotely
auto rmutate = [this, handler_ptr, timeout, response_id, my_address, &global_stats] (gms::inet_address coordinator, std::vector&& forward) {
auto msize = handler_ptr->get_mutation_size(); // can overestimate for repair writes
global_stats.queued_write_bytes += msize;
return handler_ptr->apply_remotely(coordinator, std::move(forward), response_id, timeout, handler_ptr->get_trace_state())
.finally([this, p = shared_from_this(), h = std::move(handler_ptr), msize, &global_stats] {
global_stats.queued_write_bytes -= msize;
unthrottle();
});
};
// OK, now send and/or apply locally
for (typename decltype(dc_groups)::value_type& dc_targets : all) {
auto& forward = dc_targets.second;
// last one in forward list is a coordinator
auto coordinator = forward.back();
forward.pop_back();
size_t forward_size = forward.size();
future<> f = make_ready_future<>();
if (handler.is_counter() && coordinator == my_address) {
got_response(response_id, coordinator, std::nullopt);
} else {
if (!handler.read_repair_write()) {
++stats.writes_attempts.get_ep_stat(coordinator);
} else {
++stats.read_repair_write_attempts.get_ep_stat(coordinator);
}
if (coordinator == my_address) {
f = futurize_invoke(lmutate);
} else {
f = futurize_invoke(rmutate, coordinator, std::move(forward));
}
}
// Waited on indirectly.
(void)f.handle_exception([response_id, forward_size, coordinator, handler_ptr, p = shared_from_this(), &stats] (std::exception_ptr eptr) {
++stats.writes_errors.get_ep_stat(coordinator);
error err = error::FAILURE;
try {
std::rethrow_exception(eptr);
} catch(rpc::closed_error&) {
// ignore, disconnect will be logged by gossiper
} catch(seastar::gate_closed_exception&) {
// may happen during shutdown, ignore it
} catch(timed_out_error&) {
// from lmutate(). Ignore so that logs are not flooded
// database total_writes_timedout counter was incremented.
// It needs to be recorded that the timeout occurred locally though.
err = error::TIMEOUT;
} catch(...) {
slogger.error("exception during mutation write to {}: {}", coordinator, std::current_exception());
}
p->got_failure_response(response_id, coordinator, forward_size + 1, std::nullopt, err);
});
}
}
// returns number of hints stored
template
size_t storage_proxy::hint_to_dead_endpoints(std::unique_ptr& mh, const Range& targets, db::write_type type, tracing::trace_state_ptr tr_state) noexcept
{
if (hints_enabled(type)) {
db::hints::manager& hints_manager = hints_manager_for(type);
return boost::count_if(targets, [this, &mh, tr_state = std::move(tr_state), &hints_manager] (gms::inet_address target) mutable -> bool {
return mh->store_hint(hints_manager, target, tr_state);
});
} else {
return 0;
}
}
future<> storage_proxy::schedule_repair(std::unordered_map>> diffs, db::consistency_level cl, tracing::trace_state_ptr trace_state,
service_permit permit) {
if (diffs.empty()) {
return make_ready_future<>();
}
return mutate_internal(diffs | boost::adaptors::map_values, cl, false, std::move(trace_state), std::move(permit));
}
class abstract_read_resolver {
protected:
db::consistency_level _cl;
size_t _targets_count;
promise<> _done_promise; // all target responded
bool _request_failed = false; // will be true if request fails or timeouts
timer _timeout;
schema_ptr _schema;
size_t _failed = 0;
virtual void on_failure(std::exception_ptr ex) = 0;
virtual void on_timeout() = 0;
virtual size_t response_count() const = 0;
virtual void fail_request(std::exception_ptr ex) {
_request_failed = true;
_done_promise.set_exception(ex);
_timeout.cancel();
on_failure(ex);
}
public:
abstract_read_resolver(schema_ptr schema, db::consistency_level cl, size_t target_count, storage_proxy::clock_type::time_point timeout)
: _cl(cl)
, _targets_count(target_count)
, _schema(std::move(schema))
{
_timeout.set_callback([this] {
on_timeout();
});
_timeout.arm(timeout);
}
virtual ~abstract_read_resolver() {};
virtual void on_error(gms::inet_address ep, bool disconnect) = 0;
future<> done() {
return _done_promise.get_future();
}
void error(gms::inet_address ep, std::exception_ptr eptr) {
sstring why;
bool disconnect = false;
try {
std::rethrow_exception(eptr);
} catch (rpc::closed_error&) {
// do not report connection closed exception, gossiper does that
disconnect = true;
} catch (rpc::timeout_error&) {
// do not report timeouts, the whole operation will timeout and be reported
return; // also do not report timeout as replica failure for the same reason
} catch(...) {
slogger.error("Exception when communicating with {}, to read from {}.{}: {}", ep, _schema->ks_name(), _schema->cf_name(), eptr);
}
if (!_request_failed) { // request may fail only once.
on_error(ep, disconnect);
}
}
};
struct digest_read_result {
foreign_ptr> result;
bool digests_match;
};
class digest_read_resolver : public abstract_read_resolver {
size_t _block_for;
size_t _cl_responses = 0;
promise _cl_promise; // cl is reached
bool _cl_reported = false;
foreign_ptr