mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 01:25:21 +00:00
Merge branch 'master' into replicated-mergetree-mutations
Conflicts: dbms/src/Common/ActionBlocker.h dbms/src/Storages/MergeTree/ActiveDataPartSet.cpp dbms/src/Storages/MergeTree/ActiveDataPartSet.h dbms/src/Storages/MergeTree/MergeTreeData.h dbms/src/Storages/MergeTree/MergeTreeDataPart.cpp dbms/src/Storages/MergeTree/ReplicatedMergeTreeAlterThread.cpp dbms/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp dbms/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h dbms/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp dbms/src/Storages/StorageMergeTree.cpp dbms/src/Storages/StorageReplicatedMergeTree.cpp dbms/src/Storages/StorageReplicatedMergeTree.h
This commit is contained in:
commit
f6c72f8e56
11
CHANGELOG.draft.md
Normal file
11
CHANGELOG.draft.md
Normal file
@ -0,0 +1,11 @@
|
||||
en:
|
||||
|
||||
## Improvements:
|
||||
* `clickhouse-client`: option --ask-password for interactively ask for credentials #1044
|
||||
|
||||
|
||||
|
||||
ru:
|
||||
|
||||
## Улучшения:
|
||||
* `clickhouse-client`: опция --ask-password для интерактивного ввода пароля #1044
|
@ -128,14 +128,14 @@ endif ()
|
||||
#endif ()
|
||||
|
||||
if (CMAKE_VERSION VERSION_LESS "3.8.0")
|
||||
set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -std=c++1z")
|
||||
if (NOT MSVC)
|
||||
set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -std=c++1z")
|
||||
endif ()
|
||||
else ()
|
||||
set (CMAKE_CXX_STANDARD 17)
|
||||
set (CMAKE_CXX_EXTENSIONS 0) # https://cmake.org/cmake/help/latest/prop_tgt/CXX_EXTENSIONS.html#prop_tgt:CXX_EXTENSIONS
|
||||
set (CMAKE_CXX_STANDARD_REQUIRED ON)
|
||||
set (CXX_FLAGS_INTERNAL_COMPILER "-std=c++1z")
|
||||
# This needs to propagate to vendored projects in contrib
|
||||
set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -std=c++17")
|
||||
endif ()
|
||||
|
||||
set (CMAKE_BUILD_COLOR_MAKEFILE ON)
|
||||
|
@ -1,4 +1,4 @@
|
||||
option (ENABLE_EMBEDDED_COMPILER "Set to TRUE to enable support for 'compile' option for query execution" 1)
|
||||
option (ENABLE_EMBEDDED_COMPILER "Set to TRUE to enable support for 'compile' option for query execution" 0)
|
||||
option (USE_INTERNAL_LLVM_LIBRARY "Use bundled or system LLVM library. Default: system library for quicker developer builds." ${APPLE})
|
||||
|
||||
if (ENABLE_EMBEDDED_COMPILER)
|
||||
|
2
contrib/CMakeLists.txt
vendored
2
contrib/CMakeLists.txt
vendored
@ -1,5 +1,5 @@
|
||||
if (NOT MSVC)
|
||||
set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Wno-old-style-cast")
|
||||
set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Wno-old-style-cast -std=c++1z")
|
||||
endif ()
|
||||
|
||||
if (USE_INTERNAL_BOOST_LIBRARY)
|
||||
|
@ -116,3 +116,10 @@
|
||||
/* Define to the type of an unsigned integer type of width exactly 8 bits if
|
||||
such a type exists and the standard includes do not define it. */
|
||||
/* #undef uint8_t */
|
||||
|
||||
#ifdef _MSC_VER
|
||||
#include <basetsd.h>
|
||||
typedef SSIZE_T ssize_t;
|
||||
#else
|
||||
#include <sys/types.h>
|
||||
#endif
|
||||
|
@ -17,7 +17,7 @@
|
||||
|
||||
SOURCE_PATH=${1:-.}
|
||||
DST=${2:-$SOURCE_PATH/../headers}
|
||||
BUILD_PATH=${3:-$SOURCE_PATH/build}
|
||||
BUILD_PATH=${BUILD_PATH=${3:-$SOURCE_PATH/build}}
|
||||
|
||||
PATH="/usr/local/bin:/usr/local/sbin:/usr/bin:$PATH"
|
||||
|
||||
@ -30,23 +30,29 @@ START_HEADERS=$(echo \
|
||||
$SOURCE_PATH/dbms/src/Interpreters/SpecializedAggregator.h \
|
||||
$SOURCE_PATH/dbms/src/AggregateFunctions/AggregateFunction*.h)
|
||||
|
||||
for header in $START_HEADERS; do
|
||||
START_HEADERS_INCLUDE+="-include $header "
|
||||
done
|
||||
|
||||
# Опция -mcx16 для того, чтобы выбиралось больше заголовочных файлов (с запасом).
|
||||
# The latter options are the same that are added while building packages.
|
||||
|
||||
# TODO: Does not work on macos:
|
||||
GCC_ROOT=`$CLANG -v 2>&1 | grep "Selected GCC installation"| sed -n -e 's/^.*: //p'`
|
||||
|
||||
for src_file in $(echo | $CLANG -M -xc++ -std=c++1z -Wall -Werror -msse4 -mcx16 -mpopcnt -O3 -g -fPIC -fstack-protector -D_FORTIFY_SOURCE=2 \
|
||||
-I $GCC_ROOT/include \
|
||||
-I $GCC_ROOT/include-fixed \
|
||||
$(cat "$BUILD_PATH/include_directories.txt") \
|
||||
$(echo $START_HEADERS | sed -r -e 's/[^ ]+/-include \0/g') \
|
||||
$START_HEADERS_INCLUDE \
|
||||
- |
|
||||
tr -d '\\' |
|
||||
sed -r -e 's/^-\.o://');
|
||||
sed -E -e 's/^-\.o://');
|
||||
do
|
||||
dst_file=$src_file;
|
||||
dst_file=$(echo $dst_file | sed -r -e 's/build\///') # for simplicity reasons, will put generated headers near the rest.
|
||||
mkdir -p "$DST/$(echo $dst_file | sed -r -e 's/\/[^/]*$/\//')";
|
||||
[ -n $DESTDIR ] && dst_file=$(echo $dst_file | sed -E -e "s!^$DESTDIR!!")
|
||||
dst_file=$(echo $dst_file | sed -E -e 's/build\///') # for simplicity reasons, will put generated headers near the rest.
|
||||
mkdir -p "$DST/$(echo $dst_file | sed -E -e 's/\/[^/]*$/\//')";
|
||||
cp "$src_file" "$DST/$dst_file";
|
||||
done
|
||||
|
||||
@ -56,19 +62,25 @@ done
|
||||
|
||||
for src_file in $(ls -1 $($CLANG -v -xc++ - <<<'' 2>&1 | grep '^ /' | grep 'include' | grep -E '/lib/clang/|/include/clang/')/*.h | grep -vE 'arm|altivec|Intrin');
|
||||
do
|
||||
mkdir -p "$DST/$(echo $src_file | sed -r -e 's/\/[^/]*$/\//')";
|
||||
cp "$src_file" "$DST/$src_file";
|
||||
dst_file=$src_file;
|
||||
[ -n $DESTDIR ] && dst_file=$(echo $dst_file | sed -E -e "s!^$DESTDIR!!")
|
||||
mkdir -p "$DST/$(echo $dst_file | sed -E -e 's/\/[^/]*$/\//')";
|
||||
cp "$src_file" "$DST/$dst_file";
|
||||
done
|
||||
|
||||
# Even more platform-specific headers
|
||||
for src_file in $(ls -1 $SOURCE_PATH/contrib/boost/libs/smart_ptr/include/boost/smart_ptr/detail/*);
|
||||
do
|
||||
mkdir -p "$DST/$(echo $src_file | sed -r -e 's/\/[^/]*$/\//')";
|
||||
cp "$src_file" "$DST/$src_file";
|
||||
dst_file=$src_file;
|
||||
[ -n $DESTDIR ] && dst_file=$(echo $dst_file | sed -E -e "s!^$DESTDIR!!")
|
||||
mkdir -p "$DST/$(echo $dst_file | sed -E -e 's/\/[^/]*$/\//')";
|
||||
cp "$src_file" "$DST/$dst_file";
|
||||
done
|
||||
|
||||
for src_file in $(ls -1 $SOURCE_PATH/contrib/boost/boost/smart_ptr/detail/*);
|
||||
do
|
||||
mkdir -p "$DST/$(echo $src_file | sed -r -e 's/\/[^/]*$/\//')";
|
||||
cp "$src_file" "$DST/$src_file";
|
||||
dst_file=$src_file;
|
||||
[ -n $DESTDIR ] && dst_file=$(echo $dst_file | sed -E -e "s!^$DESTDIR!!")
|
||||
mkdir -p "$DST/$(echo $dst_file | sed -E -e 's/\/[^/]*$/\//')";
|
||||
cp "$src_file" "$DST/$dst_file";
|
||||
done
|
||||
|
@ -245,6 +245,12 @@ target_include_directories (clickhouse_common_io BEFORE PUBLIC ${DOUBLE_CONVERSI
|
||||
# also for copy_headers.sh:
|
||||
target_include_directories (clickhouse_common_io BEFORE PRIVATE ${COMMON_INCLUDE_DIR})
|
||||
|
||||
if (USE_EMBEDDED_COMPILER)
|
||||
add_custom_target(copy-headers ALL env CLANG=${CMAKE_CURRENT_BINARY_DIR}/src/Server/clickhouse-clang BUILD_PATH=${ClickHouse_BINARY_DIR} DESTDIR=${ClickHouse_SOURCE_DIR} ${ClickHouse_SOURCE_DIR}/copy_headers.sh ${ClickHouse_SOURCE_DIR} ${CMAKE_CURRENT_BINARY_DIR}/headers DEPENDS clickhouse-clang WORKING_DIRECTORY ${ClickHouse_SOURCE_DIR} SOURCES ${ClickHouse_SOURCE_DIR}/copy_headers.sh)
|
||||
install(DIRECTORY ${CMAKE_CURRENT_BINARY_DIR}/headers DESTINATION ${CMAKE_INSTALL_DATAROOTDIR}/clickhouse COMPONENT clickhouse)
|
||||
add_dependencies(clickhouse-bundle copy-headers)
|
||||
endif ()
|
||||
|
||||
add_subdirectory (tests)
|
||||
|
||||
if (ENABLE_TESTS)
|
||||
|
@ -1,6 +1,8 @@
|
||||
#pragma once
|
||||
|
||||
#include <atomic>
|
||||
#include <memory>
|
||||
#include <Common/ActionLock.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -9,55 +11,26 @@ namespace DB
|
||||
/// If it is not zero then actions related with it should be considered as interrupted
|
||||
class ActionBlocker
|
||||
{
|
||||
private:
|
||||
std::atomic<int> counter{0};
|
||||
|
||||
public:
|
||||
bool isCancelled() const { return counter > 0; }
|
||||
ActionBlocker() : counter(std::make_shared<Counter>(0)) {}
|
||||
|
||||
bool isCancelled() const { return *counter > 0; }
|
||||
|
||||
/// Temporarily blocks corresponding actions (while the returned object is alive)
|
||||
struct LockHolder;
|
||||
LockHolder cancel() { return LockHolder(this); }
|
||||
friend class ActionLock;
|
||||
ActionLock cancel() { return ActionLock(*this); }
|
||||
|
||||
/// Cancel the actions forever.
|
||||
void cancelForever() { ++counter; }
|
||||
void cancelForever() { ++(*counter); }
|
||||
|
||||
/// Returns reference to the counter to allow observing it directly.
|
||||
const auto & getCounter() { return counter; }
|
||||
/// Returns reference to counter to allow to watch on it directly.
|
||||
const std::atomic<int> & getCounter() const { return *counter; }
|
||||
|
||||
/// Blocks related action while a BlockerHolder instance exists
|
||||
struct LockHolder
|
||||
{
|
||||
explicit LockHolder(ActionBlocker * var_ = nullptr) : var(var_)
|
||||
{
|
||||
if (var)
|
||||
++var->counter;
|
||||
}
|
||||
private:
|
||||
using Counter = std::atomic<int>;
|
||||
using CounterPtr = std::shared_ptr<Counter>;
|
||||
|
||||
LockHolder(LockHolder && other) noexcept
|
||||
{
|
||||
*this = std::move(other);
|
||||
}
|
||||
|
||||
LockHolder & operator=(LockHolder && other) noexcept
|
||||
{
|
||||
var = other.var;
|
||||
other.var = nullptr;
|
||||
return *this;
|
||||
}
|
||||
|
||||
LockHolder(const LockHolder & other) = delete;
|
||||
LockHolder & operator=(const LockHolder & other) = delete;
|
||||
|
||||
~LockHolder()
|
||||
{
|
||||
if (var)
|
||||
--var->counter;
|
||||
}
|
||||
|
||||
private:
|
||||
ActionBlocker * var = nullptr;
|
||||
};
|
||||
CounterPtr counter;
|
||||
};
|
||||
|
||||
}
|
||||
|
33
dbms/src/Common/ActionLock.cpp
Normal file
33
dbms/src/Common/ActionLock.cpp
Normal file
@ -0,0 +1,33 @@
|
||||
#include "ActionLock.h"
|
||||
#include <Common/ActionBlocker.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
ActionLock::ActionLock(const ActionBlocker & blocker) : counter_ptr(blocker.counter)
|
||||
{
|
||||
if (auto counter = counter_ptr.lock())
|
||||
++(*counter);
|
||||
}
|
||||
|
||||
ActionLock::ActionLock(ActionLock && other)
|
||||
{
|
||||
*this = std::move(other);
|
||||
}
|
||||
|
||||
ActionLock & ActionLock::operator=(ActionLock && other)
|
||||
{
|
||||
auto lock_lhs = this->counter_ptr.lock();
|
||||
|
||||
counter_ptr = std::move(other.counter_ptr);
|
||||
/// After move other.counter_ptr still points to counter, reset it explicitly
|
||||
other.counter_ptr.reset();
|
||||
|
||||
if (lock_lhs)
|
||||
--(*lock_lhs);
|
||||
|
||||
return *this;
|
||||
}
|
||||
|
||||
}
|
46
dbms/src/Common/ActionLock.h
Normal file
46
dbms/src/Common/ActionLock.h
Normal file
@ -0,0 +1,46 @@
|
||||
#pragma once
|
||||
#include <memory>
|
||||
#include <atomic>
|
||||
#include <Core/Types.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class ActionBlocker;
|
||||
using StorageActionBlockType = size_t;
|
||||
|
||||
/// Blocks related action while a ActionLock instance exists
|
||||
/// ActionBlocker could be destroyed before the lock, in this case ActionLock will safely do nothing in its destructor
|
||||
class ActionLock
|
||||
{
|
||||
public:
|
||||
|
||||
ActionLock() = default;
|
||||
|
||||
explicit ActionLock(const ActionBlocker & blocker);
|
||||
|
||||
ActionLock(ActionLock && other);
|
||||
ActionLock & operator=(ActionLock && other);
|
||||
|
||||
ActionLock(const ActionLock & other) = delete;
|
||||
ActionLock & operator=(const ActionLock & other) = delete;
|
||||
|
||||
bool expired() const
|
||||
{
|
||||
return counter_ptr.expired();
|
||||
}
|
||||
|
||||
~ActionLock()
|
||||
{
|
||||
if (auto counter = counter_ptr.lock())
|
||||
--(*counter);
|
||||
}
|
||||
|
||||
private:
|
||||
using Counter = std::atomic<int>;
|
||||
using CounterWeakPtr = std::weak_ptr<Counter>;
|
||||
|
||||
CounterWeakPtr counter_ptr;
|
||||
};
|
||||
|
||||
}
|
300
dbms/src/Common/BackgroundSchedulePool.cpp
Normal file
300
dbms/src/Common/BackgroundSchedulePool.cpp
Normal file
@ -0,0 +1,300 @@
|
||||
#include <Common/BackgroundSchedulePool.h>
|
||||
#include <Common/MemoryTracker.h>
|
||||
#include <Common/CurrentMetrics.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/setThreadName.h>
|
||||
#include <Common/Stopwatch.h>
|
||||
#include <common/logger_useful.h>
|
||||
#include <chrono>
|
||||
|
||||
namespace CurrentMetrics
|
||||
{
|
||||
extern const Metric BackgroundSchedulePoolTask;
|
||||
extern const Metric MemoryTrackingInBackgroundSchedulePool;
|
||||
}
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
||||
// TaskNotification
|
||||
|
||||
class TaskNotification final : public Poco::Notification
|
||||
{
|
||||
public:
|
||||
explicit TaskNotification(const BackgroundSchedulePool::TaskHandle & task) : task(task) {}
|
||||
void execute() { task->execute(); }
|
||||
|
||||
private:
|
||||
BackgroundSchedulePool::TaskHandle task;
|
||||
};
|
||||
|
||||
|
||||
// BackgroundSchedulePool::TaskInfo
|
||||
|
||||
BackgroundSchedulePool::TaskInfo::TaskInfo(BackgroundSchedulePool & pool, const std::string & name, const Task & function):
|
||||
name(name),
|
||||
pool(pool),
|
||||
function(function)
|
||||
{
|
||||
}
|
||||
|
||||
|
||||
bool BackgroundSchedulePool::TaskInfo::schedule()
|
||||
{
|
||||
std::lock_guard lock(schedule_mutex);
|
||||
|
||||
if (deactivated || scheduled)
|
||||
return false;
|
||||
|
||||
scheduled = true;
|
||||
|
||||
if(!executing)
|
||||
{
|
||||
if (delayed)
|
||||
pool.cancelDelayedTask(shared_from_this(), lock);
|
||||
|
||||
pool.queue.enqueueNotification(new TaskNotification(shared_from_this()));
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
|
||||
bool BackgroundSchedulePool::TaskInfo::scheduleAfter(size_t ms)
|
||||
{
|
||||
std::lock_guard lock(schedule_mutex);
|
||||
|
||||
if (deactivated || scheduled)
|
||||
return false;
|
||||
|
||||
pool.scheduleDelayedTask(shared_from_this(), ms, lock);
|
||||
return true;
|
||||
}
|
||||
|
||||
|
||||
void BackgroundSchedulePool::TaskInfo::deactivate()
|
||||
{
|
||||
std::lock_guard lock_exec(exec_mutex);
|
||||
std::lock_guard lock_schedule(schedule_mutex);
|
||||
|
||||
if (deactivated)
|
||||
return;
|
||||
|
||||
deactivated = true;
|
||||
scheduled = false;
|
||||
|
||||
if (delayed)
|
||||
pool.cancelDelayedTask(shared_from_this(), lock_schedule);
|
||||
}
|
||||
|
||||
|
||||
void BackgroundSchedulePool::TaskInfo::activate()
|
||||
{
|
||||
std::lock_guard lock(schedule_mutex);
|
||||
deactivated = false;
|
||||
}
|
||||
|
||||
|
||||
void BackgroundSchedulePool::TaskInfo::execute()
|
||||
{
|
||||
std::lock_guard lock_exec(exec_mutex);
|
||||
|
||||
{
|
||||
std::lock_guard lock_schedule(schedule_mutex);
|
||||
|
||||
if (deactivated)
|
||||
return;
|
||||
|
||||
scheduled = false;
|
||||
executing = true;
|
||||
}
|
||||
|
||||
CurrentMetrics::Increment metric_increment{CurrentMetrics::BackgroundSchedulePoolTask};
|
||||
|
||||
Stopwatch watch;
|
||||
function();
|
||||
UInt64 milliseconds = watch.elapsedMilliseconds();
|
||||
|
||||
/// If the task is executed longer than specified time, it will be logged.
|
||||
static const int32_t slow_execution_threshold_ms = 50;
|
||||
|
||||
if (milliseconds >= slow_execution_threshold_ms)
|
||||
LOG_INFO(&Logger::get("BackgroundSchedulePool"), "Executing " << name << " took " << milliseconds << " ms.");
|
||||
|
||||
{
|
||||
std::lock_guard lock_schedule(schedule_mutex);
|
||||
|
||||
executing = false;
|
||||
|
||||
/// In case was scheduled while executing (including a scheduleAfter which expired) we schedule the task
|
||||
/// on the queue. We don't call the function again here because this way all tasks
|
||||
/// will have their chance to execute
|
||||
|
||||
if(scheduled)
|
||||
pool.queue.enqueueNotification(new TaskNotification(shared_from_this()));
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
zkutil::WatchCallback BackgroundSchedulePool::TaskInfo::getWatchCallback()
|
||||
{
|
||||
return [t=shared_from_this()](const ZooKeeperImpl::ZooKeeper::WatchResponse &) {
|
||||
t->schedule();
|
||||
};
|
||||
}
|
||||
|
||||
|
||||
// BackgroundSchedulePool
|
||||
|
||||
BackgroundSchedulePool::BackgroundSchedulePool(size_t size)
|
||||
: size(size)
|
||||
{
|
||||
LOG_INFO(&Logger::get("BackgroundSchedulePool"), "Create BackgroundSchedulePool with " << size << " threads");
|
||||
|
||||
threads.resize(size);
|
||||
for (auto & thread : threads)
|
||||
thread = std::thread([this] { threadFunction(); });
|
||||
|
||||
delayed_thread = std::thread([this] { delayExecutionThreadFunction(); });
|
||||
}
|
||||
|
||||
|
||||
BackgroundSchedulePool::~BackgroundSchedulePool()
|
||||
{
|
||||
try
|
||||
{
|
||||
{
|
||||
std::unique_lock lock(delayed_tasks_lock);
|
||||
shutdown = true;
|
||||
wakeup_cond.notify_all();
|
||||
}
|
||||
|
||||
queue.wakeUpAll();
|
||||
delayed_thread.join();
|
||||
|
||||
LOG_TRACE(&Logger::get("BackgroundSchedulePool"), "Waiting for threads to finish.");
|
||||
for (std::thread & thread : threads)
|
||||
thread.join();
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(__PRETTY_FUNCTION__);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
BackgroundSchedulePool::TaskHandle BackgroundSchedulePool::addTask(const std::string & name, const Task & task)
|
||||
{
|
||||
return std::make_shared<TaskInfo>(*this, name, task);
|
||||
}
|
||||
|
||||
|
||||
void BackgroundSchedulePool::removeTask(const TaskHandle & task)
|
||||
{
|
||||
task->deactivate();
|
||||
}
|
||||
|
||||
|
||||
void BackgroundSchedulePool::scheduleDelayedTask(const TaskHandle & task, size_t ms, std::lock_guard<std::mutex> & /* schedule_mutex_lock */)
|
||||
{
|
||||
Poco::Timestamp current_time;
|
||||
|
||||
{
|
||||
std::lock_guard lock(delayed_tasks_lock);
|
||||
|
||||
if (task->delayed)
|
||||
delayed_tasks.erase(task->iterator);
|
||||
|
||||
task->iterator = delayed_tasks.emplace(current_time + (ms * 1000), task);
|
||||
task->delayed = true;
|
||||
}
|
||||
|
||||
wakeup_cond.notify_all();
|
||||
}
|
||||
|
||||
|
||||
void BackgroundSchedulePool::cancelDelayedTask(const TaskHandle & task, std::lock_guard<std::mutex> & /* schedule_mutex_lock */)
|
||||
{
|
||||
{
|
||||
std::lock_guard lock(delayed_tasks_lock);
|
||||
delayed_tasks.erase(task->iterator);
|
||||
task->delayed = false;
|
||||
}
|
||||
|
||||
wakeup_cond.notify_all();
|
||||
}
|
||||
|
||||
|
||||
void BackgroundSchedulePool::threadFunction()
|
||||
{
|
||||
setThreadName("BackgrSchedPool");
|
||||
|
||||
MemoryTracker memory_tracker;
|
||||
memory_tracker.setMetric(CurrentMetrics::MemoryTrackingInBackgroundSchedulePool);
|
||||
current_memory_tracker = &memory_tracker;
|
||||
|
||||
while (!shutdown)
|
||||
{
|
||||
if (Poco::AutoPtr<Poco::Notification> notification = queue.waitDequeueNotification())
|
||||
{
|
||||
TaskNotification & task_notification = static_cast<TaskNotification &>(*notification);
|
||||
task_notification.execute();
|
||||
}
|
||||
}
|
||||
|
||||
current_memory_tracker = nullptr;
|
||||
}
|
||||
|
||||
|
||||
void BackgroundSchedulePool::delayExecutionThreadFunction()
|
||||
{
|
||||
setThreadName("BckSchPoolDelay");
|
||||
|
||||
while (!shutdown)
|
||||
{
|
||||
TaskHandle task;
|
||||
bool found = false;
|
||||
|
||||
{
|
||||
std::unique_lock lock(delayed_tasks_lock);
|
||||
|
||||
while(!shutdown)
|
||||
{
|
||||
Poco::Timestamp min_time;
|
||||
|
||||
if (!delayed_tasks.empty())
|
||||
{
|
||||
auto t = delayed_tasks.begin();
|
||||
min_time = t->first;
|
||||
task = t->second;
|
||||
}
|
||||
|
||||
if (!task)
|
||||
{
|
||||
wakeup_cond.wait(lock);
|
||||
continue;
|
||||
}
|
||||
|
||||
Poco::Timestamp current_time;
|
||||
|
||||
if (min_time > current_time)
|
||||
{
|
||||
wakeup_cond.wait_for(lock, std::chrono::microseconds(min_time - current_time));
|
||||
continue;
|
||||
}
|
||||
else
|
||||
{
|
||||
/// We have a task ready for execution
|
||||
found = true;
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if(found)
|
||||
task->schedule();
|
||||
}
|
||||
}
|
||||
|
||||
}
|
120
dbms/src/Common/BackgroundSchedulePool.h
Normal file
120
dbms/src/Common/BackgroundSchedulePool.h
Normal file
@ -0,0 +1,120 @@
|
||||
#pragma once
|
||||
|
||||
#include <Poco/Notification.h>
|
||||
#include <Poco/NotificationQueue.h>
|
||||
#include <Poco/Timestamp.h>
|
||||
#include <thread>
|
||||
#include <atomic>
|
||||
#include <mutex>
|
||||
#include <condition_variable>
|
||||
#include <vector>
|
||||
#include <map>
|
||||
#include <functional>
|
||||
#include <boost/noncopyable.hpp>
|
||||
#include <Common/ZooKeeper/Types.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class TaskNotification;
|
||||
|
||||
|
||||
/** Executes functions scheduled at a specific point in time.
|
||||
* Basically all tasks are added in a queue and precessed by worker threads.
|
||||
*
|
||||
* The most important difference between this and BackgroundProcessingPool
|
||||
* is that we have the guarantee that the same function is not executed from many workers in the same time.
|
||||
*
|
||||
* The usage scenario: instead starting a separate thread for each task,
|
||||
* register a task in BackgroundSchedulePool and when you need to run the task,
|
||||
* call schedule or scheduleAfter(duration) method.
|
||||
*/
|
||||
class BackgroundSchedulePool
|
||||
{
|
||||
public:
|
||||
class TaskInfo;
|
||||
using TaskHandle = std::shared_ptr<TaskInfo>;
|
||||
using Tasks = std::multimap<Poco::Timestamp, TaskHandle>;
|
||||
using Task = std::function<void()>;
|
||||
|
||||
class TaskInfo : public std::enable_shared_from_this<TaskInfo>, private boost::noncopyable
|
||||
{
|
||||
public:
|
||||
TaskInfo(BackgroundSchedulePool & pool, const std::string & name, const Task & function);
|
||||
|
||||
/// All these methods waits for current execution of task.
|
||||
|
||||
/// Schedule for execution as soon as possible (if not already scheduled).
|
||||
/// If the task was already scheduled with delay, the delay will be ignored.
|
||||
bool schedule();
|
||||
|
||||
/// Schedule for execution after specified delay.
|
||||
bool scheduleAfter(size_t ms);
|
||||
|
||||
/// Further attempts to schedule become no-op.
|
||||
void deactivate();
|
||||
void activate();
|
||||
|
||||
/// get zkutil::WatchCallback needed for zookeeper callbacks.
|
||||
|
||||
zkutil::WatchCallback getWatchCallback();
|
||||
|
||||
private:
|
||||
friend class TaskNotification;
|
||||
friend class BackgroundSchedulePool;
|
||||
|
||||
void execute();
|
||||
|
||||
std::mutex schedule_mutex;
|
||||
std::mutex exec_mutex;
|
||||
|
||||
std::string name;
|
||||
bool deactivated = false;
|
||||
bool scheduled = false;
|
||||
bool delayed = false;
|
||||
bool executing = false;
|
||||
BackgroundSchedulePool & pool;
|
||||
Task function;
|
||||
|
||||
/// If the task is scheduled with delay, points to element of delayed_tasks.
|
||||
Tasks::iterator iterator;
|
||||
};
|
||||
|
||||
BackgroundSchedulePool(size_t size);
|
||||
~BackgroundSchedulePool();
|
||||
|
||||
TaskHandle addTask(const std::string & name, const Task & task);
|
||||
void removeTask(const TaskHandle & task);
|
||||
size_t getNumberOfThreads() const { return size; }
|
||||
|
||||
private:
|
||||
using Threads = std::vector<std::thread>;
|
||||
|
||||
void threadFunction();
|
||||
void delayExecutionThreadFunction();
|
||||
|
||||
/// Schedule task for execution after specified delay from now.
|
||||
void scheduleDelayedTask(const TaskHandle & task, size_t ms, std::lock_guard<std::mutex> &);
|
||||
|
||||
/// Remove task, that was scheduled with delay, from schedule.
|
||||
void cancelDelayedTask(const TaskHandle & task, std::lock_guard<std::mutex> &);
|
||||
|
||||
/// Number for worker threads.
|
||||
const size_t size;
|
||||
std::atomic<bool> shutdown {false};
|
||||
Threads threads;
|
||||
Poco::NotificationQueue queue;
|
||||
|
||||
/// Delayed notifications.
|
||||
|
||||
std::condition_variable wakeup_cond;
|
||||
std::mutex delayed_tasks_lock;
|
||||
/// Thread waiting for next delayed task.
|
||||
std::thread delayed_thread;
|
||||
/// Tasks ordered by scheduled time.
|
||||
Tasks delayed_tasks;
|
||||
};
|
||||
|
||||
using BackgroundSchedulePoolPtr = std::shared_ptr<BackgroundSchedulePool>;
|
||||
|
||||
}
|
@ -10,6 +10,7 @@
|
||||
M(ReplicatedSend) \
|
||||
M(ReplicatedChecks) \
|
||||
M(BackgroundPoolTask) \
|
||||
M(BackgroundSchedulePoolTask) \
|
||||
M(DiskSpaceReservedForMerge) \
|
||||
M(DistributedSend) \
|
||||
M(QueryPreempted) \
|
||||
@ -26,6 +27,7 @@
|
||||
M(LeaderReplica) \
|
||||
M(MemoryTracking) \
|
||||
M(MemoryTrackingInBackgroundProcessingPool) \
|
||||
M(MemoryTrackingInBackgroundSchedulePool) \
|
||||
M(MemoryTrackingForMerges) \
|
||||
M(LeaderElection) \
|
||||
M(EphemeralNode) \
|
||||
|
@ -6,6 +6,7 @@
|
||||
#include <memory>
|
||||
#include <common/logger_useful.h>
|
||||
#include <Common/CurrentMetrics.h>
|
||||
#include <Common/BackgroundSchedulePool.h>
|
||||
|
||||
|
||||
namespace ProfileEvents
|
||||
@ -36,9 +37,10 @@ public:
|
||||
* It means that different participants of leader election have different identifiers
|
||||
* and existence of more than one ephemeral node with same identifier indicates an error.
|
||||
*/
|
||||
LeaderElection(const std::string & path_, ZooKeeper & zookeeper_, LeadershipHandler handler_, const std::string & identifier_ = "")
|
||||
: path(path_), zookeeper(zookeeper_), handler(handler_), identifier(identifier_)
|
||||
LeaderElection(DB::BackgroundSchedulePool & pool_, const std::string & path_, ZooKeeper & zookeeper_, LeadershipHandler handler_, const std::string & identifier_ = "")
|
||||
: pool(pool_), path(path_), zookeeper(zookeeper_), handler(handler_), identifier(identifier_)
|
||||
{
|
||||
task_handle = pool.addTask("LeaderElection", [this] { threadFunction(); });
|
||||
createNode();
|
||||
}
|
||||
|
||||
@ -48,17 +50,18 @@ public:
|
||||
return;
|
||||
|
||||
shutdown_called = true;
|
||||
event->set();
|
||||
if (thread.joinable())
|
||||
thread.join();
|
||||
task_handle->deactivate();
|
||||
}
|
||||
|
||||
~LeaderElection()
|
||||
{
|
||||
releaseNode();
|
||||
pool.removeTask(task_handle);
|
||||
}
|
||||
|
||||
private:
|
||||
DB::BackgroundSchedulePool & pool;
|
||||
DB::BackgroundSchedulePool::TaskHandle task_handle;
|
||||
std::string path;
|
||||
ZooKeeper & zookeeper;
|
||||
LeadershipHandler handler;
|
||||
@ -67,9 +70,7 @@ private:
|
||||
EphemeralNodeHolderPtr node;
|
||||
std::string node_name;
|
||||
|
||||
std::thread thread;
|
||||
std::atomic<bool> shutdown_called {false};
|
||||
EventPtr event = std::make_shared<Poco::Event>();
|
||||
|
||||
CurrentMetrics::Increment metric_increment{CurrentMetrics::LeaderElection};
|
||||
|
||||
@ -81,7 +82,8 @@ private:
|
||||
std::string node_path = node->getPath();
|
||||
node_name = node_path.substr(node_path.find_last_of('/') + 1);
|
||||
|
||||
thread = std::thread(&LeaderElection::threadFunction, this);
|
||||
task_handle->activate();
|
||||
task_handle->schedule();
|
||||
}
|
||||
|
||||
void releaseNode()
|
||||
@ -92,45 +94,42 @@ private:
|
||||
|
||||
void threadFunction()
|
||||
{
|
||||
while (!shutdown_called)
|
||||
bool success = false;
|
||||
|
||||
try
|
||||
{
|
||||
bool success = false;
|
||||
Strings children = zookeeper.getChildren(path);
|
||||
std::sort(children.begin(), children.end());
|
||||
auto it = std::lower_bound(children.begin(), children.end(), node_name);
|
||||
if (it == children.end() || *it != node_name)
|
||||
throw Poco::Exception("Assertion failed in LeaderElection");
|
||||
|
||||
try
|
||||
if (it == children.begin())
|
||||
{
|
||||
Strings children = zookeeper.getChildren(path);
|
||||
std::sort(children.begin(), children.end());
|
||||
auto it = std::lower_bound(children.begin(), children.end(), node_name);
|
||||
if (it == children.end() || *it != node_name)
|
||||
throw Poco::Exception("Assertion failed in LeaderElection");
|
||||
|
||||
if (it == children.begin())
|
||||
{
|
||||
ProfileEvents::increment(ProfileEvents::LeaderElectionAcquiredLeadership);
|
||||
handler();
|
||||
return;
|
||||
}
|
||||
|
||||
if (zookeeper.exists(path + "/" + *(it - 1), nullptr, event))
|
||||
event->wait();
|
||||
|
||||
success = true;
|
||||
}
|
||||
catch (const KeeperException & e)
|
||||
{
|
||||
DB::tryLogCurrentException("LeaderElection");
|
||||
|
||||
if (e.code == ZooKeeperImpl::ZooKeeper::ZSESSIONEXPIRED)
|
||||
break;
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
DB::tryLogCurrentException("LeaderElection");
|
||||
ProfileEvents::increment(ProfileEvents::LeaderElectionAcquiredLeadership);
|
||||
handler();
|
||||
return;
|
||||
}
|
||||
|
||||
if (!success)
|
||||
event->tryWait(10 * 1000);
|
||||
if (!zookeeper.existsWatch(path + "/" + *(it - 1), nullptr, task_handle->getWatchCallback()))
|
||||
task_handle->schedule();
|
||||
|
||||
success = true;
|
||||
}
|
||||
catch (const KeeperException & e)
|
||||
{
|
||||
DB::tryLogCurrentException("LeaderElection");
|
||||
|
||||
if (e.code == ZooKeeperImpl::ZooKeeper::ZSESSIONEXPIRED)
|
||||
return;
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
DB::tryLogCurrentException("LeaderElection");
|
||||
}
|
||||
|
||||
if (!success)
|
||||
task_handle->scheduleAfter(10 * 1000);
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -188,6 +188,14 @@ Strings ZooKeeper::getChildren(
|
||||
return res;
|
||||
}
|
||||
|
||||
Strings ZooKeeper::getChildrenWatch(
|
||||
const std::string & path, Stat * stat, WatchCallback watch_callback)
|
||||
{
|
||||
Strings res;
|
||||
check(tryGetChildrenWatch(path, res, stat, watch_callback), path);
|
||||
return res;
|
||||
}
|
||||
|
||||
int32_t ZooKeeper::tryGetChildren(const std::string & path, Strings & res,
|
||||
Stat * stat, const EventPtr & watch)
|
||||
{
|
||||
@ -199,6 +207,17 @@ int32_t ZooKeeper::tryGetChildren(const std::string & path, Strings & res,
|
||||
return code;
|
||||
}
|
||||
|
||||
int32_t ZooKeeper::tryGetChildrenWatch(const std::string & path, Strings & res,
|
||||
Stat * stat, WatchCallback watch_callback)
|
||||
{
|
||||
int32_t code = getChildrenImpl(path, res, stat, watch_callback);
|
||||
|
||||
if (!(code == ZooKeeperImpl::ZooKeeper::ZOK || code == ZooKeeperImpl::ZooKeeper::ZNONODE))
|
||||
throw KeeperException(code, path);
|
||||
|
||||
return code;
|
||||
}
|
||||
|
||||
int32_t ZooKeeper::createImpl(const std::string & path, const std::string & data, int32_t mode, std::string & path_created)
|
||||
{
|
||||
int32_t code = 0;
|
||||
@ -367,6 +386,16 @@ std::string ZooKeeper::get(const std::string & path, Stat * stat, const EventPtr
|
||||
throw KeeperException("Can't get data for node " + path + ": node doesn't exist", code);
|
||||
}
|
||||
|
||||
std::string ZooKeeper::getWatch(const std::string & path, Stat * stat, WatchCallback watch_callback)
|
||||
{
|
||||
int32_t code = 0;
|
||||
std::string res;
|
||||
if (tryGetWatch(path, res, stat, watch_callback, &code))
|
||||
return res;
|
||||
else
|
||||
throw KeeperException("Can't get data for node " + path + ": node doesn't exist", code);
|
||||
}
|
||||
|
||||
bool ZooKeeper::tryGet(const std::string & path, std::string & res, Stat * stat, const EventPtr & watch, int * return_code)
|
||||
{
|
||||
return tryGetWatch(path, res, stat, callbackForEvent(watch), return_code);
|
||||
|
@ -113,6 +113,7 @@ public:
|
||||
bool existsWatch(const std::string & path, Stat * stat, WatchCallback watch_callback);
|
||||
|
||||
std::string get(const std::string & path, Stat * stat = nullptr, const EventPtr & watch = nullptr);
|
||||
std::string getWatch(const std::string & path, Stat * stat, WatchCallback watch_callback);
|
||||
|
||||
/// Doesn't not throw in the following cases:
|
||||
/// * The node doesn't exist. Returns false in this case.
|
||||
@ -136,12 +137,20 @@ public:
|
||||
Stat * stat = nullptr,
|
||||
const EventPtr & watch = nullptr);
|
||||
|
||||
Strings getChildrenWatch(const std::string & path,
|
||||
Stat * stat,
|
||||
WatchCallback watch_callback);
|
||||
|
||||
/// Doesn't not throw in the following cases:
|
||||
/// * The node doesn't exist.
|
||||
int32_t tryGetChildren(const std::string & path, Strings & res,
|
||||
Stat * stat = nullptr,
|
||||
const EventPtr & watch = nullptr);
|
||||
|
||||
int32_t tryGetChildrenWatch(const std::string & path, Strings & res,
|
||||
Stat * stat,
|
||||
WatchCallback watch_callback);
|
||||
|
||||
/// Performs several operations in a transaction.
|
||||
/// Throws on every error.
|
||||
Responses multi(const Requests & requests);
|
||||
|
@ -39,6 +39,52 @@ inline void writeHexByteLowercase(UInt8 byte, void * out)
|
||||
}
|
||||
|
||||
|
||||
/// Produces hex representation of an unsigned int with leading zeros (for checksums)
|
||||
template <typename TUInt>
|
||||
inline void writeHexUIntImpl(TUInt uint, char * out, const char * const table)
|
||||
{
|
||||
union
|
||||
{
|
||||
TUInt value;
|
||||
UInt8 uint8[sizeof(TUInt)];
|
||||
};
|
||||
|
||||
value = uint;
|
||||
|
||||
/// Use little endian
|
||||
for (size_t i = 0; i < sizeof(TUInt); ++i)
|
||||
memcpy(out + i * 2, &table[static_cast<size_t>(uint8[sizeof(TUInt) - 1 - i]) * 2], 2);
|
||||
}
|
||||
|
||||
template <typename TUInt>
|
||||
inline void writeHexUIntUppercase(TUInt uint, char * out)
|
||||
{
|
||||
writeHexUIntImpl(uint, out, hex_byte_to_char_uppercase_table);
|
||||
}
|
||||
|
||||
template <typename TUInt>
|
||||
inline void writeHexUIntLowercase(TUInt uint, char * out)
|
||||
{
|
||||
writeHexUIntImpl(uint, out, hex_byte_to_char_lowercase_table);
|
||||
}
|
||||
|
||||
template <typename TUInt>
|
||||
std::string getHexUIntUppercase(TUInt uint)
|
||||
{
|
||||
std::string res(sizeof(TUInt) * 2, '\0');
|
||||
writeHexUIntUppercase(uint, res.data());
|
||||
return res;
|
||||
}
|
||||
|
||||
template <typename TUInt>
|
||||
std::string getHexUIntLowercase(TUInt uint)
|
||||
{
|
||||
std::string res(sizeof(TUInt) * 2, '\0');
|
||||
writeHexUIntLowercase(uint, res.data());
|
||||
return res;
|
||||
}
|
||||
|
||||
|
||||
/// Maps 0..9, A..F, a..f to 0..15. Other chars are mapped to implementation specific value.
|
||||
|
||||
extern const char * const hex_char_to_digit_table;
|
||||
|
@ -1,3 +1,4 @@
|
||||
#include "localBackup.h"
|
||||
#include <sys/stat.h>
|
||||
#include <string>
|
||||
#include <iostream>
|
||||
@ -18,8 +19,12 @@ namespace ErrorCodes
|
||||
}
|
||||
|
||||
|
||||
static void localBackupImpl(const Poco::Path & source_path, const Poco::Path & destination_path, size_t level)
|
||||
static void localBackupImpl(const Poco::Path & source_path, const Poco::Path & destination_path, size_t level,
|
||||
std::optional<size_t> max_level)
|
||||
{
|
||||
if (max_level && level > max_level.value())
|
||||
return;
|
||||
|
||||
if (level >= 1000)
|
||||
throw DB::Exception("Too deep recursion", DB::ErrorCodes::TOO_DEEP_RECURSION);
|
||||
|
||||
@ -66,12 +71,12 @@ static void localBackupImpl(const Poco::Path & source_path, const Poco::Path & d
|
||||
}
|
||||
else
|
||||
{
|
||||
localBackupImpl(source, destination, level + 1);
|
||||
localBackupImpl(source, destination, level + 1, max_level);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
void localBackup(const Poco::Path & source_path, const Poco::Path & destination_path)
|
||||
void localBackup(const Poco::Path & source_path, const Poco::Path & destination_path, std::optional<size_t> max_level)
|
||||
{
|
||||
if (Poco::File(destination_path).exists()
|
||||
&& Poco::DirectoryIterator(destination_path) != Poco::DirectoryIterator())
|
||||
@ -90,7 +95,7 @@ void localBackup(const Poco::Path & source_path, const Poco::Path & destination_
|
||||
{
|
||||
try
|
||||
{
|
||||
localBackupImpl(source_path, destination_path, 0);
|
||||
localBackupImpl(source_path, destination_path, 0, max_level);
|
||||
}
|
||||
catch (const DB::ErrnoException & e)
|
||||
{
|
||||
|
@ -1,6 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <Poco/Path.h>
|
||||
#include <optional>
|
||||
|
||||
|
||||
/** Creates a local (at the same mount point) backup (snapshot) directory.
|
||||
@ -12,5 +13,9 @@
|
||||
* This protects data from accidental deletion or modification,
|
||||
* and is intended to be used as a simple means of protection against a human or program error,
|
||||
* but not from a hardware failure.
|
||||
*
|
||||
* If max_level is specified, than only files which depth relative source_path less or equal max_level will be copied.
|
||||
* So, if max_level=0 than only direct file child are copied.
|
||||
*/
|
||||
void localBackup(const Poco::Path & source_path, const Poco::Path & destination_path);
|
||||
void localBackup(const Poco::Path & source_path, const Poco::Path & destination_path, std::optional<size_t> max_level = {});
|
||||
|
||||
|
@ -18,6 +18,10 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream(
|
||||
*/
|
||||
addTableLock(storage->lockStructure(true, __PRETTY_FUNCTION__));
|
||||
|
||||
/// If the "root" table deduplactes blocks, there are no need to make deduplication for children
|
||||
/// Moreover, deduplication for AggregatingMergeTree children could produce false positives due to low size of inserting blocks
|
||||
bool disable_deduplication_for_children = !no_destination && storage->supportsDeduplication();
|
||||
|
||||
if (!table.empty())
|
||||
{
|
||||
Dependencies dependencies = context.getDependencies(database, table);
|
||||
@ -27,7 +31,8 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream(
|
||||
{
|
||||
views_context = std::make_unique<Context>(context);
|
||||
// Do not deduplicate insertions into MV if the main insertion is Ok
|
||||
views_context->getSettingsRef().insert_deduplicate = false;
|
||||
if (disable_deduplication_for_children)
|
||||
views_context->getSettingsRef().insert_deduplicate = false;
|
||||
}
|
||||
|
||||
for (const auto & database_table : dependencies)
|
||||
@ -89,4 +94,51 @@ void PushingToViewsBlockOutputStream::write(const Block & block)
|
||||
}
|
||||
}
|
||||
|
||||
void PushingToViewsBlockOutputStream::writePrefix()
|
||||
{
|
||||
if (output)
|
||||
output->writePrefix();
|
||||
|
||||
for (auto & view : views)
|
||||
{
|
||||
try
|
||||
{
|
||||
view.out->writePrefix();
|
||||
}
|
||||
catch (Exception & ex)
|
||||
{
|
||||
ex.addMessage("while write prefix to view " + view.database + "." + view.table);
|
||||
throw;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
void PushingToViewsBlockOutputStream::writeSuffix()
|
||||
{
|
||||
if (output)
|
||||
output->writeSuffix();
|
||||
|
||||
for (auto & view : views)
|
||||
{
|
||||
try
|
||||
{
|
||||
view.out->writeSuffix();
|
||||
}
|
||||
catch (Exception & ex)
|
||||
{
|
||||
ex.addMessage("while write prefix to view " + view.database + "." + view.table);
|
||||
throw;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
void PushingToViewsBlockOutputStream::flush()
|
||||
{
|
||||
if (output)
|
||||
output->flush();
|
||||
|
||||
for (auto & view : views)
|
||||
view.out->flush();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -25,32 +25,9 @@ public:
|
||||
Block getHeader() const override { return storage->getSampleBlock(); }
|
||||
void write(const Block & block) override;
|
||||
|
||||
void flush() override
|
||||
{
|
||||
if (output)
|
||||
output->flush();
|
||||
|
||||
for (auto & view : views)
|
||||
view.out->flush();
|
||||
}
|
||||
|
||||
void writePrefix() override
|
||||
{
|
||||
if (output)
|
||||
output->writePrefix();
|
||||
|
||||
for (auto & view : views)
|
||||
view.out->writePrefix();
|
||||
}
|
||||
|
||||
void writeSuffix() override
|
||||
{
|
||||
if (output)
|
||||
output->writeSuffix();
|
||||
|
||||
for (auto & view : views)
|
||||
view.out->writeSuffix();
|
||||
}
|
||||
void flush() override;
|
||||
void writePrefix() override;
|
||||
void writeSuffix() override;
|
||||
|
||||
private:
|
||||
StoragePtr storage;
|
||||
|
@ -11,12 +11,12 @@ namespace DB
|
||||
|
||||
void DataTypeDate::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr) const
|
||||
{
|
||||
writeDateText(DayNum_t(static_cast<const ColumnUInt16 &>(column).getData()[row_num]), ostr);
|
||||
writeDateText(DayNum(static_cast<const ColumnUInt16 &>(column).getData()[row_num]), ostr);
|
||||
}
|
||||
|
||||
static void deserializeText(IColumn & column, ReadBuffer & istr)
|
||||
{
|
||||
DayNum_t x;
|
||||
DayNum x;
|
||||
readDateText(x, istr);
|
||||
static_cast<ColumnUInt16 &>(column).getData().push_back(x);
|
||||
}
|
||||
@ -40,7 +40,7 @@ void DataTypeDate::serializeTextQuoted(const IColumn & column, size_t row_num, W
|
||||
|
||||
void DataTypeDate::deserializeTextQuoted(IColumn & column, ReadBuffer & istr) const
|
||||
{
|
||||
DayNum_t x;
|
||||
DayNum x;
|
||||
assertChar('\'', istr);
|
||||
readDateText(x, istr);
|
||||
assertChar('\'', istr);
|
||||
@ -56,7 +56,7 @@ void DataTypeDate::serializeTextJSON(const IColumn & column, size_t row_num, Wri
|
||||
|
||||
void DataTypeDate::deserializeTextJSON(IColumn & column, ReadBuffer & istr) const
|
||||
{
|
||||
DayNum_t x;
|
||||
DayNum x;
|
||||
assertChar('"', istr);
|
||||
readDateText(x, istr);
|
||||
assertChar('"', istr);
|
||||
|
@ -828,7 +828,7 @@ private:
|
||||
|
||||
if (is_date)
|
||||
{
|
||||
DayNum_t date;
|
||||
DayNum date;
|
||||
ReadBufferFromMemory in(string_value.data, string_value.size);
|
||||
readDateText(date, in);
|
||||
if (!in.eof())
|
||||
|
@ -112,7 +112,7 @@ struct ToDateTimeImpl
|
||||
|
||||
static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.fromDayNum(DayNum_t(d));
|
||||
return time_zone.fromDayNum(DayNum(d));
|
||||
}
|
||||
};
|
||||
|
||||
@ -175,7 +175,7 @@ struct FormatImpl<DataTypeDate>
|
||||
{
|
||||
static void execute(const DataTypeDate::FieldType x, WriteBuffer & wb, const DataTypeDate *, const DateLUTImpl *)
|
||||
{
|
||||
writeDateText(DayNum_t(x), wb);
|
||||
writeDateText(DayNum(x), wb);
|
||||
}
|
||||
};
|
||||
|
||||
@ -307,7 +307,7 @@ template <typename DataType> void parseImpl(typename DataType::FieldType & x, Re
|
||||
|
||||
template <> inline void parseImpl<DataTypeDate>(DataTypeDate::FieldType & x, ReadBuffer & rb, const DateLUTImpl *)
|
||||
{
|
||||
DayNum_t tmp(0);
|
||||
DayNum tmp(0);
|
||||
readDateText(tmp, rb);
|
||||
x = tmp;
|
||||
}
|
||||
|
@ -133,7 +133,7 @@ struct ToMondayImpl
|
||||
}
|
||||
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toFirstDayNumOfWeek(DayNum_t(d));
|
||||
return time_zone.toFirstDayNumOfWeek(DayNum(d));
|
||||
}
|
||||
|
||||
using FactorTransform = ZeroTransform;
|
||||
@ -149,7 +149,7 @@ struct ToStartOfMonthImpl
|
||||
}
|
||||
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toFirstDayNumOfMonth(DayNum_t(d));
|
||||
return time_zone.toFirstDayNumOfMonth(DayNum(d));
|
||||
}
|
||||
|
||||
using FactorTransform = ZeroTransform;
|
||||
@ -165,7 +165,7 @@ struct ToStartOfQuarterImpl
|
||||
}
|
||||
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toFirstDayNumOfQuarter(DayNum_t(d));
|
||||
return time_zone.toFirstDayNumOfQuarter(DayNum(d));
|
||||
}
|
||||
|
||||
using FactorTransform = ZeroTransform;
|
||||
@ -181,7 +181,7 @@ struct ToStartOfYearImpl
|
||||
}
|
||||
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toFirstDayNumOfYear(DayNum_t(d));
|
||||
return time_zone.toFirstDayNumOfYear(DayNum(d));
|
||||
}
|
||||
|
||||
using FactorTransform = ZeroTransform;
|
||||
@ -281,7 +281,7 @@ struct ToYearImpl
|
||||
}
|
||||
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toYear(DayNum_t(d));
|
||||
return time_zone.toYear(DayNum(d));
|
||||
}
|
||||
|
||||
using FactorTransform = ZeroTransform;
|
||||
@ -297,7 +297,7 @@ struct ToQuarterImpl
|
||||
}
|
||||
static inline UInt8 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toQuarter(DayNum_t(d));
|
||||
return time_zone.toQuarter(DayNum(d));
|
||||
}
|
||||
|
||||
using FactorTransform = ToStartOfYearImpl;
|
||||
@ -313,7 +313,7 @@ struct ToMonthImpl
|
||||
}
|
||||
static inline UInt8 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toMonth(DayNum_t(d));
|
||||
return time_zone.toMonth(DayNum(d));
|
||||
}
|
||||
|
||||
using FactorTransform = ToStartOfYearImpl;
|
||||
@ -329,7 +329,7 @@ struct ToDayOfMonthImpl
|
||||
}
|
||||
static inline UInt8 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toDayOfMonth(DayNum_t(d));
|
||||
return time_zone.toDayOfMonth(DayNum(d));
|
||||
}
|
||||
|
||||
using FactorTransform = ToStartOfMonthImpl;
|
||||
@ -345,7 +345,7 @@ struct ToDayOfWeekImpl
|
||||
}
|
||||
static inline UInt8 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toDayOfWeek(DayNum_t(d));
|
||||
return time_zone.toDayOfWeek(DayNum(d));
|
||||
}
|
||||
|
||||
using FactorTransform = ToMondayImpl;
|
||||
@ -410,7 +410,7 @@ struct ToRelativeYearNumImpl
|
||||
}
|
||||
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toYear(DayNum_t(d));
|
||||
return time_zone.toYear(DayNum(d));
|
||||
}
|
||||
|
||||
using FactorTransform = ZeroTransform;
|
||||
@ -426,7 +426,7 @@ struct ToRelativeQuarterNumImpl
|
||||
}
|
||||
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toRelativeQuarterNum(DayNum_t(d));
|
||||
return time_zone.toRelativeQuarterNum(DayNum(d));
|
||||
}
|
||||
|
||||
using FactorTransform = ZeroTransform;
|
||||
@ -442,7 +442,7 @@ struct ToRelativeMonthNumImpl
|
||||
}
|
||||
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toRelativeMonthNum(DayNum_t(d));
|
||||
return time_zone.toRelativeMonthNum(DayNum(d));
|
||||
}
|
||||
|
||||
using FactorTransform = ZeroTransform;
|
||||
@ -458,7 +458,7 @@ struct ToRelativeWeekNumImpl
|
||||
}
|
||||
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toRelativeWeekNum(DayNum_t(d));
|
||||
return time_zone.toRelativeWeekNum(DayNum(d));
|
||||
}
|
||||
|
||||
using FactorTransform = ZeroTransform;
|
||||
@ -474,7 +474,7 @@ struct ToRelativeDayNumImpl
|
||||
}
|
||||
static inline UInt16 execute(UInt16 d, const DateLUTImpl &)
|
||||
{
|
||||
return static_cast<DayNum_t>(d);
|
||||
return static_cast<DayNum>(d);
|
||||
}
|
||||
|
||||
using FactorTransform = ZeroTransform;
|
||||
@ -491,7 +491,7 @@ struct ToRelativeHourNumImpl
|
||||
}
|
||||
static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toRelativeHourNum(DayNum_t(d));
|
||||
return time_zone.toRelativeHourNum(DayNum(d));
|
||||
}
|
||||
|
||||
using FactorTransform = ZeroTransform;
|
||||
@ -507,7 +507,7 @@ struct ToRelativeMinuteNumImpl
|
||||
}
|
||||
static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toRelativeMinuteNum(DayNum_t(d));
|
||||
return time_zone.toRelativeMinuteNum(DayNum(d));
|
||||
}
|
||||
|
||||
using FactorTransform = ZeroTransform;
|
||||
@ -523,7 +523,7 @@ struct ToRelativeSecondNumImpl
|
||||
}
|
||||
static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.fromDayNum(DayNum_t(d));
|
||||
return time_zone.fromDayNum(DayNum(d));
|
||||
}
|
||||
|
||||
using FactorTransform = ZeroTransform;
|
||||
@ -539,7 +539,7 @@ struct ToYYYYMMImpl
|
||||
}
|
||||
static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toNumYYYYMM(static_cast<DayNum_t>(d));
|
||||
return time_zone.toNumYYYYMM(static_cast<DayNum>(d));
|
||||
}
|
||||
|
||||
using FactorTransform = ZeroTransform;
|
||||
@ -555,7 +555,7 @@ struct ToYYYYMMDDImpl
|
||||
}
|
||||
static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toNumYYYYMMDD(static_cast<DayNum_t>(d));
|
||||
return time_zone.toNumYYYYMMDD(static_cast<DayNum>(d));
|
||||
}
|
||||
|
||||
using FactorTransform = ZeroTransform;
|
||||
@ -571,7 +571,7 @@ struct ToYYYYMMDDhhmmssImpl
|
||||
}
|
||||
static inline UInt64 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toNumYYYYMMDDhhmmss(time_zone.toDate(static_cast<DayNum_t>(d)));
|
||||
return time_zone.toNumYYYYMMDDhhmmss(time_zone.toDate(static_cast<DayNum>(d)));
|
||||
}
|
||||
|
||||
using FactorTransform = ZeroTransform;
|
||||
@ -732,7 +732,7 @@ struct AddSecondsImpl
|
||||
|
||||
static inline UInt32 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.fromDayNum(DayNum_t(d)) + delta;
|
||||
return time_zone.fromDayNum(DayNum(d)) + delta;
|
||||
}
|
||||
};
|
||||
|
||||
@ -747,7 +747,7 @@ struct AddMinutesImpl
|
||||
|
||||
static inline UInt32 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.fromDayNum(DayNum_t(d)) + delta * 60;
|
||||
return time_zone.fromDayNum(DayNum(d)) + delta * 60;
|
||||
}
|
||||
};
|
||||
|
||||
@ -762,7 +762,7 @@ struct AddHoursImpl
|
||||
|
||||
static inline UInt32 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.fromDayNum(DayNum_t(d)) + delta * 3600;
|
||||
return time_zone.fromDayNum(DayNum(d)) + delta * 3600;
|
||||
}
|
||||
};
|
||||
|
||||
@ -807,7 +807,7 @@ struct AddMonthsImpl
|
||||
|
||||
static inline UInt16 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.addMonths(DayNum_t(d), delta);
|
||||
return time_zone.addMonths(DayNum(d), delta);
|
||||
}
|
||||
};
|
||||
|
||||
@ -822,7 +822,7 @@ struct AddYearsImpl
|
||||
|
||||
static inline UInt16 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.addYears(DayNum_t(d), delta);
|
||||
return time_zone.addYears(DayNum(d), delta);
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -483,7 +483,7 @@ inline void readDateText(LocalDate & date, ReadBuffer & buf)
|
||||
readDateTextFallback(date, buf);
|
||||
}
|
||||
|
||||
inline void readDateText(DayNum_t & date, ReadBuffer & buf)
|
||||
inline void readDateText(DayNum & date, ReadBuffer & buf)
|
||||
{
|
||||
LocalDate local_date;
|
||||
readDateText(local_date, buf);
|
||||
|
@ -537,7 +537,7 @@ inline void writeDateText(const LocalDate & date, WriteBuffer & buf)
|
||||
}
|
||||
|
||||
template <char delimiter = '-'>
|
||||
inline void writeDateText(DayNum_t date, WriteBuffer & buf)
|
||||
inline void writeDateText(DayNum date, WriteBuffer & buf)
|
||||
{
|
||||
if (unlikely(!date))
|
||||
{
|
||||
|
97
dbms/src/Interpreters/ActionLocksManager.cpp
Normal file
97
dbms/src/Interpreters/ActionLocksManager.cpp
Normal file
@ -0,0 +1,97 @@
|
||||
#include "ActionLocksManager.h"
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Databases/IDatabase.h>
|
||||
#include <Storages/IStorage.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ActionLocks
|
||||
{
|
||||
extern const StorageActionBlockType PartsMerge = 1;
|
||||
extern const StorageActionBlockType PartsFetch = 2;
|
||||
extern const StorageActionBlockType PartsSend = 3;
|
||||
extern const StorageActionBlockType ReplicationQueue = 4;
|
||||
}
|
||||
|
||||
|
||||
template <typename F>
|
||||
inline void forEachTable(Context & context, F && f)
|
||||
{
|
||||
for (auto & elem : context.getDatabases())
|
||||
for (auto iterator = elem.second->getIterator(context); iterator->isValid(); iterator->next())
|
||||
f(iterator->table());
|
||||
|
||||
}
|
||||
|
||||
void ActionLocksManager::add(StorageActionBlockType action_type)
|
||||
{
|
||||
forEachTable(global_context, [&] (const StoragePtr & table)
|
||||
{
|
||||
ActionLock action_lock = table->getActionLock(action_type);
|
||||
|
||||
if (!action_lock.expired())
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
storage_locks[table.get()][action_type] = std::move(action_lock);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
void ActionLocksManager::add(const String & database_name, const String & table_name, StorageActionBlockType action_type)
|
||||
{
|
||||
if (auto table = global_context.tryGetTable(database_name, table_name))
|
||||
{
|
||||
ActionLock action_lock = table->getActionLock(action_type);
|
||||
|
||||
if (!action_lock.expired())
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
storage_locks[table.get()][action_type] = std::move(action_lock);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
void ActionLocksManager::remove(StorageActionBlockType action_type)
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
|
||||
for (auto & storage_elem : storage_locks)
|
||||
storage_elem.second.erase(action_type);
|
||||
}
|
||||
|
||||
void ActionLocksManager::remove(const String & database_name, const String & table_name, StorageActionBlockType action_type)
|
||||
{
|
||||
if (auto table = global_context.tryGetTable(database_name, table_name))
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
|
||||
if (storage_locks.count(table.get()))
|
||||
storage_locks[table.get()].erase(action_type);
|
||||
}
|
||||
}
|
||||
|
||||
void ActionLocksManager::cleanExpired()
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
|
||||
for (auto it_storage = storage_locks.begin(); it_storage != storage_locks.end(); )
|
||||
{
|
||||
auto & locks = it_storage->second;
|
||||
for (auto it_lock = locks.begin(); it_lock != locks.end(); )
|
||||
{
|
||||
if (it_lock->second.expired())
|
||||
it_lock = locks.erase(it_lock);
|
||||
else
|
||||
++it_lock;
|
||||
}
|
||||
|
||||
if (locks.empty())
|
||||
it_storage = storage_locks.erase(it_storage);
|
||||
else
|
||||
++it_storage;
|
||||
}
|
||||
}
|
||||
|
||||
}
|
45
dbms/src/Interpreters/ActionLocksManager.h
Normal file
45
dbms/src/Interpreters/ActionLocksManager.h
Normal file
@ -0,0 +1,45 @@
|
||||
#pragma once
|
||||
#include <Core/Types.h>
|
||||
#include <Common/ActionLock.h>
|
||||
#include <unordered_map>
|
||||
#include <mutex>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class IStorage;
|
||||
class Context;
|
||||
|
||||
/// Holds ActionLocks for tables
|
||||
/// Does not store pointers to tables
|
||||
class ActionLocksManager
|
||||
{
|
||||
public:
|
||||
explicit ActionLocksManager(Context & global_context_) : global_context(global_context_) {}
|
||||
|
||||
/// Adds new locks for each table
|
||||
void add(StorageActionBlockType action_type);
|
||||
/// Add new lock for a table if it has not been already added
|
||||
void add(const String & database_name, const String & table_name, StorageActionBlockType action_type);
|
||||
|
||||
/// Remove locks for all tables
|
||||
void remove(StorageActionBlockType action_type);
|
||||
/// Removes a lock for a table if it exists
|
||||
void remove(const String & database_name, const String & table_name, StorageActionBlockType action_type);
|
||||
|
||||
/// Removes all locks of non-existing tables
|
||||
void cleanExpired();
|
||||
|
||||
private:
|
||||
Context & global_context;
|
||||
|
||||
using StorageRawPtr = const IStorage *;
|
||||
using Locks = std::unordered_map<size_t, ActionLock>;
|
||||
using StorageLocks = std::unordered_map<StorageRawPtr, Locks>;
|
||||
|
||||
mutable std::mutex mutex;
|
||||
StorageLocks storage_locks;
|
||||
};
|
||||
|
||||
}
|
@ -239,6 +239,7 @@ void Compiler::compile(
|
||||
" -isystem " INTERNAL_COMPILER_HEADERS_ROOT "/usr/include/c++/*/backward"
|
||||
" -isystem " INTERNAL_COMPILER_HEADERS_ROOT "/usr/include/clang/*/include" /// if compiler is clang (from package)
|
||||
" -isystem " INTERNAL_COMPILER_HEADERS_ROOT "/usr/local/lib/clang/*/include" /// if clang installed manually
|
||||
" -isystem " INTERNAL_COMPILER_HEADERS_ROOT "/usr/lib/clang/*/include" /// if clang build from submodules
|
||||
" -isystem " INTERNAL_COMPILER_HEADERS_ROOT "/usr/lib/gcc/" CMAKE_LIBRARY_ARCHITECTURE "/*/include-fixed"
|
||||
" -isystem " INTERNAL_COMPILER_HEADERS_ROOT "/usr/lib/gcc/" CMAKE_LIBRARY_ARCHITECTURE "/*/include"
|
||||
" -isystem " INTERNAL_COMPILER_HEADERS_ROOT "/usr/local/include" /// if something installed manually
|
||||
|
@ -15,6 +15,7 @@
|
||||
#include <Common/setThreadName.h>
|
||||
#include <Common/Stopwatch.h>
|
||||
#include <Common/formatReadable.h>
|
||||
#include <Common/BackgroundSchedulePool.h>
|
||||
#include <DataStreams/FormatFactory.h>
|
||||
#include <Databases/IDatabase.h>
|
||||
#include <Storages/IStorage.h>
|
||||
@ -24,6 +25,7 @@
|
||||
#include <Storages/MergeTree/MergeTreeSettings.h>
|
||||
#include <Storages/CompressionSettingsSelector.h>
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
#include <Interpreters/ActionLocksManager.h>
|
||||
#include <Interpreters/Settings.h>
|
||||
#include <Interpreters/RuntimeComponentsFactory.h>
|
||||
#include <Interpreters/ISecurityManager.h>
|
||||
@ -131,6 +133,7 @@ struct ContextShared
|
||||
ConfigurationPtr users_config; /// Config with the users, profiles and quotas sections.
|
||||
InterserverIOHandler interserver_io_handler; /// Handler for interserver communication.
|
||||
BackgroundProcessingPoolPtr background_pool; /// The thread pool for the background work performed by the tables.
|
||||
BackgroundSchedulePoolPtr schedule_pool; /// A thread pool that can run different jobs in background (used in replicated tables)
|
||||
MultiVersion<Macros> macros; /// Substitutions extracted from config.
|
||||
std::unique_ptr<Compiler> compiler; /// Used for dynamic compilation of queries' parts if it necessary.
|
||||
std::shared_ptr<DDLWorker> ddl_worker; /// Process ddl commands from zk.
|
||||
@ -139,7 +142,7 @@ struct ContextShared
|
||||
std::unique_ptr<MergeTreeSettings> merge_tree_settings; /// Settings of MergeTree* engines.
|
||||
size_t max_table_size_to_drop = 50000000000lu; /// Protects MergeTree tables from accidental DROP (50GB by default)
|
||||
String format_schema_path; /// Path to a directory that contains schema files used by input formats.
|
||||
|
||||
ActionLocksManagerPtr action_locks_manager; /// Set of storages' action lockers
|
||||
|
||||
/// Named sessions. The user could specify session identifier to reuse settings and temporary tables in subsequent requests.
|
||||
|
||||
@ -1328,6 +1331,14 @@ BackgroundProcessingPool & Context::getBackgroundPool()
|
||||
return *shared->background_pool;
|
||||
}
|
||||
|
||||
BackgroundSchedulePool & Context::getSchedulePool()
|
||||
{
|
||||
auto lock = getLock();
|
||||
if (!shared->schedule_pool)
|
||||
shared->schedule_pool = std::make_shared<BackgroundSchedulePool>(settings.background_schedule_pool_size);
|
||||
return *shared->schedule_pool;
|
||||
}
|
||||
|
||||
void Context::setDDLWorker(std::shared_ptr<DDLWorker> ddl_worker)
|
||||
{
|
||||
auto lock = getLock();
|
||||
@ -1721,6 +1732,16 @@ void Context::setFormatSchemaPath(const String & path)
|
||||
shared->format_schema_path = path;
|
||||
}
|
||||
|
||||
std::shared_ptr<ActionLocksManager> Context::getActionLocksManager()
|
||||
{
|
||||
auto lock = getLock();
|
||||
|
||||
if (!shared->action_locks_manager)
|
||||
shared->action_locks_manager = std::make_shared<ActionLocksManager>(getGlobalContext());
|
||||
|
||||
return shared->action_locks_manager;
|
||||
}
|
||||
|
||||
|
||||
SessionCleaner::~SessionCleaner()
|
||||
{
|
||||
|
@ -40,6 +40,7 @@ class ExternalDictionaries;
|
||||
class ExternalModels;
|
||||
class InterserverIOHandler;
|
||||
class BackgroundProcessingPool;
|
||||
class BackgroundSchedulePool;
|
||||
class MergeList;
|
||||
class Cluster;
|
||||
class Compiler;
|
||||
@ -69,6 +70,8 @@ using BlockOutputStreamPtr = std::shared_ptr<IBlockOutputStream>;
|
||||
class Block;
|
||||
struct SystemLogs;
|
||||
using SystemLogsPtr = std::shared_ptr<SystemLogs>;
|
||||
class ActionLocksManager;
|
||||
using ActionLocksManagerPtr = std::shared_ptr<ActionLocksManager>;
|
||||
|
||||
|
||||
/// (database name, table name)
|
||||
@ -328,6 +331,7 @@ public:
|
||||
void dropCaches() const;
|
||||
|
||||
BackgroundProcessingPool & getBackgroundPool();
|
||||
BackgroundSchedulePool & getSchedulePool();
|
||||
|
||||
void setDDLWorker(std::shared_ptr<DDLWorker> ddl_worker);
|
||||
DDLWorker & getDDLWorker() const;
|
||||
@ -370,6 +374,8 @@ public:
|
||||
|
||||
void shutdown();
|
||||
|
||||
ActionLocksManagerPtr getActionLocksManager();
|
||||
|
||||
enum class ApplicationType
|
||||
{
|
||||
SERVER, /// The program is run as clickhouse-server daemon (default behavior)
|
||||
|
@ -1486,7 +1486,7 @@ void ExpressionAnalyzer::tryMakeSetFromSubquery(const ASTPtr & subquery_or_table
|
||||
return;
|
||||
}
|
||||
|
||||
prepared_sets[subquery_or_table_name.get()] = std::move(set);
|
||||
prepared_sets[subquery_or_table_name->range] = std::move(set);
|
||||
}
|
||||
|
||||
|
||||
@ -1515,7 +1515,7 @@ void ExpressionAnalyzer::makeSetsForIndexImpl(const ASTPtr & node, const Block &
|
||||
{
|
||||
const ASTPtr & arg = args.children.at(1);
|
||||
|
||||
if (!prepared_sets.count(arg.get())) /// Not already prepared.
|
||||
if (!prepared_sets.count(arg->range)) /// Not already prepared.
|
||||
{
|
||||
if (typeid_cast<ASTSubquery *>(arg.get()) || typeid_cast<ASTIdentifier *>(arg.get()))
|
||||
{
|
||||
@ -1550,7 +1550,7 @@ void ExpressionAnalyzer::makeSet(const ASTFunction * node, const Block & sample_
|
||||
const ASTPtr & arg = args.children.at(1);
|
||||
|
||||
/// Already converted.
|
||||
if (prepared_sets.count(arg.get()))
|
||||
if (prepared_sets.count(arg->range))
|
||||
return;
|
||||
|
||||
/// If the subquery or table name for SELECT.
|
||||
@ -1573,7 +1573,7 @@ void ExpressionAnalyzer::makeSet(const ASTFunction * node, const Block & sample_
|
||||
|
||||
if (storage_set)
|
||||
{
|
||||
prepared_sets[arg.get()] = storage_set->getSet();
|
||||
prepared_sets[arg->range] = storage_set->getSet();
|
||||
return;
|
||||
}
|
||||
}
|
||||
@ -1584,7 +1584,7 @@ void ExpressionAnalyzer::makeSet(const ASTFunction * node, const Block & sample_
|
||||
/// If you already created a Set with the same subquery / table.
|
||||
if (subquery_for_set.set)
|
||||
{
|
||||
prepared_sets[arg.get()] = subquery_for_set.set;
|
||||
prepared_sets[arg->range] = subquery_for_set.set;
|
||||
return;
|
||||
}
|
||||
|
||||
@ -1630,7 +1630,7 @@ void ExpressionAnalyzer::makeSet(const ASTFunction * node, const Block & sample_
|
||||
}
|
||||
|
||||
subquery_for_set.set = set;
|
||||
prepared_sets[arg.get()] = set;
|
||||
prepared_sets[arg->range] = set;
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -1712,7 +1712,7 @@ void ExpressionAnalyzer::makeExplicitSet(const ASTFunction * node, const Block &
|
||||
|
||||
SetPtr set = std::make_shared<Set>(SizeLimits(settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode));
|
||||
set->createFromAST(set_element_types, elements_ast, context, create_ordered_set);
|
||||
prepared_sets[right_arg.get()] = std::move(set);
|
||||
prepared_sets[right_arg->range] = std::move(set);
|
||||
}
|
||||
|
||||
|
||||
@ -2102,12 +2102,12 @@ void ExpressionAnalyzer::getActionsImpl(const ASTPtr & ast, bool no_subqueries,
|
||||
/// Select the name in the next cycle.
|
||||
argument_names.emplace_back();
|
||||
}
|
||||
else if (prepared_sets.count(child.get()) && functionIsInOrGlobalInOperator(node->name) && arg == 1)
|
||||
else if (prepared_sets.count(child->range) && functionIsInOrGlobalInOperator(node->name) && arg == 1)
|
||||
{
|
||||
ColumnWithTypeAndName column;
|
||||
column.type = std::make_shared<DataTypeSet>();
|
||||
|
||||
const SetPtr & set = prepared_sets[child.get()];
|
||||
const SetPtr & set = prepared_sets[child->range];
|
||||
|
||||
/// If the argument is a set given by an enumeration of values (so, the set was already built), give it a unique name,
|
||||
/// so that sets with the same literal representation do not fuse together (they can have different types).
|
||||
|
@ -3,9 +3,9 @@
|
||||
#include <Interpreters/AggregateDescription.h>
|
||||
#include <Interpreters/Settings.h>
|
||||
#include <Core/Block.h>
|
||||
#include "ExpressionActions.h"
|
||||
#include "ProjectionManipulation.h"
|
||||
|
||||
#include <Interpreters/ExpressionActions.h>
|
||||
#include <Interpreters/ProjectionManipulation.h>
|
||||
#include <Parsers/StringRange.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -23,7 +23,9 @@ using ASTPtr = std::shared_ptr<IAST>;
|
||||
|
||||
class Set;
|
||||
using SetPtr = std::shared_ptr<Set>;
|
||||
using PreparedSets = std::unordered_map<IAST*, SetPtr>;
|
||||
/// Will compare sets by their position in query string. It's possible because IAST::clone() doesn't chane IAST::range.
|
||||
/// It should be taken into account when we want to change AST part which contains sets.
|
||||
using PreparedSets = std::unordered_map<StringRange, SetPtr, StringRangePointersHash, StringRangePointersEqualTo>;
|
||||
|
||||
class IBlockInputStream;
|
||||
using BlockInputStreamPtr = std::shared_ptr<IBlockInputStream>;
|
||||
|
@ -72,8 +72,16 @@ BlockIO InterpreterAlterQuery::execute()
|
||||
table->attachPartition(command.partition, command.part, context);
|
||||
break;
|
||||
|
||||
case PartitionCommand::REPLACE_PARTITION:
|
||||
{
|
||||
String from_database = command.from_database.empty() ? context.getCurrentDatabase() : command.from_database;
|
||||
auto from_storage = context.getTable(from_database, command.from_table);
|
||||
table->replacePartitionFrom(from_storage, command.partition, command.replace, context);
|
||||
}
|
||||
break;
|
||||
|
||||
case PartitionCommand::FETCH_PARTITION:
|
||||
table->fetchPartition(command.partition, command.from, context);
|
||||
table->fetchPartition(command.partition, command.from_zookeeper_path, context);
|
||||
break;
|
||||
|
||||
case PartitionCommand::FREEZE_PARTITION:
|
||||
@ -187,6 +195,11 @@ void InterpreterAlterQuery::parseAlter(
|
||||
{
|
||||
out_partition_commands.emplace_back(PartitionCommand::attachPartition(params.partition, params.part));
|
||||
}
|
||||
else if (params.type == ASTAlterQuery::REPLACE_PARTITION)
|
||||
{
|
||||
out_partition_commands.emplace_back(
|
||||
PartitionCommand::replacePartition(params.partition, params.replace, params.from_database, params.from_table));
|
||||
}
|
||||
else if (params.type == ASTAlterQuery::FETCH_PARTITION)
|
||||
{
|
||||
out_partition_commands.emplace_back(PartitionCommand::fetchPartition(params.partition, params.from));
|
||||
|
@ -28,6 +28,7 @@ private:
|
||||
{
|
||||
DROP_PARTITION,
|
||||
ATTACH_PARTITION,
|
||||
REPLACE_PARTITION,
|
||||
FETCH_PARTITION,
|
||||
FREEZE_PARTITION,
|
||||
CLEAR_COLUMN,
|
||||
@ -37,11 +38,20 @@ private:
|
||||
|
||||
ASTPtr partition;
|
||||
Field column_name;
|
||||
bool detach = false; /// true for DETACH PARTITION.
|
||||
|
||||
/// true for DETACH PARTITION.
|
||||
bool detach = false;
|
||||
|
||||
/// true for ATTACH PART (and false for PARTITION)
|
||||
bool part = false;
|
||||
|
||||
String from; /// For FETCH PARTITION - path in ZK to the shard, from which to download the partition.
|
||||
/// For ATTACH PARTITION partition FROM db.table
|
||||
String from_database;
|
||||
String from_table;
|
||||
bool replace = true;
|
||||
|
||||
/// For FETCH PARTITION - path in ZK to the shard, from which to download the partition.
|
||||
String from_zookeeper_path;
|
||||
|
||||
/// For FREEZE PARTITION
|
||||
String with_name;
|
||||
@ -73,12 +83,23 @@ private:
|
||||
return res;
|
||||
}
|
||||
|
||||
static PartitionCommand replacePartition(const ASTPtr & partition, bool replace, const String & from_database, const String & from_table)
|
||||
{
|
||||
PartitionCommand res;
|
||||
res.type = REPLACE_PARTITION;
|
||||
res.partition = partition;
|
||||
res.replace = replace;
|
||||
res.from_database = from_database;
|
||||
res.from_table = from_table;
|
||||
return res;
|
||||
}
|
||||
|
||||
static PartitionCommand fetchPartition(const ASTPtr & partition, const String & from)
|
||||
{
|
||||
PartitionCommand res;
|
||||
res.type = FETCH_PARTITION;
|
||||
res.partition = partition;
|
||||
res.from = from;
|
||||
res.from_zookeeper_path = from;
|
||||
return res;
|
||||
}
|
||||
|
||||
|
@ -1,10 +1,21 @@
|
||||
#include <Interpreters/InterpreterSystemQuery.h>
|
||||
#include <Common/DNSResolver.h>
|
||||
#include <Common/ActionLock.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/DNSResolver.h>
|
||||
#include <common/ThreadPool.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/ExternalDictionaries.h>
|
||||
#include <Interpreters/EmbeddedDictionaries.h>
|
||||
#include <Interpreters/ActionLocksManager.h>
|
||||
#include <Interpreters/InterpreterDropQuery.h>
|
||||
#include <Interpreters/InterpreterCreateQuery.h>
|
||||
#include <Databases/IDatabase.h>
|
||||
#include <Storages/StorageReplicatedMergeTree.h>
|
||||
#include <Storages/StorageFactory.h>
|
||||
#include <Parsers/ASTSystemQuery.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Parsers/ASTDropQuery.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <csignal>
|
||||
|
||||
|
||||
@ -20,6 +31,15 @@ namespace ErrorCodes
|
||||
}
|
||||
|
||||
|
||||
namespace ActionLocks
|
||||
{
|
||||
extern StorageActionBlockType PartsMerge;
|
||||
extern StorageActionBlockType PartsFetch;
|
||||
extern StorageActionBlockType PartsSend;
|
||||
extern StorageActionBlockType ReplicationQueue;
|
||||
}
|
||||
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
@ -50,11 +70,34 @@ ExecutionStatus getOverallExecutionStatusOfCommands(Callable && command, Callabl
|
||||
return ExecutionStatus(res_status, res_message);
|
||||
}
|
||||
|
||||
/// Implements SYSTEM [START|STOP] <something action from ActionLocks>
|
||||
void startStopAction(Context & context, ASTSystemQuery & query, StorageActionBlockType action_type, bool start)
|
||||
{
|
||||
auto manager = context.getActionLocksManager();
|
||||
manager->cleanExpired();
|
||||
|
||||
if (!query.target_table.empty())
|
||||
{
|
||||
String database = !query.target_database.empty() ? query.target_database : context.getCurrentDatabase();
|
||||
|
||||
if (start)
|
||||
manager->remove(database, query.target_table, action_type);
|
||||
else
|
||||
manager->add(database, query.target_table, action_type);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (start)
|
||||
manager->remove(action_type);
|
||||
else
|
||||
manager->add(action_type);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
InterpreterSystemQuery::InterpreterSystemQuery(const ASTPtr & query_ptr_, Context & context_)
|
||||
: query_ptr(query_ptr_), context(context_) {}
|
||||
: query_ptr(query_ptr_->clone()), context(context_), log(&Poco::Logger::get("InterpreterSystemQuery")) {}
|
||||
|
||||
|
||||
BlockIO InterpreterSystemQuery::execute()
|
||||
@ -67,6 +110,10 @@ BlockIO InterpreterSystemQuery::execute()
|
||||
Context system_context = context.getGlobalContext();
|
||||
system_context.setSetting("profile", context.getSystemProfileName());
|
||||
|
||||
/// Make canonical query for simpler processing
|
||||
if (!query.target_table.empty() && query.target_database.empty())
|
||||
query.target_database = context.getCurrentDatabase();
|
||||
|
||||
switch (query.type)
|
||||
{
|
||||
case Type::SHUTDOWN:
|
||||
@ -107,14 +154,43 @@ BlockIO InterpreterSystemQuery::execute()
|
||||
case Type::RELOAD_CONFIG:
|
||||
system_context.reloadConfig();
|
||||
break;
|
||||
case Type::STOP_MERGES:
|
||||
startStopAction(context, query, ActionLocks::PartsMerge, false);
|
||||
break;
|
||||
case Type::START_MERGES:
|
||||
startStopAction(context, query, ActionLocks::PartsMerge, true);
|
||||
break;
|
||||
case Type::STOP_FETCHES:
|
||||
startStopAction(context, query, ActionLocks::PartsFetch, false);
|
||||
break;
|
||||
case Type::START_FETCHES:
|
||||
startStopAction(context, query, ActionLocks::PartsFetch, true);
|
||||
break;
|
||||
case Type::STOP_REPLICATED_SENDS:
|
||||
startStopAction(context, query, ActionLocks::PartsSend, false);
|
||||
break;
|
||||
case Type::START_REPLICATEDS_SENDS:
|
||||
startStopAction(context, query, ActionLocks::PartsSend, false);
|
||||
break;
|
||||
case Type::STOP_REPLICATION_QUEUES:
|
||||
startStopAction(context, query, ActionLocks::ReplicationQueue, false);
|
||||
break;
|
||||
case Type::START_REPLICATION_QUEUES:
|
||||
startStopAction(context, query, ActionLocks::ReplicationQueue, true);
|
||||
break;
|
||||
case Type::SYNC_REPLICA:
|
||||
syncReplica(query);
|
||||
break;
|
||||
case Type::RESTART_REPLICAS:
|
||||
restartReplicas(system_context);
|
||||
break;
|
||||
case Type::RESTART_REPLICA:
|
||||
if (!tryRestartReplica(query.target_database, query.target_table, system_context))
|
||||
throw Exception("There is no " + query.target_database + "." + query.target_table + " replicated table",
|
||||
ErrorCodes::BAD_ARGUMENTS);
|
||||
break;
|
||||
case Type::STOP_LISTEN_QUERIES:
|
||||
case Type::START_LISTEN_QUERIES:
|
||||
case Type::RESTART_REPLICAS:
|
||||
case Type::SYNC_REPLICA:
|
||||
case Type::STOP_MERGES:
|
||||
case Type::START_MERGES:
|
||||
case Type::STOP_REPLICATION_QUEUES:
|
||||
case Type::START_REPLICATION_QUEUES:
|
||||
throw Exception(String(ASTSystemQuery::typeToString(query.type)) + " is not supported yet", ErrorCodes::NOT_IMPLEMENTED);
|
||||
default:
|
||||
throw Exception("Unknown type of SYSTEM query", ErrorCodes::BAD_ARGUMENTS);
|
||||
@ -124,4 +200,92 @@ BlockIO InterpreterSystemQuery::execute()
|
||||
}
|
||||
|
||||
|
||||
StoragePtr InterpreterSystemQuery::tryRestartReplica(const String & database_name, const String & table_name, Context & context)
|
||||
{
|
||||
auto database = context.getDatabase(database_name);
|
||||
auto table_ddl_guard = context.getDDLGuard(database_name, table_name, "Table " + database_name + "." + table_name + " is restarting right now");
|
||||
ASTPtr create_ast;
|
||||
|
||||
/// Detach actions
|
||||
{
|
||||
auto table = context.tryGetTable(database_name, table_name);
|
||||
|
||||
if (!table || !dynamic_cast<const StorageReplicatedMergeTree *>(table.get()))
|
||||
return nullptr;
|
||||
|
||||
table->shutdown();
|
||||
|
||||
/// If table was already dropped by anyone, an exception will be thrown
|
||||
auto table_lock = table->lockForAlter(__PRETTY_FUNCTION__);
|
||||
create_ast = context.getCreateTableQuery(database_name, table_name);
|
||||
|
||||
database->detachTable(table_name);
|
||||
}
|
||||
|
||||
/// Attach actions
|
||||
{
|
||||
/// getCreateTableQuery must return canonical CREATE query representation, there are no need for AST postprocessing
|
||||
auto & create = typeid_cast<ASTCreateQuery &>(*create_ast);
|
||||
create.attach = true;
|
||||
|
||||
std::string data_path = database->getDataPath();
|
||||
auto columns = InterpreterCreateQuery::getColumnsDescription(*create.columns, context);
|
||||
|
||||
StoragePtr table = StorageFactory::instance().get(create,
|
||||
data_path,
|
||||
table_name,
|
||||
database_name,
|
||||
context,
|
||||
context.getGlobalContext(),
|
||||
columns,
|
||||
create.attach,
|
||||
false);
|
||||
|
||||
database->createTable(context, table_name, table, create_ast);
|
||||
|
||||
table->startup();
|
||||
return table;
|
||||
}
|
||||
}
|
||||
|
||||
void InterpreterSystemQuery::restartReplicas(Context & context)
|
||||
{
|
||||
std::vector<std::pair<String, String>> replica_names;
|
||||
|
||||
for (auto & elem : context.getDatabases())
|
||||
{
|
||||
DatabasePtr & database = elem.second;
|
||||
const String & database_name = elem.first;
|
||||
|
||||
for (auto iterator = database->getIterator(context); iterator->isValid(); iterator->next())
|
||||
{
|
||||
if (dynamic_cast<const StorageReplicatedMergeTree *>(iterator->table().get()))
|
||||
replica_names.emplace_back(database_name, iterator->name());
|
||||
}
|
||||
}
|
||||
|
||||
if (replica_names.empty())
|
||||
return;
|
||||
|
||||
ThreadPool pool(std::min(getNumberOfPhysicalCPUCores(), replica_names.size()));
|
||||
for (auto & table : replica_names)
|
||||
pool.schedule([&] () { tryRestartReplica(table.first, table.second, context); });
|
||||
pool.wait();
|
||||
}
|
||||
|
||||
void InterpreterSystemQuery::syncReplica(ASTSystemQuery & query)
|
||||
{
|
||||
String database_name = !query.target_database.empty() ? query.target_database : context.getCurrentDatabase();
|
||||
const String & table_name = query.target_table;
|
||||
|
||||
StoragePtr table = context.getTable(database_name, table_name);
|
||||
|
||||
auto table_replicated = dynamic_cast<StorageReplicatedMergeTree *>(table.get());
|
||||
if (!table_replicated)
|
||||
throw Exception("Table " + database_name + "." + table_name + " is not replicated", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
table_replicated->waitForShrinkingQueueSize(0, context.getSettingsRef().receive_timeout.value.milliseconds());
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
@ -7,7 +7,10 @@ namespace DB
|
||||
|
||||
class Context;
|
||||
class IAST;
|
||||
class ASTSystemQuery;
|
||||
class IStorage;
|
||||
using ASTPtr = std::shared_ptr<IAST>;
|
||||
using StoragePtr = std::shared_ptr<IStorage>;
|
||||
|
||||
|
||||
class InterpreterSystemQuery : public IInterpreter
|
||||
@ -20,6 +23,14 @@ public:
|
||||
private:
|
||||
ASTPtr query_ptr;
|
||||
Context & context;
|
||||
Poco::Logger * log = nullptr;
|
||||
|
||||
/// Tries to get a replicated table and restart it
|
||||
/// Returns pointer to a newly created table if the restart was successful
|
||||
StoragePtr tryRestartReplica(const String & database_name, const String & table_name, Context & context);
|
||||
|
||||
void restartReplicas(Context & context);
|
||||
void syncReplica(ASTSystemQuery & query);
|
||||
};
|
||||
|
||||
|
||||
|
@ -85,25 +85,27 @@ public:
|
||||
void addEndpoint(const String & name, InterserverIOEndpointPtr endpoint)
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
if (endpoint_map.count(name))
|
||||
bool inserted = endpoint_map.try_emplace(name, std::move(endpoint)).second;
|
||||
if (!inserted)
|
||||
throw Exception("Duplicate interserver IO endpoint: " + name, ErrorCodes::DUPLICATE_INTERSERVER_IO_ENDPOINT);
|
||||
endpoint_map[name] = std::move(endpoint);
|
||||
}
|
||||
|
||||
void removeEndpoint(const String & name)
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
if (!endpoint_map.count(name))
|
||||
if (!endpoint_map.erase(name))
|
||||
throw Exception("No interserver IO endpoint named " + name, ErrorCodes::NO_SUCH_INTERSERVER_IO_ENDPOINT);
|
||||
endpoint_map.erase(name);
|
||||
}
|
||||
|
||||
InterserverIOEndpointPtr getEndpoint(const String & name)
|
||||
try
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
if (!endpoint_map.count(name))
|
||||
throw Exception("No interserver IO endpoint named " + name, ErrorCodes::NO_SUCH_INTERSERVER_IO_ENDPOINT);
|
||||
return endpoint_map[name];
|
||||
return endpoint_map.at(name);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
throw Exception("No interserver IO endpoint named " + name, ErrorCodes::NO_SUCH_INTERSERVER_IO_ENDPOINT);
|
||||
}
|
||||
|
||||
private:
|
||||
@ -129,22 +131,18 @@ public:
|
||||
}
|
||||
|
||||
~InterserverIOEndpointHolder()
|
||||
try
|
||||
{
|
||||
try
|
||||
{
|
||||
handler.removeEndpoint(name);
|
||||
/// After destroying the object, `endpoint` can still live, since its ownership is acquired during the processing of the request,
|
||||
/// see InterserverIOHTTPHandler.cpp
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException("~InterserverIOEndpointHolder");
|
||||
}
|
||||
handler.removeEndpoint(name);
|
||||
/// After destroying the object, `endpoint` can still live, since its ownership is acquired during the processing of the request,
|
||||
/// see InterserverIOHTTPHandler.cpp
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException("~InterserverIOEndpointHolder");
|
||||
}
|
||||
|
||||
ActionBlocker & getBlocker() { return endpoint->blocker; }
|
||||
void cancelForever() { getBlocker().cancelForever(); }
|
||||
ActionBlocker::LockHolder cancel() { return getBlocker().cancel(); }
|
||||
|
||||
private:
|
||||
String name;
|
||||
|
@ -89,33 +89,45 @@ void PartLogElement::appendToBlock(Block & block) const
|
||||
}
|
||||
|
||||
|
||||
bool PartLog::addNewPartToTheLog(Context & context, const MergeTreeDataPart & part, UInt64 elapsed_ns, const ExecutionStatus & execution_status)
|
||||
bool PartLog::addNewPart(Context & context, const MutableDataPartPtr & part, UInt64 elapsed_ns, const ExecutionStatus & execution_status)
|
||||
{
|
||||
return addNewParts(context, {part}, elapsed_ns, execution_status);
|
||||
}
|
||||
|
||||
bool PartLog::addNewParts(Context & context, const PartLog::MutableDataPartsVector & parts, UInt64 elapsed_ns,
|
||||
const ExecutionStatus & execution_status)
|
||||
{
|
||||
if (parts.empty())
|
||||
return true;
|
||||
|
||||
PartLog * part_log = nullptr;
|
||||
|
||||
try
|
||||
{
|
||||
part_log = context.getPartLog(part.storage.getDatabaseName());
|
||||
part_log = context.getPartLog(parts.front()->storage.getDatabaseName()); // assume parts belong to the same table
|
||||
if (!part_log)
|
||||
return false;
|
||||
|
||||
PartLogElement elem;
|
||||
for (const auto & part : parts)
|
||||
{
|
||||
PartLogElement elem;
|
||||
|
||||
elem.event_type = PartLogElement::NEW_PART;
|
||||
elem.event_time = time(nullptr);
|
||||
elem.duration_ms = elapsed_ns / 1000000;
|
||||
elem.event_type = PartLogElement::NEW_PART;
|
||||
elem.event_time = time(nullptr);
|
||||
elem.duration_ms = elapsed_ns / 1000000;
|
||||
|
||||
elem.database_name = part.storage.getDatabaseName();
|
||||
elem.table_name = part.storage.getTableName();
|
||||
elem.part_name = part.name;
|
||||
elem.database_name = part->storage.getDatabaseName();
|
||||
elem.table_name = part->storage.getTableName();
|
||||
elem.part_name = part->name;
|
||||
|
||||
elem.bytes_compressed_on_disk = part.bytes_on_disk;
|
||||
elem.rows = part.rows_count;
|
||||
elem.bytes_compressed_on_disk = part->bytes_on_disk;
|
||||
elem.rows = part->rows_count;
|
||||
|
||||
elem.error = static_cast<UInt16>(execution_status.code);
|
||||
elem.exception = execution_status.message;
|
||||
elem.error = static_cast<UInt16>(execution_status.code);
|
||||
elem.exception = execution_status.message;
|
||||
|
||||
part_log->add(elem);
|
||||
part_log->add(elem);
|
||||
}
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
|
@ -56,10 +56,15 @@ class PartLog : public SystemLog<PartLogElement>
|
||||
{
|
||||
using SystemLog<PartLogElement>::SystemLog;
|
||||
|
||||
using MutableDataPartPtr = std::shared_ptr<MergeTreeDataPart>;
|
||||
using MutableDataPartsVector = std::vector<MutableDataPartPtr>;
|
||||
|
||||
public:
|
||||
/// Add a record about creation of new part.
|
||||
static bool addNewPartToTheLog(Context & context, const MergeTreeDataPart & part, UInt64 elapsed_ns,
|
||||
const ExecutionStatus & execution_status = {});
|
||||
static bool addNewPart(Context & context, const MutableDataPartPtr & part, UInt64 elapsed_ns,
|
||||
const ExecutionStatus & execution_status = {});
|
||||
static bool addNewParts(Context & context, const MutableDataPartsVector & parts, UInt64 elapsed_ns,
|
||||
const ExecutionStatus & execution_status = {});
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -49,6 +49,7 @@ struct Settings
|
||||
M(SettingBool, use_uncompressed_cache, true, "Whether to use the cache of uncompressed blocks.") \
|
||||
M(SettingBool, replace_running_query, false, "Whether the running request should be canceled with the same id as the new one.") \
|
||||
M(SettingUInt64, background_pool_size, DBMS_DEFAULT_BACKGROUND_POOL_SIZE, "Number of threads performing background work for tables (for example, merging in merge tree). Only has meaning at server startup.") \
|
||||
M(SettingUInt64, background_schedule_pool_size, DBMS_DEFAULT_BACKGROUND_POOL_SIZE, "Number of threads performing background tasks for replicated tables. Only has meaning at server startup.") \
|
||||
\
|
||||
M(SettingMilliseconds, distributed_directory_monitor_sleep_time_ms, DBMS_DISTRIBUTED_DIRECTORY_MONITOR_SLEEP_TIME_MS, "Sleep time for StorageDistributed DirectoryMonitors in case there is no work or exception has been thrown.") \
|
||||
\
|
||||
|
@ -72,10 +72,10 @@ static Field convertNumericType(const Field & from, const IDataType & type)
|
||||
}
|
||||
|
||||
|
||||
DayNum_t stringToDate(const String & s)
|
||||
DayNum stringToDate(const String & s)
|
||||
{
|
||||
ReadBufferFromString in(s);
|
||||
DayNum_t date{};
|
||||
DayNum date{};
|
||||
|
||||
readDateText(date, in);
|
||||
if (!in.eof())
|
||||
|
@ -135,6 +135,19 @@ void ASTAlterQuery::formatQueryImpl(const FormatSettings & settings, FormatState
|
||||
<< (p.part ? "PART " : "PARTITION ") << (settings.hilite ? hilite_none : "");
|
||||
p.partition->formatImpl(settings, state, frame);
|
||||
}
|
||||
else if (p.type == ASTAlterQuery::REPLACE_PARTITION)
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << (p.replace ? "REPLACE" : "ATTACH") << " PARTITION "
|
||||
<< (settings.hilite ? hilite_none : "");
|
||||
p.partition->formatImpl(settings, state, frame);
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << " FROM " << (settings.hilite ? hilite_none : "");
|
||||
if (!p.from_database.empty())
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(p.from_database)
|
||||
<< (settings.hilite ? hilite_none : "") << ".";
|
||||
}
|
||||
settings.ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(p.from_table) << (settings.hilite ? hilite_none : "");
|
||||
}
|
||||
else if (p.type == ASTAlterQuery::FETCH_PARTITION)
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "FETCH "
|
||||
|
@ -28,6 +28,7 @@ public:
|
||||
|
||||
DROP_PARTITION,
|
||||
ATTACH_PARTITION,
|
||||
REPLACE_PARTITION,
|
||||
FETCH_PARTITION,
|
||||
FREEZE_PARTITION,
|
||||
|
||||
@ -57,7 +58,8 @@ public:
|
||||
*/
|
||||
ASTPtr primary_key;
|
||||
|
||||
/** In DROP PARTITION and RESHARD PARTITION queries, the value or ID of the partition is stored here.
|
||||
/** Used in DROP PARTITION, RESHARD PARTITION and ATTACH PARTITION FROM queries.
|
||||
* The value or ID of the partition is stored here.
|
||||
*/
|
||||
ASTPtr partition;
|
||||
|
||||
@ -80,6 +82,12 @@ public:
|
||||
*/
|
||||
String with_name;
|
||||
|
||||
/// REPLACE(ATTACH) PARTITION partition FROM db.table
|
||||
String from_database;
|
||||
String from_table;
|
||||
/// To distinguish REPLACE and ATTACH PARTITION partition FROM db.table
|
||||
bool replace = true;
|
||||
|
||||
/// deep copy
|
||||
void clone(Parameters & p) const;
|
||||
};
|
||||
|
@ -26,7 +26,7 @@ public:
|
||||
Kind kind;
|
||||
|
||||
ASTIdentifier(const String & name_, const Kind kind_ = Column)
|
||||
: name(name_), kind(kind_) {}
|
||||
: name(name_), kind(kind_) { range = StringRange(name.data(), name.data() + name.size()); }
|
||||
|
||||
/** Get the text that identifies this element. */
|
||||
String getID() const override { return "Identifier_" + name; }
|
||||
|
@ -33,6 +33,8 @@ const char * ASTSystemQuery::typeToString(Type type)
|
||||
return "START LISTEN QUERIES";
|
||||
case Type::RESTART_REPLICAS:
|
||||
return "RESTART REPLICAS";
|
||||
case Type::RESTART_REPLICA:
|
||||
return "RESTART REPLICA";
|
||||
case Type::SYNC_REPLICA:
|
||||
return "SYNC REPLICA";
|
||||
case Type::RELOAD_DICTIONARY:
|
||||
@ -47,6 +49,14 @@ const char * ASTSystemQuery::typeToString(Type type)
|
||||
return "STOP MERGES";
|
||||
case Type::START_MERGES:
|
||||
return "START MERGES";
|
||||
case Type::STOP_FETCHES:
|
||||
return "STOP FETCHES";
|
||||
case Type::START_FETCHES:
|
||||
return "START FETCHES";
|
||||
case Type::STOP_REPLICATED_SENDS:
|
||||
return "STOP REPLICATED SENDS";
|
||||
case Type::START_REPLICATEDS_SENDS:
|
||||
return "START REPLICATED SENDS";
|
||||
case Type::STOP_REPLICATION_QUEUES:
|
||||
return "STOP REPLICATION QUEUES";
|
||||
case Type::START_REPLICATION_QUEUES:
|
||||
@ -62,10 +72,38 @@ void ASTSystemQuery::formatImpl(const FormatSettings & settings, FormatState &,
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << "SYSTEM " << (settings.hilite ? hilite_none : "");
|
||||
settings.ostr << typeToString(type);
|
||||
|
||||
if (type == Type::RELOAD_DICTIONARY)
|
||||
auto print_database_table = [&] ()
|
||||
{
|
||||
settings.ostr << " ";
|
||||
|
||||
if (!target_database.empty())
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(target_database)
|
||||
<< (settings.hilite ? hilite_none : "") << ".";
|
||||
}
|
||||
|
||||
settings.ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(target_table)
|
||||
<< (settings.hilite ? hilite_none : "");
|
||||
};
|
||||
|
||||
if ( type == Type::STOP_MERGES
|
||||
|| type == Type::START_MERGES
|
||||
|| type == Type::STOP_FETCHES
|
||||
|| type == Type::START_FETCHES
|
||||
|| type == Type::STOP_REPLICATED_SENDS
|
||||
|| type == Type::START_REPLICATEDS_SENDS
|
||||
|| type == Type::STOP_REPLICATION_QUEUES
|
||||
|| type == Type::START_REPLICATION_QUEUES)
|
||||
{
|
||||
if (!target_table.empty())
|
||||
print_database_table();
|
||||
}
|
||||
else if (type == Type::RESTART_REPLICA || type == Type::SYNC_REPLICA)
|
||||
{
|
||||
print_database_table();
|
||||
}
|
||||
else if (type == Type::RELOAD_DICTIONARY)
|
||||
settings.ostr << " " << backQuoteIfNeed(target_dictionary);
|
||||
else if (type == Type::SYNC_REPLICA)
|
||||
throw Exception("SYNC_REPLICA isn't supported yet", ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
|
||||
|
@ -21,6 +21,7 @@ public:
|
||||
STOP_LISTEN_QUERIES,
|
||||
START_LISTEN_QUERIES,
|
||||
RESTART_REPLICAS,
|
||||
RESTART_REPLICA,
|
||||
SYNC_REPLICA,
|
||||
RELOAD_DICTIONARY,
|
||||
RELOAD_DICTIONARIES,
|
||||
@ -28,6 +29,10 @@ public:
|
||||
RELOAD_CONFIG,
|
||||
STOP_MERGES,
|
||||
START_MERGES,
|
||||
STOP_FETCHES,
|
||||
START_FETCHES,
|
||||
STOP_REPLICATED_SENDS,
|
||||
START_REPLICATEDS_SENDS,
|
||||
STOP_REPLICATION_QUEUES,
|
||||
START_REPLICATION_QUEUES,
|
||||
END
|
||||
@ -38,8 +43,8 @@ public:
|
||||
Type type = Type::UNKNOWN;
|
||||
|
||||
String target_dictionary;
|
||||
//String target_replica_database;
|
||||
//String target_replica_table;
|
||||
String target_database;
|
||||
String target_table;
|
||||
|
||||
String getID() const override { return "SYSTEM query"; };
|
||||
|
||||
|
@ -65,7 +65,7 @@ public:
|
||||
|
||||
ASTPtr ptr() { return shared_from_this(); }
|
||||
|
||||
/** Get a deep copy of the tree. */
|
||||
/** Get a deep copy of the tree. Cloned object must have the same range. */
|
||||
virtual ASTPtr clone() const = 0;
|
||||
|
||||
/** Get hash code, identifying this element and its subtree.
|
||||
|
@ -6,6 +6,8 @@
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTAlterQuery.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/parseDatabaseAndTableName.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -45,8 +47,6 @@ bool ParserAlterQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
ParserStringLiteral parser_string_literal;
|
||||
ParserExpression exp_elem;
|
||||
|
||||
ASTPtr table;
|
||||
ASTPtr database;
|
||||
String cluster_str;
|
||||
ASTPtr col_type;
|
||||
ASTPtr col_after;
|
||||
@ -57,24 +57,9 @@ bool ParserAlterQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
if (!s_alter_table.ignore(pos, expected))
|
||||
return false;
|
||||
|
||||
if (!table_parser.parse(pos, database, expected))
|
||||
if (!parseDatabaseAndTableName(pos, expected, query->database, query->table))
|
||||
return false;
|
||||
|
||||
/// Parse [db].name
|
||||
if (s_dot.ignore(pos))
|
||||
{
|
||||
if (!table_parser.parse(pos, table, expected))
|
||||
return false;
|
||||
|
||||
query->table = typeid_cast<ASTIdentifier &>(*table).name;
|
||||
query->database = typeid_cast<ASTIdentifier &>(*database).name;
|
||||
}
|
||||
else
|
||||
{
|
||||
table = database;
|
||||
query->table = typeid_cast<ASTIdentifier &>(*table).name;
|
||||
}
|
||||
|
||||
if (ParserKeyword{"ON"}.ignore(pos, expected))
|
||||
{
|
||||
if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected))
|
||||
@ -142,7 +127,32 @@ bool ParserAlterQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
if (!parser_partition.parse(pos, params.partition, expected))
|
||||
return false;
|
||||
|
||||
params.type = ASTAlterQuery::ATTACH_PARTITION;
|
||||
if (s_from.ignore(pos))
|
||||
{
|
||||
if (!parseDatabaseAndTableName(pos, expected, params.from_database, params.from_table))
|
||||
return false;
|
||||
|
||||
params.replace = false;
|
||||
params.type = ASTAlterQuery::REPLACE_PARTITION;
|
||||
}
|
||||
else
|
||||
{
|
||||
params.type = ASTAlterQuery::ATTACH_PARTITION;
|
||||
}
|
||||
}
|
||||
else if (ParserKeyword{"REPLACE PARTITION"}.ignore(pos, expected))
|
||||
{
|
||||
if (!parser_partition.parse(pos, params.partition, expected))
|
||||
return false;
|
||||
|
||||
if (!s_from.ignore(pos, expected))
|
||||
return false;
|
||||
|
||||
if (!parseDatabaseAndTableName(pos, expected, params.from_database, params.from_table))
|
||||
return false;
|
||||
|
||||
params.replace = true;
|
||||
params.type = ASTAlterQuery::REPLACE_PARTITION;
|
||||
}
|
||||
else if (s_attach_part.ignore(pos, expected))
|
||||
{
|
||||
|
@ -2,7 +2,10 @@
|
||||
#include <Parsers/ASTSystemQuery.h>
|
||||
#include <Parsers/CommonParsers.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/parseIdentifierOrStringLiteral.h>
|
||||
#include <Parsers/ExpressionElementParsers.h>
|
||||
#include <Parsers/parseDatabaseAndTableName.h>
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
|
||||
|
||||
@ -39,14 +42,33 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected &
|
||||
if (!found)
|
||||
return false;
|
||||
|
||||
if (res->type == Type::RELOAD_DICTIONARY)
|
||||
switch (res->type)
|
||||
{
|
||||
if (!parseIdentifierOrStringLiteral(pos, expected, res->target_dictionary))
|
||||
return false;
|
||||
}
|
||||
else if (res->type == Type::SYNC_REPLICA)
|
||||
{
|
||||
throw Exception("SYNC REPLICA is not supported yet", ErrorCodes::NOT_IMPLEMENTED);
|
||||
case Type::RELOAD_DICTIONARY:
|
||||
if (!parseIdentifierOrStringLiteral(pos, expected, res->target_dictionary))
|
||||
return false;
|
||||
break;
|
||||
|
||||
case Type::RESTART_REPLICA:
|
||||
case Type::SYNC_REPLICA:
|
||||
if (!parseDatabaseAndTableName(pos, expected, res->target_database, res->target_table))
|
||||
return false;
|
||||
break;
|
||||
|
||||
case Type::STOP_MERGES:
|
||||
case Type::START_MERGES:
|
||||
case Type::STOP_FETCHES:
|
||||
case Type::START_FETCHES:
|
||||
case Type::STOP_REPLICATED_SENDS:
|
||||
case Type::START_REPLICATEDS_SENDS:
|
||||
case Type::STOP_REPLICATION_QUEUES:
|
||||
case Type::START_REPLICATION_QUEUES:
|
||||
parseDatabaseAndTableName(pos, expected, res->target_database, res->target_table);
|
||||
break;
|
||||
|
||||
default:
|
||||
/// There are no [db.table] after COMMAND NAME
|
||||
break;
|
||||
}
|
||||
|
||||
node = std::move(res);
|
||||
|
@ -4,6 +4,7 @@
|
||||
#include <Parsers/TokenIterator.h>
|
||||
#include <map>
|
||||
#include <memory>
|
||||
#include <Common/SipHash.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -14,9 +15,10 @@ struct StringRange
|
||||
const char * first = nullptr;
|
||||
const char * second = nullptr;
|
||||
|
||||
StringRange() {}
|
||||
StringRange() = default;
|
||||
StringRange(const StringRange & other) = default;
|
||||
StringRange(const char * begin, const char * end) : first(begin), second(end) {}
|
||||
StringRange(TokenIterator token) : first(token->begin), second(token->end) {}
|
||||
explicit StringRange(TokenIterator token) : first(token->begin), second(token->end) {}
|
||||
|
||||
StringRange(TokenIterator token_begin, TokenIterator token_end)
|
||||
{
|
||||
@ -44,4 +46,27 @@ inline String toString(const StringRange & range)
|
||||
return range.first ? String(range.first, range.second) : String();
|
||||
}
|
||||
|
||||
}
|
||||
/// Hashes only the values of pointers in StringRange. Is used with StringRangePointersEqualTo comparator.
|
||||
struct StringRangePointersHash
|
||||
{
|
||||
UInt64 operator()(const StringRange & range) const
|
||||
{
|
||||
SipHash hash;
|
||||
hash.update(range.first);
|
||||
hash.update(range.second);
|
||||
return hash.get64();
|
||||
}
|
||||
};
|
||||
|
||||
/// Ranges are equal only when they point to the same memory region.
|
||||
/// It may be used when it's enough to compare substrings by their position in the same string.
|
||||
struct StringRangePointersEqualTo
|
||||
{
|
||||
constexpr bool operator()(const StringRange &lhs, const StringRange &rhs) const
|
||||
{
|
||||
return std::tie(lhs.first, lhs.second) == std::tie(rhs.first, rhs.second);
|
||||
}
|
||||
};
|
||||
|
||||
};
|
||||
|
||||
|
45
dbms/src/Parsers/parseDatabaseAndTableName.cpp
Normal file
45
dbms/src/Parsers/parseDatabaseAndTableName.cpp
Normal file
@ -0,0 +1,45 @@
|
||||
#include "parseDatabaseAndTableName.h"
|
||||
#include <Parsers/ExpressionElementParsers.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/CommonParsers.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
bool parseDatabaseAndTableName(IParser::Pos & pos, Expected & expected, String & database_str, String & table_str)
|
||||
{
|
||||
ParserToken s_dot(TokenType::Dot);
|
||||
ParserIdentifier table_parser;
|
||||
|
||||
ASTPtr database;
|
||||
ASTPtr table;
|
||||
|
||||
database_str = "";
|
||||
table_str = "";
|
||||
|
||||
if (!table_parser.parse(pos, database, expected))
|
||||
return false;
|
||||
|
||||
if (s_dot.ignore(pos))
|
||||
{
|
||||
if (!table_parser.parse(pos, table, expected))
|
||||
{
|
||||
database_str = "";
|
||||
return false;
|
||||
}
|
||||
|
||||
database_str = typeid_cast<ASTIdentifier &>(*database).name;
|
||||
table_str = typeid_cast<ASTIdentifier &>(*table).name;
|
||||
}
|
||||
else
|
||||
{
|
||||
database_str = "";
|
||||
table_str = typeid_cast<ASTIdentifier &>(*database).name;
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
}
|
10
dbms/src/Parsers/parseDatabaseAndTableName.h
Normal file
10
dbms/src/Parsers/parseDatabaseAndTableName.h
Normal file
@ -0,0 +1,10 @@
|
||||
#pragma once
|
||||
#include <Parsers/IParser.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// Parses [db].name
|
||||
bool parseDatabaseAndTableName(IParser::Pos & pos, Expected & expected, String & database_str, String & table_str);
|
||||
|
||||
}
|
@ -15,6 +15,7 @@
|
||||
#include <Poco/Util/Application.h>
|
||||
#include <common/readline_use.h>
|
||||
#include <common/find_first_symbols.h>
|
||||
#include <common/SetTerminalEcho.h>
|
||||
#include <Common/ClickHouseRevision.h>
|
||||
#include <Common/Stopwatch.h>
|
||||
#include <Common/Exception.h>
|
||||
@ -52,6 +53,7 @@
|
||||
#include "InterruptListener.h"
|
||||
#include <Functions/registerFunctions.h>
|
||||
#include <AggregateFunctions/registerAggregateFunctions.h>
|
||||
#include <ext/scope_guard.h>
|
||||
|
||||
|
||||
/// http://en.wikipedia.org/wiki/ANSI_escape_code
|
||||
@ -197,7 +199,25 @@ private:
|
||||
|
||||
default_database = config.getString("database", "");
|
||||
user = config.getString("user", "");
|
||||
password = config.getString("password", "");
|
||||
|
||||
if (config.getBool("ask-password", false))
|
||||
{
|
||||
if (config.has("password"))
|
||||
throw Exception("Specified both --password and --ask-password. Remove one of them", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
std::cout << "Password for user " << user << ": ";
|
||||
SetTerminalEcho(false);
|
||||
|
||||
SCOPE_EXIT({
|
||||
SetTerminalEcho(true);
|
||||
});
|
||||
std::getline(std::cin, password);
|
||||
std::cout << std::endl;
|
||||
}
|
||||
else
|
||||
{
|
||||
password = config.getString("password", "");
|
||||
}
|
||||
|
||||
compression = config.getBool("compression", true)
|
||||
? Protocol::Compression::Enable
|
||||
@ -461,8 +481,9 @@ private:
|
||||
query_id = config().getString("query_id", "");
|
||||
nonInteractive();
|
||||
|
||||
/// If exception code isn't zero, we should return non-zero return code anyway.
|
||||
if (last_exception)
|
||||
return last_exception->code();
|
||||
return last_exception->code() != 0 ? last_exception->code() : -1;
|
||||
|
||||
return 0;
|
||||
}
|
||||
@ -1364,18 +1385,27 @@ public:
|
||||
|
||||
ioctl(0, TIOCGWINSZ, &terminal_size);
|
||||
|
||||
unsigned line_length = boost::program_options::options_description::m_default_line_length;
|
||||
unsigned min_description_length = line_length / 2;
|
||||
if (!stdin_is_not_tty)
|
||||
{
|
||||
line_length = std::max(3U, static_cast<unsigned>(terminal_size.ws_col));
|
||||
min_description_length = std::min(min_description_length, line_length - 2);
|
||||
}
|
||||
|
||||
#define DECLARE_SETTING(TYPE, NAME, DEFAULT, DESCRIPTION) (#NAME, boost::program_options::value<std::string> (), DESCRIPTION)
|
||||
|
||||
/// Main commandline options related to client functionality and all parameters from Settings.
|
||||
boost::program_options::options_description main_description("Main options", terminal_size.ws_col);
|
||||
boost::program_options::options_description main_description("Main options", line_length, min_description_length);
|
||||
main_description.add_options()
|
||||
("help", "produce help message")
|
||||
("config-file,c", boost::program_options::value<std::string>(), "config-file path")
|
||||
("host,h", boost::program_options::value<std::string>()->default_value("localhost"), "server host")
|
||||
("port", boost::program_options::value<int>()->default_value(9000), "server port")
|
||||
("secure,s", "secure")
|
||||
("user,u", boost::program_options::value<std::string>(), "user")
|
||||
("user,u", boost::program_options::value<std::string>()->default_value("default"), "user")
|
||||
("password", boost::program_options::value<std::string>(), "password")
|
||||
("ask-password", "ask-password")
|
||||
("query_id", boost::program_options::value<std::string>(), "query_id")
|
||||
("query,q", boost::program_options::value<std::string>(), "query")
|
||||
("database,d", boost::program_options::value<std::string>(), "database")
|
||||
@ -1482,7 +1512,8 @@ public:
|
||||
config().setString("user", options["user"].as<std::string>());
|
||||
if (options.count("password"))
|
||||
config().setString("password", options["password"].as<std::string>());
|
||||
|
||||
if (options.count("ask-password"))
|
||||
config().setBool("ask-password", true);
|
||||
if (options.count("multiline"))
|
||||
config().setBool("multiline", true);
|
||||
if (options.count("multiquery"))
|
||||
|
@ -388,16 +388,23 @@ std::string LocalServer::getHelpFooter() const
|
||||
|
||||
void LocalServer::init(int argc, char ** argv)
|
||||
{
|
||||
namespace po = boost::program_options;
|
||||
|
||||
/// Don't parse options with Poco library, we prefer neat boost::program_options
|
||||
stopOptionsProcessing();
|
||||
|
||||
winsize terminal_size{};
|
||||
ioctl(0, TIOCGWINSZ, &terminal_size);
|
||||
|
||||
namespace po = boost::program_options;
|
||||
unsigned line_length = po::options_description::m_default_line_length;
|
||||
unsigned min_description_length = line_length / 2;
|
||||
if (isatty(STDIN_FILENO))
|
||||
{
|
||||
winsize terminal_size{};
|
||||
ioctl(0, TIOCGWINSZ, &terminal_size);
|
||||
line_length = std::max(3U, static_cast<unsigned>(terminal_size.ws_col));
|
||||
min_description_length = std::min(min_description_length, line_length - 2);
|
||||
}
|
||||
|
||||
#define DECLARE_SETTING(TYPE, NAME, DEFAULT, DESCRIPTION) (#NAME, po::value<std::string> (), DESCRIPTION)
|
||||
po::options_description description("Main options", terminal_size.ws_col);
|
||||
po::options_description description("Main options", line_length, min_description_length);
|
||||
description.add_options()
|
||||
("help", "produce help message")
|
||||
("config-file,c", po::value<std::string>(), "config-file path")
|
||||
|
@ -9,6 +9,7 @@
|
||||
#include <shared_mutex>
|
||||
#include <memory>
|
||||
#include <optional>
|
||||
#include <Common/ActionLock.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -26,6 +27,8 @@ class IBlockOutputStream;
|
||||
class RWLockFIFO;
|
||||
using RWLockFIFOPtr = std::shared_ptr<RWLockFIFO>;
|
||||
|
||||
using StorageActionBlockType = size_t;
|
||||
|
||||
using BlockOutputStreamPtr = std::shared_ptr<IBlockOutputStream>;
|
||||
using BlockInputStreamPtr = std::shared_ptr<IBlockInputStream>;
|
||||
using BlockInputStreams = std::vector<BlockInputStreamPtr>;
|
||||
@ -104,6 +107,9 @@ public:
|
||||
/** Returns true if the storage replicates SELECT, INSERT and ALTER commands among replicas. */
|
||||
virtual bool supportsReplication() const { return false; }
|
||||
|
||||
/** Returns true if the storage supports deduplication of inserted data blocks . */
|
||||
virtual bool supportsDeduplication() const { return false; }
|
||||
|
||||
/** Does not allow you to change the structure or name of the table.
|
||||
* If you change the data in the table, you will need to specify will_modify_data = true.
|
||||
* This will take an extra lock that does not allow starting ALTER MODIFY.
|
||||
@ -225,6 +231,12 @@ public:
|
||||
throw Exception("Method dropColumnFromPartition is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
/** Execute ALTER TABLE dst.table REPLACE(ATTACH) PARTITION partition FROM src.table */
|
||||
virtual void replacePartitionFrom(const StoragePtr & /*source_table*/, const ASTPtr & /*partition*/, bool /*replace*/, const Context &)
|
||||
{
|
||||
throw Exception("Method replacePartitionFrom is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
/** Run the query (DROP|DETACH) PARTITION.
|
||||
*/
|
||||
virtual void dropPartition(const ASTPtr & /*query*/, const ASTPtr & /*partition*/, bool /*detach*/, const Context & /*context*/)
|
||||
@ -284,6 +296,13 @@ public:
|
||||
*/
|
||||
virtual void shutdown() {}
|
||||
|
||||
/// Asks table to stop executing some action identified by action_type
|
||||
/// If table does not support such type of lock, and empty lock is returned
|
||||
virtual ActionLock getActionLock(StorageActionBlockType /* action_type */)
|
||||
{
|
||||
return {};
|
||||
}
|
||||
|
||||
bool is_dropped{false};
|
||||
|
||||
/// Does table support index for IN sections
|
||||
|
@ -198,7 +198,7 @@ void ITableDeclaration::check(const NamesAndTypesList & provided_columns, const
|
||||
throw Exception("There is no column with name " + name + ". There are columns: "
|
||||
+ listOfColumns(available_columns), ErrorCodes::NO_SUCH_COLUMN_IN_TABLE);
|
||||
|
||||
if (it->second->getName() != jt->second->getName())
|
||||
if (!it->second->equals(*jt->second))
|
||||
throw Exception("Type mismatch for column " + name + ". Column has type "
|
||||
+ jt->second->getName() + ", got type " + it->second->getName(), ErrorCodes::TYPE_MISMATCH);
|
||||
|
||||
|
@ -84,7 +84,6 @@ ActiveDataPartSet::getContainingPartImpl(const MergeTreePartInfo & part_info) co
|
||||
return part_info_to_name.end();
|
||||
}
|
||||
|
||||
|
||||
Strings ActiveDataPartSet::getPartsCoveredBy(const MergeTreePartInfo & part_info) const
|
||||
{
|
||||
auto it_middle = part_info_to_name.lower_bound(part_info);
|
||||
@ -123,7 +122,6 @@ Strings ActiveDataPartSet::getPartsCoveredBy(const MergeTreePartInfo & part_info
|
||||
return covered;
|
||||
}
|
||||
|
||||
|
||||
Strings ActiveDataPartSet::getParts() const
|
||||
{
|
||||
Strings res;
|
||||
@ -134,11 +132,9 @@ Strings ActiveDataPartSet::getParts() const
|
||||
return res;
|
||||
}
|
||||
|
||||
|
||||
size_t ActiveDataPartSet::size() const
|
||||
{
|
||||
return part_info_to_name.size();
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
@ -12,8 +12,7 @@ namespace DB
|
||||
|
||||
/** Supports multiple names of active parts of data.
|
||||
* Repeats part of the MergeTreeData functionality.
|
||||
* TODO: generalize with MergeTreeData. It is possible to leave this class approximately as is and use it from MergeTreeData.
|
||||
* Then in MergeTreeData you can make map<String, DataPartPtr> data_parts and all_data_parts.
|
||||
* TODO: generalize with MergeTreeData
|
||||
*/
|
||||
class ActiveDataPartSet
|
||||
{
|
||||
@ -62,7 +61,8 @@ public:
|
||||
|
||||
Strings getPartsCoveredBy(const MergeTreePartInfo & part_info) const;
|
||||
|
||||
Strings getParts() const; /// In ascending order of the partition_id and block number.
|
||||
/// Returns parts in ascending order of the partition_id and block number.
|
||||
Strings getParts() const;
|
||||
|
||||
size_t size() const;
|
||||
|
||||
|
@ -160,7 +160,8 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchPart(
|
||||
const String & host,
|
||||
int port,
|
||||
const ConnectionTimeouts & timeouts,
|
||||
bool to_detached)
|
||||
bool to_detached,
|
||||
const String & tmp_prefix_)
|
||||
{
|
||||
Poco::URI uri;
|
||||
uri.setScheme("http");
|
||||
@ -176,7 +177,9 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchPart(
|
||||
ReadWriteBufferFromHTTP in{uri, Poco::Net::HTTPRequest::HTTP_POST, {}, timeouts};
|
||||
|
||||
static const String TMP_PREFIX = "tmp_fetch_";
|
||||
String relative_part_path = String(to_detached ? "detached/" : "") + TMP_PREFIX + part_name;
|
||||
String tmp_prefix = tmp_prefix_.empty() ? TMP_PREFIX : tmp_prefix_;
|
||||
|
||||
String relative_part_path = String(to_detached ? "detached/" : "") + tmp_prefix + part_name;
|
||||
String absolute_part_path = data.getFullPath() + relative_part_path + "/";
|
||||
Poco::File part_file(absolute_part_path);
|
||||
|
||||
|
@ -54,7 +54,8 @@ public:
|
||||
const String & host,
|
||||
int port,
|
||||
const ConnectionTimeouts & timeouts,
|
||||
bool to_detached = false);
|
||||
bool to_detached = false,
|
||||
const String & tmp_prefix_ = "");
|
||||
|
||||
/// You need to stop the data transfer.
|
||||
ActionBlocker blocker;
|
||||
|
@ -641,8 +641,8 @@ bool KeyCondition::atomFromAST(const ASTPtr & node, const Context & context, Blo
|
||||
bool is_set_const = false;
|
||||
bool is_constant_transformed = false;
|
||||
|
||||
if (prepared_sets.count(args[1].get())
|
||||
&& isTupleIndexable(args[0], context, out, prepared_sets[args[1].get()], key_column_num))
|
||||
if (prepared_sets.count(args[1]->range)
|
||||
&& isTupleIndexable(args[0], context, out, prepared_sets[args[1]->range], key_column_num))
|
||||
{
|
||||
key_arg_pos = 0;
|
||||
is_set_const = true;
|
||||
@ -1016,7 +1016,7 @@ bool KeyCondition::mayBeTrueInRangeImpl(const std::vector<Range> & key_ranges, c
|
||||
{
|
||||
auto in_func = typeid_cast<const ASTFunction *>(element.in_function.get());
|
||||
const ASTs & args = typeid_cast<const ASTExpressionList &>(*in_func->arguments).children;
|
||||
PreparedSets::const_iterator it = prepared_sets.find(args[1].get());
|
||||
PreparedSets::const_iterator it = prepared_sets.find(args[1]->range);
|
||||
if (in_func && it != prepared_sets.end())
|
||||
{
|
||||
rpn_stack.emplace_back(element.set_index->mayBeTrueInRange(key_ranges, data_types));
|
||||
|
@ -14,7 +14,7 @@ Block MergeTreeBlockOutputStream::getHeader() const
|
||||
|
||||
void MergeTreeBlockOutputStream::write(const Block & block)
|
||||
{
|
||||
storage.data.delayInsertIfNeeded();
|
||||
storage.data.delayInsertOrThrowIfNeeded();
|
||||
|
||||
auto part_blocks = storage.writer.splitBlockIntoParts(block);
|
||||
for (auto & current_block : part_blocks)
|
||||
@ -24,7 +24,7 @@ void MergeTreeBlockOutputStream::write(const Block & block)
|
||||
MergeTreeData::MutableDataPartPtr part = storage.writer.writeTempPart(current_block);
|
||||
storage.data.renameTempPartAndAdd(part, &storage.increment);
|
||||
|
||||
PartLog::addNewPartToTheLog(storage.context, *part, watch.elapsed());
|
||||
PartLog::addNewPart(storage.context, part, watch.elapsed());
|
||||
|
||||
/// Initiate async merge - it will be done if it's good time for merge and if there are space in 'background_pool'.
|
||||
storage.merge_task_handle->wake();
|
||||
|
@ -3,6 +3,8 @@
|
||||
#include <Storages/MergeTree/MergeTreeBlockInputStream.h>
|
||||
#include <Storages/MergeTree/MergedBlockOutputStream.h>
|
||||
#include <Storages/MergeTree/checkDataPart.h>
|
||||
#include <Storages/StorageMergeTree.h>
|
||||
#include <Storages/StorageReplicatedMergeTree.h>
|
||||
#include <Storages/AlterCommands.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTNameTypePair.h>
|
||||
@ -11,6 +13,7 @@
|
||||
#include <Parsers/ASTPartition.h>
|
||||
#include <Parsers/ExpressionListParsers.h>
|
||||
#include <Parsers/parseQuery.h>
|
||||
#include <Parsers/queryToString.h>
|
||||
#include <DataStreams/ExpressionBlockInputStream.h>
|
||||
#include <DataStreams/ValuesRowInputStream.h>
|
||||
#include <DataStreams/copyData.h>
|
||||
@ -36,6 +39,7 @@
|
||||
#include <Common/Stopwatch.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/localBackup.h>
|
||||
#include <Interpreters/PartLog.h>
|
||||
|
||||
#include <Poco/DirectoryIterator.h>
|
||||
|
||||
@ -47,7 +51,6 @@
|
||||
#include <typeinfo>
|
||||
#include <typeindex>
|
||||
#include <optional>
|
||||
#include <Interpreters/PartLog.h>
|
||||
|
||||
|
||||
namespace ProfileEvents
|
||||
@ -70,11 +73,11 @@ namespace ErrorCodes
|
||||
{
|
||||
extern const int MEMORY_LIMIT_EXCEEDED;
|
||||
extern const int SYNTAX_ERROR;
|
||||
extern const int CORRUPTED_DATA;
|
||||
extern const int INVALID_PARTITION_VALUE;
|
||||
extern const int METADATA_MISMATCH;
|
||||
extern const int PART_IS_TEMPORARILY_LOCKED;
|
||||
extern const int TOO_MANY_PARTS;
|
||||
extern const int INCOMPATIBLE_COLUMNS;
|
||||
}
|
||||
|
||||
|
||||
@ -550,7 +553,7 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks)
|
||||
for (auto & part : broken_parts_to_remove)
|
||||
part->remove();
|
||||
for (auto & part : broken_parts_to_detach)
|
||||
part->renameAddPrefix(true, "");
|
||||
part->renameToDetached("");
|
||||
|
||||
/// Delete from the set of current parts those parts that are covered by another part (those parts that
|
||||
/// were merged), but that for some reason are still not deleted from the filesystem.
|
||||
@ -1367,10 +1370,10 @@ MergeTreeData::DataPartsVector MergeTreeData::getActivePartsToReplace(
|
||||
const MergeTreePartInfo & new_part_info,
|
||||
const String & new_part_name,
|
||||
DataPartPtr & out_covering_part,
|
||||
std::lock_guard<std::mutex> & /* data_parts_lock */) const
|
||||
DataPartsLock & /* data_parts_lock */) const
|
||||
{
|
||||
/// Parts contained in the part are consecutive in data_parts, intersecting the insertion place for the part itself.
|
||||
auto it_middle = data_parts_by_state_and_info.lower_bound(DataPartStateAndInfo(DataPartState::Committed, new_part_info));
|
||||
auto it_middle = data_parts_by_state_and_info.lower_bound(DataPartStateAndInfo{DataPartState::Committed, new_part_info});
|
||||
auto committed_parts_range = getDataPartsStateRange(DataPartState::Committed);
|
||||
|
||||
/// Go to the left.
|
||||
@ -1435,16 +1438,10 @@ void MergeTreeData::renameTempPartAndAdd(MutableDataPartPtr & part, SimpleIncrem
|
||||
}
|
||||
|
||||
|
||||
|
||||
MergeTreeData::DataPartsVector MergeTreeData::renameTempPartAndReplace(
|
||||
MutableDataPartPtr & part, SimpleIncrement * increment, Transaction * out_transaction)
|
||||
void MergeTreeData::renameTempPartAndReplace(
|
||||
MutableDataPartPtr & part, SimpleIncrement * increment, MergeTreeData::Transaction * out_transaction,
|
||||
std::unique_lock<std::mutex> & lock, DataPartsVector * out_covered_parts)
|
||||
{
|
||||
if (out_transaction && out_transaction->data && out_transaction->data != this)
|
||||
throw Exception("The same MergeTreeData::Transaction cannot be used for different tables",
|
||||
ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
std::lock_guard<std::mutex> lock(data_parts_mutex);
|
||||
|
||||
part->assertState({DataPartState::Temporary});
|
||||
|
||||
MergeTreePartInfo part_info = part->info;
|
||||
@ -1489,7 +1486,7 @@ MergeTreeData::DataPartsVector MergeTreeData::renameTempPartAndReplace(
|
||||
if (covering_part)
|
||||
{
|
||||
LOG_WARNING(log, "Tried to add obsolete part " << part_name << " covered by " << covering_part->getNameWithState());
|
||||
return {};
|
||||
return;
|
||||
}
|
||||
|
||||
/// All checks are passed. Now we can rename the part on disk.
|
||||
@ -1523,12 +1520,48 @@ MergeTreeData::DataPartsVector MergeTreeData::renameTempPartAndReplace(
|
||||
addPartContributionToColumnSizes(part);
|
||||
}
|
||||
|
||||
if (out_covered_parts)
|
||||
{
|
||||
for (DataPartPtr & covered_part : covered_parts)
|
||||
out_covered_parts->emplace_back(std::move(covered_part));
|
||||
}
|
||||
}
|
||||
|
||||
MergeTreeData::DataPartsVector MergeTreeData::renameTempPartAndReplace(
|
||||
MutableDataPartPtr & part, SimpleIncrement * increment, Transaction * out_transaction)
|
||||
{
|
||||
if (out_transaction && out_transaction->data && out_transaction->data != this)
|
||||
throw Exception("The same MergeTreeData::Transaction cannot be used for different tables",
|
||||
ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
DataPartsVector covered_parts;
|
||||
{
|
||||
std::unique_lock<std::mutex> lock(data_parts_mutex);
|
||||
renameTempPartAndReplace(part, increment, out_transaction, lock, &covered_parts);
|
||||
}
|
||||
return covered_parts;
|
||||
}
|
||||
|
||||
void MergeTreeData::removePartsFromWorkingSet(const DataPartsVector & remove, bool clear_without_timeout)
|
||||
void MergeTreeData::removePartsFromWorkingSet(const MergeTreeData::DataPartsVector & remove, bool clear_without_timeout, DataPartsLock & /*acquired_lock*/)
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(data_parts_mutex);
|
||||
auto remove_time = clear_without_timeout ? 0 : time(nullptr);
|
||||
|
||||
for (const DataPartPtr & part : remove)
|
||||
{
|
||||
if (part->state == MergeTreeDataPart::State::Committed)
|
||||
removePartContributionToColumnSizes(part);
|
||||
|
||||
if (part->state == MergeTreeDataPart::State::Committed || clear_without_timeout)
|
||||
part->remove_time.store(remove_time, std::memory_order_relaxed);
|
||||
|
||||
if (part->state != MergeTreeDataPart::State::Outdated)
|
||||
modifyPartState(part, MergeTreeDataPart::State::Outdated);
|
||||
}
|
||||
}
|
||||
|
||||
void MergeTreeData::removePartsFromWorkingSet(const DataPartsVector & remove, bool clear_without_timeout, DataPartsLock * acquired_lock)
|
||||
{
|
||||
auto lock = (acquired_lock) ? DataPartsLock() : lockParts();
|
||||
|
||||
for (auto & part : remove)
|
||||
{
|
||||
@ -1538,37 +1571,83 @@ void MergeTreeData::removePartsFromWorkingSet(const DataPartsVector & remove, bo
|
||||
part->assertState({DataPartState::PreCommitted, DataPartState::Committed, DataPartState::Outdated});
|
||||
}
|
||||
|
||||
auto remove_time = clear_without_timeout ? 0 : time(nullptr);
|
||||
for (const DataPartPtr & part : remove)
|
||||
{
|
||||
if (part->state == DataPartState::Committed)
|
||||
removePartContributionToColumnSizes(part);
|
||||
|
||||
modifyPartState(part, DataPartState::Outdated);
|
||||
part->remove_time.store(remove_time, std::memory_order_relaxed);
|
||||
}
|
||||
removePartsFromWorkingSet(remove, clear_without_timeout, lock);
|
||||
}
|
||||
|
||||
|
||||
void MergeTreeData::renameAndDetachPart(const DataPartPtr & part_to_detach, const String & prefix, bool restore_covered,
|
||||
bool move_to_detached)
|
||||
MergeTreeData::DataPartsVector MergeTreeData::removePartsInRangeFromWorkingSet(const MergeTreePartInfo & drop_range, bool clear_without_timeout,
|
||||
bool skip_intersecting_parts, DataPartsLock & lock)
|
||||
{
|
||||
LOG_INFO(log, "Renaming " << part_to_detach->relative_path << " to " << prefix << part_to_detach->name << " and detaching it.");
|
||||
DataPartsVector parts_to_remove;
|
||||
|
||||
std::lock_guard<std::mutex> lock(data_parts_mutex);
|
||||
if (drop_range.min_block > drop_range.max_block)
|
||||
return parts_to_remove;
|
||||
|
||||
auto partition_range = getDataPartsPartitionRange(drop_range.partition_id);
|
||||
|
||||
for (const DataPartPtr & part : partition_range)
|
||||
{
|
||||
if (part->info.partition_id != drop_range.partition_id)
|
||||
throw Exception("Unexpected partition_id of part " + part->name + ". This is a bug.", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
if (part->info.min_block < drop_range.min_block)
|
||||
{
|
||||
if (drop_range.min_block <= part->info.max_block)
|
||||
{
|
||||
/// Intersect left border
|
||||
String error = "Unexpected merged part " + part->name + " intersecting drop range " + drop_range.getPartName();
|
||||
if (!skip_intersecting_parts)
|
||||
throw Exception(error, ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
LOG_WARNING(log, error);
|
||||
}
|
||||
|
||||
continue;
|
||||
}
|
||||
|
||||
/// Stop on new parts
|
||||
if (part->info.min_block > drop_range.max_block)
|
||||
break;
|
||||
|
||||
if (part->info.min_block <= drop_range.max_block && drop_range.max_block < part->info.max_block)
|
||||
{
|
||||
/// Intersect right border
|
||||
String error = "Unexpected merged part " + part->name + " intersecting drop range " + drop_range.getPartName();
|
||||
if (!skip_intersecting_parts)
|
||||
throw Exception(error, ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
LOG_WARNING(log, error);
|
||||
continue;
|
||||
}
|
||||
|
||||
if (part->state != DataPartState::Deleting)
|
||||
parts_to_remove.emplace_back(part);
|
||||
}
|
||||
|
||||
removePartsFromWorkingSet(parts_to_remove, clear_without_timeout, lock);
|
||||
|
||||
return parts_to_remove;
|
||||
}
|
||||
|
||||
void MergeTreeData::forgetPartAndMoveToDetached(const MergeTreeData::DataPartPtr & part_to_detach, const String & prefix, bool
|
||||
restore_covered)
|
||||
{
|
||||
LOG_INFO(log, "Renaming " << part_to_detach->relative_path << " to " << prefix << part_to_detach->name << " and forgiving it.");
|
||||
|
||||
auto data_parts_lock = lockParts();
|
||||
|
||||
auto it_part = data_parts_by_info.find(part_to_detach->info);
|
||||
if (it_part == data_parts_by_info.end())
|
||||
throw Exception("No such data part " + part_to_detach->getNameWithState(), ErrorCodes::NO_SUCH_DATA_PART);
|
||||
|
||||
/// What if part_to_detach is reference to *it_part? Make a new owner just in case.
|
||||
/// What if part_to_detach is a reference to *it_part? Make a new owner just in case.
|
||||
DataPartPtr part = *it_part;
|
||||
|
||||
if (part->state == DataPartState::Committed)
|
||||
removePartContributionToColumnSizes(part);
|
||||
modifyPartState(it_part, DataPartState::Deleting);
|
||||
if (move_to_detached || !prefix.empty())
|
||||
part->renameAddPrefix(move_to_detached, prefix);
|
||||
|
||||
part->renameToDetached(prefix);
|
||||
|
||||
data_parts_indexes.erase(it_part);
|
||||
|
||||
if (restore_covered && part->info.level == 0)
|
||||
@ -1708,7 +1787,7 @@ size_t MergeTreeData::getMaxPartsCountForPartition() const
|
||||
}
|
||||
|
||||
|
||||
void MergeTreeData::delayInsertIfNeeded(Poco::Event * until)
|
||||
void MergeTreeData::delayInsertOrThrowIfNeeded(Poco::Event *until) const
|
||||
{
|
||||
const size_t parts_count = getMaxPartsCountForPartition();
|
||||
if (parts_count < settings.parts_to_delay_insert)
|
||||
@ -1738,20 +1817,28 @@ void MergeTreeData::delayInsertIfNeeded(Poco::Event * until)
|
||||
std::this_thread::sleep_for(std::chrono::milliseconds(static_cast<size_t>(delay_milliseconds)));
|
||||
}
|
||||
|
||||
MergeTreeData::DataPartPtr MergeTreeData::getActiveContainingPart(const String & part_name)
|
||||
void MergeTreeData::throwInsertIfNeeded() const
|
||||
{
|
||||
auto part_info = MergeTreePartInfo::fromPartName(part_name, format_version);
|
||||
const size_t parts_count = getMaxPartsCountForPartition();
|
||||
|
||||
std::lock_guard<std::mutex> lock(data_parts_mutex);
|
||||
if (parts_count >= settings.parts_to_throw_insert)
|
||||
{
|
||||
ProfileEvents::increment(ProfileEvents::RejectedInserts);
|
||||
throw Exception("Too many parts (" + toString(parts_count) + "). Merges are processing significantly slower than inserts.", ErrorCodes::TOO_MANY_PARTS);
|
||||
}
|
||||
}
|
||||
|
||||
auto committed_parts_range = getDataPartsStateRange(DataPartState::Committed);
|
||||
MergeTreeData::DataPartPtr MergeTreeData::getActiveContainingPart(
|
||||
const MergeTreePartInfo & part_info, MergeTreeData::DataPartState state, DataPartsLock & /*lock*/)
|
||||
{
|
||||
auto committed_parts_range = getDataPartsStateRange(state);
|
||||
|
||||
/// The part can be covered only by the previous or the next one in data_parts.
|
||||
auto it = data_parts_by_state_and_info.lower_bound(DataPartStateAndInfo(DataPartState::Committed, part_info));
|
||||
auto it = data_parts_by_state_and_info.lower_bound(DataPartStateAndInfo{state, part_info});
|
||||
|
||||
if (it != committed_parts_range.end())
|
||||
{
|
||||
if ((*it)->name == part_name)
|
||||
if ((*it)->info == part_info)
|
||||
return *it;
|
||||
if ((*it)->info.contains(part_info))
|
||||
return *it;
|
||||
@ -1767,11 +1854,28 @@ MergeTreeData::DataPartPtr MergeTreeData::getActiveContainingPart(const String &
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
|
||||
MergeTreeData::DataPartPtr MergeTreeData::getPartIfExists(const String & part_name, const MergeTreeData::DataPartStates & valid_states)
|
||||
MergeTreeData::DataPartPtr MergeTreeData::getActiveContainingPart(const String & part_name)
|
||||
{
|
||||
auto part_info = MergeTreePartInfo::fromPartName(part_name, format_version);
|
||||
|
||||
DataPartsLock data_parts_lock(data_parts_mutex);
|
||||
return getActiveContainingPart(part_info, DataPartState::Committed, data_parts_lock);
|
||||
}
|
||||
|
||||
|
||||
MergeTreeData::DataPartsVector MergeTreeData::getDataPartsVectorInPartition(MergeTreeData::DataPartState state, const String & partition_id)
|
||||
{
|
||||
DataPartStateAndPartitionID state_with_partition{state, partition_id};
|
||||
|
||||
std::lock_guard<std::mutex> lock(data_parts_mutex);
|
||||
return DataPartsVector(
|
||||
data_parts_by_state_and_info.lower_bound(state_with_partition),
|
||||
data_parts_by_state_and_info.upper_bound(state_with_partition));
|
||||
}
|
||||
|
||||
|
||||
MergeTreeData::DataPartPtr MergeTreeData::getPartIfExists(const MergeTreePartInfo & part_info, const MergeTreeData::DataPartStates & valid_states)
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(data_parts_mutex);
|
||||
|
||||
auto it = data_parts_by_info.find(part_info);
|
||||
@ -1787,6 +1891,11 @@ MergeTreeData::DataPartPtr MergeTreeData::getPartIfExists(const String & part_na
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
MergeTreeData::DataPartPtr MergeTreeData::getPartIfExists(const String & part_name, const MergeTreeData::DataPartStates & valid_states)
|
||||
{
|
||||
return getPartIfExists(MergeTreePartInfo::fromPartName(part_name, format_version), valid_states);
|
||||
}
|
||||
|
||||
|
||||
MergeTreeData::MutableDataPartPtr MergeTreeData::loadPartAndFixMetadata(const String & relative_path)
|
||||
{
|
||||
@ -2012,7 +2121,7 @@ String MergeTreeData::getPartitionIDFromQuery(const ASTPtr & ast, const Context
|
||||
String partition_id = partition.getID(*this);
|
||||
|
||||
{
|
||||
std::lock_guard<std::mutex> data_parts_lock(data_parts_mutex);
|
||||
DataPartsLock data_parts_lock(data_parts_mutex);
|
||||
DataPartPtr existing_part_in_partition = getAnyPartInPartition(partition_id, data_parts_lock);
|
||||
if (existing_part_in_partition && existing_part_in_partition->partition.value != partition.value)
|
||||
{
|
||||
@ -2098,12 +2207,9 @@ MergeTreeData::DataPartsVector MergeTreeData::getDataPartsVector() const
|
||||
}
|
||||
|
||||
MergeTreeData::DataPartPtr MergeTreeData::getAnyPartInPartition(
|
||||
const String & partition_id, std::lock_guard<std::mutex> & /*data_parts_lock*/)
|
||||
const String & partition_id, DataPartsLock & /*data_parts_lock*/)
|
||||
{
|
||||
auto min_block = std::numeric_limits<Int64>::min();
|
||||
MergeTreePartInfo dummy_part_info(partition_id, min_block, min_block, 0);
|
||||
|
||||
auto it = data_parts_by_state_and_info.lower_bound(DataPartStateAndInfo(DataPartState::Committed, dummy_part_info));
|
||||
auto it = data_parts_by_state_and_info.lower_bound(DataPartStateAndPartitionID{DataPartState::Committed, partition_id});
|
||||
|
||||
if (it != data_parts_by_state_and_info.end() && (*it)->state == DataPartState::Committed && (*it)->info.partition_id == partition_id)
|
||||
return *it;
|
||||
@ -2130,19 +2236,20 @@ void MergeTreeData::Transaction::rollback()
|
||||
clear();
|
||||
}
|
||||
|
||||
MergeTreeData::DataPartsVector MergeTreeData::Transaction::commit()
|
||||
MergeTreeData::DataPartsVector MergeTreeData::Transaction::commit(MergeTreeData::DataPartsLock * acquired_parts_lock)
|
||||
{
|
||||
DataPartsVector total_covered_parts;
|
||||
|
||||
if (!isEmpty())
|
||||
{
|
||||
std::lock_guard<std::mutex> data_parts_lock(data->data_parts_mutex);
|
||||
auto parts_lock = acquired_parts_lock ? MergeTreeData::DataPartsLock() : data->lockParts();
|
||||
auto owing_parts_lock = acquired_parts_lock ? acquired_parts_lock : &parts_lock;
|
||||
|
||||
auto current_time = time(nullptr);
|
||||
for (const DataPartPtr & part : precommitted_parts)
|
||||
{
|
||||
DataPartPtr covering_part;
|
||||
DataPartsVector covered_parts = data->getActivePartsToReplace(part->info, part->name, covering_part, data_parts_lock);
|
||||
DataPartsVector covered_parts = data->getActivePartsToReplace(part->info, part->name, covering_part, *owing_parts_lock);
|
||||
if (covering_part)
|
||||
{
|
||||
LOG_WARNING(data->log, "Tried to commit obsolete part " << part->name
|
||||
@ -2212,4 +2319,67 @@ bool MergeTreeData::mayBenefitFromIndexForIn(const ASTPtr & left_in_operand) con
|
||||
}
|
||||
}
|
||||
|
||||
MergeTreeData * MergeTreeData::checkStructureAndGetMergeTreeData(const StoragePtr & source_table) const
|
||||
{
|
||||
MergeTreeData * src_data;
|
||||
if (auto storage_merge_tree = dynamic_cast<StorageMergeTree *>(source_table.get()))
|
||||
src_data = &storage_merge_tree->data;
|
||||
else if (auto storage_replicated_merge_tree = dynamic_cast<StorageReplicatedMergeTree *>(source_table.get()))
|
||||
src_data = &storage_replicated_merge_tree->data;
|
||||
else
|
||||
{
|
||||
throw Exception("Table " + table_name + " supports attachPartitionFrom only for MergeTree or ReplicatedMergeTree engines."
|
||||
" Got " + source_table->getName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
if (getColumns().getAllPhysical().sizeOfDifference(src_data->getColumns().getAllPhysical()))
|
||||
throw Exception("Tables have different structure", ErrorCodes::INCOMPATIBLE_COLUMNS);
|
||||
|
||||
auto query_to_string = [] (const ASTPtr & ast)
|
||||
{
|
||||
return ast ? queryToString(ast) : "";
|
||||
};
|
||||
|
||||
if (query_to_string(secondary_sort_expr_ast) != query_to_string(secondary_sort_expr_ast))
|
||||
throw Exception("Tables have different ordering", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
if (query_to_string(partition_expr_ast) != query_to_string(src_data->partition_expr_ast))
|
||||
throw Exception("Tables have different partition key", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
if (format_version != src_data->format_version)
|
||||
throw Exception("Tables have different format_version", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
return src_data;
|
||||
}
|
||||
|
||||
MergeTreeData::MutableDataPartPtr MergeTreeData::cloneAndLoadDataPart(const MergeTreeData::DataPartPtr & src_part,
|
||||
const String & tmp_part_prefix,
|
||||
const MergeTreePartInfo & dst_part_info)
|
||||
{
|
||||
String dst_part_name;
|
||||
if (format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING)
|
||||
dst_part_name = dst_part_info.getPartNameV0(src_part->getMinDate(), src_part->getMaxDate());
|
||||
else
|
||||
dst_part_name = dst_part_info.getPartName();
|
||||
|
||||
String tmp_dst_part_name = tmp_part_prefix + dst_part_name;
|
||||
|
||||
Poco::Path dst_part_absolute_path = Poco::Path(full_path + tmp_dst_part_name).absolute();
|
||||
Poco::Path src_part_absolute_path = Poco::Path(src_part->getFullPath()).absolute();
|
||||
|
||||
if (Poco::File(dst_part_absolute_path).exists())
|
||||
throw Exception("Part in " + dst_part_absolute_path.toString() + " already exists", ErrorCodes::DIRECTORY_ALREADY_EXISTS);
|
||||
|
||||
LOG_DEBUG(log, "Cloning part " << src_part_absolute_path.toString() << " to " << dst_part_absolute_path.toString());
|
||||
localBackup(src_part_absolute_path, dst_part_absolute_path);
|
||||
|
||||
MergeTreeData::MutableDataPartPtr dst_data_part = std::make_shared<MergeTreeData::DataPart>(*this, dst_part_name, dst_part_info);
|
||||
dst_data_part->relative_path = tmp_dst_part_name;
|
||||
dst_data_part->is_temp = true;
|
||||
|
||||
dst_data_part->loadColumnsChecksumsIndexes(require_part_metadata, true);
|
||||
dst_data_part->modification_time = Poco::File(dst_part_absolute_path).getLastModified().epochTime();
|
||||
return dst_data_part;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -20,6 +20,8 @@
|
||||
#include <boost/multi_index/ordered_index.hpp>
|
||||
#include <boost/multi_index/global_fun.hpp>
|
||||
#include <boost/range/iterator_range_core.hpp>
|
||||
#include "../../Core/Types.h"
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -94,6 +96,7 @@ public:
|
||||
using DataPart = MergeTreeDataPart;
|
||||
|
||||
using MutableDataPartPtr = std::shared_ptr<DataPart>;
|
||||
using MutableDataPartsVector = std::vector<MutableDataPartPtr>;
|
||||
/// After the DataPart is added to the working set, it cannot be changed.
|
||||
using DataPartPtr = std::shared_ptr<const DataPart>;
|
||||
|
||||
@ -106,10 +109,17 @@ public:
|
||||
{
|
||||
DataPartState state;
|
||||
const MergeTreePartInfo & info;
|
||||
|
||||
DataPartStateAndInfo(DataPartState state, const MergeTreePartInfo & info) : state(state), info(info) {}
|
||||
};
|
||||
|
||||
/// Auxiliary structure for index comparison
|
||||
struct DataPartStateAndPartitionID
|
||||
{
|
||||
DataPartState state;
|
||||
String partition_id;
|
||||
};
|
||||
|
||||
STRONG_TYPEDEF(String, PartitionID);
|
||||
|
||||
struct LessDataPart
|
||||
{
|
||||
using is_transparent = void;
|
||||
@ -117,6 +127,8 @@ public:
|
||||
bool operator()(const DataPartPtr & lhs, const MergeTreePartInfo & rhs) const { return lhs->info < rhs; }
|
||||
bool operator()(const MergeTreePartInfo & lhs, const DataPartPtr & rhs) const { return lhs < rhs->info; }
|
||||
bool operator()(const DataPartPtr & lhs, const DataPartPtr & rhs) const { return lhs->info < rhs->info; }
|
||||
bool operator()(const MergeTreePartInfo & lhs, const PartitionID & rhs) const { return lhs.partition_id < rhs.toUnderType(); }
|
||||
bool operator()(const PartitionID & lhs, const MergeTreePartInfo & rhs) const { return lhs.toUnderType() < rhs.partition_id; }
|
||||
};
|
||||
|
||||
struct LessStateDataPart
|
||||
@ -138,11 +150,26 @@ public:
|
||||
{
|
||||
return static_cast<size_t>(state) < static_cast<size_t>(info.state);
|
||||
}
|
||||
|
||||
bool operator() (const DataPartStateAndInfo & lhs, const DataPartStateAndPartitionID & rhs) const
|
||||
{
|
||||
return std::forward_as_tuple(static_cast<UInt8>(lhs.state), lhs.info.partition_id)
|
||||
< std::forward_as_tuple(static_cast<UInt8>(rhs.state), rhs.partition_id);
|
||||
}
|
||||
|
||||
bool operator() (const DataPartStateAndPartitionID & lhs, const DataPartStateAndInfo & rhs) const
|
||||
{
|
||||
return std::forward_as_tuple(static_cast<UInt8>(lhs.state), lhs.partition_id)
|
||||
< std::forward_as_tuple(static_cast<UInt8>(rhs.state), rhs.info.partition_id);
|
||||
}
|
||||
};
|
||||
|
||||
using DataParts = std::set<DataPartPtr, LessDataPart>;
|
||||
using DataPartsVector = std::vector<DataPartPtr>;
|
||||
|
||||
using DataPartsLock = std::unique_lock<std::mutex>;
|
||||
DataPartsLock lockParts() const { return DataPartsLock(data_parts_mutex); }
|
||||
|
||||
/// Auxiliary object to add a set of parts into the working set in two steps:
|
||||
/// * First, as PreCommitted parts (the parts are ready, but not yet in the active set).
|
||||
/// * Next, if commit() is called, the parts are added to the active set and the parts that are
|
||||
@ -153,8 +180,7 @@ public:
|
||||
public:
|
||||
Transaction() {}
|
||||
|
||||
/// Return parts marked Obsolete as a result of the transaction commit.
|
||||
DataPartsVector commit();
|
||||
DataPartsVector commit(MergeTreeData::DataPartsLock * acquired_parts_lock = nullptr);
|
||||
|
||||
void rollback();
|
||||
|
||||
@ -213,7 +239,7 @@ public:
|
||||
}
|
||||
|
||||
DataPartPtr data_part;
|
||||
std::unique_lock<std::mutex> alter_lock;
|
||||
DataPartsLock alter_lock;
|
||||
|
||||
DataPart::Checksums new_checksums;
|
||||
NamesAndTypesList new_columns;
|
||||
@ -345,9 +371,14 @@ public:
|
||||
|
||||
/// Returns a committed part with the given name or a part containing it. If there is no such part, returns nullptr.
|
||||
DataPartPtr getActiveContainingPart(const String & part_name);
|
||||
DataPartPtr getActiveContainingPart(const MergeTreePartInfo & part_info, DataPartState state, DataPartsLock &lock);
|
||||
|
||||
/// Returns all parts in specified partition
|
||||
DataPartsVector getDataPartsVectorInPartition(DataPartState state, const String & partition_id);
|
||||
|
||||
/// Returns the part with the given name and state or nullptr if no such part.
|
||||
DataPartPtr getPartIfExists(const String & part_name, const DataPartStates & valid_states);
|
||||
DataPartPtr getPartIfExists(const MergeTreePartInfo & part_info, const DataPartStates & valid_states);
|
||||
|
||||
/// Total size of active parts in bytes.
|
||||
size_t getTotalActiveSizeInBytes() const;
|
||||
@ -356,7 +387,8 @@ public:
|
||||
|
||||
/// If the table contains too many active parts, sleep for a while to give them time to merge.
|
||||
/// If until is non-null, wake up from the sleep earlier if the event happened.
|
||||
void delayInsertIfNeeded(Poco::Event * until = nullptr);
|
||||
void delayInsertOrThrowIfNeeded(Poco::Event *until = nullptr) const;
|
||||
void throwInsertIfNeeded() const;
|
||||
|
||||
/// Renames temporary part to a permanent part and adds it to the parts set.
|
||||
/// It is assumed that the part does not intersect with existing parts.
|
||||
@ -372,16 +404,32 @@ public:
|
||||
DataPartsVector renameTempPartAndReplace(
|
||||
MutableDataPartPtr & part, SimpleIncrement * increment = nullptr, Transaction * out_transaction = nullptr);
|
||||
|
||||
/// Low-level version of previous one, doesn't lock mutex
|
||||
void renameTempPartAndReplace(
|
||||
MutableDataPartPtr & part, SimpleIncrement * increment, Transaction * out_transaction, DataPartsLock & lock,
|
||||
DataPartsVector * out_covered_parts = nullptr);
|
||||
|
||||
/// Removes parts from the working set parts.
|
||||
/// Parts in add must already be in data_parts with PreCommitted, Committed, or Outdated states.
|
||||
/// If clear_without_timeout is true, the parts will be deleted at once, or during the next call to
|
||||
/// clearOldParts (ignoring old_parts_lifetime).
|
||||
void removePartsFromWorkingSet(const DataPartsVector & remove, bool clear_without_timeout);
|
||||
void removePartsFromWorkingSet(const DataPartsVector & remove, bool clear_without_timeout, DataPartsLock * acquired_lock = nullptr);
|
||||
void removePartsFromWorkingSet(const DataPartsVector & remove, bool clear_without_timeout, DataPartsLock & acquired_lock);
|
||||
|
||||
/// Renames the part to detached/<prefix>_<part> and forgets about it. The data won't be deleted in
|
||||
/// clearOldParts.
|
||||
/// Removes all parts from the working set parts
|
||||
/// for which (partition_id = drop_range.partition_id && min_block >= drop_range.min_block && max_block <= drop_range.max_block).
|
||||
/// If a part intersecting drop_range.max_block is found, an exception will be thrown.
|
||||
/// Used in REPLACE PARTITION command;
|
||||
DataPartsVector removePartsInRangeFromWorkingSet(const MergeTreePartInfo & drop_range, bool clear_without_timeout,
|
||||
bool skip_intersecting_parts, DataPartsLock & lock);
|
||||
|
||||
/// Renames the part to detached/<prefix>_<part> and removes it from working set.
|
||||
void removePartsFromWorkingSetAndCloneToDetached(const DataPartsVector & parts, bool clear_without_timeout, const String & prefix = "");
|
||||
|
||||
/// Renames the part to detached/<prefix>_<part> and removes it from data_parts,
|
||||
//// so it will not be deleted in clearOldParts.
|
||||
/// If restore_covered is true, adds to the working set inactive parts, which were merged into the deleted part.
|
||||
void renameAndDetachPart(const DataPartPtr & part, const String & prefix = "", bool restore_covered = false, bool move_to_detached = true);
|
||||
void forgetPartAndMoveToDetached(const DataPartPtr & part, const String & prefix = "", bool restore_covered = false);
|
||||
|
||||
/// Returns old inactive parts that can be deleted. At the same time removes them from the list of parts
|
||||
/// but not from the disk.
|
||||
@ -475,6 +523,13 @@ public:
|
||||
/// For ATTACH/DETACH/DROP PARTITION.
|
||||
String getPartitionIDFromQuery(const ASTPtr & partition, const Context & context);
|
||||
|
||||
/// Extracts MergeTreeData of other *MergeTree* storage
|
||||
/// and checks that their structure suitable for ALTER TABLE ATTACH PARTITION FROM
|
||||
/// Tables structure should be locked.
|
||||
MergeTreeData * checkStructureAndGetMergeTreeData(const StoragePtr & source_table) const;
|
||||
|
||||
MergeTreeData::MutableDataPartPtr cloneAndLoadDataPart(const MergeTreeData::DataPartPtr & src_part, const String & tmp_part_prefix,
|
||||
const MergeTreePartInfo & dst_part_info);
|
||||
|
||||
MergeTreeDataFormatVersion format_version;
|
||||
|
||||
@ -513,6 +568,8 @@ private:
|
||||
friend class StorageMergeTree;
|
||||
friend class ReplicatedMergeTreeAlterThread;
|
||||
friend class MergeTreeDataMergerMutator;
|
||||
friend class StorageMergeTree;
|
||||
friend class StorageReplicatedMergeTree;
|
||||
|
||||
bool require_part_metadata;
|
||||
|
||||
@ -585,6 +642,13 @@ private:
|
||||
return {begin, end};
|
||||
}
|
||||
|
||||
boost::iterator_range<DataPartIteratorByInfo> getDataPartsPartitionRange(const String & partition_id) const
|
||||
{
|
||||
auto begin = data_parts_by_info.lower_bound(PartitionID(partition_id), LessDataPart());
|
||||
auto end = data_parts_by_info.upper_bound(PartitionID(partition_id), LessDataPart());
|
||||
return {begin, end};
|
||||
}
|
||||
|
||||
static decltype(auto) getStateModifier(DataPartState state)
|
||||
{
|
||||
return [state] (const DataPartPtr & part) { part->state = state; };
|
||||
@ -639,7 +703,7 @@ private:
|
||||
void removePartContributionToColumnSizes(const DataPartPtr & part);
|
||||
|
||||
/// If there is no part in the partition with ID `partition_id`, returns empty ptr. Should be called under the lock.
|
||||
DataPartPtr getAnyPartInPartition(const String & partition_id, std::lock_guard<std::mutex> & data_parts_lock);
|
||||
DataPartPtr getAnyPartInPartition(const String & partition_id, DataPartsLock & data_parts_lock);
|
||||
|
||||
/// Return parts in the Committed set that are covered by the new_part_info or the part that covers it.
|
||||
/// Will check that the new part doesn't already exist and that it doesn't intersect existing part.
|
||||
@ -647,7 +711,7 @@ private:
|
||||
const MergeTreePartInfo & new_part_info,
|
||||
const String & new_part_name,
|
||||
DataPartPtr & out_covering_part,
|
||||
std::lock_guard<std::mutex> & data_parts_lock) const;
|
||||
DataPartsLock & data_parts_lock) const;
|
||||
|
||||
/// Checks whether the column is in the primary key, possibly wrapped in a chain of functions with single argument.
|
||||
bool isPrimaryOrMinMaxKeyColumnPossiblyWrappedInFunctions(const ASTPtr & node) const;
|
||||
|
@ -98,16 +98,16 @@ void MergeTreeDataMergerMutator::FuturePart::assign(MergeTreeData::DataPartsVect
|
||||
|
||||
if (parts.front()->storage.format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING)
|
||||
{
|
||||
DayNum_t min_date = DayNum_t(std::numeric_limits<UInt16>::max());
|
||||
DayNum_t max_date = DayNum_t(std::numeric_limits<UInt16>::min());
|
||||
DayNum min_date = DayNum(std::numeric_limits<UInt16>::max());
|
||||
DayNum max_date = DayNum(std::numeric_limits<UInt16>::min());
|
||||
for (const auto & part : parts)
|
||||
{
|
||||
/// NOTE: getting min and max dates from part names (instead of part data) because we want
|
||||
/// the merged part name be determined only by source part names.
|
||||
/// It is simpler this way when the real min and max dates for the block range can change
|
||||
/// (e.g. after an ALTER DELETE command).
|
||||
DayNum_t part_min_date;
|
||||
DayNum_t part_max_date;
|
||||
DayNum part_min_date;
|
||||
DayNum part_max_date;
|
||||
MergeTreePartInfo::parseMinMaxDatesFromPartName(part->name, part_min_date, part_max_date);
|
||||
min_date = std::min(min_date, part_min_date);
|
||||
max_date = std::max(max_date, part_max_date);
|
||||
|
@ -13,6 +13,7 @@
|
||||
#include <Common/SipHash.h>
|
||||
#include <Common/escapeForFileName.h>
|
||||
#include <Common/StringUtils/StringUtils.h>
|
||||
#include <Common/localBackup.h>
|
||||
#include <Storages/MergeTree/MergeTreeDataPart.h>
|
||||
#include <Storages/MergeTree/MergeTreeData.h>
|
||||
|
||||
@ -241,8 +242,8 @@ String MergeTreeDataPart::getNewName(const MergeTreePartInfo & new_part_info) co
|
||||
/// the merged part name be determined only by source part names.
|
||||
/// It is simpler this way when the real min and max dates for the block range can change
|
||||
/// (e.g. after an ALTER DELETE command).
|
||||
DayNum_t min_date;
|
||||
DayNum_t max_date;
|
||||
DayNum min_date;
|
||||
DayNum max_date;
|
||||
MergeTreePartInfo::parseMinMaxDatesFromPartName(name, min_date, max_date);
|
||||
return new_part_info.getPartNameV0(min_date, max_date);
|
||||
}
|
||||
@ -251,21 +252,21 @@ String MergeTreeDataPart::getNewName(const MergeTreePartInfo & new_part_info) co
|
||||
}
|
||||
|
||||
|
||||
DayNum_t MergeTreeDataPart::getMinDate() const
|
||||
DayNum MergeTreeDataPart::getMinDate() const
|
||||
{
|
||||
if (storage.minmax_idx_date_column_pos != -1 && minmax_idx.initialized)
|
||||
return DayNum_t(minmax_idx.min_values[storage.minmax_idx_date_column_pos].get<UInt64>());
|
||||
return DayNum(minmax_idx.min_values[storage.minmax_idx_date_column_pos].get<UInt64>());
|
||||
else
|
||||
return DayNum_t();
|
||||
return DayNum();
|
||||
}
|
||||
|
||||
|
||||
DayNum_t MergeTreeDataPart::getMaxDate() const
|
||||
DayNum MergeTreeDataPart::getMaxDate() const
|
||||
{
|
||||
if (storage.minmax_idx_date_column_pos != -1 && minmax_idx.initialized)
|
||||
return DayNum_t(minmax_idx.max_values[storage.minmax_idx_date_column_pos].get<UInt64>());
|
||||
return DayNum(minmax_idx.max_values[storage.minmax_idx_date_column_pos].get<UInt64>());
|
||||
else
|
||||
return DayNum_t();
|
||||
return DayNum();
|
||||
}
|
||||
|
||||
|
||||
@ -387,29 +388,45 @@ void MergeTreeDataPart::renameTo(const String & new_relative_path, bool remove_n
|
||||
}
|
||||
|
||||
|
||||
void MergeTreeDataPart::renameAddPrefix(bool to_detached, const String & prefix) const
|
||||
String MergeTreeDataPart::getRelativePathForDetachedPart(const String & prefix) const
|
||||
{
|
||||
String res;
|
||||
unsigned try_no = 0;
|
||||
auto dst_name = [&, this] { return (to_detached ? "detached/" : "") + prefix + name + (try_no ? "_try" + DB::toString(try_no) : ""); };
|
||||
auto dst_name = [&, this] { return "detached/" + prefix + name + (try_no ? "_try" + DB::toString(try_no) : ""); };
|
||||
|
||||
if (to_detached)
|
||||
/** If you need to detach a part, and directory into which we want to rename it already exists,
|
||||
* we will rename to the directory with the name to which the suffix is added in the form of "_tryN".
|
||||
* This is done only in the case of `to_detached`, because it is assumed that in this case the exact name does not matter.
|
||||
* No more than 10 attempts are made so that there are not too many junk directories left.
|
||||
*/
|
||||
while (try_no < 10)
|
||||
{
|
||||
/** If you need to detach a part, and directory into which we want to rename it already exists,
|
||||
* we will rename to the directory with the name to which the suffix is added in the form of "_tryN".
|
||||
* This is done only in the case of `to_detached`, because it is assumed that in this case the exact name does not matter.
|
||||
* No more than 10 attempts are made so that there are not too many junk directories left.
|
||||
*/
|
||||
while (try_no < 10 && Poco::File(storage.full_path + dst_name()).exists())
|
||||
{
|
||||
LOG_WARNING(storage.log, "Directory " << dst_name() << " (to detach to) is already exist."
|
||||
" Will detach to directory with '_tryN' suffix.");
|
||||
++try_no;
|
||||
}
|
||||
res = dst_name();
|
||||
|
||||
if (!Poco::File(storage.full_path + res).exists())
|
||||
return res;
|
||||
|
||||
LOG_WARNING(storage.log, "Directory " << dst_name() << " (to detach to) is already exist."
|
||||
" Will detach to directory with '_tryN' suffix.");
|
||||
++try_no;
|
||||
}
|
||||
|
||||
renameTo(dst_name());
|
||||
return res;
|
||||
}
|
||||
|
||||
void MergeTreeDataPart::renameToDetached(const String & prefix) const
|
||||
{
|
||||
renameTo(getRelativePathForDetachedPart(prefix));
|
||||
}
|
||||
|
||||
|
||||
void MergeTreeDataPart::makeCloneInDetached(const String & prefix) const
|
||||
{
|
||||
Poco::Path src(getFullPath());
|
||||
Poco::Path dst(storage.full_path + getRelativePathForDetachedPart(prefix));
|
||||
/// Backup is not recursive (max_level is 0), so do not copy inner directories
|
||||
localBackup(src, dst, 0);
|
||||
}
|
||||
|
||||
void MergeTreeDataPart::loadColumnsChecksumsIndexes(bool require_columns_checksums, bool check_consistency)
|
||||
{
|
||||
@ -473,8 +490,8 @@ void MergeTreeDataPart::loadPartitionAndMinMaxIndex()
|
||||
{
|
||||
if (storage.format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING)
|
||||
{
|
||||
DayNum_t min_date;
|
||||
DayNum_t max_date;
|
||||
DayNum min_date;
|
||||
DayNum max_date;
|
||||
MergeTreePartInfo::parseMinMaxDatesFromPartName(name, min_date, max_date);
|
||||
|
||||
const auto & date_lut = DateLUT::instance();
|
||||
|
@ -8,6 +8,8 @@
|
||||
#include <Storages/MergeTree/MergeTreeDataPartChecksum.h>
|
||||
#include <Columns/IColumn.h>
|
||||
#include <shared_mutex>
|
||||
#include "../../../../contrib/poco/Foundation/include/Poco/Path.h"
|
||||
#include "../../Core/Types.h"
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -65,8 +67,8 @@ struct MergeTreeDataPart
|
||||
bool contains(const MergeTreeDataPart & other) const { return info.contains(other.info); }
|
||||
|
||||
/// If the partition key includes date column (a common case), these functions will return min and max values for this column.
|
||||
DayNum_t getMinDate() const;
|
||||
DayNum_t getMaxDate() const;
|
||||
DayNum getMinDate() const;
|
||||
DayNum getMaxDate() const;
|
||||
|
||||
bool isEmpty() const { return rows_count == 0; }
|
||||
|
||||
@ -180,7 +182,7 @@ struct MergeTreeDataPart
|
||||
MinMaxIndex() = default;
|
||||
|
||||
/// For month-based partitioning.
|
||||
MinMaxIndex(DayNum_t min_date, DayNum_t max_date)
|
||||
MinMaxIndex(DayNum min_date, DayNum max_date)
|
||||
: min_values(1, static_cast<UInt64>(min_date))
|
||||
, max_values(1, static_cast<UInt64>(max_date))
|
||||
, initialized(true)
|
||||
@ -229,8 +231,14 @@ struct MergeTreeDataPart
|
||||
/// Changes only relative_dir_name, you need to update other metadata (name, is_temp) explicitly
|
||||
void renameTo(const String & new_relative_path, bool remove_new_dir_if_exists = true) const;
|
||||
|
||||
/// Renames a part by appending a prefix to the name. To_detached - also moved to the detached directory.
|
||||
void renameAddPrefix(bool to_detached, const String & prefix) const;
|
||||
/// Generate unique path to detach part
|
||||
String getRelativePathForDetachedPart(const String & prefix) const;
|
||||
|
||||
/// Moves a part to detached/ directory and adds prefix to its name
|
||||
void renameToDetached(const String & prefix) const;
|
||||
|
||||
/// Makes clone of a part in detached/ directory via hard links
|
||||
void makeCloneInDetached(const String & prefix) const;
|
||||
|
||||
/// Populates columns_to_size map (compressed size).
|
||||
void accumulateColumnSizes(ColumnToSize & column_to_size) const;
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include "MergeTreeDataPartChecksum.h"
|
||||
#include <Common/SipHash.h>
|
||||
#include <Common/hex.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/ReadBufferFromString.h>
|
||||
@ -272,6 +273,34 @@ bool MergeTreeDataPartChecksums::isBadChecksumsErrorCode(int code)
|
||||
|| code == ErrorCodes::UNEXPECTED_FILE_IN_DATA_PART;
|
||||
}
|
||||
|
||||
/// Puts into hash "stream" length of the string and its bytes
|
||||
static void updateHash(SipHash & hash, const std::string & data)
|
||||
{
|
||||
UInt64 len = data.size();
|
||||
hash.update(len);
|
||||
hash.update(data.data(), len);
|
||||
}
|
||||
|
||||
/// Hash is the same as MinimalisticDataPartChecksums::hash_of_all_files
|
||||
String MergeTreeDataPartChecksums::getTotalChecksumHex() const
|
||||
{
|
||||
SipHash hash_of_all_files;
|
||||
|
||||
for (const auto & elem : files)
|
||||
{
|
||||
const String & name = elem.first;
|
||||
const auto & checksum = elem.second;
|
||||
|
||||
updateHash(hash_of_all_files, name);
|
||||
hash_of_all_files.update(checksum.file_hash);
|
||||
}
|
||||
|
||||
UInt64 lo, hi;
|
||||
hash_of_all_files.get128(lo, hi);
|
||||
|
||||
return getHexUIntUppercase(hi) + getHexUIntUppercase(lo);
|
||||
}
|
||||
|
||||
void MinimalisticDataPartChecksums::serialize(WriteBuffer & to) const
|
||||
{
|
||||
writeString("checksums format version: 5\n", to);
|
||||
@ -331,31 +360,24 @@ void MinimalisticDataPartChecksums::computeTotalChecksums(const MergeTreeDataPar
|
||||
SipHash hash_of_uncompressed_files_;
|
||||
SipHash uncompressed_hash_of_compressed_files_;
|
||||
|
||||
auto update_hash = [] (SipHash & hash, const std::string & data)
|
||||
{
|
||||
UInt64 len = data.size();
|
||||
hash.update(len);
|
||||
hash.update(data.data(), len);
|
||||
};
|
||||
|
||||
for (const auto & elem : full_checksums.files)
|
||||
{
|
||||
const String & name = elem.first;
|
||||
const auto & checksum = elem.second;
|
||||
|
||||
update_hash(hash_of_all_files_, name);
|
||||
updateHash(hash_of_all_files_, name);
|
||||
hash_of_all_files_.update(checksum.file_hash);
|
||||
|
||||
if (!checksum.is_compressed)
|
||||
{
|
||||
++num_uncompressed_files;
|
||||
update_hash(hash_of_uncompressed_files_, name);
|
||||
updateHash(hash_of_uncompressed_files_, name);
|
||||
hash_of_uncompressed_files_.update(checksum.file_hash);
|
||||
}
|
||||
else
|
||||
{
|
||||
++num_compressed_files;
|
||||
update_hash(uncompressed_hash_of_compressed_files_, name);
|
||||
updateHash(uncompressed_hash_of_compressed_files_, name);
|
||||
uncompressed_hash_of_compressed_files_.update(checksum.uncompressed_hash);
|
||||
}
|
||||
}
|
||||
|
@ -79,6 +79,9 @@ struct MergeTreeDataPartChecksums
|
||||
/// Checksum from the set of checksums of .bin files (for deduplication).
|
||||
void computeTotalChecksumDataOnly(SipHash & hash) const;
|
||||
|
||||
/// SipHash of all all files hashes represented as hex string
|
||||
String getTotalChecksumHex() const;
|
||||
|
||||
String getSerializedString() const;
|
||||
static MergeTreeDataPartChecksums deserializeFrom(const String & s);
|
||||
};
|
||||
|
@ -140,13 +140,13 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa
|
||||
String part_name;
|
||||
if (data.format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING)
|
||||
{
|
||||
DayNum_t min_date(minmax_idx.min_values[data.minmax_idx_date_column_pos].get<UInt64>());
|
||||
DayNum_t max_date(minmax_idx.max_values[data.minmax_idx_date_column_pos].get<UInt64>());
|
||||
DayNum min_date(minmax_idx.min_values[data.minmax_idx_date_column_pos].get<UInt64>());
|
||||
DayNum max_date(minmax_idx.max_values[data.minmax_idx_date_column_pos].get<UInt64>());
|
||||
|
||||
const auto & date_lut = DateLUT::instance();
|
||||
|
||||
DayNum_t min_month = date_lut.toFirstDayNumOfMonth(DayNum_t(min_date));
|
||||
DayNum_t max_month = date_lut.toFirstDayNumOfMonth(DayNum_t(max_date));
|
||||
DayNum min_month = date_lut.toFirstDayNumOfMonth(DayNum(min_date));
|
||||
DayNum max_month = date_lut.toFirstDayNumOfMonth(DayNum(max_date));
|
||||
|
||||
if (min_month != max_month)
|
||||
throw Exception("Logical error: part spans more than one month.");
|
||||
|
@ -89,7 +89,7 @@ bool MergeTreePartInfo::tryParsePartName(const String & dir_name, MergeTreePartI
|
||||
}
|
||||
|
||||
|
||||
void MergeTreePartInfo::parseMinMaxDatesFromPartName(const String & dir_name, DayNum_t & min_date, DayNum_t & max_date)
|
||||
void MergeTreePartInfo::parseMinMaxDatesFromPartName(const String & dir_name, DayNum & min_date, DayNum & max_date)
|
||||
{
|
||||
UInt32 min_yyyymmdd = 0;
|
||||
UInt32 max_yyyymmdd = 0;
|
||||
@ -108,8 +108,8 @@ void MergeTreePartInfo::parseMinMaxDatesFromPartName(const String & dir_name, Da
|
||||
min_date = date_lut.YYYYMMDDToDayNum(min_yyyymmdd);
|
||||
max_date = date_lut.YYYYMMDDToDayNum(max_yyyymmdd);
|
||||
|
||||
DayNum_t min_month = date_lut.toFirstDayNumOfMonth(min_date);
|
||||
DayNum_t max_month = date_lut.toFirstDayNumOfMonth(max_date);
|
||||
DayNum min_month = date_lut.toFirstDayNumOfMonth(min_date);
|
||||
DayNum max_month = date_lut.toFirstDayNumOfMonth(max_date);
|
||||
|
||||
if (min_month != max_month)
|
||||
throw Exception("Part name " + dir_name + " contains different months", ErrorCodes::BAD_DATA_PART_NAME);
|
||||
@ -146,7 +146,7 @@ String MergeTreePartInfo::getPartName() const
|
||||
}
|
||||
|
||||
|
||||
String MergeTreePartInfo::getPartNameV0(DayNum_t left_date, DayNum_t right_date) const
|
||||
String MergeTreePartInfo::getPartNameV0(DayNum left_date, DayNum right_date) const
|
||||
{
|
||||
const auto & date_lut = DateLUT::instance();
|
||||
|
||||
|
@ -69,7 +69,7 @@ struct MergeTreePartInfo
|
||||
}
|
||||
|
||||
String getPartName() const;
|
||||
String getPartNameV0(DayNum_t left_date, DayNum_t right_date) const;
|
||||
String getPartNameV0(DayNum left_date, DayNum right_date) const;
|
||||
UInt64 getBlocksCount() const
|
||||
{
|
||||
return static_cast<UInt64>(max_block - min_block + 1);
|
||||
@ -79,7 +79,7 @@ struct MergeTreePartInfo
|
||||
|
||||
static bool tryParsePartName(const String & dir_name, MergeTreePartInfo * part_info, MergeTreeDataFormatVersion format_version);
|
||||
|
||||
static void parseMinMaxDatesFromPartName(const String & part_name, DayNum_t & min_date, DayNum_t & max_date);
|
||||
static void parseMinMaxDatesFromPartName(const String & part_name, DayNum & min_date, DayNum & max_date);
|
||||
|
||||
static bool contains(const String & outer_part_name, const String & inner_part_name, MergeTreeDataFormatVersion format_version);
|
||||
};
|
||||
|
@ -52,7 +52,7 @@ String MergeTreePartition::getID(const MergeTreeData & storage) const
|
||||
result += '-';
|
||||
|
||||
if (typeid_cast<const DataTypeDate *>(storage.partition_key_sample.getByPosition(i).type.get()))
|
||||
result += toString(DateLUT::instance().toNumYYYYMMDD(DayNum_t(value[i].safeGet<UInt64>())));
|
||||
result += toString(DateLUT::instance().toNumYYYYMMDD(DayNum(value[i].safeGet<UInt64>())));
|
||||
else
|
||||
result += applyVisitor(to_string_visitor, value[i]);
|
||||
|
||||
|
@ -334,7 +334,7 @@ bool MergeTreeWhereOptimizer::isPrimaryKeyAtom(const IAST * const ast) const
|
||||
if ((primary_key_columns.count(first_arg_name) && isConstant(args[1])) ||
|
||||
(primary_key_columns.count(second_arg_name) && isConstant(args[0])) ||
|
||||
(primary_key_columns.count(first_arg_name)
|
||||
&& (prepared_sets.count(args[1].get()) || typeid_cast<const ASTSubquery *>(args[1].get()))))
|
||||
&& (prepared_sets.count(args[1]->range) || typeid_cast<const ASTSubquery *>(args[1].get()))))
|
||||
return true;
|
||||
}
|
||||
|
||||
|
@ -14,200 +14,198 @@ namespace DB
|
||||
static const auto ALTER_ERROR_SLEEP_MS = 10 * 1000;
|
||||
|
||||
|
||||
ReplicatedMergeTreeAlterThread::ReplicatedMergeTreeAlterThread(StorageReplicatedMergeTree & storage_)
|
||||
: storage(storage_),
|
||||
log(&Logger::get(storage.database_name + "." + storage.table_name + " (StorageReplicatedMergeTree, AlterThread)")),
|
||||
thread([this] { run(); }) {}
|
||||
ReplicatedMergeTreeAlterThread::ReplicatedMergeTreeAlterThread(StorageReplicatedMergeTree & storage_) :
|
||||
storage(storage_),
|
||||
log(&Logger::get(storage.database_name + "." + storage.table_name + " (StorageReplicatedMergeTree, AlterThread)"))
|
||||
{
|
||||
task_handle = storage_.context.getSchedulePool().addTask("ReplicatedMergeTreeAlterThread", [this]{run();});
|
||||
task_handle->schedule();
|
||||
}
|
||||
|
||||
ReplicatedMergeTreeAlterThread::~ReplicatedMergeTreeAlterThread()
|
||||
{
|
||||
storage.context.getSchedulePool().removeTask(task_handle);
|
||||
}
|
||||
|
||||
void ReplicatedMergeTreeAlterThread::run()
|
||||
{
|
||||
setThreadName("ReplMTAlter");
|
||||
|
||||
bool force_recheck_parts = true;
|
||||
|
||||
while (!need_stop)
|
||||
try
|
||||
{
|
||||
try
|
||||
/** We have a description of columns in ZooKeeper, common for all replicas (Example: /clickhouse/tables/02-06/visits/columns),
|
||||
* as well as a description of columns in local file with metadata (storage.data.getColumnsList()).
|
||||
*
|
||||
* If these descriptions are different - you need to do ALTER.
|
||||
*
|
||||
* If stored version of the node (columns_version) differs from the version in ZK,
|
||||
* then the description of the columns in ZK does not necessarily differ from the local
|
||||
* - this can happen with a loop from ALTER-s, which as a whole, does not change anything.
|
||||
* In this case, you need to update the stored version number,
|
||||
* and also check the structure of parts, and, if necessary, make ALTER.
|
||||
*
|
||||
* Recorded version number needs to be updated after updating the metadata, under lock.
|
||||
* This version number is checked against the current one for INSERT.
|
||||
* That is, we make sure to insert blocks with the correct structure.
|
||||
*
|
||||
* When the server starts, previous ALTER might not have been completed.
|
||||
* Therefore, for the first time, regardless of the changes, we check the structure of all parts,
|
||||
* (Example: /clickhouse/tables/02-06/visits/replicas/example02-06-1.yandex.ru/parts/20140806_20140831_131664_134988_3296/columns)
|
||||
* and do ALTER if necessary.
|
||||
*
|
||||
* TODO: Too complicated, rewrite everything.
|
||||
*/
|
||||
|
||||
auto zookeeper = storage.getZooKeeper();
|
||||
|
||||
zkutil::Stat stat;
|
||||
const String columns_str = zookeeper->getWatch(storage.zookeeper_path + "/columns", &stat, task_handle->getWatchCallback());
|
||||
auto columns_in_zk = ColumnsDescription::parse(columns_str);
|
||||
|
||||
bool changed_version = (stat.version != storage.columns_version);
|
||||
|
||||
{
|
||||
/** We have a description of columns in ZooKeeper, common for all replicas (Example: /clickhouse/tables/02-06/visits/columns),
|
||||
* as well as a description of columns in local file with metadata (storage.data.getColumnsList()).
|
||||
*
|
||||
* If these descriptions are different - you need to do ALTER.
|
||||
*
|
||||
* If stored version of the node (columns_version) differs from the version in ZK,
|
||||
* then the description of the columns in ZK does not necessarily differ from the local
|
||||
* - this can happen with a loop from ALTER-s, which as a whole, does not change anything.
|
||||
* In this case, you need to update the stored version number,
|
||||
* and also check the structure of parts, and, if necessary, make ALTER.
|
||||
*
|
||||
* Recorded version number needs to be updated after updating the metadata, under lock.
|
||||
* This version number is checked against the current one for INSERT.
|
||||
* That is, we make sure to insert blocks with the correct structure.
|
||||
*
|
||||
* When the server starts, previous ALTER might not have been completed.
|
||||
* Therefore, for the first time, regardless of the changes, we check the structure of all parts,
|
||||
* (Example: /clickhouse/tables/02-06/visits/replicas/example02-06-1.yandex.ru/parts/20140806_20140831_131664_134988_3296/columns)
|
||||
* and do ALTER if necessary.
|
||||
*
|
||||
* TODO: Too complicated, rewrite everything.
|
||||
*/
|
||||
/// If you need to lock table structure, then suspend merges.
|
||||
ActionLock merge_blocker;
|
||||
|
||||
auto zookeeper = storage.getZooKeeper();
|
||||
if (changed_version || force_recheck_parts)
|
||||
merge_blocker = storage.merger_mutator.actions_blocker.cancel();
|
||||
|
||||
zkutil::Stat stat;
|
||||
const String columns_str = zookeeper->get(storage.zookeeper_path + "/columns", &stat, wakeup_event);
|
||||
auto columns_in_zk = ColumnsDescription::parse(columns_str);
|
||||
|
||||
bool changed_version = (stat.version != storage.columns_version);
|
||||
MergeTreeData::DataParts parts;
|
||||
|
||||
/// If columns description has changed, we will update table structure locally.
|
||||
if (changed_version)
|
||||
{
|
||||
/// If you need to lock table structure, then suspend merges.
|
||||
ActionBlocker::LockHolder merge_blocker;
|
||||
|
||||
if (changed_version || force_recheck_parts)
|
||||
merge_blocker = storage.merger_mutator.actions_blocker.cancel();
|
||||
|
||||
MergeTreeData::DataParts parts;
|
||||
|
||||
/// If columns description has changed, we will update table structure locally.
|
||||
if (changed_version)
|
||||
{
|
||||
/// Temporarily cancel part checks to avoid locking for long time.
|
||||
auto temporarily_stop_part_checks = storage.part_check_thread.temporarilyStop();
|
||||
/// Temporarily cancel part checks to avoid locking for long time.
|
||||
auto temporarily_stop_part_checks = storage.part_check_thread.temporarilyStop();
|
||||
|
||||
/// Temporarily cancel parts sending
|
||||
ActionBlocker::LockHolder data_parts_exchange_blocker;
|
||||
ActionLock data_parts_exchange_blocker;
|
||||
if (storage.data_parts_exchange_endpoint_holder)
|
||||
data_parts_exchange_blocker = storage.data_parts_exchange_endpoint_holder->cancel();
|
||||
data_parts_exchange_blocker = storage.data_parts_exchange_endpoint_holder->getBlocker().cancel();
|
||||
|
||||
/// Temporarily cancel part fetches
|
||||
auto fetches_blocker = storage.fetcher.blocker.cancel();
|
||||
/// Temporarily cancel part fetches
|
||||
auto fetches_blocker = storage.fetcher.blocker.cancel();
|
||||
|
||||
LOG_INFO(log, "Changed version of 'columns' node in ZooKeeper. Waiting for structure write lock.");
|
||||
LOG_INFO(log, "Changed version of 'columns' node in ZooKeeper. Waiting for structure write lock.");
|
||||
|
||||
auto table_lock = storage.lockStructureForAlter(__PRETTY_FUNCTION__);
|
||||
auto table_lock = storage.lockStructureForAlter(__PRETTY_FUNCTION__);
|
||||
|
||||
if (columns_in_zk != storage.getColumns())
|
||||
{
|
||||
LOG_INFO(log, "Columns list changed in ZooKeeper. Applying changes locally.");
|
||||
|
||||
storage.context.getDatabase(storage.database_name)->alterTable(
|
||||
storage.context, storage.table_name, columns_in_zk, {});
|
||||
storage.setColumns(std::move(columns_in_zk));
|
||||
|
||||
/// Reinitialize primary key because primary key column types might have changed.
|
||||
storage.data.initPrimaryKey();
|
||||
|
||||
LOG_INFO(log, "Applied changes to table.");
|
||||
}
|
||||
else
|
||||
{
|
||||
LOG_INFO(log, "Columns version changed in ZooKeeper, but data wasn't changed. It's like cyclic ALTERs.");
|
||||
}
|
||||
|
||||
/// You need to get a list of parts under table lock to avoid race condition with merge.
|
||||
parts = storage.data.getDataParts();
|
||||
|
||||
storage.columns_version = stat.version;
|
||||
}
|
||||
|
||||
/// Update parts.
|
||||
if (changed_version || force_recheck_parts)
|
||||
if (columns_in_zk != storage.getColumns())
|
||||
{
|
||||
auto table_lock = storage.lockStructure(false, __PRETTY_FUNCTION__);
|
||||
LOG_INFO(log, "Columns list changed in ZooKeeper. Applying changes locally.");
|
||||
|
||||
if (changed_version)
|
||||
LOG_INFO(log, "ALTER-ing parts");
|
||||
storage.context.getDatabase(storage.database_name)->alterTable(
|
||||
storage.context, storage.table_name, columns_in_zk, {});
|
||||
storage.setColumns(std::move(columns_in_zk));
|
||||
|
||||
int changed_parts = 0;
|
||||
/// Reinitialize primary key because primary key column types might have changed.
|
||||
storage.data.initPrimaryKey();
|
||||
|
||||
if (!changed_version)
|
||||
parts = storage.data.getDataParts();
|
||||
|
||||
const auto columns_for_parts = storage.getColumns().getAllPhysical();
|
||||
|
||||
for (const MergeTreeData::DataPartPtr & part : parts)
|
||||
{
|
||||
/// Update the part and write result to temporary files.
|
||||
/// TODO: You can skip checking for too large changes if ZooKeeper has, for example,
|
||||
/// node /flags/force_alter.
|
||||
auto transaction = storage.data.alterDataPart(
|
||||
part, columns_for_parts, storage.data.primary_expr_ast, false);
|
||||
|
||||
if (!transaction)
|
||||
continue;
|
||||
|
||||
++changed_parts;
|
||||
|
||||
/// Update part metadata in ZooKeeper.
|
||||
zkutil::Requests ops;
|
||||
ops.emplace_back(zkutil::makeSetRequest(
|
||||
storage.replica_path + "/parts/" + part->name + "/columns", transaction->getNewColumns().toString(), -1));
|
||||
ops.emplace_back(zkutil::makeSetRequest(
|
||||
storage.replica_path + "/parts/" + part->name + "/checksums",
|
||||
storage.getChecksumsForZooKeeper(transaction->getNewChecksums()),
|
||||
-1));
|
||||
|
||||
try
|
||||
{
|
||||
zookeeper->multi(ops);
|
||||
}
|
||||
catch (const zkutil::KeeperException & e)
|
||||
{
|
||||
/// The part does not exist in ZK. We will add to queue for verification - maybe the part is superfluous, and it must be removed locally.
|
||||
if (e.code == ZooKeeperImpl::ZooKeeper::ZNONODE)
|
||||
storage.enqueuePartForCheck(part->name);
|
||||
|
||||
throw;
|
||||
}
|
||||
|
||||
/// Apply file changes.
|
||||
transaction->commit();
|
||||
}
|
||||
|
||||
/// Columns sizes could be quietly changed in case of MODIFY/ADD COLUMN
|
||||
storage.data.recalculateColumnSizes();
|
||||
|
||||
/// List of columns for a specific replica.
|
||||
zookeeper->set(storage.replica_path + "/columns", columns_str);
|
||||
|
||||
if (changed_version)
|
||||
{
|
||||
if (changed_parts != 0)
|
||||
LOG_INFO(log, "ALTER-ed " << changed_parts << " parts");
|
||||
else
|
||||
LOG_INFO(log, "No parts ALTER-ed");
|
||||
}
|
||||
|
||||
force_recheck_parts = false;
|
||||
LOG_INFO(log, "Applied changes to table.");
|
||||
}
|
||||
else
|
||||
{
|
||||
LOG_INFO(log, "Columns version changed in ZooKeeper, but data wasn't changed. It's like cyclic ALTERs.");
|
||||
}
|
||||
|
||||
/// It's important that parts and merge_blocker are destroyed before the wait.
|
||||
/// You need to get a list of parts under table lock to avoid race condition with merge.
|
||||
parts = storage.data.getDataParts();
|
||||
|
||||
storage.columns_version = stat.version;
|
||||
}
|
||||
|
||||
wakeup_event->wait();
|
||||
}
|
||||
catch (const zkutil::KeeperException & e)
|
||||
{
|
||||
tryLogCurrentException(log, __PRETTY_FUNCTION__);
|
||||
/// Update parts.
|
||||
if (changed_version || force_recheck_parts)
|
||||
{
|
||||
auto table_lock = storage.lockStructure(false, __PRETTY_FUNCTION__);
|
||||
|
||||
if (e.code == ZooKeeperImpl::ZooKeeper::ZSESSIONEXPIRED)
|
||||
break;
|
||||
if (changed_version)
|
||||
LOG_INFO(log, "ALTER-ing parts");
|
||||
|
||||
force_recheck_parts = true;
|
||||
wakeup_event->tryWait(ALTER_ERROR_SLEEP_MS);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(log, __PRETTY_FUNCTION__);
|
||||
int changed_parts = 0;
|
||||
|
||||
force_recheck_parts = true;
|
||||
wakeup_event->tryWait(ALTER_ERROR_SLEEP_MS);
|
||||
if (!changed_version)
|
||||
parts = storage.data.getDataParts();
|
||||
|
||||
const auto columns_for_parts = storage.getColumns().getAllPhysical();
|
||||
|
||||
for (const MergeTreeData::DataPartPtr & part : parts)
|
||||
{
|
||||
/// Update the part and write result to temporary files.
|
||||
/// TODO: You can skip checking for too large changes if ZooKeeper has, for example,
|
||||
/// node /flags/force_alter.
|
||||
auto transaction = storage.data.alterDataPart(
|
||||
part, columns_for_parts, storage.data.primary_expr_ast, false);
|
||||
|
||||
if (!transaction)
|
||||
continue;
|
||||
|
||||
++changed_parts;
|
||||
|
||||
/// Update part metadata in ZooKeeper.
|
||||
zkutil::Requests ops;
|
||||
ops.emplace_back(zkutil::makeSetRequest(
|
||||
storage.replica_path + "/parts/" + part->name + "/columns", transaction->getNewColumns().toString(), -1));
|
||||
ops.emplace_back(zkutil::makeSetRequest(
|
||||
storage.replica_path + "/parts/" + part->name + "/checksums",
|
||||
storage.getChecksumsForZooKeeper(transaction->getNewChecksums()),
|
||||
-1));
|
||||
|
||||
try
|
||||
{
|
||||
zookeeper->multi(ops);
|
||||
}
|
||||
catch (const zkutil::KeeperException & e)
|
||||
{
|
||||
/// The part does not exist in ZK. We will add to queue for verification - maybe the part is superfluous, and it must be removed locally.
|
||||
if (e.code == ZooKeeperImpl::ZooKeeper::ZNONODE)
|
||||
storage.enqueuePartForCheck(part->name);
|
||||
|
||||
throw;
|
||||
}
|
||||
|
||||
/// Apply file changes.
|
||||
transaction->commit();
|
||||
}
|
||||
|
||||
/// Columns sizes could be quietly changed in case of MODIFY/ADD COLUMN
|
||||
storage.data.recalculateColumnSizes();
|
||||
|
||||
/// List of columns for a specific replica.
|
||||
zookeeper->set(storage.replica_path + "/columns", columns_str);
|
||||
|
||||
if (changed_version)
|
||||
{
|
||||
if (changed_parts != 0)
|
||||
LOG_INFO(log, "ALTER-ed " << changed_parts << " parts");
|
||||
else
|
||||
LOG_INFO(log, "No parts ALTER-ed");
|
||||
}
|
||||
|
||||
force_recheck_parts = false;
|
||||
}
|
||||
|
||||
/// It's important that parts and merge_blocker are destroyed before the wait.
|
||||
}
|
||||
}
|
||||
catch (const zkutil::KeeperException & e)
|
||||
{
|
||||
tryLogCurrentException(log, __PRETTY_FUNCTION__);
|
||||
|
||||
LOG_DEBUG(log, "Alter thread finished");
|
||||
if (e.code == ZooKeeperImpl::ZooKeeper::ZSESSIONEXPIRED)
|
||||
return;
|
||||
|
||||
force_recheck_parts = true;
|
||||
task_handle->scheduleAfter(ALTER_ERROR_SLEEP_MS);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(log, __PRETTY_FUNCTION__);
|
||||
|
||||
force_recheck_parts = true;
|
||||
task_handle->scheduleAfter(ALTER_ERROR_SLEEP_MS);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -1,6 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <thread>
|
||||
#include <Common/BackgroundSchedulePool.h>
|
||||
#include <Common/ZooKeeper/Types.h>
|
||||
#include <Core/Types.h>
|
||||
#include <common/logger_useful.h>
|
||||
@ -21,25 +22,14 @@ class ReplicatedMergeTreeAlterThread
|
||||
{
|
||||
public:
|
||||
ReplicatedMergeTreeAlterThread(StorageReplicatedMergeTree & storage_);
|
||||
|
||||
~ReplicatedMergeTreeAlterThread()
|
||||
{
|
||||
need_stop = true;
|
||||
wakeup_event->set();
|
||||
if (thread.joinable())
|
||||
thread.join();
|
||||
}
|
||||
~ReplicatedMergeTreeAlterThread();
|
||||
|
||||
private:
|
||||
void run();
|
||||
|
||||
StorageReplicatedMergeTree & storage;
|
||||
Logger * log;
|
||||
|
||||
zkutil::EventPtr wakeup_event { std::make_shared<Poco::Event>() };
|
||||
std::atomic<bool> need_stop { false };
|
||||
|
||||
std::thread thread;
|
||||
BackgroundSchedulePool::TaskHandle task_handle;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -108,7 +108,7 @@ void ReplicatedMergeTreeBlockOutputStream::write(const Block & block)
|
||||
last_block_is_duplicate = false;
|
||||
|
||||
/// TODO Is it possible to not lock the table structure here?
|
||||
storage.data.delayInsertIfNeeded(&storage.restarting_thread->getWakeupEvent());
|
||||
storage.data.delayInsertOrThrowIfNeeded(&storage.restarting_thread->getWakeupEvent());
|
||||
|
||||
auto zookeeper = storage.getZooKeeper();
|
||||
assertSessionIsNotExpired(zookeeper);
|
||||
@ -161,11 +161,11 @@ void ReplicatedMergeTreeBlockOutputStream::write(const Block & block)
|
||||
|
||||
/// Set a special error code if the block is duplicate
|
||||
int error = (deduplicate && last_block_is_duplicate) ? ErrorCodes::INSERT_WAS_DEDUPLICATED : 0;
|
||||
PartLog::addNewPartToTheLog(storage.context, *part, watch.elapsed(), ExecutionStatus(error));
|
||||
PartLog::addNewPart(storage.context, part, watch.elapsed(), ExecutionStatus(error));
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
PartLog::addNewPartToTheLog(storage.context, *part, watch.elapsed(), ExecutionStatus::fromCurrentException(__PRETTY_FUNCTION__));
|
||||
PartLog::addNewPart(storage.context, part, watch.elapsed(), ExecutionStatus::fromCurrentException(__PRETTY_FUNCTION__));
|
||||
throw;
|
||||
}
|
||||
}
|
||||
@ -176,7 +176,7 @@ void ReplicatedMergeTreeBlockOutputStream::writeExistingPart(MergeTreeData::Muta
|
||||
{
|
||||
last_block_is_duplicate = false;
|
||||
|
||||
/// NOTE No delay in this case. That's Ok.
|
||||
/// NOTE: No delay in this case. That's Ok.
|
||||
|
||||
auto zookeeper = storage.getZooKeeper();
|
||||
assertSessionIsNotExpired(zookeeper);
|
||||
@ -189,11 +189,11 @@ void ReplicatedMergeTreeBlockOutputStream::writeExistingPart(MergeTreeData::Muta
|
||||
try
|
||||
{
|
||||
commitPart(zookeeper, part, "");
|
||||
PartLog::addNewPartToTheLog(storage.context, *part, watch.elapsed());
|
||||
PartLog::addNewPart(storage.context, part, watch.elapsed());
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
PartLog::addNewPartToTheLog(storage.context, *part, watch.elapsed(), ExecutionStatus::fromCurrentException(__PRETTY_FUNCTION__));
|
||||
PartLog::addNewPart(storage.context, part, watch.elapsed(), ExecutionStatus::fromCurrentException(__PRETTY_FUNCTION__));
|
||||
throw;
|
||||
}
|
||||
}
|
||||
@ -208,47 +208,20 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart(zkutil::ZooKeeperPtr & zoo
|
||||
/// We remove the lock just after renaming the part. In case of exception, block number will be marked as abandoned.
|
||||
/// Also, make deduplication check. If a duplicate is detected, no nodes are created.
|
||||
|
||||
/// Deduplication stuff
|
||||
/// Allocate new block number and check for duplicates
|
||||
bool deduplicate_block = !block_id.empty();
|
||||
String block_id_path;
|
||||
zkutil::Requests deduplication_check_ops;
|
||||
zkutil::Requests * deduplication_check_ops_ptr = nullptr;
|
||||
String block_id_path = deduplicate_block ? storage.zookeeper_path + "/blocks/" + block_id : "";
|
||||
auto block_number_lock = storage.allocateBlockNumber(part->info.partition_id, zookeeper, block_id_path);
|
||||
|
||||
if (deduplicate_block)
|
||||
if (!block_number_lock)
|
||||
{
|
||||
block_id_path = storage.zookeeper_path + "/blocks/" + block_id;
|
||||
|
||||
/// Lets check for duplicates in advance, to avoid superflous block numbers allocation
|
||||
deduplication_check_ops.emplace_back(zkutil::makeCreateRequest(block_id_path, "", zkutil::CreateMode::Persistent));
|
||||
deduplication_check_ops.emplace_back(zkutil::makeRemoveRequest(block_id_path, -1));
|
||||
deduplication_check_ops_ptr = &deduplication_check_ops;
|
||||
part->is_duplicate = true;
|
||||
last_block_is_duplicate = true;
|
||||
ProfileEvents::increment(ProfileEvents::DuplicatedInsertedBlocks);
|
||||
return;
|
||||
}
|
||||
|
||||
AbandonableLockInZooKeeper block_number_lock;
|
||||
try
|
||||
{
|
||||
/// 2 RTT
|
||||
block_number_lock = storage.allocateBlockNumber(part->info.partition_id, zookeeper, deduplication_check_ops_ptr);
|
||||
}
|
||||
catch (const zkutil::KeeperMultiException & e)
|
||||
{
|
||||
if (deduplicate_block && e.code == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS && e.getPathForFirstFailedOp() == block_id_path)
|
||||
{
|
||||
LOG_INFO(log, "Block with ID " << block_id << " already exists; ignoring it (skip the insertion)");
|
||||
part->is_duplicate = true;
|
||||
last_block_is_duplicate = true;
|
||||
ProfileEvents::increment(ProfileEvents::DuplicatedInsertedBlocks);
|
||||
return;
|
||||
}
|
||||
|
||||
throw Exception("Cannot allocate block number in ZooKeeper: " + e.displayText(), ErrorCodes::KEEPER_EXCEPTION);
|
||||
}
|
||||
catch (const zkutil::KeeperException & e)
|
||||
{
|
||||
throw Exception("Cannot allocate block number in ZooKeeper: " + e.displayText(), ErrorCodes::KEEPER_EXCEPTION);
|
||||
}
|
||||
|
||||
Int64 block_number = block_number_lock.getNumber();
|
||||
Int64 block_number = block_number_lock->getNumber();
|
||||
|
||||
/// Set part attributes according to part_number. Prepare an entry for log.
|
||||
|
||||
@ -272,33 +245,7 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart(zkutil::ZooKeeperPtr & zoo
|
||||
/// Information about the part.
|
||||
zkutil::Requests ops;
|
||||
|
||||
if (deduplicate_block)
|
||||
{
|
||||
/// Make final duplicate check and commit block_id
|
||||
ops.emplace_back(
|
||||
zkutil::makeCreateRequest(
|
||||
block_id_path,
|
||||
toString(block_number), /// We will able to know original part number for duplicate blocks, if we want.
|
||||
zkutil::CreateMode::Persistent));
|
||||
}
|
||||
|
||||
/// Information about the part, in the replica data.
|
||||
|
||||
ops.emplace_back(zkutil::makeCheckRequest(
|
||||
storage.zookeeper_path + "/columns",
|
||||
storage.columns_version));
|
||||
ops.emplace_back(zkutil::makeCreateRequest(
|
||||
storage.replica_path + "/parts/" + part->name,
|
||||
"",
|
||||
zkutil::CreateMode::Persistent));
|
||||
ops.emplace_back(zkutil::makeCreateRequest(
|
||||
storage.replica_path + "/parts/" + part->name + "/columns",
|
||||
part->columns.toString(),
|
||||
zkutil::CreateMode::Persistent));
|
||||
ops.emplace_back(zkutil::makeCreateRequest(
|
||||
storage.replica_path + "/parts/" + part->name + "/checksums",
|
||||
storage.getChecksumsForZooKeeper(part->checksums),
|
||||
zkutil::CreateMode::Persistent));
|
||||
storage.getCommitPartOps(ops, part, block_id_path);
|
||||
|
||||
/// Replication log.
|
||||
ops.emplace_back(zkutil::makeCreateRequest(
|
||||
@ -307,7 +254,7 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart(zkutil::ZooKeeperPtr & zoo
|
||||
zkutil::CreateMode::PersistentSequential));
|
||||
|
||||
/// Deletes the information that the block number is used for writing.
|
||||
block_number_lock.getUnlockOps(ops);
|
||||
block_number_lock->getUnlockOps(ops);
|
||||
|
||||
/** If you need a quorum - create a node in which the quorum is monitored.
|
||||
* (If such a node already exists, then someone has managed to make another quorum record at the same time, but for it the quorum has not yet been reached.
|
||||
@ -357,10 +304,10 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart(zkutil::ZooKeeperPtr & zoo
|
||||
if (multi_code == ZooKeeperImpl::ZooKeeper::ZOK)
|
||||
{
|
||||
transaction.commit();
|
||||
storage.merge_selecting_event.set();
|
||||
storage.merge_selecting_task_handle->schedule();
|
||||
|
||||
/// Lock nodes have been already deleted, do not delete them in destructor
|
||||
block_number_lock.assumeUnlocked();
|
||||
block_number_lock->assumeUnlocked();
|
||||
}
|
||||
else if (zkutil::isUserError(multi_code))
|
||||
{
|
||||
@ -450,5 +397,10 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart(zkutil::ZooKeeperPtr & zoo
|
||||
}
|
||||
}
|
||||
|
||||
void ReplicatedMergeTreeBlockOutputStream::writePrefix()
|
||||
{
|
||||
storage.data.throwInsertIfNeeded();
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
@ -26,6 +26,7 @@ public:
|
||||
bool deduplicate_);
|
||||
|
||||
Block getHeader() const override;
|
||||
void writePrefix() override;
|
||||
void write(const Block & block) override;
|
||||
|
||||
/// For ATTACHing existing data on filesystem.
|
||||
|
@ -17,41 +17,40 @@ namespace ErrorCodes
|
||||
|
||||
ReplicatedMergeTreeCleanupThread::ReplicatedMergeTreeCleanupThread(StorageReplicatedMergeTree & storage_)
|
||||
: storage(storage_),
|
||||
log(&Logger::get(storage.database_name + "." + storage.table_name + " (StorageReplicatedMergeTree, CleanupThread)")),
|
||||
thread([this] { run(); })
|
||||
log(&Logger::get(storage.database_name + "." + storage.table_name + " (StorageReplicatedMergeTree, CleanupThread)"))
|
||||
{
|
||||
task_handle = storage.context.getSchedulePool().addTask("ReplicatedMergeTreeCleanupThread", [this]{ run(); });
|
||||
task_handle->schedule();
|
||||
}
|
||||
|
||||
ReplicatedMergeTreeCleanupThread::~ReplicatedMergeTreeCleanupThread()
|
||||
{
|
||||
storage.context.getSchedulePool().removeTask(task_handle);
|
||||
}
|
||||
|
||||
void ReplicatedMergeTreeCleanupThread::run()
|
||||
{
|
||||
setThreadName("ReplMTCleanup");
|
||||
|
||||
const auto CLEANUP_SLEEP_MS = storage.data.settings.cleanup_delay_period * 1000
|
||||
+ std::uniform_int_distribution<UInt64>(0, storage.data.settings.cleanup_delay_period_random_add * 1000)(rng);
|
||||
|
||||
while (!storage.shutdown_called)
|
||||
try
|
||||
{
|
||||
try
|
||||
{
|
||||
iterate();
|
||||
}
|
||||
catch (const zkutil::KeeperException & e)
|
||||
{
|
||||
tryLogCurrentException(log, __PRETTY_FUNCTION__);
|
||||
iterate();
|
||||
}
|
||||
catch (const zkutil::KeeperException & e)
|
||||
{
|
||||
tryLogCurrentException(log, __PRETTY_FUNCTION__);
|
||||
|
||||
if (e.code == ZooKeeperImpl::ZooKeeper::ZSESSIONEXPIRED)
|
||||
break;
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(log, __PRETTY_FUNCTION__);
|
||||
}
|
||||
|
||||
storage.cleanup_thread_event.tryWait(CLEANUP_SLEEP_MS);
|
||||
if (e.code == ZooKeeperImpl::ZooKeeper::ZSESSIONEXPIRED)
|
||||
return;
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(log, __PRETTY_FUNCTION__);
|
||||
}
|
||||
|
||||
LOG_DEBUG(log, "Cleanup thread finished");
|
||||
task_handle->scheduleAfter(CLEANUP_SLEEP_MS);
|
||||
|
||||
}
|
||||
|
||||
|
||||
@ -243,11 +242,4 @@ void ReplicatedMergeTreeCleanupThread::getBlocksSortedByTime(zkutil::ZooKeeper &
|
||||
std::sort(timed_blocks.begin(), timed_blocks.end(), NodeWithStat::greaterByTime);
|
||||
}
|
||||
|
||||
|
||||
ReplicatedMergeTreeCleanupThread::~ReplicatedMergeTreeCleanupThread()
|
||||
{
|
||||
if (thread.joinable())
|
||||
thread.join();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -4,6 +4,7 @@
|
||||
#include <Common/ZooKeeper/Types.h>
|
||||
#include <Common/ZooKeeper/ZooKeeper.h>
|
||||
#include <common/logger_useful.h>
|
||||
#include <Common/BackgroundSchedulePool.h>
|
||||
#include <thread>
|
||||
#include <map>
|
||||
|
||||
@ -25,10 +26,12 @@ public:
|
||||
|
||||
~ReplicatedMergeTreeCleanupThread();
|
||||
|
||||
void schedule() { task_handle->schedule(); }
|
||||
|
||||
private:
|
||||
StorageReplicatedMergeTree & storage;
|
||||
Logger * log;
|
||||
std::thread thread;
|
||||
BackgroundSchedulePool::TaskHandle task_handle;
|
||||
pcg64 rng;
|
||||
|
||||
void run();
|
||||
|
@ -46,6 +46,11 @@ void ReplicatedMergeTreeLogEntryData::writeText(WriteBuffer & out) const
|
||||
<< new_part_name;
|
||||
break;
|
||||
|
||||
case REPLACE_RANGE:
|
||||
out << typeToString(REPLACE_RANGE) << "\n";
|
||||
replace_range_entry->writeText(out);
|
||||
break;
|
||||
|
||||
case MUTATE_PART:
|
||||
out << "mutate\n"
|
||||
<< source_parts.at(0) << "\n"
|
||||
@ -120,16 +125,11 @@ void ReplicatedMergeTreeLogEntryData::readText(ReadBuffer & in)
|
||||
type = CLEAR_COLUMN;
|
||||
in >> escape >> column_name >> "\nfrom\n" >> new_part_name;
|
||||
}
|
||||
else if (type_str == "attach")
|
||||
else if (type_str == typeToString(REPLACE_RANGE))
|
||||
{
|
||||
/// Obsolete. TODO: Remove after half year.
|
||||
type = ATTACH_PART;
|
||||
String source_type;
|
||||
in >> source_type;
|
||||
if (source_type != "detached")
|
||||
throw Exception("Bad format: expected 'detached', found '" + source_type + "'", ErrorCodes::CANNOT_PARSE_TEXT);
|
||||
String source_part_name;
|
||||
in >> "\n" >> source_part_name >> "\ninto\n" >> new_part_name;
|
||||
type = REPLACE_RANGE;
|
||||
replace_range_entry = std::make_shared<ReplaceRangeEntry>();
|
||||
replace_range_entry->readText(in);
|
||||
}
|
||||
else if (type_str == "mutate")
|
||||
{
|
||||
@ -148,6 +148,48 @@ void ReplicatedMergeTreeLogEntryData::readText(ReadBuffer & in)
|
||||
in >> "quorum: " >> quorum >> "\n";
|
||||
}
|
||||
|
||||
void ReplicatedMergeTreeLogEntryData::ReplaceRangeEntry::writeText(WriteBuffer & out) const
|
||||
{
|
||||
out << "drop_range_name: " << drop_range_part_name << "\n";
|
||||
out << "from_database: " << escape << from_database << "\n";
|
||||
out << "from_table: " << escape << from_table << "\n";
|
||||
|
||||
out << "source_parts: ";
|
||||
writeQuoted(src_part_names, out);
|
||||
out << "\n";
|
||||
|
||||
out << "new_parts: ";
|
||||
writeQuoted(new_part_names, out);
|
||||
out << "\n";
|
||||
|
||||
out << "part_checksums: ";
|
||||
writeQuoted(part_names_checksums, out);
|
||||
out << "\n";
|
||||
|
||||
out << "columns_version: " << columns_version;
|
||||
}
|
||||
|
||||
void ReplicatedMergeTreeLogEntryData::ReplaceRangeEntry::readText(ReadBuffer & in)
|
||||
{
|
||||
in >> "drop_range_name: " >> drop_range_part_name >> "\n";
|
||||
in >> "from_database: " >> escape >> from_database >> "\n";
|
||||
in >> "from_table: " >> escape >> from_table >> "\n";
|
||||
|
||||
in >> "source_parts: ";
|
||||
readQuoted(src_part_names, in);
|
||||
in >> "\n";
|
||||
|
||||
in >> "new_parts: ";
|
||||
readQuoted(new_part_names, in);
|
||||
in >> "\n";
|
||||
|
||||
in >> "part_checksums: ";
|
||||
readQuoted(part_names_checksums, in);
|
||||
in >> "\n";
|
||||
|
||||
in >> "columns_version: " >> columns_version;
|
||||
}
|
||||
|
||||
String ReplicatedMergeTreeLogEntryData::toString() const
|
||||
{
|
||||
WriteBufferFromOwnString out;
|
||||
|
@ -33,26 +33,31 @@ struct ReplicatedMergeTreeLogEntryData
|
||||
GET_PART, /// Get the part from another replica.
|
||||
MERGE_PARTS, /// Merge the parts.
|
||||
DROP_RANGE, /// Delete the parts in the specified partition in the specified number range.
|
||||
ATTACH_PART, /// Move a part from the `detached` directory. Obsolete. TODO: Remove after half year.
|
||||
CLEAR_COLUMN, /// Drop specific column from specified partition.
|
||||
REPLACE_RANGE, /// Drop certain range of partitions and replace them by new ones
|
||||
MUTATE_PART, /// Apply one or several mutations to the part.
|
||||
};
|
||||
|
||||
String typeToString() const
|
||||
static String typeToString(Type type)
|
||||
{
|
||||
switch (type)
|
||||
{
|
||||
case ReplicatedMergeTreeLogEntryData::GET_PART: return "GET_PART";
|
||||
case ReplicatedMergeTreeLogEntryData::MERGE_PARTS: return "MERGE_PARTS";
|
||||
case ReplicatedMergeTreeLogEntryData::DROP_RANGE: return "DROP_RANGE";
|
||||
case ReplicatedMergeTreeLogEntryData::ATTACH_PART: return "ATTACH_PART";
|
||||
case ReplicatedMergeTreeLogEntryData::CLEAR_COLUMN: return "CLEAR_COLUMN";
|
||||
case ReplicatedMergeTreeLogEntryData::MUTATE_PART: return "MUTATE_PART";
|
||||
case ReplicatedMergeTreeLogEntryData::GET_PART: return "GET_PART";
|
||||
case ReplicatedMergeTreeLogEntryData::MERGE_PARTS: return "MERGE_PARTS";
|
||||
case ReplicatedMergeTreeLogEntryData::DROP_RANGE: return "DROP_RANGE";
|
||||
case ReplicatedMergeTreeLogEntryData::CLEAR_COLUMN: return "CLEAR_COLUMN";
|
||||
case ReplicatedMergeTreeLogEntryData::REPLACE_RANGE: return "REPLACE_RANGE";
|
||||
case ReplicatedMergeTreeLogEntryData::MUTATE_PART: return "MUTATE_PART";
|
||||
default:
|
||||
throw Exception("Unknown log entry type: " + DB::toString<int>(type), ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
}
|
||||
|
||||
String typeToString() const
|
||||
{
|
||||
return typeToString(type);
|
||||
}
|
||||
|
||||
void writeText(WriteBuffer & out) const;
|
||||
void readText(ReadBuffer & in);
|
||||
String toString() const;
|
||||
@ -62,8 +67,8 @@ struct ReplicatedMergeTreeLogEntryData
|
||||
Type type = EMPTY;
|
||||
String source_replica; /// Empty string means that this entry was added to the queue immediately, and not copied from the log.
|
||||
|
||||
/// The name of resulting part.
|
||||
/// For DROP_RANGE, the name of a non-existent part. You need to remove all the parts covered by it.
|
||||
/// The name of resulting part for GET_PART and MERGE_PARTS
|
||||
/// Part range for DROP_RANGE and CLEAR_COLUMN
|
||||
String new_part_name;
|
||||
String block_id; /// For parts of level zero, the block identifier for deduplication (node name in /blocks/).
|
||||
mutable String actual_new_part_name; /// GET_PART could actually fetch a part covering 'new_part_name'.
|
||||
@ -75,6 +80,41 @@ struct ReplicatedMergeTreeLogEntryData
|
||||
/// For DROP_RANGE, true means that the parts need not be deleted, but moved to the `detached` directory.
|
||||
bool detach = false;
|
||||
|
||||
/// REPLACE PARTITION FROM command
|
||||
struct ReplaceRangeEntry
|
||||
{
|
||||
String drop_range_part_name;
|
||||
|
||||
String from_database;
|
||||
String from_table;
|
||||
Strings src_part_names; // as in from_table
|
||||
Strings new_part_names;
|
||||
Strings part_names_checksums;
|
||||
int columns_version;
|
||||
|
||||
void writeText(WriteBuffer & out) const;
|
||||
void readText(ReadBuffer & in);
|
||||
};
|
||||
|
||||
std::shared_ptr<ReplaceRangeEntry> replace_range_entry;
|
||||
|
||||
/// Part names that supposed to be added to virtual_parts and future_parts
|
||||
Strings getVirtualPartNames() const
|
||||
{
|
||||
/// TODO: Instead of new_part_name use another field for these commands
|
||||
if (type == DROP_RANGE || type == CLEAR_COLUMN)
|
||||
return {new_part_name};
|
||||
|
||||
if (type == REPLACE_RANGE)
|
||||
{
|
||||
Strings res = replace_range_entry->new_part_names;
|
||||
res.emplace_back(replace_range_entry->drop_range_part_name);
|
||||
return res;
|
||||
}
|
||||
|
||||
return {new_part_name};
|
||||
}
|
||||
|
||||
/// Access under queue_mutex, see ReplicatedMergeTreeQueue.
|
||||
bool currently_executing = false; /// Whether the action is executing now.
|
||||
/// These several fields are informational only (for viewing by the user using system tables).
|
||||
|
@ -21,34 +21,34 @@ ReplicatedMergeTreePartCheckThread::ReplicatedMergeTreePartCheckThread(StorageRe
|
||||
: storage(storage_),
|
||||
log(&Logger::get(storage.database_name + "." + storage.table_name + " (StorageReplicatedMergeTree, PartCheckThread)"))
|
||||
{
|
||||
task_handle = storage.context.getSchedulePool().addTask("ReplicatedMergeTreePartCheckThread", [this] { run(); });
|
||||
task_handle->schedule();
|
||||
}
|
||||
|
||||
ReplicatedMergeTreePartCheckThread::~ReplicatedMergeTreePartCheckThread()
|
||||
{
|
||||
stop();
|
||||
storage.context.getSchedulePool().removeTask(task_handle);
|
||||
}
|
||||
|
||||
void ReplicatedMergeTreePartCheckThread::start()
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(start_stop_mutex);
|
||||
|
||||
if (need_stop)
|
||||
need_stop = false;
|
||||
else
|
||||
thread = std::thread([this] { run(); });
|
||||
need_stop = false;
|
||||
task_handle->activate();
|
||||
task_handle->schedule();
|
||||
}
|
||||
|
||||
|
||||
void ReplicatedMergeTreePartCheckThread::stop()
|
||||
{
|
||||
//based on discussion on https://github.com/yandex/ClickHouse/pull/1489#issuecomment-344756259
|
||||
//using the schedule pool there is no problem in case stop is called two time in row and the start multiple times
|
||||
|
||||
std::lock_guard<std::mutex> lock(start_stop_mutex);
|
||||
|
||||
need_stop = true;
|
||||
if (thread.joinable())
|
||||
{
|
||||
wakeup_event.set();
|
||||
thread.join();
|
||||
need_stop = false;
|
||||
}
|
||||
task_handle->deactivate();
|
||||
}
|
||||
|
||||
|
||||
void ReplicatedMergeTreePartCheckThread::enqueuePart(const String & name, time_t delay_to_check_seconds)
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(parts_mutex);
|
||||
@ -58,7 +58,7 @@ void ReplicatedMergeTreePartCheckThread::enqueuePart(const String & name, time_t
|
||||
|
||||
parts_queue.emplace_back(name, time(nullptr) + delay_to_check_seconds);
|
||||
parts_set.insert(name);
|
||||
wakeup_event.set();
|
||||
task_handle->schedule();
|
||||
}
|
||||
|
||||
|
||||
@ -243,7 +243,7 @@ void ReplicatedMergeTreePartCheckThread::checkPart(const String & part_name)
|
||||
storage.removePartAndEnqueueFetch(part_name);
|
||||
|
||||
/// Delete part locally.
|
||||
storage.data.renameAndDetachPart(part, "broken_");
|
||||
storage.data.forgetPartAndMoveToDetached(part, "broken_");
|
||||
}
|
||||
}
|
||||
else if (part->modification_time + MAX_AGE_OF_LOCAL_PART_THAT_WASNT_ADDED_TO_ZOOKEEPER < time(nullptr))
|
||||
@ -254,7 +254,7 @@ void ReplicatedMergeTreePartCheckThread::checkPart(const String & part_name)
|
||||
ProfileEvents::increment(ProfileEvents::ReplicatedPartChecksFailed);
|
||||
|
||||
LOG_ERROR(log, "Unexpected part " << part_name << " in filesystem. Removing.");
|
||||
storage.data.renameAndDetachPart(part, "unexpected_");
|
||||
storage.data.forgetPartAndMoveToDetached(part, "unexpected_");
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -279,95 +279,83 @@ void ReplicatedMergeTreePartCheckThread::checkPart(const String & part_name)
|
||||
|
||||
void ReplicatedMergeTreePartCheckThread::run()
|
||||
{
|
||||
setThreadName("ReplMTPartCheck");
|
||||
if (need_stop)
|
||||
return;
|
||||
|
||||
while (!need_stop)
|
||||
try
|
||||
{
|
||||
try
|
||||
time_t current_time = time(nullptr);
|
||||
|
||||
/// 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();
|
||||
|
||||
{
|
||||
time_t current_time = time(nullptr);
|
||||
|
||||
/// 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();
|
||||
std::lock_guard<std::mutex> lock(parts_mutex);
|
||||
|
||||
if (parts_queue.empty())
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(parts_mutex);
|
||||
|
||||
if (parts_queue.empty())
|
||||
if (!parts_set.empty())
|
||||
{
|
||||
if (!parts_set.empty())
|
||||
LOG_ERROR(log, "Non-empty parts_set with empty parts_queue. This is a bug.");
|
||||
parts_set.clear();
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
for (auto it = parts_queue.begin(); it != parts_queue.end(); ++it)
|
||||
{
|
||||
if (it->second <= current_time)
|
||||
{
|
||||
LOG_ERROR(log, "Non-empty parts_set with empty parts_queue. This is a bug.");
|
||||
parts_set.clear();
|
||||
selected = it;
|
||||
break;
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
for (auto it = parts_queue.begin(); it != parts_queue.end(); ++it)
|
||||
{
|
||||
if (it->second <= current_time)
|
||||
{
|
||||
selected = it;
|
||||
break;
|
||||
}
|
||||
|
||||
if (it->second < min_check_time)
|
||||
min_check_time = it->second;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (selected == parts_queue.end())
|
||||
{
|
||||
/// Poco::Event is triggered immediately if `signal` was before the `wait` call.
|
||||
/// We can wait a little more than we need due to the use of the old `current_time`.
|
||||
|
||||
if (min_check_time != std::numeric_limits<time_t>::max() && min_check_time > current_time)
|
||||
wakeup_event.tryWait(1000 * (min_check_time - current_time));
|
||||
else
|
||||
wakeup_event.wait();
|
||||
|
||||
continue;
|
||||
}
|
||||
|
||||
checkPart(selected->first);
|
||||
|
||||
if (need_stop)
|
||||
break;
|
||||
|
||||
/// Remove the part from check queue.
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(parts_mutex);
|
||||
|
||||
if (parts_queue.empty())
|
||||
{
|
||||
LOG_ERROR(log, "Someone erased cheking part from parts_queue. This is a bug.");
|
||||
}
|
||||
else
|
||||
{
|
||||
parts_set.erase(selected->first);
|
||||
parts_queue.erase(selected);
|
||||
if (it->second < min_check_time)
|
||||
min_check_time = it->second;
|
||||
}
|
||||
}
|
||||
}
|
||||
catch (const zkutil::KeeperException & e)
|
||||
|
||||
if (selected == parts_queue.end())
|
||||
return;
|
||||
|
||||
checkPart(selected->first);
|
||||
|
||||
if (need_stop)
|
||||
return;
|
||||
|
||||
/// Remove the part from check queue.
|
||||
{
|
||||
tryLogCurrentException(log, __PRETTY_FUNCTION__);
|
||||
std::lock_guard<std::mutex> lock(parts_mutex);
|
||||
|
||||
if (e.code == ZooKeeperImpl::ZooKeeper::ZSESSIONEXPIRED)
|
||||
break;
|
||||
if (parts_queue.empty())
|
||||
{
|
||||
LOG_ERROR(log, "Someone erased cheking part from parts_queue. This is a bug.");
|
||||
}
|
||||
else
|
||||
{
|
||||
parts_set.erase(selected->first);
|
||||
parts_queue.erase(selected);
|
||||
}
|
||||
}
|
||||
|
||||
wakeup_event.tryWait(PART_CHECK_ERROR_SLEEP_MS);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(log, __PRETTY_FUNCTION__);
|
||||
wakeup_event.tryWait(PART_CHECK_ERROR_SLEEP_MS);
|
||||
}
|
||||
task_handle->schedule();
|
||||
}
|
||||
catch (const zkutil::KeeperException & e)
|
||||
{
|
||||
tryLogCurrentException(log, __PRETTY_FUNCTION__);
|
||||
|
||||
LOG_DEBUG(log, "Part check thread finished");
|
||||
if (e.code == ZooKeeperImpl::ZooKeeper::ZSESSIONEXPIRED)
|
||||
return;
|
||||
|
||||
task_handle->scheduleAfter(PART_CHECK_ERROR_SLEEP_MS);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(log, __PRETTY_FUNCTION__);
|
||||
task_handle->scheduleAfter(PART_CHECK_ERROR_SLEEP_MS);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -10,7 +10,7 @@
|
||||
#include <Poco/Event.h>
|
||||
#include <Core/Types.h>
|
||||
#include <common/logger_useful.h>
|
||||
|
||||
#include <Common/BackgroundSchedulePool.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -29,6 +29,7 @@ class ReplicatedMergeTreePartCheckThread
|
||||
{
|
||||
public:
|
||||
ReplicatedMergeTreePartCheckThread(StorageReplicatedMergeTree & storage_);
|
||||
~ReplicatedMergeTreePartCheckThread();
|
||||
|
||||
/// Processing of the queue to be checked is done in the background thread, which you must first start.
|
||||
void start();
|
||||
@ -65,10 +66,7 @@ public:
|
||||
/// Get the number of parts in the queue for check.
|
||||
size_t size() const;
|
||||
|
||||
~ReplicatedMergeTreePartCheckThread()
|
||||
{
|
||||
stop();
|
||||
}
|
||||
|
||||
|
||||
private:
|
||||
void run();
|
||||
@ -91,11 +89,10 @@ private:
|
||||
mutable std::mutex parts_mutex;
|
||||
StringSet parts_set;
|
||||
PartsToCheckQueue parts_queue;
|
||||
Poco::Event wakeup_event;
|
||||
|
||||
std::mutex start_stop_mutex;
|
||||
std::atomic<bool> need_stop { false };
|
||||
std::thread thread;
|
||||
BackgroundSchedulePool::TaskHandle task_handle;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -25,7 +25,7 @@ ReplicatedMergeTreeQueue::ReplicatedMergeTreeQueue(StorageReplicatedMergeTree &
|
||||
{}
|
||||
|
||||
|
||||
void ReplicatedMergeTreeQueue::initVirtualParts(const MergeTreeData::DataParts & parts)
|
||||
void ReplicatedMergeTreeQueue::addVirtualParts(const MergeTreeData::DataParts & parts)
|
||||
{
|
||||
std::lock_guard lock(target_state_mutex);
|
||||
|
||||
@ -87,7 +87,7 @@ void ReplicatedMergeTreeQueue::initialize(
|
||||
logger_name = logger_name_;
|
||||
log = &Logger::get(logger_name);
|
||||
|
||||
initVirtualParts(parts);
|
||||
addVirtualParts(parts);
|
||||
load(zookeeper);
|
||||
}
|
||||
|
||||
@ -97,7 +97,8 @@ void ReplicatedMergeTreeQueue::insertUnlocked(
|
||||
std::lock_guard<std::mutex> & /* target_state_lock */,
|
||||
std::lock_guard<std::mutex> & /* queue_lock */)
|
||||
{
|
||||
virtual_parts.add(entry->new_part_name);
|
||||
for (const String & virtual_part_name : entry->getVirtualPartNames())
|
||||
virtual_parts.add(virtual_part_name);
|
||||
|
||||
/// Put 'DROP PARTITION' entries at the beginning of the queue not to make superfluous fetches of parts that will be eventually deleted
|
||||
if (entry->type != LogEntry::DROP_RANGE)
|
||||
@ -205,6 +206,9 @@ void ReplicatedMergeTreeQueue::remove(zkutil::ZooKeeperPtr zookeeper, LogEntryPt
|
||||
std::optional<time_t> min_unprocessed_insert_time_changed;
|
||||
std::optional<time_t> max_processed_insert_time_changed;
|
||||
|
||||
bool found = false;
|
||||
size_t queue_size = 0;
|
||||
|
||||
{
|
||||
std::unique_lock<std::mutex> lock(queue_mutex);
|
||||
|
||||
@ -218,6 +222,8 @@ void ReplicatedMergeTreeQueue::remove(zkutil::ZooKeeperPtr zookeeper, LogEntryPt
|
||||
if (*it == entry)
|
||||
{
|
||||
queue.erase(it);
|
||||
found = true;
|
||||
queue_size = queue.size();
|
||||
break;
|
||||
}
|
||||
}
|
||||
@ -225,6 +231,11 @@ void ReplicatedMergeTreeQueue::remove(zkutil::ZooKeeperPtr zookeeper, LogEntryPt
|
||||
updateTimesOnRemoval(entry, min_unprocessed_insert_time_changed, max_processed_insert_time_changed, lock);
|
||||
}
|
||||
|
||||
if (!found)
|
||||
throw Exception("Can't find " + entry->znode_name + " in the memory queue. It is a bug", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
notifySubscribers(queue_size);
|
||||
|
||||
updateTimesInZooKeeper(zookeeper, min_unprocessed_insert_time_changed, max_processed_insert_time_changed);
|
||||
}
|
||||
|
||||
@ -232,6 +243,7 @@ void ReplicatedMergeTreeQueue::remove(zkutil::ZooKeeperPtr zookeeper, LogEntryPt
|
||||
bool ReplicatedMergeTreeQueue::remove(zkutil::ZooKeeperPtr zookeeper, const String & part_name)
|
||||
{
|
||||
LogEntryPtr found;
|
||||
size_t queue_size = 0;
|
||||
|
||||
std::optional<time_t> min_unprocessed_insert_time_changed;
|
||||
std::optional<time_t> max_processed_insert_time_changed;
|
||||
@ -248,6 +260,7 @@ bool ReplicatedMergeTreeQueue::remove(zkutil::ZooKeeperPtr zookeeper, const Stri
|
||||
{
|
||||
found = *it;
|
||||
queue.erase(it++);
|
||||
queue_size = queue.size();
|
||||
updateTimesOnRemoval(found, min_unprocessed_insert_time_changed, max_processed_insert_time_changed, queue_lock);
|
||||
break;
|
||||
}
|
||||
@ -259,6 +272,8 @@ bool ReplicatedMergeTreeQueue::remove(zkutil::ZooKeeperPtr zookeeper, const Stri
|
||||
if (!found)
|
||||
return false;
|
||||
|
||||
notifySubscribers(queue_size);
|
||||
|
||||
zookeeper->tryRemove(replica_path + "/queue/" + found->znode_name);
|
||||
updateTimesInZooKeeper(zookeeper, min_unprocessed_insert_time_changed, max_processed_insert_time_changed);
|
||||
|
||||
@ -273,14 +288,17 @@ bool ReplicatedMergeTreeQueue::removeFromVirtualParts(const MergeTreePartInfo &
|
||||
}
|
||||
|
||||
|
||||
void ReplicatedMergeTreeQueue::pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper, zkutil::EventPtr next_update_event)
|
||||
void ReplicatedMergeTreeQueue::pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper, BackgroundSchedulePool::TaskHandle update_task_handle)
|
||||
{
|
||||
std::lock_guard lock(pull_logs_to_queue_mutex);
|
||||
|
||||
String index_str = zookeeper->get(replica_path + "/log_pointer");
|
||||
UInt64 index;
|
||||
|
||||
Strings log_entries = zookeeper->getChildren(zookeeper_path + "/log", nullptr, next_update_event);
|
||||
zkutil::WatchCallback watch_callback;
|
||||
if (update_task_handle)
|
||||
watch_callback = update_task_handle->getWatchCallback();
|
||||
Strings log_entries = zookeeper->getChildrenWatch(zookeeper_path + "/log", nullptr, watch_callback);
|
||||
|
||||
/// We update mutations after we have loaded the list of log entries, but before we insert them
|
||||
/// in the queue.
|
||||
@ -416,12 +434,14 @@ void ReplicatedMergeTreeQueue::pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper, z
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void ReplicatedMergeTreeQueue::updateMutations(zkutil::ZooKeeperPtr zookeeper, zkutil::EventPtr next_update_event)
|
||||
void ReplicatedMergeTreeQueue::updateMutations(zkutil::ZooKeeperPtr zookeeper, BackgroundSchedulePool::TaskHandle update_task_handle)
|
||||
{
|
||||
std::lock_guard lock(update_mutations_mutex);
|
||||
|
||||
Strings entries_in_zk = zookeeper->getChildren(zookeeper_path + "/mutations", nullptr, next_update_event);
|
||||
zkutil::WatchCallback watch_callback;
|
||||
if (update_task_handle)
|
||||
watch_callback = update_task_handle->getWatchCallback();
|
||||
Strings entries_in_zk = zookeeper->getChildrenWatch(zookeeper_path + "/mutations", nullptr, watch_callback);
|
||||
StringSet entries_in_zk_set(entries_in_zk.begin(), entries_in_zk.end());
|
||||
|
||||
/// Compare with the local state, delete obsolete entries and determine which new entries to load.
|
||||
@ -486,7 +506,7 @@ void ReplicatedMergeTreeQueue::updateMutations(zkutil::ZooKeeperPtr zookeeper, z
|
||||
}
|
||||
}
|
||||
|
||||
storage.merge_selecting_event.set();
|
||||
storage.merge_selecting_task_handle->schedule();
|
||||
}
|
||||
}
|
||||
|
||||
@ -535,7 +555,7 @@ ReplicatedMergeTreeQueue::StringSet ReplicatedMergeTreeQueue::moveSiblingPartsFo
|
||||
}
|
||||
|
||||
|
||||
void ReplicatedMergeTreeQueue::removePartProducingOpsInRange(zkutil::ZooKeeperPtr zookeeper, const String & part_name)
|
||||
void ReplicatedMergeTreeQueue::removePartProducingOpsInRange(zkutil::ZooKeeperPtr zookeeper, const MergeTreePartInfo & part_info)
|
||||
{
|
||||
Queue to_wait;
|
||||
size_t removed_entries = 0;
|
||||
@ -546,8 +566,10 @@ void ReplicatedMergeTreeQueue::removePartProducingOpsInRange(zkutil::ZooKeeperPt
|
||||
std::unique_lock<std::mutex> lock(queue_mutex);
|
||||
for (Queue::iterator it = queue.begin(); it != queue.end();)
|
||||
{
|
||||
if (((*it)->type == LogEntry::GET_PART || (*it)->type == LogEntry::MERGE_PARTS || (*it)->type == LogEntry::MUTATE_PART) &&
|
||||
MergeTreePartInfo::contains(part_name, (*it)->new_part_name, format_version))
|
||||
auto type = (*it)->type;
|
||||
|
||||
if ((type == LogEntry::GET_PART || type == LogEntry::MERGE_PARTS || type == LogEntry::MUTATE_PART)
|
||||
&& part_info.contains(MergeTreePartInfo::fromPartName((*it)->new_part_name, format_version)))
|
||||
{
|
||||
if ((*it)->currently_executing)
|
||||
to_wait.push_back(*it);
|
||||
@ -575,31 +597,23 @@ void ReplicatedMergeTreeQueue::removePartProducingOpsInRange(zkutil::ZooKeeperPt
|
||||
}
|
||||
|
||||
|
||||
ReplicatedMergeTreeQueue::Queue ReplicatedMergeTreeQueue::getConflictsForClearColumnCommand(
|
||||
const LogEntry & entry, String * out_conflicts_description, std::lock_guard<std::mutex> & /* queue_lock */) const
|
||||
size_t ReplicatedMergeTreeQueue::getConflictsCountForRange(
|
||||
const MergeTreePartInfo & range, const String & range_znode,
|
||||
String * out_conflicts_description, std::lock_guard<std::mutex> & /* queue_lock */) const
|
||||
{
|
||||
Queue conflicts;
|
||||
std::vector<std::pair<LogEntryPtr, String>> conflicts;
|
||||
|
||||
for (auto & elem : queue)
|
||||
{
|
||||
if (elem->currently_executing && elem->znode_name != entry.znode_name)
|
||||
if (!elem->currently_executing || elem->znode_name == range_znode)
|
||||
continue;
|
||||
|
||||
for (const String & new_part_name : elem->getVirtualPartNames())
|
||||
{
|
||||
if (elem->type == LogEntry::MERGE_PARTS
|
||||
|| elem->type == LogEntry::GET_PART
|
||||
|| elem->type == LogEntry::MUTATE_PART
|
||||
|| elem->type == LogEntry::ATTACH_PART)
|
||||
if (!range.isDisjoint(MergeTreePartInfo::fromPartName(new_part_name, format_version)))
|
||||
{
|
||||
if (MergeTreePartInfo::contains(entry.new_part_name, elem->new_part_name, format_version))
|
||||
conflicts.emplace_back(elem);
|
||||
}
|
||||
|
||||
if (elem->type == LogEntry::CLEAR_COLUMN)
|
||||
{
|
||||
auto cur_part = MergeTreePartInfo::fromPartName(elem->new_part_name, format_version);
|
||||
auto part = MergeTreePartInfo::fromPartName(entry.new_part_name, format_version);
|
||||
|
||||
if (part.partition_id == cur_part.partition_id)
|
||||
conflicts.emplace_back(elem);
|
||||
conflicts.emplace_back(elem, new_part_name);
|
||||
continue;
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -607,28 +621,25 @@ ReplicatedMergeTreeQueue::Queue ReplicatedMergeTreeQueue::getConflictsForClearCo
|
||||
if (out_conflicts_description)
|
||||
{
|
||||
std::stringstream ss;
|
||||
ss << "Can't execute " << entry.typeToString() << " entry " << entry.znode_name << ". ";
|
||||
ss << "Can't execute command for range " << range.getPartName() << " (entry " << range_znode << "). ";
|
||||
ss << "There are " << conflicts.size() << " currently executing entries blocking it: ";
|
||||
for (const auto & conflict : conflicts)
|
||||
ss << conflict->typeToString() << " " << conflict->new_part_name << " " << conflict->znode_name << ", ";
|
||||
ss << conflict.first->typeToString() << " part " << conflict.second << ", ";
|
||||
|
||||
*out_conflicts_description = ss.str();
|
||||
}
|
||||
|
||||
return conflicts;
|
||||
return conflicts.size();
|
||||
}
|
||||
|
||||
|
||||
void ReplicatedMergeTreeQueue::disableMergesAndFetchesInRange(const LogEntry & entry)
|
||||
void ReplicatedMergeTreeQueue::checkThereAreNoConflictsInRange(const MergeTreePartInfo & range, const String & range_znode_name)
|
||||
{
|
||||
std::lock_guard lock(queue_mutex);
|
||||
String conflicts_description;
|
||||
std::lock_guard<std::mutex> lock(queue_mutex);
|
||||
|
||||
if (!getConflictsForClearColumnCommand(entry, &conflicts_description, lock).empty())
|
||||
if (0 != getConflictsCountForRange(range, range_znode_name, &conflicts_description, lock))
|
||||
throw Exception(conflicts_description, ErrorCodes::UNFINISHED);
|
||||
|
||||
if (!future_parts.count(entry.new_part_name))
|
||||
throw Exception("Expected that merges and fetches should be blocked in range " + entry.new_part_name + ". This is a bug", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
|
||||
@ -659,6 +670,8 @@ bool ReplicatedMergeTreeQueue::isNotCoveredByFuturePartsImpl(const String & new_
|
||||
|
||||
if (future_part.contains(result_part))
|
||||
{
|
||||
out_reason = "Not executing log entry for part " + new_part_name + " because it is covered by part "
|
||||
+ future_part_name + " that is currently executing";
|
||||
return false;
|
||||
}
|
||||
}
|
||||
@ -689,14 +702,16 @@ bool ReplicatedMergeTreeQueue::shouldExecuteLogEntry(
|
||||
{
|
||||
if (entry.type == LogEntry::MERGE_PARTS
|
||||
|| entry.type == LogEntry::GET_PART
|
||||
|| entry.type == LogEntry::ATTACH_PART
|
||||
|| entry.type == LogEntry::MUTATE_PART)
|
||||
{
|
||||
if (!isNotCoveredByFuturePartsImpl(entry.new_part_name, out_postpone_reason, queue_lock))
|
||||
for (const String & new_part_name : entry.getVirtualPartNames())
|
||||
{
|
||||
if (!out_postpone_reason.empty())
|
||||
LOG_DEBUG(log, out_postpone_reason);
|
||||
return false;
|
||||
if (!isNotCoveredByFuturePartsImpl(new_part_name, out_postpone_reason, queue_lock))
|
||||
{
|
||||
if (!out_postpone_reason.empty())
|
||||
LOG_DEBUG(log, out_postpone_reason);
|
||||
return false;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@ -749,10 +764,14 @@ bool ReplicatedMergeTreeQueue::shouldExecuteLogEntry(
|
||||
}
|
||||
}
|
||||
|
||||
if (entry.type == LogEntry::CLEAR_COLUMN)
|
||||
/// TODO: it makes sense to check DROP_RANGE also
|
||||
if (entry.type == LogEntry::CLEAR_COLUMN || entry.type == LogEntry::REPLACE_RANGE)
|
||||
{
|
||||
String conflicts_description;
|
||||
if (!getConflictsForClearColumnCommand(entry, &conflicts_description, queue_lock).empty())
|
||||
String range_name = (entry.type == LogEntry::REPLACE_RANGE) ? entry.replace_range_entry->drop_range_part_name : entry.new_part_name;
|
||||
auto range = MergeTreePartInfo::fromPartName(range_name, format_version);
|
||||
|
||||
if (0 != getConflictsCountForRange(range, entry.znode_name, &conflicts_description, queue_lock))
|
||||
{
|
||||
LOG_DEBUG(log, conflicts_description);
|
||||
return false;
|
||||
@ -793,8 +812,11 @@ ReplicatedMergeTreeQueue::CurrentlyExecuting::CurrentlyExecuting(ReplicatedMerge
|
||||
++entry->num_tries;
|
||||
entry->last_attempt_time = time(nullptr);
|
||||
|
||||
if (!queue.future_parts.insert(entry->new_part_name).second)
|
||||
throw Exception("Tagging already tagged future part " + entry->new_part_name + ". This is a bug.", ErrorCodes::LOGICAL_ERROR);
|
||||
for (const String & new_part_name : entry->getVirtualPartNames())
|
||||
{
|
||||
if (!queue.future_parts.insert(new_part_name).second)
|
||||
throw Exception("Tagging already tagged future part " + new_part_name + ". This is a bug.", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@ -822,8 +844,11 @@ ReplicatedMergeTreeQueue::CurrentlyExecuting::~CurrentlyExecuting()
|
||||
entry->currently_executing = false;
|
||||
entry->execution_complete.notify_all();
|
||||
|
||||
if (!queue.future_parts.erase(entry->new_part_name))
|
||||
LOG_ERROR(queue.log, "Untagging already untagged future part " + entry->new_part_name + ". This is a bug.");
|
||||
for (const String & new_part_name : entry->getVirtualPartNames())
|
||||
{
|
||||
if (!queue.future_parts.erase(new_part_name))
|
||||
LOG_ERROR(queue.log, "Untagging already untagged future part " + new_part_name + ". This is a bug.");
|
||||
}
|
||||
|
||||
if (!entry->actual_new_part_name.empty())
|
||||
{
|
||||
@ -837,10 +862,10 @@ ReplicatedMergeTreeQueue::CurrentlyExecuting::~CurrentlyExecuting()
|
||||
|
||||
ReplicatedMergeTreeQueue::SelectedEntry ReplicatedMergeTreeQueue::selectEntryToProcess(MergeTreeDataMergerMutator & merger_mutator, MergeTreeData & data)
|
||||
{
|
||||
std::lock_guard lock(queue_mutex);
|
||||
|
||||
LogEntryPtr entry;
|
||||
|
||||
std::lock_guard<std::mutex> lock(queue_mutex);
|
||||
|
||||
for (auto it = queue.begin(); it != queue.end(); ++it)
|
||||
{
|
||||
if ((*it)->currently_executing)
|
||||
@ -1280,11 +1305,41 @@ std::optional<Int64> ReplicatedMergeTreeMergePredicate::getDesiredMutationVersio
|
||||
return max_version;
|
||||
}
|
||||
|
||||
ReplicatedMergeTreeQueue::SubscriberHandler
|
||||
ReplicatedMergeTreeQueue::addSubscriber(ReplicatedMergeTreeQueue::SubscriberCallBack && callback)
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(queue_mutex);
|
||||
std::lock_guard<std::mutex> lock_subscribers(subscribers_mutex);
|
||||
|
||||
auto it = subscribers.emplace(subscribers.end(), std::move(callback));
|
||||
|
||||
/// Atomically notify about current size
|
||||
(*it)(queue.size());
|
||||
|
||||
return SubscriberHandler(it, *this);
|
||||
}
|
||||
|
||||
ReplicatedMergeTreeQueue::SubscriberHandler::~SubscriberHandler()
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(queue.subscribers_mutex);
|
||||
queue.subscribers.erase(it);
|
||||
}
|
||||
|
||||
void ReplicatedMergeTreeQueue::notifySubscribers(size_t new_queue_size)
|
||||
{
|
||||
std::lock_guard<std::mutex> lock_subscribers(subscribers_mutex);
|
||||
for (auto & subscriber_callback : subscribers)
|
||||
subscriber_callback(new_queue_size);
|
||||
}
|
||||
|
||||
ReplicatedMergeTreeQueue::~ReplicatedMergeTreeQueue()
|
||||
{
|
||||
notifySubscribers(0);
|
||||
}
|
||||
|
||||
String padIndex(Int64 index)
|
||||
{
|
||||
String index_str = toString(index);
|
||||
return std::string(10 - index_str.size(), '0') + index_str;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -2,12 +2,14 @@
|
||||
|
||||
#include <optional>
|
||||
|
||||
#include <Common/ActionBlocker.h>
|
||||
#include <Storages/MergeTree/ReplicatedMergeTreeLogEntry.h>
|
||||
#include <Storages/MergeTree/ReplicatedMergeTreeMutationEntry.h>
|
||||
#include <Storages/MergeTree/ActiveDataPartSet.h>
|
||||
#include <Storages/MergeTree/MergeTreeData.h>
|
||||
|
||||
#include <Common/ZooKeeper/ZooKeeper.h>
|
||||
#include <Common/BackgroundSchedulePool.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -71,6 +73,7 @@ private:
|
||||
/// Used to not perform other actions at the same time with these parts.
|
||||
StringSet future_parts;
|
||||
|
||||
|
||||
/// Protects virtual_parts, log_pointer, mutations.
|
||||
/// If you intend to lock both target_state_mutex and queue_mutex, lock target_state_mutex first.
|
||||
mutable std::mutex target_state_mutex;
|
||||
@ -90,14 +93,41 @@ private:
|
||||
std::map<String, ReplicatedMergeTreeMutationEntry> mutations_by_znode;
|
||||
std::unordered_map<String, std::map<Int64, const ReplicatedMergeTreeMutationEntry *>> mutations_by_partition;
|
||||
|
||||
|
||||
/// Provides only one simultaneous call to pullLogsToQueue.
|
||||
std::mutex pull_logs_to_queue_mutex;
|
||||
|
||||
|
||||
/// List of subscribers
|
||||
/// A subscriber callback is called when an entry queue is deleted
|
||||
mutable std::mutex subscribers_mutex;
|
||||
|
||||
using SubscriberCallBack = std::function<void(size_t /* queue_size */)>;
|
||||
using Subscribers = std::list<SubscriberCallBack>;
|
||||
using SubscriberIterator = Subscribers::iterator;
|
||||
|
||||
friend class SubscriberHandler;
|
||||
struct SubscriberHandler : public boost::noncopyable
|
||||
{
|
||||
SubscriberHandler(SubscriberIterator it, ReplicatedMergeTreeQueue & queue) : it(it), queue(queue) {}
|
||||
~SubscriberHandler();
|
||||
|
||||
private:
|
||||
SubscriberIterator it;
|
||||
ReplicatedMergeTreeQueue & queue;
|
||||
};
|
||||
|
||||
Subscribers subscribers;
|
||||
|
||||
/// Notify subscribers about queue change
|
||||
void notifySubscribers(size_t new_queue_size);
|
||||
|
||||
|
||||
/// Ensures that only one thread is simultaneously updating mutations.
|
||||
std::mutex update_mutations_mutex;
|
||||
|
||||
/// Put a set of (already existing) parts in virtual_parts.
|
||||
void initVirtualParts(const MergeTreeData::DataParts & parts);
|
||||
void addVirtualParts(const MergeTreeData::DataParts & parts);
|
||||
|
||||
/// Load (initialize) a queue from ZooKeeper (/replicas/me/queue/).
|
||||
bool load(zkutil::ZooKeeperPtr zookeeper);
|
||||
@ -138,8 +168,10 @@ private:
|
||||
std::optional<time_t> min_unprocessed_insert_time_changed,
|
||||
std::optional<time_t> max_processed_insert_time_changed) const;
|
||||
|
||||
/// Returns list of currently executing entries blocking execution of specified CLEAR_COLUMN command
|
||||
Queue getConflictsForClearColumnCommand(const LogEntry & entry, String * out_conflicts_description, std::lock_guard<std::mutex> & queue_lock) const;
|
||||
/// Returns list of currently executing entries blocking execution a command modifying specified range
|
||||
size_t getConflictsCountForRange(
|
||||
const MergeTreePartInfo & range, const String & range_znode, String * out_conflicts_description,
|
||||
std::lock_guard<std::mutex> & queue_lock) const;
|
||||
|
||||
/// Marks the element of the queue as running.
|
||||
class CurrentlyExecuting
|
||||
@ -163,6 +195,8 @@ private:
|
||||
public:
|
||||
ReplicatedMergeTreeQueue(StorageReplicatedMergeTree & storage_);
|
||||
|
||||
~ReplicatedMergeTreeQueue();
|
||||
|
||||
void initialize(const String & zookeeper_path_, const String & replica_path_, const String & logger_name_,
|
||||
const MergeTreeData::DataParts & parts, zkutil::ZooKeeperPtr zookeeper);
|
||||
|
||||
@ -180,24 +214,24 @@ public:
|
||||
bool removeFromVirtualParts(const MergeTreePartInfo & part_info);
|
||||
|
||||
/** Copy the new entries from the shared log to the queue of this replica. Set the log_pointer to the appropriate value.
|
||||
* If next_update_event != nullptr, will call this event when new entries appear in the log.
|
||||
* If update_task_handle != nullptr, will schedule this task when new entries appear in the log.
|
||||
* If there were new entries, notifies storage.queue_task_handle.
|
||||
* Additionally loads mutations (so that the set of mutations is always more recent than the queue).
|
||||
*/
|
||||
void pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper, zkutil::EventPtr next_update_event);
|
||||
void pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper, BackgroundSchedulePool::TaskHandle update_task_handle);
|
||||
|
||||
/// Load new mutation entries. If something new is loaded, notify storage.merge_selecting_event.
|
||||
void updateMutations(zkutil::ZooKeeperPtr zookeeper, zkutil::EventPtr next_update_event);
|
||||
/// Load new mutation entries. If something new is loaded, schedule storage.merge_selecting_task_handle.
|
||||
/// If update_task_handle != nullptr, will schedule this task when new mutations appear in ZK.
|
||||
void updateMutations(zkutil::ZooKeeperPtr zookeeper, BackgroundSchedulePool::TaskHandle update_task_handle);
|
||||
|
||||
/** Remove the action from the queue with the parts covered by part_name (from ZK and from the RAM).
|
||||
* And also wait for the completion of their execution, if they are now being executed.
|
||||
*/
|
||||
void removePartProducingOpsInRange(zkutil::ZooKeeperPtr zookeeper, const String & part_name);
|
||||
void removePartProducingOpsInRange(zkutil::ZooKeeperPtr zookeeper, const MergeTreePartInfo & part_info);
|
||||
|
||||
/** Disables future merges and fetches inside entry.new_part_name
|
||||
* If there are currently executing merges or fetches then throws exception.
|
||||
/** Throws and exception if there are currently executing entries in the range .
|
||||
*/
|
||||
void disableMergesAndFetchesInRange(const LogEntry & entry);
|
||||
void checkThereAreNoConflictsInRange(const MergeTreePartInfo & range, const String & range_znode_name);
|
||||
|
||||
/** In the case where there are not enough parts to perform the merge in part_name
|
||||
* - move actions with merged parts to the end of the queue
|
||||
@ -237,6 +271,12 @@ public:
|
||||
*/
|
||||
bool addFuturePartIfNotCoveredByThem(const String & part_name, const LogEntry & entry, String & reject_reason);
|
||||
|
||||
/// A blocker that stops selects from the queue
|
||||
ActionBlocker actions_blocker;
|
||||
|
||||
/// Adds a subscriber
|
||||
SubscriberHandler addSubscriber(SubscriberCallBack && callback);
|
||||
|
||||
struct Status
|
||||
{
|
||||
UInt32 future_parts;
|
||||
|
@ -28,6 +28,10 @@ namespace ErrorCodes
|
||||
extern const int REPLICA_IS_ALREADY_ACTIVE;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
constexpr auto retry_period_ms = 10 * 1000;
|
||||
}
|
||||
|
||||
/// Used to check whether it's us who set node `is_active`, or not.
|
||||
static String generateActiveNodeIdentifier()
|
||||
@ -35,137 +39,145 @@ static String generateActiveNodeIdentifier()
|
||||
return "pid: " + toString(getpid()) + ", random: " + toString(randomSeed());
|
||||
}
|
||||
|
||||
|
||||
ReplicatedMergeTreeRestartingThread::ReplicatedMergeTreeRestartingThread(StorageReplicatedMergeTree & storage_)
|
||||
: storage(storage_),
|
||||
log(&Logger::get(storage.database_name + "." + storage.table_name + " (StorageReplicatedMergeTree, RestartingThread)")),
|
||||
active_node_identifier(generateActiveNodeIdentifier()),
|
||||
thread([this] { run(); })
|
||||
active_node_identifier(generateActiveNodeIdentifier())
|
||||
{
|
||||
}
|
||||
|
||||
|
||||
void ReplicatedMergeTreeRestartingThread::run()
|
||||
{
|
||||
constexpr auto retry_period_ms = 10 * 1000;
|
||||
|
||||
/// The frequency of checking expiration of session in ZK.
|
||||
Int64 check_period_ms = storage.data.settings.zookeeper_session_expiration_check_period.totalSeconds() * 1000;
|
||||
check_period_ms = storage.data.settings.zookeeper_session_expiration_check_period.totalSeconds() * 1000;
|
||||
|
||||
/// Periodicity of checking lag of replica.
|
||||
if (check_period_ms > static_cast<Int64>(storage.data.settings.check_delay_period) * 1000)
|
||||
check_period_ms = storage.data.settings.check_delay_period * 1000;
|
||||
|
||||
setThreadName("ReplMTRestart");
|
||||
storage.queue_updating_task_handle = storage.context.getSchedulePool().addTask("StorageReplicatedMergeTree::queueUpdatingThread", [this]{ storage.queueUpdatingThread(); });
|
||||
storage.queue_updating_task_handle->deactivate();
|
||||
|
||||
bool first_time = true; /// Activate replica for the first time.
|
||||
time_t prev_time_of_check_delay = 0;
|
||||
storage.mutations_updating_task_handle = storage.context.getSchedulePool().addTask("StorageReplicatedMergeTree::mutationsUpdatingThread", [this]{ storage.mutationsUpdatingThread(); });
|
||||
|
||||
/// Starts the replica when the server starts/creates a table. Restart the replica when session expires with ZK.
|
||||
while (!need_stop)
|
||||
{
|
||||
try
|
||||
{
|
||||
if (first_time || storage.getZooKeeper()->expired())
|
||||
{
|
||||
if (first_time)
|
||||
{
|
||||
LOG_DEBUG(log, "Activating replica.");
|
||||
}
|
||||
else
|
||||
{
|
||||
LOG_WARNING(log, "ZooKeeper session has expired. Switching to a new session.");
|
||||
task_handle = storage.context.getSchedulePool().addTask("ReplicatedMergeTreeRestartingThread", [this]{ run(); });
|
||||
task_handle->schedule();
|
||||
}
|
||||
|
||||
bool old_val = false;
|
||||
if (storage.is_readonly.compare_exchange_strong(old_val, true))
|
||||
CurrentMetrics::add(CurrentMetrics::ReadonlyReplica);
|
||||
ReplicatedMergeTreeRestartingThread::~ReplicatedMergeTreeRestartingThread()
|
||||
{
|
||||
storage.context.getSchedulePool().removeTask(task_handle);
|
||||
completeShutdown();
|
||||
storage.context.getSchedulePool().removeTask(storage.queue_updating_task_handle);
|
||||
}
|
||||
|
||||
partialShutdown();
|
||||
}
|
||||
|
||||
while (!need_stop)
|
||||
{
|
||||
try
|
||||
{
|
||||
storage.setZooKeeper(storage.context.getZooKeeper());
|
||||
}
|
||||
catch (const zkutil::KeeperException & e)
|
||||
{
|
||||
/// The exception when you try to zookeeper_init usually happens if DNS does not work. We will try to do it again.
|
||||
tryLogCurrentException(log, __PRETTY_FUNCTION__);
|
||||
|
||||
if (first_time)
|
||||
storage.startup_event.set();
|
||||
wakeup_event.tryWait(retry_period_ms);
|
||||
continue;
|
||||
}
|
||||
|
||||
if (!need_stop && !tryStartup())
|
||||
{
|
||||
if (first_time)
|
||||
storage.startup_event.set();
|
||||
wakeup_event.tryWait(retry_period_ms);
|
||||
continue;
|
||||
}
|
||||
|
||||
if (first_time)
|
||||
storage.startup_event.set();
|
||||
break;
|
||||
}
|
||||
|
||||
if (need_stop)
|
||||
break;
|
||||
|
||||
bool old_val = true;
|
||||
if (storage.is_readonly.compare_exchange_strong(old_val, false))
|
||||
CurrentMetrics::sub(CurrentMetrics::ReadonlyReplica);
|
||||
|
||||
first_time = false;
|
||||
}
|
||||
|
||||
time_t current_time = time(nullptr);
|
||||
if (current_time >= prev_time_of_check_delay + static_cast<time_t>(storage.data.settings.check_delay_period))
|
||||
{
|
||||
/// Find out lag of replicas.
|
||||
time_t absolute_delay = 0;
|
||||
time_t relative_delay = 0;
|
||||
|
||||
storage.getReplicaDelays(absolute_delay, relative_delay);
|
||||
|
||||
if (absolute_delay)
|
||||
LOG_TRACE(log, "Absolute delay: " << absolute_delay << ". Relative delay: " << relative_delay << ".");
|
||||
|
||||
prev_time_of_check_delay = current_time;
|
||||
|
||||
/// We give up leadership if the relative lag is greater than threshold.
|
||||
if (storage.is_leader
|
||||
&& relative_delay > static_cast<time_t>(storage.data.settings.min_relative_delay_to_yield_leadership))
|
||||
{
|
||||
LOG_INFO(log, "Relative replica delay (" << relative_delay << " seconds) is bigger than threshold ("
|
||||
<< storage.data.settings.min_relative_delay_to_yield_leadership << "). Will yield leadership.");
|
||||
|
||||
ProfileEvents::increment(ProfileEvents::ReplicaYieldLeadership);
|
||||
|
||||
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();
|
||||
}
|
||||
}
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
storage.startup_event.set();
|
||||
tryLogCurrentException(log, __PRETTY_FUNCTION__);
|
||||
}
|
||||
|
||||
wakeup_event.tryWait(check_period_ms);
|
||||
}
|
||||
void ReplicatedMergeTreeRestartingThread::run()
|
||||
{
|
||||
if (need_stop)
|
||||
return;
|
||||
|
||||
try
|
||||
{
|
||||
storage.data_parts_exchange_endpoint_holder->cancelForever();
|
||||
if (first_time || storage.getZooKeeper()->expired())
|
||||
{
|
||||
startup_completed = false;
|
||||
|
||||
if (first_time)
|
||||
{
|
||||
LOG_DEBUG(log, "Activating replica.");
|
||||
}
|
||||
else
|
||||
{
|
||||
LOG_WARNING(log, "ZooKeeper session has expired. Switching to a new session.");
|
||||
|
||||
bool old_val = false;
|
||||
if (storage.is_readonly.compare_exchange_strong(old_val, true))
|
||||
CurrentMetrics::add(CurrentMetrics::ReadonlyReplica);
|
||||
|
||||
partialShutdown();
|
||||
}
|
||||
|
||||
if (!startup_completed)
|
||||
{
|
||||
try
|
||||
{
|
||||
storage.setZooKeeper(storage.context.getZooKeeper());
|
||||
}
|
||||
catch (const zkutil::KeeperException & e)
|
||||
{
|
||||
/// The exception when you try to zookeeper_init usually happens if DNS does not work. We will try to do it again.
|
||||
tryLogCurrentException(log, __PRETTY_FUNCTION__);
|
||||
|
||||
if (first_time)
|
||||
storage.startup_event.set();
|
||||
task_handle->scheduleAfter(retry_period_ms);
|
||||
return;
|
||||
}
|
||||
|
||||
if (!need_stop && !tryStartup())
|
||||
{
|
||||
if (first_time)
|
||||
storage.startup_event.set();
|
||||
task_handle->scheduleAfter(retry_period_ms);
|
||||
return;
|
||||
}
|
||||
|
||||
if (first_time)
|
||||
storage.startup_event.set();
|
||||
|
||||
startup_completed = true;
|
||||
}
|
||||
|
||||
if (need_stop)
|
||||
return;
|
||||
|
||||
bool old_val = true;
|
||||
if (storage.is_readonly.compare_exchange_strong(old_val, false))
|
||||
CurrentMetrics::sub(CurrentMetrics::ReadonlyReplica);
|
||||
|
||||
first_time = false;
|
||||
}
|
||||
|
||||
time_t current_time = time(nullptr);
|
||||
if (current_time >= prev_time_of_check_delay + static_cast<time_t>(storage.data.settings.check_delay_period))
|
||||
{
|
||||
/// Find out lag of replicas.
|
||||
time_t absolute_delay = 0;
|
||||
time_t relative_delay = 0;
|
||||
|
||||
storage.getReplicaDelays(absolute_delay, relative_delay);
|
||||
|
||||
if (absolute_delay)
|
||||
LOG_TRACE(log, "Absolute delay: " << absolute_delay << ". Relative delay: " << relative_delay << ".");
|
||||
|
||||
prev_time_of_check_delay = current_time;
|
||||
|
||||
/// We give up leadership if the relative lag is greater than threshold.
|
||||
if (storage.is_leader
|
||||
&& relative_delay > static_cast<time_t>(storage.data.settings.min_relative_delay_to_yield_leadership))
|
||||
{
|
||||
LOG_INFO(log, "Relative replica delay (" << relative_delay << " seconds) is bigger than threshold ("
|
||||
<< storage.data.settings.min_relative_delay_to_yield_leadership << "). Will yield leadership.");
|
||||
|
||||
ProfileEvents::increment(ProfileEvents::ReplicaYieldLeadership);
|
||||
|
||||
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();
|
||||
}
|
||||
}
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
storage.startup_event.set();
|
||||
tryLogCurrentException(log, __PRETTY_FUNCTION__);
|
||||
}
|
||||
|
||||
task_handle->scheduleAfter(check_period_ms);
|
||||
}
|
||||
|
||||
void ReplicatedMergeTreeRestartingThread::completeShutdown()
|
||||
{
|
||||
try
|
||||
{
|
||||
storage.data_parts_exchange_endpoint_holder->getBlocker().cancelForever();
|
||||
storage.data_parts_exchange_endpoint_holder = nullptr;
|
||||
|
||||
/// Cancel fetches, merges and mutations to force the queue_task to finish ASAP.
|
||||
@ -182,8 +194,6 @@ void ReplicatedMergeTreeRestartingThread::run()
|
||||
{
|
||||
tryLogCurrentException(log, __PRETTY_FUNCTION__);
|
||||
}
|
||||
|
||||
LOG_DEBUG(log, "Restarting thread finished");
|
||||
}
|
||||
|
||||
|
||||
@ -204,8 +214,10 @@ bool ReplicatedMergeTreeRestartingThread::tryStartup()
|
||||
storage.shutdown_called = false;
|
||||
storage.shutdown_event.reset();
|
||||
|
||||
storage.queue_updating_thread = std::thread(&StorageReplicatedMergeTree::queueUpdatingThread, &storage);
|
||||
storage.mutations_updating_thread = std::thread(&StorageReplicatedMergeTree::mutationsUpdatingThread, &storage);
|
||||
storage.queue_updating_task_handle->activate();
|
||||
storage.queue_updating_task_handle->schedule();
|
||||
storage.mutations_updating_task_handle->activate();
|
||||
storage.mutations_updating_task_handle->schedule();
|
||||
storage.part_check_thread.start();
|
||||
storage.alter_thread = std::make_unique<ReplicatedMergeTreeAlterThread>(storage);
|
||||
storage.cleanup_thread = std::make_unique<ReplicatedMergeTreeCleanupThread>(storage);
|
||||
@ -249,22 +261,18 @@ void ReplicatedMergeTreeRestartingThread::removeFailedQuorumParts()
|
||||
if (zookeeper->tryGetChildren(storage.zookeeper_path + "/quorum/failed_parts", failed_parts) != ZooKeeperImpl::ZooKeeper::ZOK)
|
||||
return;
|
||||
|
||||
/// Firstly, remove parts from ZooKeeper
|
||||
storage.tryRemovePartsFromZooKeeperWithRetries(failed_parts);
|
||||
|
||||
for (auto part_name : failed_parts)
|
||||
{
|
||||
auto part = storage.data.getPartIfExists(
|
||||
part_name, {MergeTreeDataPartState::PreCommitted, MergeTreeDataPartState::Committed, MergeTreeDataPartState::Outdated});
|
||||
|
||||
if (part)
|
||||
{
|
||||
LOG_DEBUG(log, "Found part " << part_name << " with failed quorum. Moving to detached. This shouldn't happen often.");
|
||||
|
||||
zkutil::Requests ops;
|
||||
zkutil::Responses responses;
|
||||
storage.removePartFromZooKeeper(part_name, ops);
|
||||
auto code = zookeeper->tryMulti(ops, responses);
|
||||
if (code == ZooKeeperImpl::ZooKeeper::ZNONODE)
|
||||
LOG_WARNING(log, "Part " << part_name << " with failed quorum is not in ZooKeeper. This shouldn't happen often.");
|
||||
|
||||
storage.data.renameAndDetachPart(part, "noquorum_");
|
||||
storage.data.forgetPartAndMoveToDetached(part, "noquorum_");
|
||||
storage.queue.removeFromVirtualParts(part->info);
|
||||
}
|
||||
}
|
||||
@ -350,22 +358,15 @@ void ReplicatedMergeTreeRestartingThread::partialShutdown()
|
||||
|
||||
storage.shutdown_called = true;
|
||||
storage.shutdown_event.set();
|
||||
storage.merge_selecting_event.set();
|
||||
storage.queue_updating_event->set();
|
||||
storage.mutations_updating_event->set();
|
||||
storage.alter_query_event->set();
|
||||
storage.cleanup_thread_event.set();
|
||||
storage.replica_is_active_node = nullptr;
|
||||
|
||||
LOG_TRACE(log, "Waiting for threads to finish");
|
||||
|
||||
storage.exitLeaderElection();
|
||||
|
||||
if (storage.queue_updating_thread.joinable())
|
||||
storage.queue_updating_thread.join();
|
||||
|
||||
if (storage.mutations_updating_thread.joinable())
|
||||
storage.mutations_updating_thread.join();
|
||||
storage.queue_updating_task_handle->deactivate();
|
||||
storage.mutations_updating_task_handle->deactivate();
|
||||
|
||||
storage.cleanup_thread.reset();
|
||||
storage.alter_thread.reset();
|
||||
|
@ -2,6 +2,7 @@
|
||||
|
||||
#include <Poco/Event.h>
|
||||
#include <common/logger_useful.h>
|
||||
#include <Common/BackgroundSchedulePool.h>
|
||||
#include <Core/Types.h>
|
||||
#include <thread>
|
||||
#include <atomic>
|
||||
@ -22,16 +23,12 @@ class ReplicatedMergeTreeRestartingThread
|
||||
{
|
||||
public:
|
||||
ReplicatedMergeTreeRestartingThread(StorageReplicatedMergeTree & storage_);
|
||||
|
||||
~ReplicatedMergeTreeRestartingThread()
|
||||
{
|
||||
if (thread.joinable())
|
||||
thread.join();
|
||||
}
|
||||
~ReplicatedMergeTreeRestartingThread();
|
||||
|
||||
void wakeup()
|
||||
{
|
||||
wakeup_event.set();
|
||||
task_handle->schedule();
|
||||
}
|
||||
|
||||
Poco::Event & getWakeupEvent()
|
||||
@ -42,7 +39,7 @@ public:
|
||||
void stop()
|
||||
{
|
||||
need_stop = true;
|
||||
wakeup();
|
||||
wakeup_event.set();
|
||||
}
|
||||
|
||||
private:
|
||||
@ -54,9 +51,14 @@ private:
|
||||
/// The random data we wrote into `/replicas/me/is_active`.
|
||||
String active_node_identifier;
|
||||
|
||||
std::thread thread;
|
||||
BackgroundSchedulePool::TaskHandle task_handle;
|
||||
Int64 check_period_ms; /// The frequency of checking expiration of session in ZK.
|
||||
bool first_time = true; /// Activate replica for the first time.
|
||||
time_t prev_time_of_check_delay = 0;
|
||||
bool startup_completed = false;
|
||||
|
||||
void run();
|
||||
void completeShutdown();
|
||||
|
||||
/// Start or stop background threads. Used for partial reinitialization when re-creating a session in ZooKeeper.
|
||||
bool tryStartup(); /// Returns false if ZooKeeper is not available.
|
||||
|
@ -2,7 +2,7 @@
|
||||
|
||||
#include <memory>
|
||||
#include <unordered_map>
|
||||
|
||||
#include <Parsers/StringRange.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -14,7 +14,7 @@ class Set;
|
||||
using SetPtr = std::shared_ptr<Set>;
|
||||
|
||||
/// Information about calculated sets in right hand side of IN.
|
||||
using PreparedSets = std::unordered_map<IAST*, SetPtr>;
|
||||
using PreparedSets = std::unordered_map<StringRange, SetPtr, StringRangePointersHash, StringRangePointersEqualTo>;
|
||||
|
||||
|
||||
/** Query along with some additional data,
|
||||
|
@ -84,33 +84,6 @@ bool StorageMerge::isRemote() const
|
||||
}
|
||||
|
||||
|
||||
namespace
|
||||
{
|
||||
using NodeHashToSet = std::map<IAST::Hash, SetPtr>;
|
||||
|
||||
void relinkSetsImpl(const ASTPtr & query, const NodeHashToSet & node_hash_to_set, PreparedSets & new_sets)
|
||||
{
|
||||
auto hash = query->getTreeHash();
|
||||
auto it = node_hash_to_set.find(hash);
|
||||
if (node_hash_to_set.end() != it)
|
||||
new_sets[query.get()] = it->second;
|
||||
|
||||
for (const auto & child : query->children)
|
||||
relinkSetsImpl(child, node_hash_to_set, new_sets);
|
||||
}
|
||||
|
||||
/// Re-link prepared sets onto cloned and modified AST.
|
||||
void relinkSets(const ASTPtr & query, const PreparedSets & old_sets, PreparedSets & new_sets)
|
||||
{
|
||||
NodeHashToSet node_hash_to_set;
|
||||
for (const auto & node_set : old_sets)
|
||||
node_hash_to_set.emplace(node_set.first->getTreeHash(), node_set.second);
|
||||
|
||||
relinkSetsImpl(query, node_hash_to_set, new_sets);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
bool StorageMerge::mayBenefitFromIndexForIn(const ASTPtr & left_in_operand) const
|
||||
{
|
||||
/// It's beneficial if it is true for at least one table.
|
||||
@ -210,8 +183,7 @@ BlockInputStreams StorageMerge::read(
|
||||
|
||||
SelectQueryInfo modified_query_info;
|
||||
modified_query_info.query = modified_query_ast;
|
||||
|
||||
relinkSets(modified_query_info.query, query_info.sets, modified_query_info.sets);
|
||||
modified_query_info.sets = query_info.sets;
|
||||
|
||||
BlockInputStreams source_streams;
|
||||
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <optional>
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <Common/localBackup.h>
|
||||
#include <Storages/StorageMergeTree.h>
|
||||
#include <Storages/MergeTree/MergeTreeBlockOutputStream.h>
|
||||
#include <Storages/MergeTree/DiskSpaceMonitor.h>
|
||||
@ -16,6 +17,7 @@
|
||||
|
||||
#include <Poco/DirectoryIterator.h>
|
||||
#include <Poco/File.h>
|
||||
#include <Parsers/queryToString.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -28,6 +30,12 @@ namespace ErrorCodes
|
||||
extern const int INCORRECT_DATA;
|
||||
extern const int INCORRECT_FILE_NAME;
|
||||
extern const int CANNOT_ASSIGN_OPTIMIZE;
|
||||
extern const int INCOMPATIBLE_COLUMNS;
|
||||
}
|
||||
|
||||
namespace ActionLocks
|
||||
{
|
||||
extern const StorageActionBlockType PartsMerge;
|
||||
}
|
||||
|
||||
|
||||
@ -379,6 +387,9 @@ bool StorageMergeTree::mergeTask()
|
||||
if (shutdown_called)
|
||||
return false;
|
||||
|
||||
if (merger.actions_blocker.isCancelled())
|
||||
return false;
|
||||
|
||||
try
|
||||
{
|
||||
size_t aio_threshold = context.getSettings().min_bytes_to_use_direct_io;
|
||||
@ -407,7 +418,7 @@ void StorageMergeTree::clearColumnInPartition(const ASTPtr & partition, const Fi
|
||||
auto lock_read_structure = lockStructure(false, __PRETTY_FUNCTION__);
|
||||
|
||||
String partition_id = data.getPartitionIDFromQuery(partition, context);
|
||||
MergeTreeData::DataParts parts = data.getDataParts();
|
||||
auto parts = data.getDataPartsVectorInPartition(MergeTreeDataPartState::Committed, partition_id);
|
||||
|
||||
std::vector<MergeTreeData::AlterDataPartTransactionPtr> transactions;
|
||||
|
||||
@ -422,7 +433,7 @@ void StorageMergeTree::clearColumnInPartition(const ASTPtr & partition, const Fi
|
||||
for (const auto & part : parts)
|
||||
{
|
||||
if (part->info.partition_id != partition_id)
|
||||
continue;
|
||||
throw Exception("Unexpected partition ID " + part->info.partition_id + ". This is a bug.", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
if (auto transaction = data.alterDataPart(part, columns_for_parts, data.primary_expr_ast, false))
|
||||
transactions.push_back(std::move(transaction));
|
||||
@ -462,32 +473,33 @@ bool StorageMergeTree::optimize(
|
||||
|
||||
void StorageMergeTree::dropPartition(const ASTPtr & /*query*/, const ASTPtr & partition, bool detach, const Context & context)
|
||||
{
|
||||
/// Asks to complete merges and does not allow them to start.
|
||||
/// This protects against "revival" of data for a removed partition after completion of merge.
|
||||
auto merge_blocker = merger.actions_blocker.cancel();
|
||||
/// Waits for completion of merge and does not start new ones.
|
||||
auto lock = lockForAlter(__PRETTY_FUNCTION__);
|
||||
|
||||
String partition_id = data.getPartitionIDFromQuery(partition, context);
|
||||
|
||||
size_t removed_parts = 0;
|
||||
MergeTreeData::DataParts parts = data.getDataParts();
|
||||
|
||||
for (const auto & part : parts)
|
||||
{
|
||||
if (part->info.partition_id != partition_id)
|
||||
continue;
|
||||
/// Asks to complete merges and does not allow them to start.
|
||||
/// This protects against "revival" of data for a removed partition after completion of merge.
|
||||
auto merge_blocker = merger.actions_blocker.cancel();
|
||||
/// Waits for completion of merge and does not start new ones.
|
||||
auto lock = lockForAlter(__PRETTY_FUNCTION__);
|
||||
|
||||
LOG_DEBUG(log, "Removing part " << part->name);
|
||||
++removed_parts;
|
||||
String partition_id = data.getPartitionIDFromQuery(partition, context);
|
||||
|
||||
/// TODO: should we include PreComitted parts like in Replicated case?
|
||||
auto parts_to_remove = data.getDataPartsVectorInPartition(MergeTreeDataPartState::Committed, partition_id);
|
||||
data.removePartsFromWorkingSet(parts_to_remove, true);
|
||||
|
||||
if (detach)
|
||||
data.renameAndDetachPart(part, "");
|
||||
else
|
||||
data.removePartsFromWorkingSet({part}, false);
|
||||
{
|
||||
/// If DETACH clone parts to detached/ directory
|
||||
for (const auto & part : parts_to_remove)
|
||||
{
|
||||
LOG_INFO(log, "Detaching " << part->relative_path);
|
||||
part->makeCloneInDetached("");
|
||||
}
|
||||
}
|
||||
|
||||
LOG_INFO(log, (detach ? "Detached " : "Removed ") << parts_to_remove.size() << " parts inside partition ID " << partition_id << ".");
|
||||
}
|
||||
|
||||
LOG_INFO(log, (detach ? "Detached " : "Removed ") << removed_parts << " parts inside partition ID " << partition_id << ".");
|
||||
data.clearOldPartsFromFilesystem();
|
||||
}
|
||||
|
||||
|
||||
@ -551,4 +563,79 @@ void StorageMergeTree::freezePartition(const ASTPtr & partition, const String &
|
||||
data.freezePartition(partition, with_name, context);
|
||||
}
|
||||
|
||||
void StorageMergeTree::replacePartitionFrom(const StoragePtr & source_table, const ASTPtr & partition, bool replace, const Context & context)
|
||||
{
|
||||
auto lock1 = lockStructure(false, __PRETTY_FUNCTION__);
|
||||
auto lock2 = source_table->lockStructure(false, __PRETTY_FUNCTION__);
|
||||
|
||||
Stopwatch watch;
|
||||
MergeTreeData * src_data = data.checkStructureAndGetMergeTreeData(source_table);
|
||||
String partition_id = data.getPartitionIDFromQuery(partition, context);
|
||||
|
||||
MergeTreeData::DataPartsVector src_parts = src_data->getDataPartsVectorInPartition(MergeTreeDataPartState::Committed, partition_id);
|
||||
MergeTreeData::MutableDataPartsVector dst_parts;
|
||||
|
||||
static const String TMP_PREFIX = "tmp_replace_from_";
|
||||
|
||||
for (const MergeTreeData::DataPartPtr & src_part : src_parts)
|
||||
{
|
||||
/// This will generate unique name in scope of current server process.
|
||||
Int64 temp_index = data.insert_increment.get();
|
||||
MergeTreePartInfo dst_part_info(partition_id, temp_index, temp_index, src_part->info.level);
|
||||
|
||||
std::shared_lock<std::shared_mutex> part_lock(src_part->columns_lock);
|
||||
dst_parts.emplace_back(data.cloneAndLoadDataPart(src_part, TMP_PREFIX, dst_part_info));
|
||||
}
|
||||
|
||||
/// ATTACH empty part set
|
||||
if (!replace && dst_parts.empty())
|
||||
return;
|
||||
|
||||
MergeTreePartInfo drop_range;
|
||||
if (replace)
|
||||
{
|
||||
drop_range.partition_id = partition_id;
|
||||
drop_range.min_block = 0;
|
||||
drop_range.max_block = increment.get(); // there will be a "hole" in block numbers
|
||||
drop_range.level = std::numeric_limits<decltype(drop_range.level)>::max();
|
||||
}
|
||||
|
||||
/// Atomically add new parts and remove old ones
|
||||
try
|
||||
{
|
||||
{
|
||||
/// Here we use the transaction just like RAII since rare errors in renameTempPartAndReplace() are possible
|
||||
/// and we should be able to rollback already added (Precomitted) parts
|
||||
MergeTreeData::Transaction transaction;
|
||||
|
||||
auto data_parts_lock = data.lockParts();
|
||||
|
||||
/// Populate transaction
|
||||
for (MergeTreeData::MutableDataPartPtr & part : dst_parts)
|
||||
data.renameTempPartAndReplace(part, &increment, &transaction, data_parts_lock);
|
||||
|
||||
transaction.commit(&data_parts_lock);
|
||||
|
||||
/// If it is REPLACE (not ATTACH), remove all parts which max_block_number less then min_block_number of the first new block
|
||||
if (replace)
|
||||
data.removePartsInRangeFromWorkingSet(drop_range, true, false, data_parts_lock);
|
||||
}
|
||||
|
||||
PartLog::addNewParts(this->context, dst_parts, watch.elapsed());
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
PartLog::addNewParts(this->context, dst_parts, watch.elapsed(), ExecutionStatus::fromCurrentException());
|
||||
throw;
|
||||
}
|
||||
}
|
||||
|
||||
ActionLock StorageMergeTree::getActionLock(StorageActionBlockType action_type)
|
||||
{
|
||||
if (action_type == ActionLocks::PartsMerge)
|
||||
return merger.actions_blocker.cancel();
|
||||
|
||||
return {};
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -19,8 +19,6 @@ namespace DB
|
||||
*/
|
||||
class StorageMergeTree : public ext::shared_ptr_helper<StorageMergeTree>, public IStorage
|
||||
{
|
||||
friend class MergeTreeBlockOutputStream;
|
||||
|
||||
public:
|
||||
void startup() override;
|
||||
void shutdown() override;
|
||||
@ -69,6 +67,7 @@ public:
|
||||
void dropPartition(const ASTPtr & query, const ASTPtr & partition, bool detach, const Context & context) override;
|
||||
void clearColumnInPartition(const ASTPtr & partition, const Field & column_name, const Context & context) override;
|
||||
void attachPartition(const ASTPtr & partition, bool part, const Context & context) override;
|
||||
void replacePartitionFrom(const StoragePtr & source_table, const ASTPtr & partition, bool replace, const Context & context) override;
|
||||
void freezePartition(const ASTPtr & partition, const String & with_name, const Context & context) override;
|
||||
|
||||
void drop() override;
|
||||
@ -79,6 +78,8 @@ public:
|
||||
|
||||
bool checkTableCanBeDropped() const override;
|
||||
|
||||
ActionLock getActionLock(StorageActionBlockType action_type) override;
|
||||
|
||||
MergeTreeData & getData() { return data; }
|
||||
const MergeTreeData & getData() const { return data; }
|
||||
|
||||
@ -113,8 +114,6 @@ private:
|
||||
|
||||
BackgroundProcessingPool::TaskHandle merge_task_handle;
|
||||
|
||||
friend struct CurrentlyMergingPartsTagger;
|
||||
|
||||
/** Determines what parts should be merged and merges it.
|
||||
* If aggressive - when selects parts don't takes into account their ratio size and novelty (used for OPTIMIZE query).
|
||||
* Returns true if merge is finished successfully.
|
||||
@ -124,6 +123,10 @@ private:
|
||||
|
||||
bool mergeTask();
|
||||
|
||||
friend class MergeTreeBlockOutputStream;
|
||||
friend class MergeTreeData;
|
||||
friend struct CurrentlyMergingPartsTagger;
|
||||
|
||||
protected:
|
||||
/** Attach the table with the appropriate name, along the appropriate path (with / at the end),
|
||||
* (correctness of names and paths are not checked)
|
||||
|
File diff suppressed because it is too large
Load Diff
@ -23,6 +23,7 @@
|
||||
#include <Common/randomSeed.h>
|
||||
#include <Common/ZooKeeper/ZooKeeper.h>
|
||||
#include <Common/ZooKeeper/LeaderElection.h>
|
||||
#include <Common/BackgroundSchedulePool.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -84,6 +85,7 @@ public:
|
||||
bool supportsFinal() const override { return data.supportsFinal(); }
|
||||
bool supportsPrewhere() const override { return data.supportsPrewhere(); }
|
||||
bool supportsReplication() const override { return true; }
|
||||
bool supportsDeduplication() const override { return true; }
|
||||
|
||||
const ColumnsDescription & getColumns() const override { return data.getColumns(); }
|
||||
void setColumns(ColumnsDescription columns_) override { return data.setColumns(std::move(columns_)); }
|
||||
@ -115,6 +117,7 @@ public:
|
||||
void clearColumnInPartition(const ASTPtr & partition, const Field & column_name, const Context & context) override;
|
||||
void dropPartition(const ASTPtr & query, const ASTPtr & partition, bool detach, const Context & context) override;
|
||||
void attachPartition(const ASTPtr & partition, bool part, const Context & context) override;
|
||||
void replacePartitionFrom(const StoragePtr & source_table, const ASTPtr & partition, bool replace, const Context & context) override;
|
||||
void fetchPartition(const ASTPtr & partition, const String & from, const Context & context) override;
|
||||
void freezePartition(const ASTPtr & partition, const String & with_name, const Context & context) override;
|
||||
|
||||
@ -131,6 +134,12 @@ public:
|
||||
|
||||
bool checkTableCanBeDropped() const override;
|
||||
|
||||
ActionLock getActionLock(StorageActionBlockType action_type) override;
|
||||
|
||||
/// Wait when replication queue size becomes less or equal than queue_size
|
||||
/// If timeout is exceeded returns false
|
||||
bool waitForShrinkingQueueSize(size_t queue_size = 0, UInt64 max_wait_milliseconds = 0);
|
||||
|
||||
MergeTreeData & getData() { return data; }
|
||||
const MergeTreeData & getData() const { return data; }
|
||||
|
||||
@ -189,6 +198,8 @@ private:
|
||||
friend struct ReplicatedMergeTreeLogEntry;
|
||||
friend class ScopedPartitionMergeLock;
|
||||
friend class ReplicatedMergeTreeQueue;
|
||||
friend class ReplicatedMergeTreeMergeSelectingThread;
|
||||
friend class MergeTreeData;
|
||||
|
||||
using LogEntry = ReplicatedMergeTreeLogEntry;
|
||||
using LogEntryPtr = LogEntry::Ptr;
|
||||
@ -256,26 +267,23 @@ private:
|
||||
|
||||
/// Threads.
|
||||
|
||||
/// A thread that keeps track of the updates in the logs of all replicas and loads them into the queue.
|
||||
std::thread queue_updating_thread;
|
||||
zkutil::EventPtr queue_updating_event = std::make_shared<Poco::Event>();
|
||||
/// A task that keeps track of the updates in the logs of all replicas and loads them into the queue.
|
||||
bool queue_update_in_progress = false;
|
||||
BackgroundSchedulePool::TaskHandle queue_updating_task_handle;
|
||||
|
||||
std::thread mutations_updating_thread;
|
||||
zkutil::EventPtr mutations_updating_event = std::make_shared<Poco::Event>();
|
||||
BackgroundSchedulePool::TaskHandle mutations_updating_task_handle;
|
||||
|
||||
/// A task that performs actions from the queue.
|
||||
BackgroundProcessingPool::TaskHandle queue_task_handle;
|
||||
|
||||
/// A thread that selects parts to merge.
|
||||
std::thread merge_selecting_thread;
|
||||
Poco::Event merge_selecting_event;
|
||||
/// A task that selects parts to merge.
|
||||
BackgroundSchedulePool::TaskHandle merge_selecting_task_handle;
|
||||
|
||||
/// It is acquired for each iteration of the selection of parts to merge or each OPTIMIZE query.
|
||||
std::mutex merge_selecting_mutex;
|
||||
|
||||
/// A thread that removes old parts, log entries, and blocks.
|
||||
std::unique_ptr<ReplicatedMergeTreeCleanupThread> cleanup_thread;
|
||||
/// Is used to wakeup cleanup_thread
|
||||
Poco::Event cleanup_thread_event;
|
||||
|
||||
/// A thread that processes reconnection to ZooKeeper when the session expires.
|
||||
std::unique_ptr<ReplicatedMergeTreeRestartingThread> restarting_thread;
|
||||
@ -291,8 +299,6 @@ private:
|
||||
|
||||
Logger * log;
|
||||
|
||||
/// Initialization.
|
||||
|
||||
/** Creates the minimum set of nodes in ZooKeeper.
|
||||
*/
|
||||
void createTableIfNotExists();
|
||||
@ -326,12 +332,17 @@ private:
|
||||
void checkPartChecksumsAndAddCommitOps(const zkutil::ZooKeeperPtr & zookeeper, const MergeTreeData::DataPartPtr & part,
|
||||
zkutil::Requests & ops, String part_name = "", NameSet * absent_replicas_paths = nullptr);
|
||||
|
||||
String getChecksumsForZooKeeper(const MergeTreeDataPartChecksums & checksums);
|
||||
String getChecksumsForZooKeeper(const MergeTreeDataPartChecksums & checksums) const;
|
||||
|
||||
/// Accepts a PreComitted part, atomically checks its checksums with ones on other replicas and commit the part
|
||||
MergeTreeData::DataPartsVector checkPartChecksumsAndCommit(MergeTreeData::Transaction & transaction,
|
||||
const MergeTreeData::DataPartPtr & part);
|
||||
|
||||
void getCommitPartOps(
|
||||
zkutil::Requests & ops,
|
||||
MergeTreeData::MutableDataPartPtr & part,
|
||||
const String & block_id_path = "") const;
|
||||
|
||||
/// Adds actions to `ops` that remove a part from ZooKeeper.
|
||||
void removePartFromZooKeeper(const String & part_name, zkutil::Requests & ops);
|
||||
|
||||
@ -339,6 +350,9 @@ private:
|
||||
void removePartsFromZooKeeper(zkutil::ZooKeeperPtr & zookeeper, const Strings & part_names,
|
||||
NameSet * parts_should_be_retried = nullptr);
|
||||
|
||||
bool tryRemovePartsFromZooKeeperWithRetries(const Strings & part_names, size_t max_retries = 5);
|
||||
bool tryRemovePartsFromZooKeeperWithRetries(MergeTreeData::DataPartsVector & parts, size_t max_retries = 5);
|
||||
|
||||
/// Removes a part from ZooKeeper and adds a task to the queue to download it. It is supposed to do this with broken parts.
|
||||
void removePartAndEnqueueFetch(const String & part_name);
|
||||
|
||||
@ -367,6 +381,8 @@ private:
|
||||
|
||||
void executeClearColumnInPartition(const LogEntry & entry);
|
||||
|
||||
bool executeReplaceRange(const LogEntry & entry);
|
||||
|
||||
/** Updates the queue.
|
||||
*/
|
||||
void queueUpdatingThread();
|
||||
@ -416,6 +432,7 @@ private:
|
||||
* If not found, returns empty string.
|
||||
*/
|
||||
String findReplicaHavingCoveringPart(const LogEntry & entry, bool active);
|
||||
String findReplicaHavingCoveringPart(const String & part_name, bool active, String & found_part_name);
|
||||
|
||||
/** Download the specified part from the specified replica.
|
||||
* If `to_detached`, the part is placed in the `detached` directory.
|
||||
@ -431,9 +448,9 @@ private:
|
||||
/// With the quorum being tracked, add a replica to the quorum for the part.
|
||||
void updateQuorum(const String & part_name);
|
||||
|
||||
/// Creates new block number and additionally perform precheck_ops while creates 'abandoned node'
|
||||
AbandonableLockInZooKeeper allocateBlockNumber(const String & partition_id, zkutil::ZooKeeperPtr & zookeeper,
|
||||
zkutil::Requests * precheck_ops = nullptr);
|
||||
/// Creates new block number if block with such block_id does not exist
|
||||
std::optional<AbandonableLockInZooKeeper> allocateBlockNumber(const String & partition_id, zkutil::ZooKeeperPtr & zookeeper,
|
||||
const String & zookeeper_block_id_path = "");
|
||||
|
||||
/** Wait until all replicas, including this, execute the specified action from the log.
|
||||
* If replicas are added at the same time, it can not wait the added replica .
|
||||
@ -450,10 +467,9 @@ private:
|
||||
/// Throw an exception if the table is readonly.
|
||||
void assertNotReadonly() const;
|
||||
|
||||
/// The name of an imaginary part covering all parts in the specified partition (at the call moment).
|
||||
/// Returns empty string if the partition doesn't exist yet.
|
||||
String getFakePartNameCoveringAllPartsInPartition(
|
||||
const String & partition_id, Int64 * out_min_block = nullptr, Int64 * out_max_block = nullptr);
|
||||
/// Produce an imaginary part info covering all parts in the specified partition (at the call moment).
|
||||
/// Returns false if the partition doesn't exist yet.
|
||||
bool getFakePartCoveringAllPartsInPartition(const String & partition_id, MergeTreePartInfo & part_info);
|
||||
|
||||
/// Check for a node in ZK. If it is, remember this information, and then immediately answer true.
|
||||
std::unordered_set<std::string> existing_nodes_cache;
|
||||
|
@ -5,9 +5,9 @@
|
||||
|
||||
int main(int, char **)
|
||||
{
|
||||
DayNum_t today = DateLUT::instance().toDayNum(time(nullptr));
|
||||
DayNum today = DateLUT::instance().toDayNum(time(nullptr));
|
||||
|
||||
for (DayNum_t date = today; DayNum_t(date + 10) > today; --date)
|
||||
for (DayNum date = today; DayNum(date + 10) > today; --date)
|
||||
{
|
||||
DB::MergeTreePartInfo part_info("partition", 0, 0, 0);
|
||||
std::string name = part_info.getPartNameV0(date, date);
|
||||
|
@ -359,5 +359,10 @@ if __name__ == '__main__':
|
||||
if args.client is None:
|
||||
args.client = args.binary + '-client'
|
||||
if args.configclient:
|
||||
args.client += ' -c' + args.configclient
|
||||
args.client += ' --config-file=' + args.configclient
|
||||
if os.getenv("CLICKHOUSE_HOST"):
|
||||
args.client += ' --host=' + os.getenv("CLICKHOUSE_HOST")
|
||||
if os.getenv("CLICKHOUSE_PORT_TCP"):
|
||||
args.client += ' --port=' + os.getenv("CLICKHOUSE_PORT_TCP")
|
||||
|
||||
main(args)
|
||||
|
@ -21,10 +21,27 @@ CONFIG_CLIENT=${CONFIG_CLIENT:=${CONFIG_CLIENT_DIR}client-test.xml}
|
||||
export CLICKHOUSE_CONFIG=${CLICKHOUSE_CONFIG:=${CONFIG_SERVER_DIR}server-test.xml}
|
||||
[ -x "$CUR_DIR/clickhouse-test" ] && TEST_DIR=${TEST_DIR=$CUR_DIR/}
|
||||
[ -d "$CUR_DIR/queries" ] && QUERIES_DIR=${QUERIES_DIR=$CUR_DIR/queries}
|
||||
[ ! -d "$QUERIES_DIR" ] && QUERIES_DIR=${QUERIES_DIR=/usr/local/share/clickhouse-test/queries}
|
||||
[ ! -d "$QUERIES_DIR" ] && QUERIES_DIR=${QUERIES_DIR=/usr/share/clickhouse-test/queries}
|
||||
[ ! -d "$QUERIES_DIR" ] && [ -d "/usr/local/share/clickhouse-test/queries" ] && QUERIES_DIR=${QUERIES_DIR=/usr/local/share/clickhouse-test/queries}
|
||||
[ ! -d "$QUERIES_DIR" ] && [ -d "/usr/share/clickhouse-test/queries" ] && QUERIES_DIR=${QUERIES_DIR=/usr/share/clickhouse-test/queries}
|
||||
CLICKHOUSE_EXTRACT_CONFIG=${CLICKHOUSE_EXTRACT_CONFIG:="${BIN_DIR}${CLICKHOUSE_BINARY}-extract-from-config --config=$CLICKHOUSE_CONFIG"}
|
||||
|
||||
PORT_RANDOM=${PORT_RANDOM=1}
|
||||
if [ "${PORT_RANDOM}" ]; then
|
||||
CLICKHOUSE_PORT_BASE=${CLICKHOUSE_PORT_BASE:=$(( ( RANDOM % 50000 ) + 10000 ))}
|
||||
CLICKHOUSE_PORT_TCP=${CLICKHOUSE_PORT_TCP:=$(($CLICKHOUSE_PORT_BASE + 1))}
|
||||
CLICKHOUSE_PORT_HTTP=${CLICKHOUSE_PORT_HTTP:=$(($CLICKHOUSE_PORT_BASE + 2))}
|
||||
CLICKHOUSE_PORT_INTERSERVER=${CLICKHOUSE_PORT_INTERSERVER:=$(($CLICKHOUSE_PORT_BASE + 3))}
|
||||
CLICKHOUSE_PORT_TCP_SECURE=${CLICKHOUSE_PORT_TCP_SECURE:=$(($CLICKHOUSE_PORT_BASE + 4))}
|
||||
CLICKHOUSE_PORT_HTTPS=${CLICKHOUSE_PORT_HTTPS:=$(($CLICKHOUSE_PORT_BASE + 5))}
|
||||
fi
|
||||
|
||||
export CLICKHOUSE_PORT_TCP=${CLICKHOUSE_PORT_TCP:=`$CLICKHOUSE_EXTRACT_CONFIG --key=tcp_port`}
|
||||
export CLICKHOUSE_PORT_HTTP=${CLICKHOUSE_PORT_HTTP:=`$CLICKHOUSE_EXTRACT_CONFIG --key=http_port`}
|
||||
export CLICKHOUSE_PORT_INTERSERVER=${CLICKHOUSE_PORT_INTERSERVER:=`$CLICKHOUSE_EXTRACT_CONFIG --key=interserver_http_port`}
|
||||
export CLICKHOUSE_PORT_TCP_SECURE=${CLICKHOUSE_PORT_TCP_SECURE:=`$CLICKHOUSE_EXTRACT_CONFIG --key=tcp_port_secure`}
|
||||
export CLICKHOUSE_PORT_HTTPS=${CLICKHOUSE_PORT_HTTPS:=`$CLICKHOUSE_EXTRACT_CONFIG --key=https_port`}
|
||||
|
||||
|
||||
rm -rf $DATA_DIR
|
||||
mkdir -p $LOG_DIR
|
||||
|
||||
@ -42,7 +59,7 @@ fi
|
||||
|
||||
# Start a local clickhouse server which will be used to run tests
|
||||
#PATH=$PATH:$BIN_DIR \
|
||||
$GDB ${BIN_DIR}clickhouse-server --config-file=$CLICKHOUSE_CONFIG > $LOG_DIR/stdout 2>&1 &
|
||||
$GDB ${BIN_DIR}clickhouse-server --config-file=$CLICKHOUSE_CONFIG -- --http_port=$CLICKHOUSE_PORT_HTTP --tcp_port=$CLICKHOUSE_PORT_TCP --https_port=$CLICKHOUSE_PORT_HTTPS --tcp_port_secure=$CLICKHOUSE_PORT_TCP_SECURE --interserver_http_port=$CLICKHOUSE_PORT_INTERSERVER > $LOG_DIR/stdout 2>&1 &
|
||||
CH_PID=$!
|
||||
sleep 3
|
||||
|
||||
@ -71,7 +88,7 @@ if [ -n "$*" ]; then
|
||||
else
|
||||
TEST_RUN=${TEST_RUN=1}
|
||||
TEST_PERF=${TEST_PERF=1}
|
||||
${BIN_DIR}clickhouse-client --config ${CONFIG_CLIENT} -q 'SELECT * from system.build_options;'
|
||||
${BIN_DIR}clickhouse-client --config ${CONFIG_CLIENT} --port $CLICKHOUSE_PORT_TCP -q 'SELECT * from system.build_options;'
|
||||
[ "$TEST_RUN" ] && env PATH=$PATH:$BIN_DIR ${TEST_DIR}clickhouse-test --binary ${BIN_DIR}clickhouse --configclient $CONFIG_CLIENT --configserver $CLICKHOUSE_CONFIG --tmp $DATA_DIR/tmp --queries $QUERIES_DIR $TEST_OPT0 $TEST_OPT
|
||||
( [ "$TEST_PERF" ] && ${BIN_DIR}clickhouse-performance-test --port `$CLICKHOUSE_EXTRACT_CONFIG --key=tcp_port` --r $CUR_DIR/performance --skip-tags=long $* ) || true
|
||||
( [ "$TEST_PERF" ] && ${BIN_DIR}clickhouse-performance-test --port $CLICKHOUSE_PORT_TCP --r $CUR_DIR/performance --skip-tags=long $* ) || true
|
||||
fi
|
||||
|
@ -16,6 +16,8 @@ Don't use Docker from your system repository.
|
||||
* [py.test](https://docs.pytest.org/) testing framework. To install: `sudo -H pip install pytest`
|
||||
* [docker-compose](https://docs.docker.com/compose/) and additional python libraries. To install: `sudo -H pip install docker-compose docker dicttoxml kazoo PyMySQL`
|
||||
|
||||
If you want install on modern debian/ubuntu: `sudo apt install -y docker docker-compose python-pytest python-dicttoxml python-docker python-pymysql python-kazoo`
|
||||
|
||||
If you want to run the tests under a non-privileged user, you must add this user to `docker` group: `sudo usermod -aG docker $USER` and re-login.
|
||||
(You must close all your sessions (for example, restart your computer))
|
||||
To check, that you have access to Docker, run `docker ps`.
|
||||
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user