2017-04-01 09:19:00 +00:00
# include <Storages/MergeTree/ReplicatedMergeTreePartCheckThread.h>
2017-12-03 00:48:19 +00:00
# include <Storages/MergeTree/checkDataPart.h>
2018-12-11 13:30:20 +00:00
# include <Storages/MergeTree/ReplicatedMergeTreePartHeader.h>
2017-04-01 09:19:00 +00:00
# include <Storages/StorageReplicatedMergeTree.h>
2020-05-20 20:16:32 +00:00
# include <Interpreters/Context.h>
2016-04-09 03:50:02 +00:00
2016-10-24 02:02:37 +00:00
namespace ProfileEvents
{
2017-04-01 07:20:54 +00:00
extern const Event ReplicatedPartChecks ;
extern const Event ReplicatedPartChecksFailed ;
extern const Event ReplicatedDataLoss ;
2016-10-24 02:02:37 +00:00
}
2016-04-09 03:50:02 +00:00
namespace DB
{
2018-11-22 21:19:58 +00:00
namespace ErrorCodes
{
extern const int TABLE_DIFFERS_TOO_MUCH ;
2021-01-27 18:54:05 +00:00
extern const int LOGICAL_ERROR ;
2018-11-22 21:19:58 +00:00
}
2016-04-09 04:22:11 +00:00
static const auto PART_CHECK_ERROR_SLEEP_MS = 5 * 1000 ;
2016-04-09 03:50:02 +00:00
ReplicatedMergeTreePartCheckThread : : ReplicatedMergeTreePartCheckThread ( 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 ( ) + " (ReplicatedMergeTreePartCheckThread) " )
2020-05-30 21:57:37 +00:00
, log ( & Poco : : Logger : : get ( log_name ) )
2016-04-09 03:50:02 +00:00
{
2019-01-04 12:10:00 +00:00
task = storage . global_context . getSchedulePool ( ) . createTask ( log_name , [ this ] { run ( ) ; } ) ;
2018-05-31 13:05:05 +00:00
task - > schedule ( ) ;
2016-04-09 03:50:02 +00:00
}
2017-12-29 22:32:04 +00:00
ReplicatedMergeTreePartCheckThread : : ~ ReplicatedMergeTreePartCheckThread ( )
{
stop ( ) ;
}
2016-04-09 03:50:02 +00:00
void ReplicatedMergeTreePartCheckThread : : start ( )
{
2019-01-02 06:44:36 +00:00
std : : lock_guard lock ( start_stop_mutex ) ;
2017-12-29 22:32:04 +00:00
need_stop = false ;
2018-08-22 13:43:27 +00:00
task - > activateAndSchedule ( ) ;
2016-04-09 03:50:02 +00:00
}
void ReplicatedMergeTreePartCheckThread : : stop ( )
{
2019-09-23 16:18:19 +00:00
//based on discussion on https://github.com/ClickHouse/ClickHouse/pull/1489#issuecomment-344756259
2017-12-29 22:32:04 +00:00
//using the schedule pool there is no problem in case stop is called two time in row and the start multiple times
2017-12-21 18:17:06 +00:00
2019-01-02 06:44:36 +00:00
std : : lock_guard lock ( start_stop_mutex ) ;
2017-07-12 16:15:16 +00:00
need_stop = true ;
2018-05-31 13:05:05 +00:00
task - > deactivate ( ) ;
2016-04-09 03:50:02 +00:00
}
void ReplicatedMergeTreePartCheckThread : : enqueuePart ( const String & name , time_t delay_to_check_seconds )
{
2019-01-02 06:44:36 +00:00
std : : lock_guard lock ( parts_mutex ) ;
2016-04-09 03:50:02 +00:00
2017-04-01 07:20:54 +00:00
if ( parts_set . count ( name ) )
return ;
2016-04-09 03:50:02 +00:00
2017-12-18 04:07:26 +00:00
parts_queue . emplace_back ( name , time ( nullptr ) + delay_to_check_seconds ) ;
2017-04-01 07:20:54 +00:00
parts_set . insert ( name ) ;
2018-05-31 13:05:05 +00:00
task - > schedule ( ) ;
2016-04-09 03:50:02 +00:00
}
size_t ReplicatedMergeTreePartCheckThread : : size ( ) const
{
2019-01-02 06:44:36 +00:00
std : : lock_guard lock ( parts_mutex ) ;
2017-04-01 07:20:54 +00:00
return parts_set . size ( ) ;
2016-04-09 03:50:02 +00:00
}
2021-01-27 10:07:18 +00:00
ReplicatedMergeTreePartCheckThread : : MissingPartSearchResult ReplicatedMergeTreePartCheckThread : : searchForMissingPartOnOtherReplicas ( const String & part_name )
2016-04-09 03:50:02 +00:00
{
2017-04-01 07:20:54 +00:00
auto zookeeper = storage . getZooKeeper ( ) ;
/// If the part is not in ZooKeeper, we'll check if it's at least somewhere.
2019-05-03 02:00:57 +00:00
auto part_info = MergeTreePartInfo : : fromPartName ( part_name , storage . format_version ) ;
2017-04-01 07:20:54 +00:00
2017-09-15 20:36:03 +00:00
/** The logic is as follows:
2017-04-01 07:20:54 +00:00
* - if some live or inactive replica has such a part , or a part covering it
* - it is Ok , nothing is needed , it is then downloaded when processing the queue , when the replica comes to life ;
* - or , if the replica never comes to life , then the administrator will delete or create a new replica with the same address and see everything from the beginning ;
* - if no one has such part or a part covering it , then
2017-09-15 20:36:03 +00:00
* - if there are two smaller parts , one with the same min block and the other with the same
* max block , we hope that all parts in between are present too and the needed part
* will appear on other replicas as a result of a merge .
* - otherwise , consider the part lost and delete the entry from the queue .
*
* Note that this logic is not perfect - some part in the interior may be missing and the
* needed part will never appear . But precisely determining whether the part will appear as
* a result of a merge is complicated - we can ' t just check if all block numbers covered
* by the missing part are present somewhere ( because gaps between blocks are possible )
* and to determine the constituent parts of the merge we need to query the replication log
* ( both the common log and the queues of the individual replicas ) and then , if the
* constituent parts are in turn not found , solve the problem recursively for them .
*
* Considering the part lost when it is not in fact lost is very dangerous because it leads
* to divergent replicas and intersecting parts . So we err on the side of caution
* and don ' t delete the queue entry when in doubt .
2017-04-01 07:20:54 +00:00
*/
2021-01-27 10:07:18 +00:00
LOG_WARNING ( log , " Checking if anyone has a part {} or covering part. " , part_name ) ;
2017-04-01 07:20:54 +00:00
2017-09-15 20:36:03 +00:00
bool found_part_with_the_same_min_block = false ;
bool found_part_with_the_same_max_block = false ;
2017-04-01 07:20:54 +00:00
Strings replicas = zookeeper - > getChildren ( storage . zookeeper_path + " /replicas " ) ;
for ( const String & replica : replicas )
{
2021-01-27 10:07:18 +00:00
String replica_path = storage . zookeeper_path + " /replicas/ " + replica ;
Strings parts = zookeeper - > getChildren ( replica_path + " /parts " ) ;
2017-04-01 07:20:54 +00:00
for ( const String & part_on_replica : parts )
{
2019-05-03 02:00:57 +00:00
auto part_on_replica_info = MergeTreePartInfo : : fromPartName ( part_on_replica , storage . format_version ) ;
2017-08-14 18:16:11 +00:00
2021-01-27 10:07:18 +00:00
if ( part_info = = part_on_replica_info )
{
2021-01-27 10:11:48 +00:00
/// Found missing part at ourself. If we are here then something wrong with this part, so skipping.
2021-01-27 10:07:18 +00:00
if ( replica_path = = storage . replica_path )
continue ;
LOG_WARNING ( log , " Found the missing part {} at {} on {} " , part_name , part_on_replica , replica ) ;
return MissingPartSearchResult : : FoundAndNeedFetch ;
}
2017-09-15 20:36:03 +00:00
if ( part_on_replica_info . contains ( part_info ) )
2017-04-01 07:20:54 +00:00
{
2020-05-23 22:24:01 +00:00
LOG_WARNING ( log , " Found part {} on {} that covers the missing part {} " , part_on_replica , replica , part_name ) ;
2021-01-27 10:07:18 +00:00
return MissingPartSearchResult : : FoundAndDontNeedFetch ;
2017-04-01 07:20:54 +00:00
}
2017-08-14 18:16:11 +00:00
if ( part_info . contains ( part_on_replica_info ) )
2017-04-01 07:20:54 +00:00
{
2017-09-15 20:36:03 +00:00
if ( part_on_replica_info . min_block = = part_info . min_block )
found_part_with_the_same_min_block = true ;
if ( part_on_replica_info . max_block = = part_info . max_block )
found_part_with_the_same_max_block = true ;
if ( found_part_with_the_same_min_block & & found_part_with_the_same_max_block )
{
2020-05-23 22:24:01 +00:00
LOG_WARNING ( log , " Found parts with the same min block and with the same max block as the missing part {}. Hoping that it will eventually appear as a result of a merge. " , part_name ) ;
2021-01-27 10:07:18 +00:00
return MissingPartSearchResult : : FoundAndDontNeedFetch ;
2017-09-15 20:36:03 +00:00
}
2017-04-01 07:20:54 +00:00
}
}
}
2017-09-15 20:36:03 +00:00
/// No one has such a part and the merge is impossible.
String not_found_msg ;
2018-06-04 15:03:21 +00:00
if ( found_part_with_the_same_max_block )
2017-09-15 20:36:03 +00:00
not_found_msg = " a smaller part with the same max block. " ;
else if ( found_part_with_the_same_min_block )
not_found_msg = " a smaller part with the same min block. " ;
else
not_found_msg = " smaller parts with either the same min block or the same max block. " ;
2020-05-23 22:24:01 +00:00
LOG_ERROR ( log , " No replica has part covering {} and a merge is impossible: we didn't find {} " , part_name , not_found_msg ) ;
2017-04-01 07:20:54 +00:00
2021-01-27 10:07:18 +00:00
return MissingPartSearchResult : : LostForever ;
}
void ReplicatedMergeTreePartCheckThread : : searchForMissingPartAndFetchIfPossible ( const String & part_name )
{
auto zookeeper = storage . getZooKeeper ( ) ;
String part_path = storage . replica_path + " /parts/ " + part_name ;
2017-04-01 07:20:54 +00:00
2021-01-27 10:07:18 +00:00
auto missing_part_search_result = searchForMissingPartOnOtherReplicas ( part_name ) ;
/// If the part is in ZooKeeper, remove it from there and add the task to download it to the queue.
if ( zookeeper - > exists ( part_path ) )
2017-04-01 07:20:54 +00:00
{
2021-01-27 10:07:18 +00:00
/// If part found on some other replica
if ( missing_part_search_result = = MissingPartSearchResult : : FoundAndNeedFetch )
{
LOG_WARNING ( log , " Part {} exists in ZooKeeper but not locally and found on other replica. Removing from ZooKeeper and queueing a fetch. " , part_name ) ;
storage . removePartAndEnqueueFetch ( part_name ) ;
}
else /// If we have covering part on other replica or part is lost forever we don't need to fetch anything
{
LOG_WARNING ( log , " Part {} exists in ZooKeeper but not locally and not found on other replica. Removing it from ZooKeeper. " , part_name ) ;
storage . removePartFromZooKeeper ( part_name ) ;
}
2017-04-01 07:20:54 +00:00
}
2021-01-27 10:07:18 +00:00
ProfileEvents : : increment ( ProfileEvents : : ReplicatedPartChecksFailed ) ;
if ( missing_part_search_result = = MissingPartSearchResult : : LostForever )
{
/// Is it in the replication queue? If there is - delete, because the task can not be processed.
if ( ! storage . queue . remove ( zookeeper , part_name ) )
{
/// The part was not in our queue. Why did it happen?
2021-01-28 08:07:18 +00:00
# ifdef NDEBUG
LOG_ERROR ( log , " Missing part {} is not in our queue. " , part_name ) ;
# else
2021-01-27 18:54:05 +00:00
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Missing part {} is not in our queue. " , part_name ) ;
2021-01-28 08:07:18 +00:00
# endif
2021-01-27 10:07:18 +00:00
}
/** This situation is possible if on all the replicas where the part was, it deteriorated.
* For example , a replica that has just written it has power turned off and the data has not been written from cache to disk .
*/
LOG_ERROR ( log , " Part {} is lost forever. " , part_name ) ;
ProfileEvents : : increment ( ProfileEvents : : ReplicatedDataLoss ) ;
}
2016-04-09 03:50:02 +00:00
}
2019-07-03 13:17:19 +00:00
CheckResult ReplicatedMergeTreePartCheckThread : : checkPart ( const String & part_name )
2016-04-09 03:50:02 +00:00
{
2020-05-23 22:24:01 +00:00
LOG_WARNING ( log , " Checking part {} " , part_name ) ;
2017-04-01 07:20:54 +00:00
ProfileEvents : : increment ( ProfileEvents : : ReplicatedPartChecks ) ;
2017-12-20 17:40:36 +00:00
/// If the part is still in the PreCommitted -> Committed transition, it is not lost
/// and there is no need to go searching for it on other replicas. To definitely find the needed part
/// if it exists (or a part containing it) we first search among the PreCommitted parts.
2019-05-03 02:00:57 +00:00
auto part = storage . getPartIfExists ( part_name , { MergeTreeDataPartState : : PreCommitted } ) ;
2017-12-19 14:55:50 +00:00
if ( ! part )
2019-05-03 02:00:57 +00:00
part = storage . getActiveContainingPart ( part_name ) ;
2017-04-01 07:20:54 +00:00
/// We do not have this or a covering part.
if ( ! part )
{
2021-01-27 10:07:18 +00:00
searchForMissingPartAndFetchIfPossible ( part_name ) ;
2019-07-03 13:17:19 +00:00
return { part_name , false , " Part is missing, will search for it " } ;
2017-04-01 07:20:54 +00:00
}
/// We have this part, and it's active. We will check whether we need this part and whether it has the right data.
else if ( part - > name = = part_name )
{
auto zookeeper = storage . getZooKeeper ( ) ;
2020-04-09 18:10:27 +00:00
2020-06-18 16:10:47 +00:00
auto table_lock = storage . lockForShare ( RWLockImpl : : NO_QUERY , storage . getSettings ( ) - > lock_acquire_timeout_for_background_operations ) ;
2017-04-01 07:20:54 +00:00
2018-12-11 13:30:20 +00:00
auto local_part_header = ReplicatedMergeTreePartHeader : : fromColumnsAndChecksums (
2020-01-16 16:15:01 +00:00
part - > getColumns ( ) , part - > checksums ) ;
2018-12-11 13:30:20 +00:00
String part_path = storage . replica_path + " /parts/ " + part_name ;
String part_znode ;
2017-04-01 07:20:54 +00:00
/// If the part is in ZooKeeper, check its data with its checksums, and them with ZooKeeper.
2018-12-11 13:30:20 +00:00
if ( zookeeper - > tryGet ( part_path , part_znode ) )
2017-04-01 07:20:54 +00:00
{
2020-05-23 22:24:01 +00:00
LOG_WARNING ( log , " Checking data of part {}. " , part_name ) ;
2017-04-01 07:20:54 +00:00
try
{
2018-12-11 13:30:20 +00:00
ReplicatedMergeTreePartHeader zk_part_header ;
if ( ! part_znode . empty ( ) )
zk_part_header = ReplicatedMergeTreePartHeader : : fromString ( part_znode ) ;
else
{
String columns_znode = zookeeper - > get ( part_path + " /columns " ) ;
String checksums_znode = zookeeper - > get ( part_path + " /checksums " ) ;
zk_part_header = ReplicatedMergeTreePartHeader : : fromColumnsAndChecksumsZNodes (
columns_znode , checksums_znode ) ;
}
2017-04-01 07:20:54 +00:00
2018-12-11 13:30:20 +00:00
if ( local_part_header . getColumnsHash ( ) ! = zk_part_header . getColumnsHash ( ) )
2018-11-22 21:19:58 +00:00
throw Exception ( " Columns of local part " + part_name + " are different from ZooKeeper " , ErrorCodes : : TABLE_DIFFERS_TOO_MUCH ) ;
2017-04-01 07:20:54 +00:00
2018-12-11 13:30:20 +00:00
zk_part_header . getChecksums ( ) . checkEqual ( local_part_header . getChecksums ( ) , true ) ;
2017-12-03 00:46:34 +00:00
checkDataPart (
2019-03-20 16:18:13 +00:00
part ,
2017-12-03 00:46:34 +00:00
true ,
[ this ] { return need_stop . load ( ) ; } ) ;
2017-04-01 07:20:54 +00:00
if ( need_stop )
{
2020-05-23 22:24:01 +00:00
LOG_INFO ( log , " Checking part was cancelled. " ) ;
2019-07-03 13:17:19 +00:00
return { part_name , false , " Checking part was cancelled " } ;
2017-04-01 07:20:54 +00:00
}
2020-05-23 22:24:01 +00:00
LOG_INFO ( log , " Part {} looks good. " , part_name ) ;
2017-04-01 07:20:54 +00:00
}
2020-04-20 01:44:24 +00:00
catch ( const Exception & e )
2017-04-01 07:20:54 +00:00
{
2020-04-20 01:44:24 +00:00
/// Don't count the part as broken if there is not enough memory to load it.
/// In fact, there can be many similar situations.
/// But it is OK, because there is a safety guard against deleting too many parts.
if ( isNotEnoughMemoryErrorCode ( e . code ( ) ) )
throw ;
2017-12-03 00:46:34 +00:00
2018-04-06 21:46:57 +00:00
tryLogCurrentException ( log , __PRETTY_FUNCTION__ ) ;
2017-04-01 07:20:54 +00:00
2021-01-27 10:07:18 +00:00
String message = " Part " + part_name + " looks broken. Removing it and will try to fetch. " ;
2020-05-23 22:24:01 +00:00
LOG_ERROR ( log , message ) ;
2017-04-01 07:20:54 +00:00
2021-01-27 10:07:18 +00:00
/// Part is broken, let's try to find it and fetch.
searchForMissingPartAndFetchIfPossible ( part_name ) ;
2017-04-01 07:20:54 +00:00
/// Delete part locally.
2019-05-20 16:24:36 +00:00
storage . forgetPartAndMoveToDetached ( part , " broken " ) ;
2019-07-03 13:17:19 +00:00
return { part_name , false , message } ;
2017-04-01 07:20:54 +00:00
}
}
2017-12-18 04:07:26 +00:00
else if ( part - > modification_time + MAX_AGE_OF_LOCAL_PART_THAT_WASNT_ADDED_TO_ZOOKEEPER < time ( nullptr ) )
2017-04-01 07:20:54 +00:00
{
/// If the part is not in ZooKeeper, delete it locally.
/// Probably, someone just wrote down the part, and has not yet added to ZK.
/// Therefore, delete only if the part is old (not very reliable).
ProfileEvents : : increment ( ProfileEvents : : ReplicatedPartChecksFailed ) ;
2019-07-03 13:17:19 +00:00
String message = " Unexpected part " + part_name + " in filesystem. Removing. " ;
2020-05-23 22:24:01 +00:00
LOG_ERROR ( log , message ) ;
2019-05-20 16:24:36 +00:00
storage . forgetPartAndMoveToDetached ( part , " unexpected " ) ;
2019-07-03 13:17:19 +00:00
return { part_name , false , message } ;
2017-04-01 07:20:54 +00:00
}
else
{
/// TODO You need to make sure that the part is still checked after a while.
/// Otherwise, it's possible that the part was not added to ZK,
2017-06-21 19:07:08 +00:00
/// but remained in the filesystem and in a number of active parts.
2017-04-01 07:20:54 +00:00
/// And then for a long time (before restarting), the data on the replicas will be different.
2020-05-23 22:24:01 +00:00
LOG_TRACE ( log , " Young part {} with age {} seconds hasn't been added to ZooKeeper yet. It's ok. " , part_name , ( time ( nullptr ) - part - > modification_time ) ) ;
2017-04-01 07:20:54 +00:00
}
}
else
{
/// If we have a covering part, ignore all the problems with this part.
/// In the worst case, errors will still appear `old_parts_lifetime` seconds in error log until the part is removed as the old one.
2020-05-23 22:24:01 +00:00
LOG_WARNING ( log , " We have part {} covering part {} " , part - > name , part_name ) ;
2017-04-01 07:20:54 +00:00
}
2019-07-03 13:17:19 +00:00
return { part_name , true , " " } ;
2016-04-09 03:50:02 +00:00
}
void ReplicatedMergeTreePartCheckThread : : 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
2017-04-01 07:20:54 +00:00
{
2017-12-29 22:32:04 +00:00
time_t current_time = time ( nullptr ) ;
2017-12-21 18:17:06 +00:00
2017-12-29 22:32:04 +00:00
/// Take part from the queue for verification.
PartsToCheckQueue : : iterator selected = parts_queue . end ( ) ; /// end from std::list is not get invalidated
time_t min_check_time = std : : numeric_limits < time_t > : : max ( ) ;
2017-04-01 07:20:54 +00:00
2017-12-29 22:32:04 +00:00
{
2019-01-02 06:44:36 +00:00
std : : lock_guard lock ( parts_mutex ) ;
2017-12-21 18:17:06 +00:00
2017-12-29 22:32:04 +00:00
if ( parts_queue . empty ( ) )
{
if ( ! parts_set . empty ( ) )
2017-04-01 07:20:54 +00:00
{
2020-05-23 22:24:01 +00:00
LOG_ERROR ( log , " Non-empty parts_set with empty parts_queue. This is a bug. " ) ;
2017-12-29 22:32:04 +00:00
parts_set . clear ( ) ;
2017-04-01 07:20:54 +00:00
}
2017-12-29 22:32:04 +00:00
}
else
{
for ( auto it = parts_queue . begin ( ) ; it ! = parts_queue . end ( ) ; + + it )
2017-04-01 07:20:54 +00:00
{
2017-12-29 22:32:04 +00:00
if ( it - > second < = current_time )
2017-04-01 07:20:54 +00:00
{
2017-12-29 22:32:04 +00:00
selected = it ;
break ;
2017-04-01 07:20:54 +00:00
}
2017-12-29 22:32:04 +00:00
if ( it - > second < min_check_time )
min_check_time = it - > second ;
2017-04-01 07:20:54 +00:00
}
}
2017-12-29 22:32:04 +00:00
}
2017-04-01 07:20:54 +00:00
2017-12-29 22:32:04 +00:00
if ( selected = = parts_queue . end ( ) )
return ;
2017-04-01 07:20:54 +00:00
2017-12-29 22:32:04 +00:00
checkPart ( selected - > first ) ;
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
/// Remove the part from check queue.
{
2019-01-02 06:44:36 +00:00
std : : lock_guard lock ( parts_mutex ) ;
2017-04-01 07:20:54 +00:00
2017-12-29 22:32:04 +00:00
if ( parts_queue . empty ( ) )
2017-04-01 07:20:54 +00:00
{
2020-08-08 00:47:03 +00:00
LOG_ERROR ( log , " Someone erased checking part from parts_queue. This is a bug. " ) ;
2017-12-29 22:32:04 +00:00
}
else
{
parts_set . erase ( selected - > first ) ;
parts_queue . erase ( selected ) ;
2017-04-01 07:20:54 +00:00
}
}
2018-04-19 18:16:18 +00:00
2018-05-31 13:05:05 +00:00
task - > schedule ( ) ;
2017-11-17 08:58:35 +00:00
}
2018-08-25 01:58:14 +00:00
catch ( const Coordination : : Exception & e )
2018-04-24 17:11:59 +00:00
{
tryLogCurrentException ( log , __PRETTY_FUNCTION__ ) ;
2020-06-12 15:09:12 +00:00
if ( e . code = = Coordination : : Error : : ZSESSIONEXPIRED )
2018-04-24 17:11:59 +00:00
return ;
2017-12-21 18:17:06 +00:00
2018-05-31 13:05:05 +00:00
task - > scheduleAfter ( PART_CHECK_ERROR_SLEEP_MS ) ;
2018-04-24 17:11:59 +00:00
}
2017-12-29 22:32:04 +00:00
catch ( . . . )
{
2018-04-10 13:20:14 +00:00
tryLogCurrentException ( log , __PRETTY_FUNCTION__ ) ;
2018-05-31 13:05:05 +00:00
task - > scheduleAfter ( PART_CHECK_ERROR_SLEEP_MS ) ;
2017-12-29 22:32:04 +00:00
}
2016-04-09 03:50:02 +00:00
}
}