#pragma once #include #include #include #include #include #include #include #include #include #include #include #include #include namespace DB { namespace ErrorCodes { extern const int ALL_CONNECTION_TRIES_FAILED; extern const int ALL_REPLICAS_ARE_STALE; extern const int LOGICAL_ERROR; } } namespace ProfileEvents { extern const Event DistributedConnectionFailTry; extern const Event DistributedConnectionFailAtAll; } /// This class provides a pool with fault tolerance. It is used for pooling of connections to replicated DB. /// Initialized by several PoolBase objects. /// When a connection is requested, tries to create or choose an alive connection from one of the nested pools. /// Pools are tried in the order consistent with lexicographical order of (error count, priority, random number) tuples. /// Number of tries for a single pool is limited by max_tries parameter. /// The client can set nested pool priority by passing a GetPriority functor. /// /// NOTE: if one of the nested pools blocks because it is empty, this pool will also block. /// /// The client must provide a TryGetEntryFunc functor, which should perform a single try to get a connection from a nested pool. /// This functor can also check if the connection satisfies some eligibility criterion (e.g. check if /// the replica is up-to-date). template class PoolWithFailoverBase : private boost::noncopyable { public: using NestedPool = TNestedPool; using NestedPoolPtr = std::shared_ptr; using Entry = typename NestedPool::Entry; using NestedPools = std::vector; PoolWithFailoverBase( NestedPools nested_pools_, size_t max_tries_, time_t decrease_error_period_, Logger * log_) : nested_pools(std::move(nested_pools_)) , max_tries(max_tries_) , decrease_error_period(decrease_error_period_) , shared_pool_states(nested_pools.size()) , log(log_) { } struct TryResult { TryResult() = default; explicit TryResult(Entry entry_) : entry(std::move(entry)) , is_usable(true) , is_up_to_date(true) { } void reset() { entry = Entry(); is_usable = false; is_up_to_date = false; staleness = 0.0; } Entry entry; bool is_usable = false; /// If false, the entry is unusable for current request /// (but may be usable for other requests, so error counts are not incremented) bool is_up_to_date = false; /// If true, the entry is a connection to up-to-date replica. double staleness = 0.0; /// Helps choosing the "least stale" option when all replicas are stale. }; /// This functor must be provided by a client. It must perform a single try that takes a connection /// from the provided pool and checks that it is good. using TryGetEntryFunc = std::function; /// The client can provide this functor to affect load balancing - the index of a pool is passed to /// this functor. The pools with lower result value will be tried first. using GetPriorityFunc = std::function; /// Returns a single connection. Entry get(const TryGetEntryFunc & try_get_entry, const GetPriorityFunc & get_priority = GetPriorityFunc()); /// Returns at least min_entries and at most max_entries connections (at most one connection per nested pool). /// The method will throw if it is unable to get min_entries alive connections or /// if fallback_to_stale_replicas is false and it is unable to get min_entries connections to up-to-date replicas. std::vector getMany( size_t min_entries, size_t max_entries, const TryGetEntryFunc & try_get_entry, const GetPriorityFunc & get_priority = GetPriorityFunc(), bool fallback_to_stale_replicas = true); void reportError(const Entry & entry); protected: struct PoolState; using PoolStates = std::vector; /// This function returns a copy of pool states to avoid race conditions when modifying shared pool states. PoolStates updatePoolStates(); NestedPools nested_pools; const size_t max_tries; const time_t decrease_error_period; std::mutex pool_states_mutex; PoolStates shared_pool_states; /// The time when error counts were last decreased. time_t last_error_decrease_time = 0; Logger * log; }; template typename TNestedPool::Entry PoolWithFailoverBase::get(const TryGetEntryFunc & try_get_entry, const GetPriorityFunc & get_priority) { std::vector entries = getMany(1, 1, try_get_entry, get_priority); if (entries.empty() || entries[0].isNull()) throw DB::Exception( "PoolWithFailoverBase::getMany() returned less than min_entries entries.", DB::ErrorCodes::LOGICAL_ERROR); return entries[0]; } template std::vector PoolWithFailoverBase::getMany( size_t min_entries, size_t max_entries, const TryGetEntryFunc & try_get_entry, const GetPriorityFunc & get_priority, bool fallback_to_stale_replicas) { /// Update random numbers and error counts. PoolStates pool_states = updatePoolStates(); if (get_priority) { for (size_t i = 0; i < pool_states.size(); ++i) pool_states[i].priority = get_priority(i); } struct ShuffledPool { NestedPool * pool; const PoolState * state; size_t index; size_t error_count = 0; }; /// Sort the pools into order in which they will be tried (based on respective PoolStates). std::vector shuffled_pools; shuffled_pools.reserve(nested_pools.size()); for (size_t i = 0; i < nested_pools.size(); ++i) shuffled_pools.push_back(ShuffledPool{nested_pools[i].get(), &pool_states[i], i, 0}); std::sort( shuffled_pools.begin(), shuffled_pools.end(), [](const ShuffledPool & lhs, const ShuffledPool & rhs) { return PoolState::compare(*lhs.state, *rhs.state); }); /// We will try to get a connection from each pool until a connection is produced or max_tries is reached. std::vector try_results(shuffled_pools.size()); size_t entries_count = 0; size_t usable_count = 0; size_t up_to_date_count = 0; size_t failed_pools_count = 0; /// At exit update shared error counts with error counts occured during this call. SCOPE_EXIT( { std::lock_guard lock(pool_states_mutex); for (const ShuffledPool & pool: shuffled_pools) shared_pool_states[pool.index].error_count += pool.error_count; }); std::string fail_messages; bool finished = false; while (!finished) { for (size_t i = 0; i < shuffled_pools.size(); ++i) { if (up_to_date_count >= max_entries /// Already enough good entries. || entries_count + failed_pools_count >= nested_pools.size()) /// No more good entries will be produced. { finished = true; break; } ShuffledPool & shuffled_pool = shuffled_pools[i]; TryResult & result = try_results[i]; if (shuffled_pool.error_count >= max_tries || !result.entry.isNull()) continue; std::string fail_message; result = try_get_entry(*shuffled_pool.pool, fail_message); if (!fail_message.empty()) fail_messages += fail_message + '\n'; if (!result.entry.isNull()) { ++entries_count; if (result.is_usable) { ++usable_count; if (result.is_up_to_date) ++up_to_date_count; } } else { LOG_WARNING(log, "Connection failed at try №" << (shuffled_pool.error_count + 1) << ", reason: " << fail_message); ProfileEvents::increment(ProfileEvents::DistributedConnectionFailTry); ++shuffled_pool.error_count; if (shuffled_pool.error_count >= max_tries) { ++failed_pools_count; ProfileEvents::increment(ProfileEvents::DistributedConnectionFailAtAll); } } } } if (usable_count < min_entries) throw DB::NetException( "All connection tries failed. Log: \n\n" + fail_messages + "\n", DB::ErrorCodes::ALL_CONNECTION_TRIES_FAILED); try_results.erase( std::remove_if( try_results.begin(), try_results.end(), [](const TryResult & r) { return r.entry.isNull() || !r.is_usable; }), try_results.end()); std::vector entries; if (up_to_date_count >= min_entries) { /// There is enough up-to-date entries. entries.reserve(up_to_date_count); for (const TryResult & result: try_results) { if (result.is_up_to_date) entries.push_back(result.entry); } } else if (fallback_to_stale_replicas) { /// There is not enough up-to-date entries but we are allowed to return stale entries. /// Gather all up-to-date ones and least-bad stale ones. std::stable_sort( try_results.begin(), try_results.end(), [](const TryResult & left, const TryResult & right) { return std::forward_as_tuple(!left.is_up_to_date, left.staleness) < std::forward_as_tuple(!right.is_up_to_date, right.staleness); }); size_t size = std::min(try_results.size(), max_entries); entries.reserve(size); for (size_t i = 0; i < size; ++i) entries.push_back(try_results[i].entry); } else throw DB::Exception( "Could not find enough connections to up-to-date replicas. Got: " + std::to_string(up_to_date_count) + ", needed: " + std::to_string(min_entries), DB::ErrorCodes::ALL_REPLICAS_ARE_STALE); return entries; } template void PoolWithFailoverBase::reportError(const Entry & entry) { for (size_t i = 0; i < nested_pools.size(); ++i) { if (nested_pools[i]->contains(entry)) { std::lock_guard lock(pool_states_mutex); ++shared_pool_states[i].error_count; return; } } throw DB::Exception("Can't find pool to report error."); } template struct PoolWithFailoverBase::PoolState { UInt64 error_count = 0; Int64 priority = 0; UInt32 random = 0; void randomize() { random = rng(); } static bool compare(const PoolState & lhs, const PoolState & rhs) { return std::forward_as_tuple(lhs.error_count, lhs.priority, lhs.random) < std::forward_as_tuple(rhs.error_count, rhs.priority, rhs.random); } private: std::minstd_rand rng = std::minstd_rand(randomSeed()); }; template typename PoolWithFailoverBase::PoolStates PoolWithFailoverBase::updatePoolStates() { PoolStates result; result.reserve(nested_pools.size()); { std::lock_guard lock(pool_states_mutex); for (auto & state : shared_pool_states) state.randomize(); time_t current_time = time(nullptr); if (last_error_decrease_time) { time_t delta = current_time - last_error_decrease_time; if (delta >= 0) { /// Divide error counts by 2 every decrease_error_period seconds. size_t shift_amount = delta / decrease_error_period; /// Update time but don't do it more often than once a period. /// Else if the function is called often enough, error count will never decrease. if (shift_amount) last_error_decrease_time = current_time; if (shift_amount >= sizeof(UInt64) * CHAR_BIT) { for (auto & state : shared_pool_states) state.error_count = 0; } else if (shift_amount) { for (auto & state : shared_pool_states) state.error_count >>= shift_amount; } } } else last_error_decrease_time = current_time; result.assign(shared_pool_states.begin(), shared_pool_states.end()); } return result; }