2017-04-01 09:19:00 +00:00
|
|
|
#include <Storages/MergeTree/ReplicatedMergeTreeCleanupThread.h>
|
|
|
|
#include <Storages/StorageReplicatedMergeTree.h>
|
|
|
|
#include <Common/setThreadName.h>
|
2017-07-24 20:12:59 +00:00
|
|
|
#include <Poco/Timestamp.h>
|
2014-10-15 01:22:06 +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 NOT_FOUND_NODE;
|
2016-01-11 21:46:36 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2014-10-15 01:22:06 +00:00
|
|
|
ReplicatedMergeTreeCleanupThread::ReplicatedMergeTreeCleanupThread(StorageReplicatedMergeTree & storage_)
|
2017-04-01 07:20:54 +00:00
|
|
|
: storage(storage_),
|
|
|
|
log(&Logger::get(storage.database_name + "." + storage.table_name + " (StorageReplicatedMergeTree, CleanupThread)")),
|
|
|
|
thread([this] { run(); }) {}
|
2014-10-15 01:22:06 +00:00
|
|
|
|
|
|
|
|
|
|
|
void ReplicatedMergeTreeCleanupThread::run()
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
setThreadName("ReplMTCleanup");
|
2015-09-24 18:54:21 +00:00
|
|
|
|
2017-07-24 20:12:59 +00:00
|
|
|
const auto CLEANUP_SLEEP_MS = storage.data.settings.cleanup_delay_period * 1000;
|
2014-10-15 01:22:06 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
while (!storage.shutdown_called)
|
|
|
|
{
|
|
|
|
try
|
|
|
|
{
|
|
|
|
iterate();
|
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
|
|
|
tryLogCurrentException(__PRETTY_FUNCTION__);
|
|
|
|
}
|
2014-10-15 01:22:06 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
storage.shutdown_event.tryWait(CLEANUP_SLEEP_MS);
|
|
|
|
}
|
2014-10-15 01:22:06 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
LOG_DEBUG(log, "Cleanup thread finished");
|
2014-10-15 01:22:06 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
void ReplicatedMergeTreeCleanupThread::iterate()
|
|
|
|
{
|
2017-05-24 20:19:29 +00:00
|
|
|
storage.clearOldPartsAndRemoveFromZK(log);
|
2017-04-01 07:20:54 +00:00
|
|
|
storage.data.clearOldTemporaryDirectories();
|
|
|
|
|
|
|
|
if (storage.is_leader_node)
|
|
|
|
{
|
|
|
|
clearOldLogs();
|
|
|
|
clearOldBlocks();
|
|
|
|
}
|
2014-10-15 01:22:06 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
void ReplicatedMergeTreeCleanupThread::clearOldLogs()
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
auto zookeeper = storage.getZooKeeper();
|
|
|
|
|
|
|
|
zkutil::Stat stat;
|
|
|
|
if (!zookeeper->exists(storage.zookeeper_path + "/log", &stat))
|
|
|
|
throw Exception(storage.zookeeper_path + "/log doesn't exist", ErrorCodes::NOT_FOUND_NODE);
|
|
|
|
|
|
|
|
int children_count = stat.numChildren;
|
|
|
|
|
|
|
|
/// We will wait for 1.1 times more records to accumulate than necessary.
|
|
|
|
if (static_cast<double>(children_count) < storage.data.settings.replicated_logs_to_keep * 1.1)
|
|
|
|
return;
|
|
|
|
|
|
|
|
Strings replicas = zookeeper->getChildren(storage.zookeeper_path + "/replicas", &stat);
|
|
|
|
UInt64 min_pointer = std::numeric_limits<UInt64>::max();
|
|
|
|
for (const String & replica : replicas)
|
|
|
|
{
|
|
|
|
String pointer = zookeeper->get(storage.zookeeper_path + "/replicas/" + replica + "/log_pointer");
|
|
|
|
if (pointer.empty())
|
|
|
|
return;
|
|
|
|
min_pointer = std::min(min_pointer, parse<UInt64>(pointer));
|
|
|
|
}
|
|
|
|
|
|
|
|
Strings entries = zookeeper->getChildren(storage.zookeeper_path + "/log");
|
|
|
|
std::sort(entries.begin(), entries.end());
|
|
|
|
|
|
|
|
/// We will not touch the last `replicated_logs_to_keep` records.
|
|
|
|
entries.erase(entries.end() - std::min(entries.size(), storage.data.settings.replicated_logs_to_keep), entries.end());
|
|
|
|
/// We will not touch records that are no less than `min_pointer`.
|
|
|
|
entries.erase(std::lower_bound(entries.begin(), entries.end(), "log-" + padIndex(min_pointer)), entries.end());
|
|
|
|
|
|
|
|
if (entries.empty())
|
|
|
|
return;
|
|
|
|
|
|
|
|
zkutil::Ops ops;
|
|
|
|
for (size_t i = 0; i < entries.size(); ++i)
|
|
|
|
{
|
|
|
|
ops.emplace_back(std::make_unique<zkutil::Op::Remove>(storage.zookeeper_path + "/log/" + entries[i], -1));
|
|
|
|
|
|
|
|
if (ops.size() > 400 || i + 1 == entries.size())
|
|
|
|
{
|
|
|
|
/// Simultaneously with clearing the log, we check to see if replica was added since we received replicas list.
|
|
|
|
ops.emplace_back(std::make_unique<zkutil::Op::Check>(storage.zookeeper_path + "/replicas", stat.version));
|
|
|
|
zookeeper->multi(ops);
|
|
|
|
ops.clear();
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
LOG_DEBUG(log, "Removed " << entries.size() << " old log entries: " << entries.front() << " - " << entries.back());
|
2014-10-15 01:22:06 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
void ReplicatedMergeTreeCleanupThread::clearOldBlocks()
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
auto zookeeper = storage.getZooKeeper();
|
2014-12-12 20:50:32 +00:00
|
|
|
|
2017-08-06 21:40:38 +00:00
|
|
|
Strings blocks;
|
2017-04-01 07:20:54 +00:00
|
|
|
zkutil::Stat stat;
|
2017-08-06 21:40:38 +00:00
|
|
|
if (ZOK != zookeeper->tryGetChildren(storage.zookeeper_path + "/blocks", blocks, &stat))
|
2017-04-01 07:20:54 +00:00
|
|
|
throw Exception(storage.zookeeper_path + "/blocks doesn't exist", ErrorCodes::NOT_FOUND_NODE);
|
2014-10-15 01:22:06 +00:00
|
|
|
|
2017-08-06 21:40:38 +00:00
|
|
|
/// Clear already deleted blocks from the cache, cached_block_ctime should be subset of blocks
|
|
|
|
{
|
|
|
|
NameSet blocks_set(blocks.begin(), blocks.end());
|
|
|
|
for (auto it = cached_block_ctime.begin(); it != cached_block_ctime.end();)
|
|
|
|
{
|
|
|
|
if (!blocks_set.count(it->first))
|
|
|
|
it = cached_block_ctime.erase(it);
|
|
|
|
else
|
|
|
|
++it;
|
|
|
|
}
|
|
|
|
}
|
2014-10-15 01:22:06 +00:00
|
|
|
|
2017-08-06 21:40:38 +00:00
|
|
|
auto not_cached_blocks = stat.numChildren - cached_block_ctime.size();
|
|
|
|
LOG_TRACE(log, "Checking " << stat.numChildren << " blocks (" << not_cached_blocks << " are not cached)"
|
|
|
|
<< " to clear old ones from ZooKeeper. This might take several minutes.");
|
2014-10-15 01:22:06 +00:00
|
|
|
|
2017-07-26 20:41:46 +00:00
|
|
|
/// Time -> block hash from ZooKeeper (from node name)
|
2017-07-24 20:12:59 +00:00
|
|
|
using TimedBlock = std::pair<Int64, String>;
|
|
|
|
using TimedBlocksComparator = std::greater<TimedBlock>;
|
|
|
|
std::vector<TimedBlock> timed_blocks;
|
2014-10-15 01:22:06 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
for (const String & block : blocks)
|
|
|
|
{
|
2017-08-06 21:40:38 +00:00
|
|
|
auto it = cached_block_ctime.find(block);
|
|
|
|
|
|
|
|
if (it == cached_block_ctime.end())
|
|
|
|
{
|
|
|
|
/// New block. Fetch its stat and put it into the cache
|
|
|
|
zkutil::Stat block_stat;
|
|
|
|
zookeeper->exists(storage.zookeeper_path + "/blocks/" + block, &block_stat);
|
|
|
|
cached_block_ctime.emplace(block, block_stat.ctime);
|
|
|
|
timed_blocks.emplace_back(block_stat.ctime, block);
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
/// Cached block
|
|
|
|
timed_blocks.emplace_back(it->second, block);
|
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2014-10-15 01:22:06 +00:00
|
|
|
|
2017-07-24 20:12:59 +00:00
|
|
|
if (timed_blocks.empty())
|
|
|
|
return;
|
|
|
|
|
|
|
|
std::sort(timed_blocks.begin(), timed_blocks.end(), TimedBlocksComparator());
|
|
|
|
|
|
|
|
/// Use ZooKeeper's first node (last according to time) timestamp as "current" time.
|
|
|
|
Int64 current_time = timed_blocks.front().first;
|
2017-07-26 20:40:09 +00:00
|
|
|
Int64 time_threshold = std::max(0L, current_time - static_cast<Int64>(storage.data.settings.replicated_deduplication_window_seconds));
|
|
|
|
TimedBlock block_threshold(time_threshold, "");
|
2017-07-24 20:12:59 +00:00
|
|
|
|
2017-07-26 21:42:16 +00:00
|
|
|
size_t current_deduplication_window = std::min(timed_blocks.size(), storage.data.settings.replicated_deduplication_window);
|
|
|
|
auto first_outdated_block_fixed_threshold = timed_blocks.begin() + current_deduplication_window;
|
2017-07-26 20:40:09 +00:00
|
|
|
auto first_outdated_block_time_threshold = std::upper_bound(timed_blocks.begin(), timed_blocks.end(), block_threshold, TimedBlocksComparator());
|
|
|
|
auto first_outdated_block = std::min(first_outdated_block_fixed_threshold, first_outdated_block_time_threshold);
|
2017-07-24 20:12:59 +00:00
|
|
|
|
|
|
|
for (auto it = first_outdated_block; it != timed_blocks.end(); ++it)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2017-06-25 00:01:10 +00:00
|
|
|
/// TODO After about half a year, we could replace this to multi op, because there will be no obsolete children nodes.
|
2017-07-24 20:12:59 +00:00
|
|
|
zookeeper->removeRecursive(storage.zookeeper_path + "/blocks/" + it->second);
|
2017-08-06 21:40:38 +00:00
|
|
|
cached_block_ctime.erase(it->second);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2014-10-15 01:22:06 +00:00
|
|
|
|
2017-07-24 20:12:59 +00:00
|
|
|
LOG_TRACE(log, "Cleared " << timed_blocks.end() - first_outdated_block << " old blocks from ZooKeeper");
|
2014-10-15 01:22:06 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
}
|