2017-04-01 09:19:00 +00:00
# include <IO/Operators.h>
# include <Storages/StorageReplicatedMergeTree.h>
# include <Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h>
# include <Storages/MergeTree/ReplicatedMergeTreeQuorumEntry.h>
# include <Storages/MergeTree/ReplicatedMergeTreeAddress.h>
2020-05-20 20:16:32 +00:00
# include <Interpreters/Context.h>
2018-04-03 17:35:48 +00:00
# include <Common/ZooKeeper/KeeperException.h>
2017-04-01 09:19:00 +00:00
# include <Common/randomSeed.h>
2014-10-17 01:05:51 +00:00
2016-10-24 02:02:37 +00:00
namespace ProfileEvents
{
2017-04-01 07:20:54 +00:00
extern const Event ReplicaYieldLeadership ;
extern const Event ReplicaPartialShutdown ;
2016-10-24 02:02:37 +00:00
}
2016-10-24 04:06:27 +00:00
namespace CurrentMetrics
{
2017-04-01 07:20:54 +00:00
extern const Metric ReadonlyReplica ;
2016-10-24 04:06:27 +00:00
}
2014-10-17 01:05:51 +00:00
namespace DB
{
2016-01-11 21:46:36 +00:00
namespace ErrorCodes
{
2017-04-01 07:20:54 +00:00
extern const int REPLICA_IS_ALREADY_ACTIVE ;
2016-01-11 21:46:36 +00:00
}
2017-12-29 22:32:04 +00:00
namespace
{
constexpr auto retry_period_ms = 10 * 1000 ;
}
2014-10-17 01:05:51 +00:00
2017-03-12 19:18:07 +00:00
/// Used to check whether it's us who set node `is_active`, or not.
2014-10-17 01:05:51 +00:00
static String generateActiveNodeIdentifier ( )
{
2017-04-01 07:20:54 +00:00
return " pid: " + toString ( getpid ( ) ) + " , random: " + toString ( randomSeed ( ) ) ;
2014-10-17 01:05:51 +00:00
}
ReplicatedMergeTreeRestartingThread : : ReplicatedMergeTreeRestartingThread ( StorageReplicatedMergeTree & storage_ )
2018-05-31 13:05:05 +00:00
: storage ( storage_ )
2019-12-03 16:25:32 +00:00
, log_name ( storage . getStorageID ( ) . getFullTableName ( ) + " (ReplicatedMergeTreeRestartingThread) " )
2018-05-31 13:05:05 +00:00
, log ( & Logger : : get ( log_name ) )
, active_node_identifier ( generateActiveNodeIdentifier ( ) )
2014-10-17 01:05:51 +00:00
{
2019-08-26 14:24:29 +00:00
const auto storage_settings = storage . getSettings ( ) ;
2019-08-13 10:29:31 +00:00
check_period_ms = storage_settings - > zookeeper_session_expiration_check_period . totalSeconds ( ) * 1000 ;
2017-12-29 22:32:04 +00:00
/// Periodicity of checking lag of replica.
2019-08-13 10:29:31 +00:00
if ( check_period_ms > static_cast < Int64 > ( storage_settings - > check_delay_period ) * 1000 )
check_period_ms = storage_settings - > check_delay_period * 1000 ;
2017-12-29 22:32:04 +00:00
2019-01-04 12:10:00 +00:00
task = storage . global_context . getSchedulePool ( ) . createTask ( log_name , [ this ] { run ( ) ; } ) ;
2017-12-29 22:32:04 +00:00
}
2014-10-17 01:05:51 +00:00
void ReplicatedMergeTreeRestartingThread : : run ( )
{
2017-12-29 22:32:04 +00:00
if ( need_stop )
return ;
2017-04-01 07:20:54 +00:00
2017-12-29 22:32:04 +00:00
try
{
if ( first_time | | storage . getZooKeeper ( ) - > expired ( ) )
{
startup_completed = false ;
2017-04-01 07:20:54 +00:00
2017-12-29 22:32:04 +00:00
if ( first_time )
{
2020-05-23 22:24:01 +00:00
LOG_DEBUG ( log , " Activating replica. " ) ;
2017-12-29 22:32:04 +00:00
}
else
{
2020-05-23 22:24:01 +00:00
LOG_WARNING ( log , " ZooKeeper session has expired. Switching to a new session. " ) ;
2017-04-01 07:20:54 +00:00
2017-12-29 22:32:04 +00:00
bool old_val = false ;
if ( storage . is_readonly . compare_exchange_strong ( old_val , true ) )
CurrentMetrics : : add ( CurrentMetrics : : ReadonlyReplica ) ;
2017-04-01 07:20:54 +00:00
2017-12-29 22:32:04 +00:00
partialShutdown ( ) ;
}
2017-04-01 07:20:54 +00:00
2017-12-29 22:32:04 +00:00
if ( ! startup_completed )
2017-04-01 07:20:54 +00:00
{
2017-12-29 22:32:04 +00:00
try
2017-04-01 07:20:54 +00:00
{
2019-01-04 12:10:00 +00:00
storage . setZooKeeper ( storage . global_context . getZooKeeper ( ) ) ;
2017-04-01 07:20:54 +00:00
}
2018-08-25 01:58:14 +00:00
catch ( const Coordination : : Exception & )
2017-04-01 07:20:54 +00:00
{
2017-12-29 22:32:04 +00:00
/// The exception when you try to zookeeper_init usually happens if DNS does not work. We will try to do it again.
2018-04-10 13:20:14 +00:00
tryLogCurrentException ( log , __PRETTY_FUNCTION__ ) ;
2017-12-21 18:17:06 +00:00
2018-04-10 13:20:14 +00:00
if ( first_time )
storage . startup_event . set ( ) ;
2018-05-31 13:05:05 +00:00
task - > scheduleAfter ( retry_period_ms ) ;
2017-12-29 22:32:04 +00:00
return ;
2017-04-01 07:20:54 +00:00
}
2017-12-29 22:32:04 +00:00
if ( ! need_stop & & ! tryStartup ( ) )
2017-04-01 07:20:54 +00:00
{
2018-03-22 19:46:01 +00:00
if ( first_time )
storage . startup_event . set ( ) ;
2018-05-31 13:05:05 +00:00
task - > scheduleAfter ( retry_period_ms ) ;
2017-12-29 22:32:04 +00:00
return ;
2017-04-01 07:20:54 +00:00
}
2018-04-10 13:20:14 +00:00
if ( first_time )
storage . startup_event . set ( ) ;
2017-11-01 13:58:02 +00:00
2017-12-29 22:32:04 +00:00
startup_completed = true ;
2017-12-21 18:17:06 +00:00
}
2017-04-01 07:20:54 +00:00
2017-12-29 22:32:04 +00:00
if ( need_stop )
return ;
2017-04-01 07:20:54 +00:00
2017-12-29 22:32:04 +00:00
bool old_val = true ;
if ( storage . is_readonly . compare_exchange_strong ( old_val , false ) )
CurrentMetrics : : sub ( CurrentMetrics : : ReadonlyReplica ) ;
2017-04-01 07:20:54 +00:00
2017-12-29 22:32:04 +00:00
first_time = false ;
}
2017-04-01 07:20:54 +00:00
2017-12-29 22:32:04 +00:00
time_t current_time = time ( nullptr ) ;
2019-08-26 14:24:29 +00:00
const auto storage_settings = storage . getSettings ( ) ;
2019-08-13 10:29:31 +00:00
if ( current_time > = prev_time_of_check_delay + static_cast < time_t > ( storage_settings - > check_delay_period ) )
2017-12-29 22:32:04 +00:00
{
/// Find out lag of replicas.
time_t absolute_delay = 0 ;
time_t relative_delay = 0 ;
2017-04-01 07:20:54 +00:00
2017-12-29 22:32:04 +00:00
storage . getReplicaDelays ( absolute_delay , relative_delay ) ;
2017-04-01 07:20:54 +00:00
2017-12-29 22:32:04 +00:00
if ( absolute_delay )
2020-05-23 22:24:01 +00:00
LOG_TRACE ( log , " Absolute delay: {}. Relative delay: {}. " , absolute_delay , relative_delay ) ;
2017-04-01 07:20:54 +00:00
2017-12-29 22:32:04 +00:00
prev_time_of_check_delay = current_time ;
/// We give up leadership if the relative lag is greater than threshold.
2018-04-10 13:20:14 +00:00
if ( storage . is_leader
2019-08-13 10:29:31 +00:00
& & relative_delay > static_cast < time_t > ( storage_settings - > min_relative_delay_to_yield_leadership ) )
2017-12-29 22:32:04 +00:00
{
2020-05-23 22:24:01 +00:00
LOG_INFO ( log , " Relative replica delay ({} seconds) is bigger than threshold ({}). Will yield leadership. " , relative_delay , storage_settings - > min_relative_delay_to_yield_leadership ) ;
2017-04-01 07:20:54 +00:00
2017-12-29 22:32:04 +00:00
ProfileEvents : : increment ( ProfileEvents : : ReplicaYieldLeadership ) ;
2017-04-01 07:20:54 +00:00
2018-04-10 13:20:14 +00:00
storage . exitLeaderElection ( ) ;
/// NOTE: enterLeaderElection() can throw if node creation in ZK fails.
/// This is bad because we can end up without a leader on any replica.
/// In this case we rely on the fact that the session will expire and we will reconnect.
storage . enterLeaderElection ( ) ;
2017-04-01 07:20:54 +00:00
}
}
2017-12-21 18:17:06 +00:00
}
2017-12-29 22:32:04 +00:00
catch ( . . . )
{
2018-04-10 13:20:14 +00:00
storage . startup_event . set ( ) ;
tryLogCurrentException ( log , __PRETTY_FUNCTION__ ) ;
2017-12-21 18:17:06 +00:00
}
2017-11-17 08:58:35 +00:00
2018-05-31 13:05:05 +00:00
task - > scheduleAfter ( check_period_ms ) ;
2017-12-29 22:32:04 +00:00
}
2017-11-17 08:58:35 +00:00
2017-12-21 18:17:06 +00:00
2014-10-17 01:05:51 +00:00
bool ReplicatedMergeTreeRestartingThread : : tryStartup ( )
{
2017-04-01 07:20:54 +00:00
try
{
removeFailedQuorumParts ( ) ;
activateReplica ( ) ;
2018-08-27 19:06:32 +00:00
2018-08-09 15:06:39 +00:00
const auto & zookeeper = storage . getZooKeeper ( ) ;
2019-08-26 14:24:29 +00:00
const auto storage_settings = storage . getSettings ( ) ;
2018-08-09 15:06:39 +00:00
2018-08-22 14:01:54 +00:00
storage . cloneReplicaIfNeeded ( zookeeper ) ;
2018-08-09 15:06:39 +00:00
storage . queue . load ( zookeeper ) ;
2018-08-27 23:59:49 +00:00
/// pullLogsToQueue() after we mark replica 'is_active' (and after we repair if it was lost);
/// because cleanup_thread doesn't delete log_pointer of active replicas.
2018-08-20 13:31:24 +00:00
storage . queue . pullLogsToQueue ( zookeeper ) ;
2020-02-18 19:57:48 +00:00
storage . queue . removeCurrentPartsFromMutations ( ) ;
2018-08-08 13:53:06 +00:00
storage . last_queue_update_finish_time . store ( time ( nullptr ) ) ;
2018-08-09 15:06:39 +00:00
2017-04-01 07:20:54 +00:00
updateQuorumIfWeHavePart ( ) ;
2019-08-13 10:29:31 +00:00
if ( storage_settings - > replicated_can_become_leader )
2018-04-06 16:06:07 +00:00
storage . enterLeaderElection ( ) ;
2019-02-13 13:05:58 +00:00
else
2020-05-23 22:24:01 +00:00
LOG_INFO ( log , " Will not enter leader election because replicated_can_become_leader=0 " ) ;
2017-04-01 07:20:54 +00:00
/// Anything above can throw a KeeperException if something is wrong with ZK.
/// Anything below should not throw exceptions.
2018-07-30 18:30:33 +00:00
storage . partial_shutdown_called = false ;
storage . partial_shutdown_event . reset ( ) ;
2017-04-01 07:20:54 +00:00
2018-08-22 13:43:27 +00:00
storage . queue_updating_task - > activateAndSchedule ( ) ;
storage . mutations_updating_task - > activateAndSchedule ( ) ;
2018-08-22 13:57:30 +00:00
storage . mutations_finalizing_task - > activateAndSchedule ( ) ;
2018-07-30 17:34:55 +00:00
storage . cleanup_thread . start ( ) ;
2017-06-23 06:57:55 +00:00
storage . part_check_thread . start ( ) ;
2017-04-01 07:20:54 +00:00
return true ;
}
catch ( . . . )
{
2018-11-26 00:56:50 +00:00
storage . replica_is_active_node = nullptr ;
2017-04-01 07:20:54 +00:00
try
{
throw ;
}
2018-08-25 01:58:14 +00:00
catch ( const Coordination : : Exception & e )
2017-04-01 07:20:54 +00:00
{
2020-05-23 22:24:01 +00:00
LOG_ERROR ( log , " Couldn't start replication: {}. {} " , e . what ( ) , DB : : getCurrentExceptionMessage ( true ) ) ;
2017-04-01 07:20:54 +00:00
return false ;
}
catch ( const Exception & e )
{
if ( e . code ( ) ! = ErrorCodes : : REPLICA_IS_ALREADY_ACTIVE )
throw ;
2020-05-23 22:24:01 +00:00
LOG_ERROR ( log , " Couldn't start replication: {}. {} " , e . what ( ) , DB : : getCurrentExceptionMessage ( true ) ) ;
2017-04-01 07:20:54 +00:00
return false ;
}
}
2014-10-17 01:05:51 +00:00
}
2015-09-20 11:02:59 +00:00
void ReplicatedMergeTreeRestartingThread : : removeFailedQuorumParts ( )
{
2017-04-01 07:20:54 +00:00
auto zookeeper = storage . getZooKeeper ( ) ;
Strings failed_parts ;
2018-08-25 01:58:14 +00:00
if ( zookeeper - > tryGetChildren ( storage . zookeeper_path + " /quorum/failed_parts " , failed_parts ) ! = Coordination : : ZOK )
2017-04-01 07:20:54 +00:00
return ;
2018-05-21 13:49:54 +00:00
/// Firstly, remove parts from ZooKeeper
storage . tryRemovePartsFromZooKeeperWithRetries ( failed_parts ) ;
2020-03-09 02:55:28 +00:00
for ( const auto & part_name : failed_parts )
2017-04-01 07:20:54 +00:00
{
2019-05-03 02:00:57 +00:00
auto part = storage . getPartIfExists (
2017-12-18 17:26:46 +00:00
part_name , { MergeTreeDataPartState : : PreCommitted , MergeTreeDataPartState : : Committed , MergeTreeDataPartState : : Outdated } ) ;
2018-05-21 13:49:54 +00:00
2017-04-01 07:20:54 +00:00
if ( part )
{
2020-05-23 22:24:01 +00:00
LOG_DEBUG ( log , " Found part {} with failed quorum. Moving to detached. This shouldn't happen often. " , part_name ) ;
2019-05-20 16:24:36 +00:00
storage . forgetPartAndMoveToDetached ( part , " noquorum " ) ;
2018-05-10 15:01:10 +00:00
storage . queue . removeFromVirtualParts ( part - > info ) ;
2017-04-01 07:20:54 +00:00
}
}
2015-09-20 11:02:59 +00:00
}
void ReplicatedMergeTreeRestartingThread : : updateQuorumIfWeHavePart ( )
{
2017-04-01 07:20:54 +00:00
auto zookeeper = storage . getZooKeeper ( ) ;
String quorum_str ;
if ( zookeeper - > tryGet ( storage . zookeeper_path + " /quorum/status " , quorum_str ) )
{
ReplicatedMergeTreeQuorumEntry quorum_entry ;
quorum_entry . fromString ( quorum_str ) ;
if ( ! quorum_entry . replicas . count ( storage . replica_name )
& & zookeeper - > exists ( storage . replica_path + " /parts/ " + quorum_entry . part_name ) )
{
2020-05-23 22:24:01 +00:00
LOG_WARNING ( log , " We have part {} but we is not in quorum. Updating quorum. This shouldn't happen often. " , quorum_entry . part_name ) ;
2017-04-01 07:20:54 +00:00
storage . updateQuorum ( quorum_entry . part_name ) ;
}
}
2015-09-20 11:02:59 +00:00
}
2014-10-17 01:05:51 +00:00
void ReplicatedMergeTreeRestartingThread : : activateReplica ( )
{
2017-04-01 07:20:54 +00:00
auto zookeeper = storage . getZooKeeper ( ) ;
2018-04-17 17:59:42 +00:00
/// How other replicas can access this one.
ReplicatedMergeTreeAddress address = storage . getReplicatedMergeTreeAddress ( ) ;
2017-04-01 07:20:54 +00:00
String is_active_path = storage . replica_path + " /is_active " ;
/** If the node is marked as active, but the mark is made in the same instance, delete it.
* This is possible only when session in ZooKeeper expires .
*/
String data ;
2018-08-25 01:58:14 +00:00
Coordination : : Stat stat ;
2017-04-01 07:20:54 +00:00
bool has_is_active = zookeeper - > tryGet ( is_active_path , data , & stat ) ;
if ( has_is_active & & data = = active_node_identifier )
{
auto code = zookeeper - > tryRemove ( is_active_path , stat . version ) ;
2018-08-25 01:58:14 +00:00
if ( code = = Coordination : : ZBADVERSION )
2017-04-01 07:20:54 +00:00
throw Exception ( " Another instance of replica " + storage . replica_path + " was created just now. "
" You shouldn't run multiple instances of same replica. You need to check configuration files. " ,
ErrorCodes : : REPLICA_IS_ALREADY_ACTIVE ) ;
2018-08-25 01:58:14 +00:00
if ( code & & code ! = Coordination : : ZNONODE )
throw Coordination : : Exception ( code , is_active_path ) ;
2017-04-01 07:20:54 +00:00
}
/// Simultaneously declare that this replica is active, and update the host.
2018-08-25 01:58:14 +00:00
Coordination : : Requests ops ;
2018-03-24 00:45:04 +00:00
ops . emplace_back ( zkutil : : makeCreateRequest ( is_active_path , active_node_identifier , zkutil : : CreateMode : : Ephemeral ) ) ;
ops . emplace_back ( zkutil : : makeSetRequest ( storage . replica_path + " /host " , address . toString ( ) , - 1 ) ) ;
2017-04-01 07:20:54 +00:00
try
{
zookeeper - > multi ( ops ) ;
}
2018-08-25 01:58:14 +00:00
catch ( const Coordination : : Exception & e )
2017-04-01 07:20:54 +00:00
{
2018-08-25 01:58:14 +00:00
if ( e . code = = Coordination : : ZNODEEXISTS )
2017-04-01 07:20:54 +00:00
throw Exception ( " Replica " + storage . replica_path + " appears to be already active. If you're sure it's not, "
" try again in a minute or remove znode " + storage . replica_path + " /is_active manually " , ErrorCodes : : REPLICA_IS_ALREADY_ACTIVE ) ;
throw ;
}
/// `current_zookeeper` lives for the lifetime of `replica_is_active_node`,
/// since before changing `current_zookeeper`, `replica_is_active_node` object is destroyed in `partialShutdown` method.
storage . replica_is_active_node = zkutil : : EphemeralNodeHolder : : existing ( is_active_path , * storage . current_zookeeper ) ;
2014-10-17 01:05:51 +00:00
}
void ReplicatedMergeTreeRestartingThread : : partialShutdown ( )
{
2017-04-01 07:20:54 +00:00
ProfileEvents : : increment ( ProfileEvents : : ReplicaPartialShutdown ) ;
2018-07-30 18:30:33 +00:00
storage . partial_shutdown_called = true ;
storage . partial_shutdown_event . set ( ) ;
2017-04-01 07:20:54 +00:00
storage . replica_is_active_node = nullptr ;
2020-05-23 22:24:01 +00:00
LOG_TRACE ( log , " Waiting for threads to finish " ) ;
2017-04-01 07:20:54 +00:00
2018-04-06 16:06:07 +00:00
storage . exitLeaderElection ( ) ;
2018-05-31 13:05:05 +00:00
storage . queue_updating_task - > deactivate ( ) ;
storage . mutations_updating_task - > deactivate ( ) ;
2018-08-22 13:57:30 +00:00
storage . mutations_finalizing_task - > deactivate ( ) ;
2018-04-19 14:20:18 +00:00
2018-07-30 17:34:55 +00:00
storage . cleanup_thread . stop ( ) ;
2017-04-01 07:20:54 +00:00
storage . part_check_thread . stop ( ) ;
2020-05-23 22:24:01 +00:00
LOG_TRACE ( log , " Threads finished " ) ;
2014-10-17 01:05:51 +00:00
}
2018-08-21 14:03:06 +00:00
void ReplicatedMergeTreeRestartingThread : : shutdown ( )
{
/// Stop restarting_thread before stopping other tasks - so that it won't restart them again.
need_stop = true ;
task - > deactivate ( ) ;
2020-05-23 22:24:01 +00:00
LOG_TRACE ( log , " Restarting thread finished " ) ;
2018-08-21 14:03:06 +00:00
/// Stop other tasks.
partialShutdown ( ) ;
}
2014-10-17 01:05:51 +00:00
}