Merge branch 'master' of https://github.com/yandex/ClickHouse into CLICKHOUSE-2720

This commit is contained in:
Ivan Blinkov 2018-05-28 19:13:44 +03:00
commit dcbf276c58
180 changed files with 4672 additions and 1479 deletions

11
CHANGELOG.draft.md Normal file
View File

@ -0,0 +1,11 @@
en:
## Improvements:
* `clickhouse-client`: option --ask-password for interactively ask for credentials #1044
ru:
## Улучшения:
* `clickhouse-client`: опция --ask-password для интерактивного ввода пароля #1044

View File

@ -1,3 +1,8 @@
# ClickHouse release 1.1.54383, 2018-05-22
## Bug fixes:
* Fixed a slowdown of replication queue if a table has many replicas.
# ClickHouse release 1.1.54381, 2018-05-14 # ClickHouse release 1.1.54381, 2018-05-14
## Bug fixes: ## Bug fixes:

View File

@ -1,3 +1,7 @@
# ClickHouse release 1.1.54383, 2018-05-22
## Исправление ошибок:
* Исправлена деградация скорости выполнения очереди репликации при большом количестве реплик
# ClickHouse release 1.1.54381, 2018-05-14 # ClickHouse release 1.1.54381, 2018-05-14
## Исправление ошибок: ## Исправление ошибок:

View File

@ -128,14 +128,14 @@ endif ()
#endif () #endif ()
if (CMAKE_VERSION VERSION_LESS "3.8.0") if (CMAKE_VERSION VERSION_LESS "3.8.0")
if (NOT MSVC)
set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -std=c++1z") set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -std=c++1z")
endif ()
else () else ()
set (CMAKE_CXX_STANDARD 17) 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_EXTENSIONS 0) # https://cmake.org/cmake/help/latest/prop_tgt/CXX_EXTENSIONS.html#prop_tgt:CXX_EXTENSIONS
set (CMAKE_CXX_STANDARD_REQUIRED ON) set (CMAKE_CXX_STANDARD_REQUIRED ON)
set (CXX_FLAGS_INTERNAL_COMPILER "-std=c++1z") 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 () endif ()
set (CMAKE_BUILD_COLOR_MAKEFILE ON) set (CMAKE_BUILD_COLOR_MAKEFILE ON)

View File

@ -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}) option (USE_INTERNAL_LLVM_LIBRARY "Use bundled or system LLVM library. Default: system library for quicker developer builds." ${APPLE})
if (ENABLE_EMBEDDED_COMPILER) if (ENABLE_EMBEDDED_COMPILER)

View File

@ -60,7 +60,6 @@ elseif (NOT MISSING_INTERNAL_POCO_LIBRARY)
) )
if (NOT DEFINED ENABLE_POCO_MONGODB OR ENABLE_POCO_MONGODB) if (NOT DEFINED ENABLE_POCO_MONGODB OR ENABLE_POCO_MONGODB)
set (USE_POCO_MONGODB 1)
set (Poco_MongoDB_LIBRARY PocoMongoDB) set (Poco_MongoDB_LIBRARY PocoMongoDB)
set (Poco_MongoDB_INCLUDE_DIRS "${ClickHouse_SOURCE_DIR}/contrib/poco/MongoDB/include/") set (Poco_MongoDB_INCLUDE_DIRS "${ClickHouse_SOURCE_DIR}/contrib/poco/MongoDB/include/")
endif () endif ()
@ -73,7 +72,6 @@ elseif (NOT MISSING_INTERNAL_POCO_LIBRARY)
"${ClickHouse_SOURCE_DIR}/contrib/poco/Data/include" "${ClickHouse_SOURCE_DIR}/contrib/poco/Data/include"
) )
if ((NOT DEFINED ENABLE_POCO_ODBC OR ENABLE_POCO_ODBC) AND ODBC_FOUND) if ((NOT DEFINED ENABLE_POCO_ODBC OR ENABLE_POCO_ODBC) AND ODBC_FOUND)
set (USE_POCO_SQLODBC 1)
set (Poco_SQLODBC_INCLUDE_DIRS set (Poco_SQLODBC_INCLUDE_DIRS
"${ClickHouse_SOURCE_DIR}/contrib/poco/SQL/ODBC/include/" "${ClickHouse_SOURCE_DIR}/contrib/poco/SQL/ODBC/include/"
"${ClickHouse_SOURCE_DIR}/contrib/poco/Data/ODBC/include/" "${ClickHouse_SOURCE_DIR}/contrib/poco/Data/ODBC/include/"
@ -97,7 +95,6 @@ elseif (NOT MISSING_INTERNAL_POCO_LIBRARY)
# TODO! fix internal ssl # TODO! fix internal ssl
if (OPENSSL_FOUND AND NOT USE_INTERNAL_SSL_LIBRARY AND (NOT DEFINED ENABLE_POCO_NETSSL OR ENABLE_POCO_NETSSL)) if (OPENSSL_FOUND AND NOT USE_INTERNAL_SSL_LIBRARY AND (NOT DEFINED ENABLE_POCO_NETSSL OR ENABLE_POCO_NETSSL))
set (USE_POCO_NETSSL 1)
set (Poco_NetSSL_LIBRARY PocoNetSSL) set (Poco_NetSSL_LIBRARY PocoNetSSL)
set (Poco_Crypto_LIBRARY PocoCrypto) set (Poco_Crypto_LIBRARY PocoCrypto)
endif () endif ()
@ -115,7 +112,20 @@ elseif (NOT MISSING_INTERNAL_POCO_LIBRARY)
set (Poco_XML_LIBRARY PocoXML) set (Poco_XML_LIBRARY PocoXML)
endif () endif ()
message(STATUS "Using Poco: ${Poco_INCLUDE_DIRS} : ${Poco_Foundation_LIBRARY},${Poco_Util_LIBRARY},${Poco_Net_LIBRARY},${Poco_NetSSL_LIBRARY},${Poco_XML_LIBRARY},${Poco_Data_LIBRARY},${Poco_DataODBC_LIBRARY},${Poco_MongoDB_LIBRARY}; MongoDB=${USE_POCO_MONGODB}, DataODBC=${Poco_DataODBC_FOUND}, NetSSL=${USE_POCO_NETSSL}") if (Poco_NetSSL_LIBRARY AND Poco_Crypto_LIBRARY)
set (USE_POCO_NETSSL 1)
endif ()
if (Poco_MongoDB_LIBRARY)
set (USE_POCO_MONGODB 1)
endif ()
if (Poco_DataODBC_LIBRARY)
set (USE_POCO_DATAODBC 1)
endif ()
if (Poco_SQLODBC_LIBRARY)
set (USE_POCO_SQLODBC 1)
endif ()
message(STATUS "Using Poco: ${Poco_INCLUDE_DIRS} : ${Poco_Foundation_LIBRARY},${Poco_Util_LIBRARY},${Poco_Net_LIBRARY},${Poco_NetSSL_LIBRARY},${Poco_Crypto_LIBRARY},${Poco_XML_LIBRARY},${Poco_Data_LIBRARY},${Poco_DataODBC_LIBRARY},${Poco_SQL_LIBRARY},${Poco_SQLODBC_LIBRARY},${Poco_MongoDB_LIBRARY}; MongoDB=${USE_POCO_MONGODB}, DataODBC=${USE_POCO_DATAODBC}, NetSSL=${USE_POCO_NETSSL}")
# How to make sutable poco: # How to make sutable poco:
# use branch: # use branch:

View File

@ -1,5 +1,5 @@
if (NOT MSVC) 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 () endif ()
if (USE_INTERNAL_BOOST_LIBRARY) if (USE_INTERNAL_BOOST_LIBRARY)

View File

@ -116,3 +116,10 @@
/* Define to the type of an unsigned integer type of width exactly 8 bits if /* 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. */ such a type exists and the standard includes do not define it. */
/* #undef uint8_t */ /* #undef uint8_t */
#ifdef _MSC_VER
#include <basetsd.h>
typedef SSIZE_T ssize_t;
#else
#include <sys/types.h>
#endif

View File

@ -17,7 +17,7 @@
SOURCE_PATH=${1:-.} SOURCE_PATH=${1:-.}
DST=${2:-$SOURCE_PATH/../headers} 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" 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/Interpreters/SpecializedAggregator.h \
$SOURCE_PATH/dbms/src/AggregateFunctions/AggregateFunction*.h) $SOURCE_PATH/dbms/src/AggregateFunctions/AggregateFunction*.h)
for header in $START_HEADERS; do
START_HEADERS_INCLUDE+="-include $header "
done
# Опция -mcx16 для того, чтобы выбиралось больше заголовочных файлов (с запасом). # Опция -mcx16 для того, чтобы выбиралось больше заголовочных файлов (с запасом).
# The latter options are the same that are added while building packages. # 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'` 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 \ 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 \
-I $GCC_ROOT/include-fixed \ -I $GCC_ROOT/include-fixed \
$(cat "$BUILD_PATH/include_directories.txt") \ $(cat "$BUILD_PATH/include_directories.txt") \
$(echo $START_HEADERS | sed -r -e 's/[^ ]+/-include \0/g') \ $START_HEADERS_INCLUDE \
- | - |
tr -d '\\' | tr -d '\\' |
sed -r -e 's/^-\.o://'); sed -E -e 's/^-\.o://');
do do
dst_file=$src_file; dst_file=$src_file;
dst_file=$(echo $dst_file | sed -r -e 's/build\///') # for simplicity reasons, will put generated headers near the rest. [ -n $DESTDIR ] && dst_file=$(echo $dst_file | sed -E -e "s!^$DESTDIR!!")
mkdir -p "$DST/$(echo $dst_file | sed -r -e 's/\/[^/]*$/\//')"; 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"; cp "$src_file" "$DST/$dst_file";
done 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'); 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 do
mkdir -p "$DST/$(echo $src_file | sed -r -e 's/\/[^/]*$/\//')"; dst_file=$src_file;
cp "$src_file" "$DST/$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 done
# Even more platform-specific headers # Even more platform-specific headers
for src_file in $(ls -1 $SOURCE_PATH/contrib/boost/libs/smart_ptr/include/boost/smart_ptr/detail/*); for src_file in $(ls -1 $SOURCE_PATH/contrib/boost/libs/smart_ptr/include/boost/smart_ptr/detail/*);
do do
mkdir -p "$DST/$(echo $src_file | sed -r -e 's/\/[^/]*$/\//')"; dst_file=$src_file;
cp "$src_file" "$DST/$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 done
for src_file in $(ls -1 $SOURCE_PATH/contrib/boost/boost/smart_ptr/detail/*); for src_file in $(ls -1 $SOURCE_PATH/contrib/boost/boost/smart_ptr/detail/*);
do do
mkdir -p "$DST/$(echo $src_file | sed -r -e 's/\/[^/]*$/\//')"; dst_file=$src_file;
cp "$src_file" "$DST/$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 done

View File

@ -198,7 +198,7 @@ if (USE_POCO_MONGODB)
endif() endif()
if (USE_POCO_NETSSL) if (USE_POCO_NETSSL)
target_link_libraries (clickhouse_common_io ${Poco_NetSSL_LIBRARY}) target_link_libraries (clickhouse_common_io ${Poco_NetSSL_LIBRARY} ${Poco_Crypto_LIBRARY})
endif() endif()
target_link_libraries (dbms ${Poco_Foundation_LIBRARY}) target_link_libraries (dbms ${Poco_Foundation_LIBRARY})
@ -245,6 +245,12 @@ target_include_directories (clickhouse_common_io BEFORE PUBLIC ${DOUBLE_CONVERSI
# also for copy_headers.sh: # also for copy_headers.sh:
target_include_directories (clickhouse_common_io BEFORE PRIVATE ${COMMON_INCLUDE_DIR}) 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) add_subdirectory (tests)
if (ENABLE_TESTS) if (ENABLE_TESTS)

View File

@ -1,6 +1,8 @@
#pragma once #pragma once
#include <atomic> #include <atomic>
#include <memory>
#include <Common/ActionLock.h>
namespace DB namespace DB
{ {
@ -10,54 +12,25 @@ namespace DB
class ActionBlocker class ActionBlocker
{ {
private: private:
mutable std::atomic<int> counter{0}; using Counter = std::atomic<int>;
using CounterPtr = std::shared_ptr<Counter>;
mutable CounterPtr counter;
public: 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) /// Temporarily blocks corresponding actions (while the returned object is alive)
struct LockHolder; friend class ActionLock;
LockHolder cancel() const { return LockHolder(this); } ActionLock cancel() const { return ActionLock(*this); }
/// Cancel the actions forever. /// Cancel the actions forever.
void cancelForever() const { ++counter; } void cancelForever() const { ++(*counter); }
/// Returns reference to counter to allow to watch on it directly. /// Returns reference to counter to allow to watch on it directly.
auto & getCounter() { return counter; } auto & getCounter() { return *counter; }
/// Blocks related action while a BlockerHolder instance exists
struct LockHolder
{
explicit LockHolder(const ActionBlocker * var_ = nullptr) : var(var_)
{
if (var)
++var->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:
const ActionBlocker * var = nullptr;
};
}; };
} }

View 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;
}
}

View 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;
};
}

View 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();
}
}
}

View 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>;
}

View File

@ -9,6 +9,7 @@
M(ReplicatedSend) \ M(ReplicatedSend) \
M(ReplicatedChecks) \ M(ReplicatedChecks) \
M(BackgroundPoolTask) \ M(BackgroundPoolTask) \
M(BackgroundSchedulePoolTask) \
M(DiskSpaceReservedForMerge) \ M(DiskSpaceReservedForMerge) \
M(DistributedSend) \ M(DistributedSend) \
M(QueryPreempted) \ M(QueryPreempted) \
@ -25,6 +26,7 @@
M(LeaderReplica) \ M(LeaderReplica) \
M(MemoryTracking) \ M(MemoryTracking) \
M(MemoryTrackingInBackgroundProcessingPool) \ M(MemoryTrackingInBackgroundProcessingPool) \
M(MemoryTrackingInBackgroundSchedulePool) \
M(MemoryTrackingForMerges) \ M(MemoryTrackingForMerges) \
M(LeaderElection) \ M(LeaderElection) \
M(EphemeralNode) \ M(EphemeralNode) \

View File

@ -6,6 +6,7 @@
#include <memory> #include <memory>
#include <common/logger_useful.h> #include <common/logger_useful.h>
#include <Common/CurrentMetrics.h> #include <Common/CurrentMetrics.h>
#include <Common/BackgroundSchedulePool.h>
namespace ProfileEvents namespace ProfileEvents
@ -36,9 +37,10 @@ public:
* It means that different participants of leader election have different identifiers * 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. * 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_ = "") LeaderElection(DB::BackgroundSchedulePool & pool_, const std::string & path_, ZooKeeper & zookeeper_, LeadershipHandler handler_, const std::string & identifier_ = "")
: path(path_), zookeeper(zookeeper_), handler(handler_), identifier(identifier_) : pool(pool_), path(path_), zookeeper(zookeeper_), handler(handler_), identifier(identifier_)
{ {
task_handle = pool.addTask("LeaderElection", [this] { threadFunction(); });
createNode(); createNode();
} }
@ -48,17 +50,18 @@ public:
return; return;
shutdown_called = true; shutdown_called = true;
event->set(); task_handle->deactivate();
if (thread.joinable())
thread.join();
} }
~LeaderElection() ~LeaderElection()
{ {
releaseNode(); releaseNode();
pool.removeTask(task_handle);
} }
private: private:
DB::BackgroundSchedulePool & pool;
DB::BackgroundSchedulePool::TaskHandle task_handle;
std::string path; std::string path;
ZooKeeper & zookeeper; ZooKeeper & zookeeper;
LeadershipHandler handler; LeadershipHandler handler;
@ -67,9 +70,7 @@ private:
EphemeralNodeHolderPtr node; EphemeralNodeHolderPtr node;
std::string node_name; std::string node_name;
std::thread thread;
std::atomic<bool> shutdown_called {false}; std::atomic<bool> shutdown_called {false};
EventPtr event = std::make_shared<Poco::Event>();
CurrentMetrics::Increment metric_increment{CurrentMetrics::LeaderElection}; CurrentMetrics::Increment metric_increment{CurrentMetrics::LeaderElection};
@ -81,7 +82,8 @@ private:
std::string node_path = node->getPath(); std::string node_path = node->getPath();
node_name = node_path.substr(node_path.find_last_of('/') + 1); 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() void releaseNode()
@ -91,8 +93,6 @@ private:
} }
void threadFunction() void threadFunction()
{
while (!shutdown_called)
{ {
bool success = false; bool success = false;
@ -111,8 +111,8 @@ private:
return; return;
} }
if (zookeeper.exists(path + "/" + *(it - 1), nullptr, event)) if (!zookeeper.existsWatch(path + "/" + *(it - 1), nullptr, task_handle->getWatchCallback()))
event->wait(); task_handle->schedule();
success = true; success = true;
} }
@ -121,7 +121,7 @@ private:
DB::tryLogCurrentException("LeaderElection"); DB::tryLogCurrentException("LeaderElection");
if (e.code == ZooKeeperImpl::ZooKeeper::ZSESSIONEXPIRED) if (e.code == ZooKeeperImpl::ZooKeeper::ZSESSIONEXPIRED)
break; return;
} }
catch (...) catch (...)
{ {
@ -129,8 +129,7 @@ private:
} }
if (!success) if (!success)
event->tryWait(10 * 1000); task_handle->scheduleAfter(10 * 1000);
}
} }
}; };

View File

@ -367,6 +367,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); 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) 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); return tryGetWatch(path, res, stat, callbackForEvent(watch), return_code);

View File

@ -113,6 +113,7 @@ public:
bool existsWatch(const std::string & path, Stat * stat, WatchCallback watch_callback); 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 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: /// Doesn't not throw in the following cases:
/// * The node doesn't exist. Returns false in this case. /// * The node doesn't exist. Returns false in this case.

View File

@ -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. /// 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; extern const char * const hex_char_to_digit_table;

View File

@ -1,3 +1,4 @@
#include "localBackup.h"
#include <sys/stat.h> #include <sys/stat.h>
#include <string> #include <string>
#include <iostream> #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) if (level >= 1000)
throw DB::Exception("Too deep recursion", DB::ErrorCodes::TOO_DEEP_RECURSION); 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 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() if (Poco::File(destination_path).exists()
&& Poco::DirectoryIterator(destination_path) != Poco::DirectoryIterator()) && Poco::DirectoryIterator(destination_path) != Poco::DirectoryIterator())
@ -90,7 +95,7 @@ void localBackup(const Poco::Path & source_path, const Poco::Path & destination_
{ {
try try
{ {
localBackupImpl(source_path, destination_path, 0); localBackupImpl(source_path, destination_path, 0, max_level);
} }
catch (const DB::ErrnoException & e) catch (const DB::ErrnoException & e)
{ {

View File

@ -1,6 +1,7 @@
#pragma once #pragma once
#include <Poco/Path.h> #include <Poco/Path.h>
#include <optional>
/** Creates a local (at the same mount point) backup (snapshot) directory. /** Creates a local (at the same mount point) backup (snapshot) directory.
@ -12,5 +13,9 @@
* This protects data from accidental deletion or modification, * 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, * and is intended to be used as a simple means of protection against a human or program error,
* but not from a hardware failure. * 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 = {});

View File

@ -14,6 +14,7 @@ namespace ErrorCodes
extern const int CANNOT_READ_ARRAY_FROM_TEXT; extern const int CANNOT_READ_ARRAY_FROM_TEXT;
extern const int CANNOT_PARSE_NUMBER; extern const int CANNOT_PARSE_NUMBER;
extern const int CANNOT_PARSE_UUID; extern const int CANNOT_PARSE_UUID;
extern const int TOO_LARGE_STRING_SIZE;
} }
@ -37,7 +38,8 @@ static bool isParseError(int code)
|| code == ErrorCodes::CANNOT_PARSE_DATETIME || code == ErrorCodes::CANNOT_PARSE_DATETIME
|| code == ErrorCodes::CANNOT_READ_ARRAY_FROM_TEXT || code == ErrorCodes::CANNOT_READ_ARRAY_FROM_TEXT
|| code == ErrorCodes::CANNOT_PARSE_NUMBER || code == ErrorCodes::CANNOT_PARSE_NUMBER
|| code == ErrorCodes::CANNOT_PARSE_UUID; || code == ErrorCodes::CANNOT_PARSE_UUID
|| code == ErrorCodes::TOO_LARGE_STRING_SIZE;
} }

View File

@ -130,8 +130,16 @@ bool JSONEachRowRowInputStream::read(MutableColumns & columns)
read_columns[index] = true; read_columns[index] = true;
try
{
header.getByPosition(index).type->deserializeTextJSON(*columns[index], istr); header.getByPosition(index).type->deserializeTextJSON(*columns[index], istr);
} }
catch (Exception & e)
{
e.addMessage("(while read the value of key " + name_ref.toString() + ")");
throw;
}
}
/// Fill non-visited columns with the default values. /// Fill non-visited columns with the default values.
for (size_t i = 0; i < num_columns; ++i) for (size_t i = 0; i < num_columns; ++i)

View File

@ -18,6 +18,10 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream(
*/ */
addTableLock(storage->lockStructure(true, __PRETTY_FUNCTION__)); 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()) if (!table.empty())
{ {
Dependencies dependencies = context.getDependencies(database, table); Dependencies dependencies = context.getDependencies(database, table);
@ -27,6 +31,7 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream(
{ {
views_context = std::make_unique<Context>(context); views_context = std::make_unique<Context>(context);
// Do not deduplicate insertions into MV if the main insertion is Ok // Do not deduplicate insertions into MV if the main insertion is Ok
if (disable_deduplication_for_children)
views_context->getSettingsRef().insert_deduplicate = false; views_context->getSettingsRef().insert_deduplicate = false;
} }
@ -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();
}
} }

View File

@ -25,32 +25,9 @@ public:
Block getHeader() const override { return storage->getSampleBlock(); } Block getHeader() const override { return storage->getSampleBlock(); }
void write(const Block & block) override; void write(const Block & block) override;
void flush() override void flush() override;
{ void writePrefix() override;
if (output) void writeSuffix() override;
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();
}
private: private:
StoragePtr storage; StoragePtr storage;

View File

@ -11,12 +11,12 @@ namespace DB
void DataTypeDate::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr) const 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) static void deserializeText(IColumn & column, ReadBuffer & istr)
{ {
DayNum_t x; DayNum x;
readDateText(x, istr); readDateText(x, istr);
static_cast<ColumnUInt16 &>(column).getData().push_back(x); 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 void DataTypeDate::deserializeTextQuoted(IColumn & column, ReadBuffer & istr) const
{ {
DayNum_t x; DayNum x;
assertChar('\'', istr); assertChar('\'', istr);
readDateText(x, istr); readDateText(x, istr);
assertChar('\'', 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 void DataTypeDate::deserializeTextJSON(IColumn & column, ReadBuffer & istr) const
{ {
DayNum_t x; DayNum x;
assertChar('"', istr); assertChar('"', istr);
readDateText(x, istr); readDateText(x, istr);
assertChar('"', istr); assertChar('"', istr);

View File

@ -29,16 +29,6 @@ HTTPDictionarySource::HTTPDictionarySource(const DictionaryStructure & dict_stru
context(context), context(context),
timeouts(ConnectionTimeouts::getHTTPTimeouts(context.getSettingsRef())) timeouts(ConnectionTimeouts::getHTTPTimeouts(context.getSettingsRef()))
{ {
if (update_field.empty())
return;
/// TODO This code is totally wrong and ignorant.
/// What if URL contains fragment (#). What if update_field contains characters that must be %-encoded.
std::string::size_type option = url.find("?");
if (option == std::string::npos)
update_field = '?' + update_field;
else
update_field = '&' + update_field;
} }
HTTPDictionarySource::HTTPDictionarySource(const HTTPDictionarySource & other) HTTPDictionarySource::HTTPDictionarySource(const HTTPDictionarySource & other)
@ -54,7 +44,7 @@ HTTPDictionarySource::HTTPDictionarySource(const HTTPDictionarySource & other)
{ {
} }
std::string HTTPDictionarySource::getUpdateFieldAndDate() void HTTPDictionarySource::getUpdateFieldAndDate(Poco::URI & uri)
{ {
if (update_time != std::chrono::system_clock::from_time_t(0)) if (update_time != std::chrono::system_clock::from_time_t(0))
{ {
@ -64,14 +54,14 @@ std::string HTTPDictionarySource::getUpdateFieldAndDate()
char buffer [80]; char buffer [80];
struct tm * timeinfo; struct tm * timeinfo;
timeinfo = localtime (&hr_time); timeinfo = localtime (&hr_time);
strftime(buffer, 80, "=%Y-%m-%d%%20%H:%M:%S", timeinfo); strftime(buffer, 80, "%Y-%m-%d %H:%M:%S", timeinfo);
std::string str_time(buffer); std::string str_time(buffer);
return url + update_field + str_time; uri.addQueryParameter(update_field, str_time);
} }
else else
{ {
update_time = std::chrono::system_clock::now(); update_time = std::chrono::system_clock::now();
return url + update_field + "=0000-00-00%2000:00:00"; ///for initial load uri.addQueryParameter(update_field, "0000-00-00 00:00:00");
} }
} }
@ -87,9 +77,9 @@ BlockInputStreamPtr HTTPDictionarySource::loadAll()
BlockInputStreamPtr HTTPDictionarySource::loadUpdatedAll() BlockInputStreamPtr HTTPDictionarySource::loadUpdatedAll()
{ {
std::string url_update = getUpdateFieldAndDate(); Poco::URI uri(url);
LOG_TRACE(log, "loadUpdatedAll " + url_update); getUpdateFieldAndDate(uri);
Poco::URI uri(url_update); LOG_TRACE(log, "loadUpdatedAll " + uri.toString());
auto in_ptr = std::make_unique<ReadWriteBufferFromHTTP>(uri, Poco::Net::HTTPRequest::HTTP_GET, auto in_ptr = std::make_unique<ReadWriteBufferFromHTTP>(uri, Poco::Net::HTTPRequest::HTTP_GET,
ReadWriteBufferFromHTTP::OutStreamCallback(), timeouts); ReadWriteBufferFromHTTP::OutStreamCallback(), timeouts);
auto input_stream = context.getInputFormat(format, *in_ptr, sample_block, max_block_size); auto input_stream = context.getInputFormat(format, *in_ptr, sample_block, max_block_size);

View File

@ -1,5 +1,6 @@
#pragma once #pragma once
#include <Poco/URI.h>
#include <Dictionaries/IDictionarySource.h> #include <Dictionaries/IDictionarySource.h>
#include <Dictionaries/DictionaryStructure.h> #include <Dictionaries/DictionaryStructure.h>
#include <common/LocalDateTime.h> #include <common/LocalDateTime.h>
@ -43,7 +44,7 @@ public:
std::string toString() const override; std::string toString() const override;
private: private:
std::string getUpdateFieldAndDate(); void getUpdateFieldAndDate(Poco::URI & uri);
Poco::Logger * log; Poco::Logger * log;

View File

@ -828,7 +828,7 @@ private:
if (is_date) if (is_date)
{ {
DayNum_t date; DayNum date;
ReadBufferFromMemory in(string_value.data, string_value.size); ReadBufferFromMemory in(string_value.data, string_value.size);
readDateText(date, in); readDateText(date, in);
if (!in.eof()) if (!in.eof())

View File

@ -112,7 +112,7 @@ struct ToDateTimeImpl
static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone) 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 *) 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 *) template <> inline void parseImpl<DataTypeDate>(DataTypeDate::FieldType & x, ReadBuffer & rb, const DateLUTImpl *)
{ {
DayNum_t tmp(0); DayNum tmp(0);
readDateText(tmp, rb); readDateText(tmp, rb);
x = tmp; x = tmp;
} }

View File

@ -133,7 +133,7 @@ struct ToMondayImpl
} }
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone) 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; using FactorTransform = ZeroTransform;
@ -149,7 +149,7 @@ struct ToStartOfMonthImpl
} }
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone) 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; using FactorTransform = ZeroTransform;
@ -165,7 +165,7 @@ struct ToStartOfQuarterImpl
} }
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone) 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; using FactorTransform = ZeroTransform;
@ -181,7 +181,7 @@ struct ToStartOfYearImpl
} }
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone) 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; using FactorTransform = ZeroTransform;
@ -281,7 +281,7 @@ struct ToYearImpl
} }
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone) 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; using FactorTransform = ZeroTransform;
@ -297,7 +297,7 @@ struct ToQuarterImpl
} }
static inline UInt8 execute(UInt16 d, const DateLUTImpl & time_zone) 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; using FactorTransform = ToStartOfYearImpl;
@ -313,7 +313,7 @@ struct ToMonthImpl
} }
static inline UInt8 execute(UInt16 d, const DateLUTImpl & time_zone) 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; using FactorTransform = ToStartOfYearImpl;
@ -329,7 +329,7 @@ struct ToDayOfMonthImpl
} }
static inline UInt8 execute(UInt16 d, const DateLUTImpl & time_zone) 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; using FactorTransform = ToStartOfMonthImpl;
@ -345,7 +345,7 @@ struct ToDayOfWeekImpl
} }
static inline UInt8 execute(UInt16 d, const DateLUTImpl & time_zone) 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; using FactorTransform = ToMondayImpl;
@ -410,7 +410,7 @@ struct ToRelativeYearNumImpl
} }
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone) 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; using FactorTransform = ZeroTransform;
@ -426,7 +426,7 @@ struct ToRelativeQuarterNumImpl
} }
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone) 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; using FactorTransform = ZeroTransform;
@ -442,7 +442,7 @@ struct ToRelativeMonthNumImpl
} }
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone) 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; using FactorTransform = ZeroTransform;
@ -458,7 +458,7 @@ struct ToRelativeWeekNumImpl
} }
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone) 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; using FactorTransform = ZeroTransform;
@ -474,7 +474,7 @@ struct ToRelativeDayNumImpl
} }
static inline UInt16 execute(UInt16 d, const DateLUTImpl &) static inline UInt16 execute(UInt16 d, const DateLUTImpl &)
{ {
return static_cast<DayNum_t>(d); return static_cast<DayNum>(d);
} }
using FactorTransform = ZeroTransform; using FactorTransform = ZeroTransform;
@ -491,7 +491,7 @@ struct ToRelativeHourNumImpl
} }
static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone) 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; using FactorTransform = ZeroTransform;
@ -507,7 +507,7 @@ struct ToRelativeMinuteNumImpl
} }
static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone) 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; using FactorTransform = ZeroTransform;
@ -523,7 +523,7 @@ struct ToRelativeSecondNumImpl
} }
static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone) 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; using FactorTransform = ZeroTransform;
@ -539,7 +539,7 @@ struct ToYYYYMMImpl
} }
static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone) 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; using FactorTransform = ZeroTransform;
@ -555,7 +555,7 @@ struct ToYYYYMMDDImpl
} }
static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone) 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; using FactorTransform = ZeroTransform;
@ -571,7 +571,7 @@ struct ToYYYYMMDDhhmmssImpl
} }
static inline UInt64 execute(UInt16 d, const DateLUTImpl & time_zone) 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; using FactorTransform = ZeroTransform;
@ -732,7 +732,7 @@ struct AddSecondsImpl
static inline UInt32 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone) 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) 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) 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) 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) 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);
} }
}; };

View File

@ -483,7 +483,7 @@ inline void readDateText(LocalDate & date, ReadBuffer & buf)
readDateTextFallback(date, buf); readDateTextFallback(date, buf);
} }
inline void readDateText(DayNum_t & date, ReadBuffer & buf) inline void readDateText(DayNum & date, ReadBuffer & buf)
{ {
LocalDate local_date; LocalDate local_date;
readDateText(local_date, buf); readDateText(local_date, buf);

View File

@ -537,7 +537,7 @@ inline void writeDateText(const LocalDate & date, WriteBuffer & buf)
} }
template <char delimiter = '-'> template <char delimiter = '-'>
inline void writeDateText(DayNum_t date, WriteBuffer & buf) inline void writeDateText(DayNum date, WriteBuffer & buf)
{ {
if (unlikely(!date)) if (unlikely(!date))
{ {

View 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;
}
}
}

View 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;
};
}

View File

@ -239,6 +239,7 @@ void Compiler::compile(
" -isystem " INTERNAL_COMPILER_HEADERS_ROOT "/usr/include/c++/*/backward" " -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/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/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-fixed"
" -isystem " INTERNAL_COMPILER_HEADERS_ROOT "/usr/lib/gcc/" CMAKE_LIBRARY_ARCHITECTURE "/*/include" " -isystem " INTERNAL_COMPILER_HEADERS_ROOT "/usr/lib/gcc/" CMAKE_LIBRARY_ARCHITECTURE "/*/include"
" -isystem " INTERNAL_COMPILER_HEADERS_ROOT "/usr/local/include" /// if something installed manually " -isystem " INTERNAL_COMPILER_HEADERS_ROOT "/usr/local/include" /// if something installed manually

View File

@ -15,6 +15,7 @@
#include <Common/setThreadName.h> #include <Common/setThreadName.h>
#include <Common/Stopwatch.h> #include <Common/Stopwatch.h>
#include <Common/formatReadable.h> #include <Common/formatReadable.h>
#include <Common/BackgroundSchedulePool.h>
#include <DataStreams/FormatFactory.h> #include <DataStreams/FormatFactory.h>
#include <Databases/IDatabase.h> #include <Databases/IDatabase.h>
#include <Storages/IStorage.h> #include <Storages/IStorage.h>
@ -24,6 +25,7 @@
#include <Storages/MergeTree/MergeTreeSettings.h> #include <Storages/MergeTree/MergeTreeSettings.h>
#include <Storages/CompressionSettingsSelector.h> #include <Storages/CompressionSettingsSelector.h>
#include <TableFunctions/TableFunctionFactory.h> #include <TableFunctions/TableFunctionFactory.h>
#include <Interpreters/ActionLocksManager.h>
#include <Interpreters/Settings.h> #include <Interpreters/Settings.h>
#include <Interpreters/RuntimeComponentsFactory.h> #include <Interpreters/RuntimeComponentsFactory.h>
#include <Interpreters/ISecurityManager.h> #include <Interpreters/ISecurityManager.h>
@ -131,6 +133,7 @@ struct ContextShared
ConfigurationPtr users_config; /// Config with the users, profiles and quotas sections. ConfigurationPtr users_config; /// Config with the users, profiles and quotas sections.
InterserverIOHandler interserver_io_handler; /// Handler for interserver communication. InterserverIOHandler interserver_io_handler; /// Handler for interserver communication.
BackgroundProcessingPoolPtr background_pool; /// The thread pool for the background work performed by the tables. 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. MultiVersion<Macros> macros; /// Substitutions extracted from config.
std::unique_ptr<Compiler> compiler; /// Used for dynamic compilation of queries' parts if it necessary. 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. 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. 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) 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. 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. /// 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; 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) void Context::setDDLWorker(std::shared_ptr<DDLWorker> ddl_worker)
{ {
auto lock = getLock(); auto lock = getLock();
@ -1721,6 +1732,16 @@ void Context::setFormatSchemaPath(const String & path)
shared->format_schema_path = 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() SessionCleaner::~SessionCleaner()
{ {

View File

@ -40,6 +40,7 @@ class ExternalDictionaries;
class ExternalModels; class ExternalModels;
class InterserverIOHandler; class InterserverIOHandler;
class BackgroundProcessingPool; class BackgroundProcessingPool;
class BackgroundSchedulePool;
class MergeList; class MergeList;
class Cluster; class Cluster;
class Compiler; class Compiler;
@ -69,6 +70,8 @@ using BlockOutputStreamPtr = std::shared_ptr<IBlockOutputStream>;
class Block; class Block;
struct SystemLogs; struct SystemLogs;
using SystemLogsPtr = std::shared_ptr<SystemLogs>; using SystemLogsPtr = std::shared_ptr<SystemLogs>;
class ActionLocksManager;
using ActionLocksManagerPtr = std::shared_ptr<ActionLocksManager>;
/// (database name, table name) /// (database name, table name)
@ -328,6 +331,7 @@ public:
void dropCaches() const; void dropCaches() const;
BackgroundProcessingPool & getBackgroundPool(); BackgroundProcessingPool & getBackgroundPool();
BackgroundSchedulePool & getSchedulePool();
void setDDLWorker(std::shared_ptr<DDLWorker> ddl_worker); void setDDLWorker(std::shared_ptr<DDLWorker> ddl_worker);
DDLWorker & getDDLWorker() const; DDLWorker & getDDLWorker() const;
@ -370,6 +374,8 @@ public:
void shutdown(); void shutdown();
ActionLocksManagerPtr getActionLocksManager();
enum class ApplicationType enum class ApplicationType
{ {
SERVER, /// The program is run as clickhouse-server daemon (default behavior) SERVER, /// The program is run as clickhouse-server daemon (default behavior)

View File

@ -1486,7 +1486,7 @@ void ExpressionAnalyzer::tryMakeSetFromSubquery(const ASTPtr & subquery_or_table
return; 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); 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())) 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); const ASTPtr & arg = args.children.at(1);
/// Already converted. /// Already converted.
if (prepared_sets.count(arg.get())) if (prepared_sets.count(arg->range))
return; return;
/// If the subquery or table name for SELECT. /// If the subquery or table name for SELECT.
@ -1573,7 +1573,7 @@ void ExpressionAnalyzer::makeSet(const ASTFunction * node, const Block & sample_
if (storage_set) if (storage_set)
{ {
prepared_sets[arg.get()] = storage_set->getSet(); prepared_sets[arg->range] = storage_set->getSet();
return; 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 you already created a Set with the same subquery / table.
if (subquery_for_set.set) if (subquery_for_set.set)
{ {
prepared_sets[arg.get()] = subquery_for_set.set; prepared_sets[arg->range] = subquery_for_set.set;
return; return;
} }
@ -1630,7 +1630,7 @@ void ExpressionAnalyzer::makeSet(const ASTFunction * node, const Block & sample_
} }
subquery_for_set.set = set; subquery_for_set.set = set;
prepared_sets[arg.get()] = set; prepared_sets[arg->range] = set;
} }
else 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)); 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); 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. /// Select the name in the next cycle.
argument_names.emplace_back(); 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; ColumnWithTypeAndName column;
column.type = std::make_shared<DataTypeSet>(); 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, /// 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). /// so that sets with the same literal representation do not fuse together (they can have different types).

View File

@ -3,9 +3,9 @@
#include <Interpreters/AggregateDescription.h> #include <Interpreters/AggregateDescription.h>
#include <Interpreters/Settings.h> #include <Interpreters/Settings.h>
#include <Core/Block.h> #include <Core/Block.h>
#include "ExpressionActions.h" #include <Interpreters/ExpressionActions.h>
#include "ProjectionManipulation.h" #include <Interpreters/ProjectionManipulation.h>
#include <Parsers/StringRange.h>
namespace DB namespace DB
{ {
@ -23,7 +23,9 @@ using ASTPtr = std::shared_ptr<IAST>;
class Set; class Set;
using SetPtr = std::shared_ptr<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; class IBlockInputStream;
using BlockInputStreamPtr = std::shared_ptr<IBlockInputStream>; using BlockInputStreamPtr = std::shared_ptr<IBlockInputStream>;

View File

@ -72,8 +72,16 @@ BlockIO InterpreterAlterQuery::execute()
table->attachPartition(command.partition, command.part, context); table->attachPartition(command.partition, command.part, context);
break; 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: case PartitionCommand::FETCH_PARTITION:
table->fetchPartition(command.partition, command.from, context); table->fetchPartition(command.partition, command.from_zookeeper_path, context);
break; break;
case PartitionCommand::FREEZE_PARTITION: case PartitionCommand::FREEZE_PARTITION:
@ -187,6 +195,11 @@ void InterpreterAlterQuery::parseAlter(
{ {
out_partition_commands.emplace_back(PartitionCommand::attachPartition(params.partition, params.part)); 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) else if (params.type == ASTAlterQuery::FETCH_PARTITION)
{ {
out_partition_commands.emplace_back(PartitionCommand::fetchPartition(params.partition, params.from)); out_partition_commands.emplace_back(PartitionCommand::fetchPartition(params.partition, params.from));

View File

@ -28,6 +28,7 @@ private:
{ {
DROP_PARTITION, DROP_PARTITION,
ATTACH_PARTITION, ATTACH_PARTITION,
REPLACE_PARTITION,
FETCH_PARTITION, FETCH_PARTITION,
FREEZE_PARTITION, FREEZE_PARTITION,
CLEAR_COLUMN, CLEAR_COLUMN,
@ -37,11 +38,20 @@ private:
ASTPtr partition; ASTPtr partition;
Field column_name; 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; 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 /// For FREEZE PARTITION
String with_name; String with_name;
@ -73,12 +83,23 @@ private:
return res; 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) static PartitionCommand fetchPartition(const ASTPtr & partition, const String & from)
{ {
PartitionCommand res; PartitionCommand res;
res.type = FETCH_PARTITION; res.type = FETCH_PARTITION;
res.partition = partition; res.partition = partition;
res.from = from; res.from_zookeeper_path = from;
return res; return res;
} }

View File

@ -1,10 +1,21 @@
#include <Interpreters/InterpreterSystemQuery.h> #include <Interpreters/InterpreterSystemQuery.h>
#include <Common/DNSResolver.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/Context.h>
#include <Interpreters/ExternalDictionaries.h> #include <Interpreters/ExternalDictionaries.h>
#include <Interpreters/EmbeddedDictionaries.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 <Parsers/ASTSystemQuery.h>
#include <Common/typeid_cast.h> #include <Parsers/ASTDropQuery.h>
#include <Parsers/ASTCreateQuery.h>
#include <csignal> #include <csignal>
@ -20,6 +31,15 @@ namespace ErrorCodes
} }
namespace ActionLocks
{
extern StorageActionBlockType PartsMerge;
extern StorageActionBlockType PartsFetch;
extern StorageActionBlockType PartsSend;
extern StorageActionBlockType ReplicationQueue;
}
namespace namespace
{ {
@ -50,11 +70,34 @@ ExecutionStatus getOverallExecutionStatusOfCommands(Callable && command, Callabl
return ExecutionStatus(res_status, res_message); 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_) 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() BlockIO InterpreterSystemQuery::execute()
@ -67,6 +110,10 @@ BlockIO InterpreterSystemQuery::execute()
Context system_context = context.getGlobalContext(); Context system_context = context.getGlobalContext();
system_context.setSetting("profile", context.getSystemProfileName()); 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) switch (query.type)
{ {
case Type::SHUTDOWN: case Type::SHUTDOWN:
@ -107,14 +154,43 @@ BlockIO InterpreterSystemQuery::execute()
case Type::RELOAD_CONFIG: case Type::RELOAD_CONFIG:
system_context.reloadConfig(); system_context.reloadConfig();
break; 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::STOP_LISTEN_QUERIES:
case Type::START_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); throw Exception(String(ASTSystemQuery::typeToString(query.type)) + " is not supported yet", ErrorCodes::NOT_IMPLEMENTED);
default: default:
throw Exception("Unknown type of SYSTEM query", ErrorCodes::BAD_ARGUMENTS); 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());
}
} }

View File

@ -7,7 +7,10 @@ namespace DB
class Context; class Context;
class IAST; class IAST;
class ASTSystemQuery;
class IStorage;
using ASTPtr = std::shared_ptr<IAST>; using ASTPtr = std::shared_ptr<IAST>;
using StoragePtr = std::shared_ptr<IStorage>;
class InterpreterSystemQuery : public IInterpreter class InterpreterSystemQuery : public IInterpreter
@ -20,6 +23,14 @@ public:
private: private:
ASTPtr query_ptr; ASTPtr query_ptr;
Context & context; 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);
}; };

View File

@ -85,25 +85,27 @@ public:
void addEndpoint(const String & name, InterserverIOEndpointPtr endpoint) void addEndpoint(const String & name, InterserverIOEndpointPtr endpoint)
{ {
std::lock_guard<std::mutex> lock(mutex); 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); throw Exception("Duplicate interserver IO endpoint: " + name, ErrorCodes::DUPLICATE_INTERSERVER_IO_ENDPOINT);
endpoint_map[name] = std::move(endpoint);
} }
void removeEndpoint(const String & name) void removeEndpoint(const String & name)
{ {
std::lock_guard<std::mutex> lock(mutex); 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); throw Exception("No interserver IO endpoint named " + name, ErrorCodes::NO_SUCH_INTERSERVER_IO_ENDPOINT);
endpoint_map.erase(name);
} }
InterserverIOEndpointPtr getEndpoint(const String & name) InterserverIOEndpointPtr getEndpoint(const String & name)
try
{ {
std::lock_guard<std::mutex> lock(mutex); std::lock_guard<std::mutex> lock(mutex);
if (!endpoint_map.count(name)) return endpoint_map.at(name);
}
catch (...)
{
throw Exception("No interserver IO endpoint named " + name, ErrorCodes::NO_SUCH_INTERSERVER_IO_ENDPOINT); throw Exception("No interserver IO endpoint named " + name, ErrorCodes::NO_SUCH_INTERSERVER_IO_ENDPOINT);
return endpoint_map[name];
} }
private: private:
@ -129,7 +131,6 @@ public:
} }
~InterserverIOEndpointHolder() ~InterserverIOEndpointHolder()
{
try try
{ {
handler.removeEndpoint(name); handler.removeEndpoint(name);
@ -140,11 +141,8 @@ public:
{ {
tryLogCurrentException("~InterserverIOEndpointHolder"); tryLogCurrentException("~InterserverIOEndpointHolder");
} }
}
ActionBlocker & getBlocker() { return endpoint->blocker; } ActionBlocker & getBlocker() { return endpoint->blocker; }
void cancelForever() { getBlocker().cancelForever(); }
ActionBlocker::LockHolder cancel() { return getBlocker().cancel(); }
private: private:
String name; String name;

View File

@ -88,34 +88,46 @@ 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; PartLog * part_log = nullptr;
try 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) if (!part_log)
return false; return false;
for (const auto & part : parts)
{
PartLogElement elem; PartLogElement elem;
elem.event_type = PartLogElement::NEW_PART; elem.event_type = PartLogElement::NEW_PART;
elem.event_time = time(nullptr); elem.event_time = time(nullptr);
elem.duration_ms = elapsed_ns / 1000000; elem.duration_ms = elapsed_ns / 1000000;
elem.database_name = part.storage.getDatabaseName(); elem.database_name = part->storage.getDatabaseName();
elem.table_name = part.storage.getTableName(); elem.table_name = part->storage.getTableName();
elem.part_name = part.name; elem.part_name = part->name;
elem.bytes_compressed_on_disk = part.bytes_on_disk; elem.bytes_compressed_on_disk = part->bytes_on_disk;
elem.rows = part.rows_count; elem.rows = part->rows_count;
elem.error = static_cast<UInt16>(execution_status.code); elem.error = static_cast<UInt16>(execution_status.code);
elem.exception = execution_status.message; elem.exception = execution_status.message;
part_log->add(elem); part_log->add(elem);
} }
}
catch (...) catch (...)
{ {
tryLogCurrentException(part_log ? part_log->log : &Logger::get("PartLog"), __PRETTY_FUNCTION__); tryLogCurrentException(part_log ? part_log->log : &Logger::get("PartLog"), __PRETTY_FUNCTION__);

View File

@ -55,9 +55,14 @@ class PartLog : public SystemLog<PartLogElement>
{ {
using SystemLog<PartLogElement>::SystemLog; using SystemLog<PartLogElement>::SystemLog;
using MutableDataPartPtr = std::shared_ptr<MergeTreeDataPart>;
using MutableDataPartsVector = std::vector<MutableDataPartPtr>;
public: public:
/// Add a record about creation of new part. /// Add a record about creation of new part.
static bool addNewPartToTheLog(Context & context, const MergeTreeDataPart & part, UInt64 elapsed_ns, 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 = {}); const ExecutionStatus & execution_status = {});
}; };

View File

@ -49,6 +49,7 @@ struct Settings
M(SettingBool, use_uncompressed_cache, true, "Whether to use the cache of uncompressed blocks.") \ 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(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_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.") \ 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.") \
\ \

View File

@ -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); ReadBufferFromString in(s);
DayNum_t date{}; DayNum date{};
readDateText(date, in); readDateText(date, in);
if (!in.eof()) if (!in.eof())

View File

@ -135,6 +135,19 @@ void ASTAlterQuery::formatQueryImpl(const FormatSettings & settings, FormatState
<< (p.part ? "PART " : "PARTITION ") << (settings.hilite ? hilite_none : ""); << (p.part ? "PART " : "PARTITION ") << (settings.hilite ? hilite_none : "");
p.partition->formatImpl(settings, state, frame); 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) else if (p.type == ASTAlterQuery::FETCH_PARTITION)
{ {
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "FETCH " settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "FETCH "

View File

@ -28,6 +28,7 @@ public:
DROP_PARTITION, DROP_PARTITION,
ATTACH_PARTITION, ATTACH_PARTITION,
REPLACE_PARTITION,
FETCH_PARTITION, FETCH_PARTITION,
FREEZE_PARTITION, FREEZE_PARTITION,
@ -57,7 +58,8 @@ public:
*/ */
ASTPtr primary_key; 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; ASTPtr partition;
@ -80,6 +82,12 @@ public:
*/ */
String with_name; 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 /// deep copy
void clone(Parameters & p) const; void clone(Parameters & p) const;
}; };

View File

@ -10,7 +10,12 @@ class ASTAsterisk : public IAST
{ {
public: public:
String getID() const override { return "Asterisk"; } String getID() const override { return "Asterisk"; }
ASTPtr clone() const override { return std::make_shared<ASTAsterisk>(*this); } ASTPtr clone() const override
{
auto clone = std::make_shared<ASTAsterisk>(*this);
clone->cloneChildren();
return std::move(clone);
}
String getColumnName() const override { return "*"; } String getColumnName() const override { return "*"; }
protected: protected:

View File

@ -6,13 +6,9 @@ namespace DB
ASTPtr ASTExpressionList::clone() const ASTPtr ASTExpressionList::clone() const
{ {
const auto res = std::make_shared<ASTExpressionList>(*this); auto clone = std::make_shared<ASTExpressionList>(*this);
res->children.clear(); clone->cloneChildren();
return std::move(clone);
for (const auto & child : children)
res->children.emplace_back(child->clone());
return res;
} }
void ASTExpressionList::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const void ASTExpressionList::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const

View File

@ -26,7 +26,7 @@ public:
Kind kind; Kind kind;
ASTIdentifier(const String & name_, const Kind kind_ = Column) 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. */ /** Get the text that identifies this element. */
String getID() const override { return "Identifier_" + name; } String getID() const override { return "Identifier_" + name; }

View File

@ -11,7 +11,13 @@ public:
bool sync = false; // SYNC or ASYNC mode bool sync = false; // SYNC or ASYNC mode
bool test = false; // does it TEST mode? (doesn't cancel queries just checks and shows them) bool test = false; // does it TEST mode? (doesn't cancel queries just checks and shows them)
ASTPtr clone() const override { return std::make_shared<ASTKillQueryQuery>(*this); } ASTPtr clone() const override
{
auto clone = std::make_shared<ASTKillQueryQuery>(*this);
clone->where_expression = where_expression->clone();
clone->children = {clone->where_expression};
return std::move(clone);
}
String getID() const override; String getID() const override;

View File

@ -5,7 +5,6 @@
namespace DB namespace DB
{ {
/** Element of expression with ASC or DESC, /** Element of expression with ASC or DESC,
* and possibly with COLLATE. * and possibly with COLLATE.
*/ */
@ -20,21 +19,27 @@ public:
ASTPtr collation; ASTPtr collation;
ASTOrderByElement( ASTOrderByElement(
const int direction_, const int direction_, const int nulls_direction_, const bool nulls_direction_was_explicitly_specified_, ASTPtr & collation_)
const int nulls_direction_, : direction(direction_)
const bool nulls_direction_was_explicitly_specified_, , nulls_direction(nulls_direction_)
ASTPtr & collation_) , nulls_direction_was_explicitly_specified(nulls_direction_was_explicitly_specified_)
: , collation(collation_)
direction(direction_), nulls_direction(nulls_direction_), {
nulls_direction_was_explicitly_specified(nulls_direction_was_explicitly_specified_), }
collation(collation_) {}
String getID() const override { return "OrderByElement"; } String getID() const override
{
return "OrderByElement";
}
ASTPtr clone() const override { return std::make_shared<ASTOrderByElement>(*this); } ASTPtr clone() const override
{
auto clone = std::make_shared<ASTOrderByElement>(*this);
clone->cloneChildren();
return std::move(clone);
}
protected: protected:
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
}; };
} }

View File

@ -13,7 +13,12 @@ class ASTQualifiedAsterisk : public IAST
{ {
public: public:
String getID() const override { return "QualifiedAsterisk"; } String getID() const override { return "QualifiedAsterisk"; }
ASTPtr clone() const override { return std::make_shared<ASTQualifiedAsterisk>(*this); } ASTPtr clone() const override
{
auto clone = std::make_shared<ASTQualifiedAsterisk>(*this);
clone->cloneChildren();
return std::move(clone);
}
String getColumnName() const override; String getColumnName() const override;
protected: protected:

View File

@ -33,6 +33,8 @@ const char * ASTSystemQuery::typeToString(Type type)
return "START LISTEN QUERIES"; return "START LISTEN QUERIES";
case Type::RESTART_REPLICAS: case Type::RESTART_REPLICAS:
return "RESTART REPLICAS"; return "RESTART REPLICAS";
case Type::RESTART_REPLICA:
return "RESTART REPLICA";
case Type::SYNC_REPLICA: case Type::SYNC_REPLICA:
return "SYNC REPLICA"; return "SYNC REPLICA";
case Type::RELOAD_DICTIONARY: case Type::RELOAD_DICTIONARY:
@ -47,6 +49,14 @@ const char * ASTSystemQuery::typeToString(Type type)
return "STOP MERGES"; return "STOP MERGES";
case Type::START_MERGES: case Type::START_MERGES:
return "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: case Type::STOP_REPLICATION_QUEUES:
return "STOP REPLICATION QUEUES"; return "STOP REPLICATION QUEUES";
case Type::START_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 << (settings.hilite ? hilite_keyword : "") << "SYSTEM " << (settings.hilite ? hilite_none : "");
settings.ostr << typeToString(type); 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); settings.ostr << " " << backQuoteIfNeed(target_dictionary);
else if (type == Type::SYNC_REPLICA)
throw Exception("SYNC_REPLICA isn't supported yet", ErrorCodes::NOT_IMPLEMENTED);
} }

View File

@ -21,6 +21,7 @@ public:
STOP_LISTEN_QUERIES, STOP_LISTEN_QUERIES,
START_LISTEN_QUERIES, START_LISTEN_QUERIES,
RESTART_REPLICAS, RESTART_REPLICAS,
RESTART_REPLICA,
SYNC_REPLICA, SYNC_REPLICA,
RELOAD_DICTIONARY, RELOAD_DICTIONARY,
RELOAD_DICTIONARIES, RELOAD_DICTIONARIES,
@ -28,6 +29,10 @@ public:
RELOAD_CONFIG, RELOAD_CONFIG,
STOP_MERGES, STOP_MERGES,
START_MERGES, START_MERGES,
STOP_FETCHES,
START_FETCHES,
STOP_REPLICATED_SENDS,
START_REPLICATEDS_SENDS,
STOP_REPLICATION_QUEUES, STOP_REPLICATION_QUEUES,
START_REPLICATION_QUEUES, START_REPLICATION_QUEUES,
END END
@ -38,8 +43,8 @@ public:
Type type = Type::UNKNOWN; Type type = Type::UNKNOWN;
String target_dictionary; String target_dictionary;
//String target_replica_database; String target_database;
//String target_replica_table; String target_table;
String getID() const override { return "SYSTEM query"; }; String getID() const override { return "SYSTEM query"; };

View File

@ -94,4 +94,11 @@ size_t IAST::checkDepthImpl(size_t max_depth, size_t level) const
return res; return res;
} }
void IAST::cloneChildren()
{
for (auto & child : children)
child = child->clone();
}
} }

View File

@ -65,7 +65,7 @@ public:
ASTPtr ptr() { return shared_from_this(); } 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; virtual ASTPtr clone() const = 0;
/** Get hash code, identifying this element and its subtree. /** Get hash code, identifying this element and its subtree.
@ -192,6 +192,8 @@ public:
void writeAlias(const String & name, std::ostream & s, bool hilite) const; void writeAlias(const String & name, std::ostream & s, bool hilite) const;
void cloneChildren();
public: public:
/// For syntax highlighting. /// For syntax highlighting.
static const char * hilite_keyword; static const char * hilite_keyword;

View File

@ -6,6 +6,8 @@
#include <Parsers/ASTIdentifier.h> #include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTAlterQuery.h> #include <Parsers/ASTAlterQuery.h>
#include <Parsers/ASTLiteral.h> #include <Parsers/ASTLiteral.h>
#include <Parsers/parseDatabaseAndTableName.h>
namespace DB namespace DB
{ {
@ -45,8 +47,6 @@ bool ParserAlterQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
ParserStringLiteral parser_string_literal; ParserStringLiteral parser_string_literal;
ParserExpression exp_elem; ParserExpression exp_elem;
ASTPtr table;
ASTPtr database;
String cluster_str; String cluster_str;
ASTPtr col_type; ASTPtr col_type;
ASTPtr col_after; ASTPtr col_after;
@ -57,24 +57,9 @@ bool ParserAlterQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
if (!s_alter_table.ignore(pos, expected)) if (!s_alter_table.ignore(pos, expected))
return false; return false;
if (!table_parser.parse(pos, database, expected)) if (!parseDatabaseAndTableName(pos, expected, query->database, query->table))
return false; 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 (ParserKeyword{"ON"}.ignore(pos, expected))
{ {
if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected)) if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected))
@ -142,8 +127,33 @@ bool ParserAlterQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
if (!parser_partition.parse(pos, params.partition, expected)) if (!parser_partition.parse(pos, params.partition, expected))
return false; return false;
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; 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)) else if (s_attach_part.ignore(pos, expected))
{ {
if (!parser_string_literal.parse(pos, params.partition, expected)) if (!parser_string_literal.parse(pos, params.partition, expected))

View File

@ -23,6 +23,8 @@ bool ParserKillQueryQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expect
if (!p_where_expression.parse(pos, query->where_expression, expected)) if (!p_where_expression.parse(pos, query->where_expression, expected))
return false; return false;
query->children.emplace_back(query->where_expression);
if (ParserKeyword{"SYNC"}.ignore(pos)) if (ParserKeyword{"SYNC"}.ignore(pos))
query->sync = true; query->sync = true;
else if (ParserKeyword{"ASYNC"}.ignore(pos)) else if (ParserKeyword{"ASYNC"}.ignore(pos))

View File

@ -2,7 +2,10 @@
#include <Parsers/ASTSystemQuery.h> #include <Parsers/ASTSystemQuery.h>
#include <Parsers/CommonParsers.h> #include <Parsers/CommonParsers.h>
#include <Parsers/ASTLiteral.h> #include <Parsers/ASTLiteral.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/parseIdentifierOrStringLiteral.h> #include <Parsers/parseIdentifierOrStringLiteral.h>
#include <Parsers/ExpressionElementParsers.h>
#include <Parsers/parseDatabaseAndTableName.h>
#include <Interpreters/evaluateConstantExpression.h> #include <Interpreters/evaluateConstantExpression.h>
@ -39,14 +42,33 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected &
if (!found) if (!found)
return false; return false;
if (res->type == Type::RELOAD_DICTIONARY) switch (res->type)
{ {
case Type::RELOAD_DICTIONARY:
if (!parseIdentifierOrStringLiteral(pos, expected, res->target_dictionary)) if (!parseIdentifierOrStringLiteral(pos, expected, res->target_dictionary))
return false; return false;
} break;
else if (res->type == Type::SYNC_REPLICA)
{ case Type::RESTART_REPLICA:
throw Exception("SYNC REPLICA is not supported yet", ErrorCodes::NOT_IMPLEMENTED); 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); node = std::move(res);

View File

@ -4,6 +4,7 @@
#include <Parsers/TokenIterator.h> #include <Parsers/TokenIterator.h>
#include <map> #include <map>
#include <memory> #include <memory>
#include <Common/SipHash.h>
namespace DB namespace DB
@ -14,9 +15,10 @@ struct StringRange
const char * first = nullptr; const char * first = nullptr;
const char * second = 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(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) 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(); 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);
}
};
};

View 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;
}
}

View 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);
}

View File

@ -15,6 +15,7 @@
#include <Poco/Util/Application.h> #include <Poco/Util/Application.h>
#include <common/readline_use.h> #include <common/readline_use.h>
#include <common/find_first_symbols.h> #include <common/find_first_symbols.h>
#include <common/SetTerminalEcho.h>
#include <Common/ClickHouseRevision.h> #include <Common/ClickHouseRevision.h>
#include <Common/Stopwatch.h> #include <Common/Stopwatch.h>
#include <Common/Exception.h> #include <Common/Exception.h>
@ -52,6 +53,7 @@
#include "InterruptListener.h" #include "InterruptListener.h"
#include <Functions/registerFunctions.h> #include <Functions/registerFunctions.h>
#include <AggregateFunctions/registerAggregateFunctions.h> #include <AggregateFunctions/registerAggregateFunctions.h>
#include <ext/scope_guard.h>
/// http://en.wikipedia.org/wiki/ANSI_escape_code /// http://en.wikipedia.org/wiki/ANSI_escape_code
@ -197,7 +199,25 @@ private:
default_database = config.getString("database", ""); default_database = config.getString("database", "");
user = config.getString("user", ""); user = config.getString("user", "");
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", ""); password = config.getString("password", "");
}
compression = config.getBool("compression", true) compression = config.getBool("compression", true)
? Protocol::Compression::Enable ? Protocol::Compression::Enable
@ -461,8 +481,9 @@ private:
query_id = config().getString("query_id", ""); query_id = config().getString("query_id", "");
nonInteractive(); nonInteractive();
/// If exception code isn't zero, we should return non-zero return code anyway.
if (last_exception) if (last_exception)
return last_exception->code(); return last_exception->code() != 0 ? last_exception->code() : -1;
return 0; return 0;
} }
@ -1364,18 +1385,27 @@ public:
ioctl(0, TIOCGWINSZ, &terminal_size); 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) #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. /// 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() main_description.add_options()
("help", "produce help message") ("help", "produce help message")
("config-file,c", boost::program_options::value<std::string>(), "config-file path") ("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") ("host,h", boost::program_options::value<std::string>()->default_value("localhost"), "server host")
("port", boost::program_options::value<int>()->default_value(9000), "server port") ("port", boost::program_options::value<int>()->default_value(9000), "server port")
("secure,s", "secure") ("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") ("password", boost::program_options::value<std::string>(), "password")
("ask-password", "ask-password")
("query_id", boost::program_options::value<std::string>(), "query_id") ("query_id", boost::program_options::value<std::string>(), "query_id")
("query,q", boost::program_options::value<std::string>(), "query") ("query,q", boost::program_options::value<std::string>(), "query")
("database,d", boost::program_options::value<std::string>(), "database") ("database,d", boost::program_options::value<std::string>(), "database")
@ -1482,7 +1512,8 @@ public:
config().setString("user", options["user"].as<std::string>()); config().setString("user", options["user"].as<std::string>());
if (options.count("password")) if (options.count("password"))
config().setString("password", options["password"].as<std::string>()); config().setString("password", options["password"].as<std::string>());
if (options.count("ask-password"))
config().setBool("ask-password", true);
if (options.count("multiline")) if (options.count("multiline"))
config().setBool("multiline", true); config().setBool("multiline", true);
if (options.count("multiquery")) if (options.count("multiquery"))

View File

@ -388,16 +388,23 @@ std::string LocalServer::getHelpFooter() const
void LocalServer::init(int argc, char ** argv) 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 /// Don't parse options with Poco library, we prefer neat boost::program_options
stopOptionsProcessing(); stopOptionsProcessing();
unsigned line_length = po::options_description::m_default_line_length;
unsigned min_description_length = line_length / 2;
if (isatty(STDIN_FILENO))
{
winsize terminal_size{}; winsize terminal_size{};
ioctl(0, TIOCGWINSZ, &terminal_size); ioctl(0, TIOCGWINSZ, &terminal_size);
line_length = std::max(3U, static_cast<unsigned>(terminal_size.ws_col));
namespace po = boost::program_options; min_description_length = std::min(min_description_length, line_length - 2);
}
#define DECLARE_SETTING(TYPE, NAME, DEFAULT, DESCRIPTION) (#NAME, po::value<std::string> (), DESCRIPTION) #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() description.add_options()
("help", "produce help message") ("help", "produce help message")
("config-file,c", po::value<std::string>(), "config-file path") ("config-file,c", po::value<std::string>(), "config-file path")

View File

@ -374,7 +374,8 @@ void DistributedBlockOutputStream::writeSuffix()
{ {
if (job.stream) if (job.stream)
{ {
pool->schedule([&job] () { pool->schedule([&job] ()
{
job.stream->writeSuffix(); job.stream->writeSuffix();
}); });
} }

View File

@ -9,6 +9,7 @@
#include <shared_mutex> #include <shared_mutex>
#include <memory> #include <memory>
#include <optional> #include <optional>
#include <Common/ActionLock.h>
namespace DB namespace DB
@ -26,6 +27,8 @@ class IBlockOutputStream;
class RWLockFIFO; class RWLockFIFO;
using RWLockFIFOPtr = std::shared_ptr<RWLockFIFO>; using RWLockFIFOPtr = std::shared_ptr<RWLockFIFO>;
using StorageActionBlockType = size_t;
using BlockOutputStreamPtr = std::shared_ptr<IBlockOutputStream>; using BlockOutputStreamPtr = std::shared_ptr<IBlockOutputStream>;
using BlockInputStreamPtr = std::shared_ptr<IBlockInputStream>; using BlockInputStreamPtr = std::shared_ptr<IBlockInputStream>;
using BlockInputStreams = std::vector<BlockInputStreamPtr>; using BlockInputStreams = std::vector<BlockInputStreamPtr>;
@ -104,6 +107,9 @@ public:
/** Returns true if the storage replicates SELECT, INSERT and ALTER commands among replicas. */ /** Returns true if the storage replicates SELECT, INSERT and ALTER commands among replicas. */
virtual bool supportsReplication() const { return false; } 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. /** 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. * 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. * 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); 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. /** Run the query (DROP|DETACH) PARTITION.
*/ */
virtual void dropPartition(const ASTPtr & /*query*/, const ASTPtr & /*partition*/, bool /*detach*/, const Context & /*context*/) virtual void dropPartition(const ASTPtr & /*query*/, const ASTPtr & /*partition*/, bool /*detach*/, const Context & /*context*/)
@ -284,6 +296,13 @@ public:
*/ */
virtual void shutdown() {} 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 */) const
{
return {};
}
bool is_dropped{false}; bool is_dropped{false};
/// Does table support index for IN sections /// Does table support index for IN sections

View File

@ -198,7 +198,7 @@ void ITableDeclaration::check(const NamesAndTypesList & provided_columns, const
throw Exception("There is no column with name " + name + ". There are columns: " throw Exception("There is no column with name " + name + ". There are columns: "
+ listOfColumns(available_columns), ErrorCodes::NO_SUCH_COLUMN_IN_TABLE); + 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 " throw Exception("Type mismatch for column " + name + ". Column has type "
+ jt->second->getName() + ", got type " + it->second->getName(), ErrorCodes::TYPE_MISMATCH); + jt->second->getName() + ", got type " + it->second->getName(), ErrorCodes::TYPE_MISMATCH);

View File

@ -8,22 +8,22 @@ ActiveDataPartSet::ActiveDataPartSet(MergeTreeDataFormatVersion format_version_,
: format_version(format_version_) : format_version(format_version_)
{ {
for (const auto & name : names) for (const auto & name : names)
addImpl(name); addUnlocked(name);
} }
void ActiveDataPartSet::add(const String & name) void ActiveDataPartSet::add(const String & name)
{ {
std::lock_guard<std::mutex> lock(mutex); std::lock_guard<std::mutex> lock(mutex);
addImpl(name); addUnlocked(name);
} }
void ActiveDataPartSet::addImpl(const String & name) void ActiveDataPartSet::addUnlocked(const String & name)
{ {
auto part_info = MergeTreePartInfo::fromPartName(name, format_version); auto part_info = MergeTreePartInfo::fromPartName(name, format_version);
if (!getContainingPartImpl(part_info).empty()) if (!getContainingPartUnlocked(part_info).empty())
return; return;
/// Parts contained in `part` are located contiguously in `part_info_to_name`, overlapping with the place where the part itself would be inserted. /// Parts contained in `part` are located contiguously in `part_info_to_name`, overlapping with the place where the part itself would be inserted.
@ -54,11 +54,11 @@ void ActiveDataPartSet::addImpl(const String & name)
String ActiveDataPartSet::getContainingPart(const String & part_name) const String ActiveDataPartSet::getContainingPart(const String & part_name) const
{ {
std::lock_guard<std::mutex> lock(mutex); std::lock_guard<std::mutex> lock(mutex);
return getContainingPartImpl(MergeTreePartInfo::fromPartName(part_name, format_version)); return getContainingPartUnlocked(MergeTreePartInfo::fromPartName(part_name, format_version));
} }
String ActiveDataPartSet::getContainingPartImpl(const MergeTreePartInfo & part_info) const String ActiveDataPartSet::getContainingPartUnlocked(const MergeTreePartInfo & part_info) const
{ {
/// A part can only be covered/overlapped by the previous or next one in `part_info_to_name`. /// A part can only be covered/overlapped by the previous or next one in `part_info_to_name`.
auto it = part_info_to_name.lower_bound(part_info); auto it = part_info_to_name.lower_bound(part_info);
@ -79,11 +79,8 @@ String ActiveDataPartSet::getContainingPartImpl(const MergeTreePartInfo & part_i
return String(); return String();
} }
Strings ActiveDataPartSet::getPartsUnlocked() const
Strings ActiveDataPartSet::getParts() const
{ {
std::lock_guard<std::mutex> lock(mutex);
Strings res; Strings res;
res.reserve(part_info_to_name.size()); res.reserve(part_info_to_name.size());
for (const auto & kv : part_info_to_name) for (const auto & kv : part_info_to_name)
@ -92,6 +89,12 @@ Strings ActiveDataPartSet::getParts() const
return res; return res;
} }
Strings ActiveDataPartSet::getParts() const
{
std::lock_guard<std::mutex> lock(mutex);
return getPartsUnlocked();
}
size_t ActiveDataPartSet::size() const size_t ActiveDataPartSet::size() const
{ {

View File

@ -12,8 +12,7 @@ namespace DB
/** Supports multiple names of active parts of data. /** Supports multiple names of active parts of data.
* Repeats part of the MergeTreeData functionality. * 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. * TODO: generalize with MergeTreeData
* Then in MergeTreeData you can make map<String, DataPartPtr> data_parts and all_data_parts.
*/ */
class ActiveDataPartSet class ActiveDataPartSet
{ {
@ -26,19 +25,21 @@ public:
/// If not found, returns an empty string. /// If not found, returns an empty string.
String getContainingPart(const String & name) const; String getContainingPart(const String & name) 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; size_t size() const;
/// Do not block mutex.
void addUnlocked(const String & name);
String getContainingPartUnlocked(const MergeTreePartInfo & part_info) const;
Strings getPartsUnlocked() const;
private: private:
MergeTreeDataFormatVersion format_version; MergeTreeDataFormatVersion format_version;
mutable std::mutex mutex; mutable std::mutex mutex;
std::map<MergeTreePartInfo, String> part_info_to_name; std::map<MergeTreePartInfo, String> part_info_to_name;
/// Do not block mutex.
void addImpl(const String & name);
String getContainingPartImpl(const MergeTreePartInfo & part_info) const;
}; };
} }

View File

@ -160,7 +160,8 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchPart(
const String & host, const String & host,
int port, int port,
const ConnectionTimeouts & timeouts, const ConnectionTimeouts & timeouts,
bool to_detached) bool to_detached,
const String & tmp_prefix_)
{ {
Poco::URI uri; Poco::URI uri;
uri.setScheme("http"); uri.setScheme("http");
@ -176,7 +177,9 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchPart(
ReadWriteBufferFromHTTP in{uri, Poco::Net::HTTPRequest::HTTP_POST, {}, timeouts}; ReadWriteBufferFromHTTP in{uri, Poco::Net::HTTPRequest::HTTP_POST, {}, timeouts};
static const String TMP_PREFIX = "tmp_fetch_"; 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 + "/"; String absolute_part_path = data.getFullPath() + relative_part_path + "/";
Poco::File part_file(absolute_part_path); Poco::File part_file(absolute_part_path);

View File

@ -54,7 +54,8 @@ public:
const String & host, const String & host,
int port, int port,
const ConnectionTimeouts & timeouts, const ConnectionTimeouts & timeouts,
bool to_detached = false); bool to_detached = false,
const String & tmp_prefix_ = "");
/// You need to stop the data transfer. /// You need to stop the data transfer.
ActionBlocker blocker; ActionBlocker blocker;

View File

@ -641,8 +641,8 @@ bool KeyCondition::atomFromAST(const ASTPtr & node, const Context & context, Blo
bool is_set_const = false; bool is_set_const = false;
bool is_constant_transformed = false; bool is_constant_transformed = false;
if (prepared_sets.count(args[1].get()) if (prepared_sets.count(args[1]->range)
&& isTupleIndexable(args[0], context, out, prepared_sets[args[1].get()], key_column_num)) && isTupleIndexable(args[0], context, out, prepared_sets[args[1]->range], key_column_num))
{ {
key_arg_pos = 0; key_arg_pos = 0;
is_set_const = true; 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()); auto in_func = typeid_cast<const ASTFunction *>(element.in_function.get());
const ASTs & args = typeid_cast<const ASTExpressionList &>(*in_func->arguments).children; 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()) if (in_func && it != prepared_sets.end())
{ {
rpn_stack.emplace_back(element.set_index->mayBeTrueInRange(key_ranges, data_types)); rpn_stack.emplace_back(element.set_index->mayBeTrueInRange(key_ranges, data_types));

View File

@ -14,7 +14,7 @@ Block MergeTreeBlockOutputStream::getHeader() const
void MergeTreeBlockOutputStream::write(const Block & block) void MergeTreeBlockOutputStream::write(const Block & block)
{ {
storage.data.delayInsertIfNeeded(); storage.data.delayInsertOrThrowIfNeeded();
auto part_blocks = storage.writer.splitBlockIntoParts(block); auto part_blocks = storage.writer.splitBlockIntoParts(block);
for (auto & current_block : part_blocks) for (auto & current_block : part_blocks)
@ -24,7 +24,7 @@ void MergeTreeBlockOutputStream::write(const Block & block)
MergeTreeData::MutableDataPartPtr part = storage.writer.writeTempPart(current_block); MergeTreeData::MutableDataPartPtr part = storage.writer.writeTempPart(current_block);
storage.data.renameTempPartAndAdd(part, &storage.increment); 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'. /// 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(); storage.merge_task_handle->wake();

View File

@ -3,6 +3,8 @@
#include <Storages/MergeTree/MergeTreeBlockInputStream.h> #include <Storages/MergeTree/MergeTreeBlockInputStream.h>
#include <Storages/MergeTree/MergedBlockOutputStream.h> #include <Storages/MergeTree/MergedBlockOutputStream.h>
#include <Storages/MergeTree/checkDataPart.h> #include <Storages/MergeTree/checkDataPart.h>
#include <Storages/StorageMergeTree.h>
#include <Storages/StorageReplicatedMergeTree.h>
#include <Storages/AlterCommands.h> #include <Storages/AlterCommands.h>
#include <Parsers/ASTIdentifier.h> #include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTNameTypePair.h> #include <Parsers/ASTNameTypePair.h>
@ -11,6 +13,7 @@
#include <Parsers/ASTPartition.h> #include <Parsers/ASTPartition.h>
#include <Parsers/ExpressionListParsers.h> #include <Parsers/ExpressionListParsers.h>
#include <Parsers/parseQuery.h> #include <Parsers/parseQuery.h>
#include <Parsers/queryToString.h>
#include <DataStreams/ExpressionBlockInputStream.h> #include <DataStreams/ExpressionBlockInputStream.h>
#include <DataStreams/ValuesRowInputStream.h> #include <DataStreams/ValuesRowInputStream.h>
#include <DataStreams/copyData.h> #include <DataStreams/copyData.h>
@ -36,6 +39,7 @@
#include <Common/Stopwatch.h> #include <Common/Stopwatch.h>
#include <Common/typeid_cast.h> #include <Common/typeid_cast.h>
#include <Common/localBackup.h> #include <Common/localBackup.h>
#include <Interpreters/PartLog.h>
#include <Poco/DirectoryIterator.h> #include <Poco/DirectoryIterator.h>
@ -47,7 +51,6 @@
#include <typeinfo> #include <typeinfo>
#include <typeindex> #include <typeindex>
#include <optional> #include <optional>
#include <Interpreters/PartLog.h>
namespace ProfileEvents namespace ProfileEvents
@ -70,11 +73,11 @@ namespace ErrorCodes
{ {
extern const int MEMORY_LIMIT_EXCEEDED; extern const int MEMORY_LIMIT_EXCEEDED;
extern const int SYNTAX_ERROR; extern const int SYNTAX_ERROR;
extern const int CORRUPTED_DATA;
extern const int INVALID_PARTITION_VALUE; extern const int INVALID_PARTITION_VALUE;
extern const int METADATA_MISMATCH; extern const int METADATA_MISMATCH;
extern const int PART_IS_TEMPORARILY_LOCKED; extern const int PART_IS_TEMPORARILY_LOCKED;
extern const int TOO_MANY_PARTS; 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) for (auto & part : broken_parts_to_remove)
part->remove(); part->remove();
for (auto & part : broken_parts_to_detach) 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 /// 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. /// 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 MergeTreePartInfo & new_part_info,
const String & new_part_name, const String & new_part_name,
DataPartPtr & out_covering_part, 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. /// 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); auto committed_parts_range = getDataPartsStateRange(DataPartState::Committed);
/// Go to the left. /// Go to the left.
@ -1435,16 +1438,10 @@ void MergeTreeData::renameTempPartAndAdd(MutableDataPartPtr & part, SimpleIncrem
} }
void MergeTreeData::renameTempPartAndReplace(
MergeTreeData::DataPartsVector MergeTreeData::renameTempPartAndReplace( MutableDataPartPtr & part, SimpleIncrement * increment, MergeTreeData::Transaction * out_transaction,
MutableDataPartPtr & part, SimpleIncrement * increment, 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}); part->assertState({DataPartState::Temporary});
MergeTreePartInfo part_info = part->info; MergeTreePartInfo part_info = part->info;
@ -1489,7 +1486,7 @@ MergeTreeData::DataPartsVector MergeTreeData::renameTempPartAndReplace(
if (covering_part) if (covering_part)
{ {
LOG_WARNING(log, "Tried to add obsolete part " << part_name << " covered by " << covering_part->getNameWithState()); 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. /// All checks are passed. Now we can rename the part on disk.
@ -1523,12 +1520,48 @@ MergeTreeData::DataPartsVector MergeTreeData::renameTempPartAndReplace(
addPartContributionToColumnSizes(part); 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; 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) for (auto & part : remove)
{ {
@ -1538,37 +1571,83 @@ void MergeTreeData::removePartsFromWorkingSet(const DataPartsVector & remove, bo
part->assertState({DataPartState::PreCommitted, DataPartState::Committed, DataPartState::Outdated}); part->assertState({DataPartState::PreCommitted, DataPartState::Committed, DataPartState::Outdated});
} }
auto remove_time = clear_without_timeout ? 0 : time(nullptr); removePartsFromWorkingSet(remove, clear_without_timeout, lock);
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);
}
} }
MergeTreeData::DataPartsVector MergeTreeData::removePartsInRangeFromWorkingSet(const MergeTreePartInfo & drop_range, bool clear_without_timeout,
void MergeTreeData::renameAndDetachPart(const DataPartPtr & part_to_detach, const String & prefix, bool restore_covered, bool skip_intersecting_parts, DataPartsLock & lock)
bool move_to_detached)
{ {
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); auto it_part = data_parts_by_info.find(part_to_detach->info);
if (it_part == data_parts_by_info.end()) if (it_part == data_parts_by_info.end())
throw Exception("No such data part " + part_to_detach->getNameWithState(), ErrorCodes::NO_SUCH_DATA_PART); 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; DataPartPtr part = *it_part;
if (part->state == DataPartState::Committed) if (part->state == DataPartState::Committed)
removePartContributionToColumnSizes(part); removePartContributionToColumnSizes(part);
modifyPartState(it_part, DataPartState::Deleting); 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); data_parts_indexes.erase(it_part);
if (restore_covered && part->info.level == 0) 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(); const size_t parts_count = getMaxPartsCountForPartition();
if (parts_count < settings.parts_to_delay_insert) 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))); 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. /// 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 != committed_parts_range.end())
{ {
if ((*it)->name == part_name) if ((*it)->info == part_info)
return *it; return *it;
if ((*it)->info.contains(part_info)) if ((*it)->info.contains(part_info))
return *it; return *it;
@ -1767,11 +1854,28 @@ MergeTreeData::DataPartPtr MergeTreeData::getActiveContainingPart(const String &
return nullptr; return nullptr;
} }
MergeTreeData::DataPartPtr MergeTreeData::getActiveContainingPart(const String & part_name)
MergeTreeData::DataPartPtr MergeTreeData::getPartIfExists(const String & part_name, const MergeTreeData::DataPartStates & valid_states)
{ {
auto part_info = MergeTreePartInfo::fromPartName(part_name, format_version); 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); std::lock_guard<std::mutex> lock(data_parts_mutex);
auto it = data_parts_by_info.find(part_info); auto it = data_parts_by_info.find(part_info);
@ -1787,6 +1891,11 @@ MergeTreeData::DataPartPtr MergeTreeData::getPartIfExists(const String & part_na
return nullptr; 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) 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); 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); DataPartPtr existing_part_in_partition = getAnyPartInPartition(partition_id, data_parts_lock);
if (existing_part_in_partition && existing_part_in_partition->partition.value != partition.value) 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( 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(); auto it = data_parts_by_state_and_info.lower_bound(DataPartStateAndPartitionID{DataPartState::Committed, partition_id});
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));
if (it != data_parts_by_state_and_info.end() && (*it)->state == DataPartState::Committed && (*it)->info.partition_id == partition_id) if (it != data_parts_by_state_and_info.end() && (*it)->state == DataPartState::Committed && (*it)->info.partition_id == partition_id)
return *it; return *it;
@ -2130,19 +2236,20 @@ void MergeTreeData::Transaction::rollback()
clear(); clear();
} }
MergeTreeData::DataPartsVector MergeTreeData::Transaction::commit() MergeTreeData::DataPartsVector MergeTreeData::Transaction::commit(MergeTreeData::DataPartsLock * acquired_parts_lock)
{ {
DataPartsVector total_covered_parts; DataPartsVector total_covered_parts;
if (!isEmpty()) 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); auto current_time = time(nullptr);
for (const DataPartPtr & part : precommitted_parts) for (const DataPartPtr & part : precommitted_parts)
{ {
DataPartPtr covering_part; 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) if (covering_part)
{ {
LOG_WARNING(data->log, "Tried to commit obsolete part " << part->name 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;
}
} }

View File

@ -20,6 +20,8 @@
#include <boost/multi_index/ordered_index.hpp> #include <boost/multi_index/ordered_index.hpp>
#include <boost/multi_index/global_fun.hpp> #include <boost/multi_index/global_fun.hpp>
#include <boost/range/iterator_range_core.hpp> #include <boost/range/iterator_range_core.hpp>
#include "../../Core/Types.h"
namespace DB namespace DB
{ {
@ -94,6 +96,7 @@ public:
using DataPart = MergeTreeDataPart; using DataPart = MergeTreeDataPart;
using MutableDataPartPtr = std::shared_ptr<DataPart>; using MutableDataPartPtr = std::shared_ptr<DataPart>;
using MutableDataPartsVector = std::vector<MutableDataPartPtr>;
/// After the DataPart is added to the working set, it cannot be changed. /// After the DataPart is added to the working set, it cannot be changed.
using DataPartPtr = std::shared_ptr<const DataPart>; using DataPartPtr = std::shared_ptr<const DataPart>;
@ -106,10 +109,17 @@ public:
{ {
DataPartState state; DataPartState state;
const MergeTreePartInfo & info; 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 struct LessDataPart
{ {
using is_transparent = void; 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 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 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 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 struct LessStateDataPart
@ -138,11 +150,26 @@ public:
{ {
return static_cast<size_t>(state) < static_cast<size_t>(info.state); 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 DataParts = std::set<DataPartPtr, LessDataPart>;
using DataPartsVector = std::vector<DataPartPtr>; 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: /// 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). /// * 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 /// * Next, if commit() is called, the parts are added to the active set and the parts that are
@ -153,8 +180,7 @@ public:
public: public:
Transaction() {} Transaction() {}
/// Return parts marked Obsolete as a result of the transaction commit. DataPartsVector commit(MergeTreeData::DataPartsLock * acquired_parts_lock = nullptr);
DataPartsVector commit();
void rollback(); void rollback();
@ -215,7 +241,7 @@ public:
} }
DataPartPtr data_part; DataPartPtr data_part;
std::unique_lock<std::mutex> alter_lock; DataPartsLock alter_lock;
DataPart::Checksums new_checksums; DataPart::Checksums new_checksums;
NamesAndTypesList new_columns; NamesAndTypesList new_columns;
@ -347,9 +373,14 @@ public:
/// Returns a committed part with the given name or a part containing it. If there is no such part, returns nullptr. /// 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 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. /// 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 String & part_name, const DataPartStates & valid_states);
DataPartPtr getPartIfExists(const MergeTreePartInfo & part_info, const DataPartStates & valid_states);
/// Total size of active parts in bytes. /// Total size of active parts in bytes.
size_t getTotalActiveSizeInBytes() const; size_t getTotalActiveSizeInBytes() const;
@ -358,7 +389,8 @@ public:
/// If the table contains too many active parts, sleep for a while to give them time to merge. /// 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. /// 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. /// 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. /// It is assumed that the part does not intersect with existing parts.
@ -374,16 +406,32 @@ public:
DataPartsVector renameTempPartAndReplace( DataPartsVector renameTempPartAndReplace(
MutableDataPartPtr & part, SimpleIncrement * increment = nullptr, Transaction * out_transaction = nullptr); 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. /// Removes parts from the working set parts.
/// Parts in add must already be in data_parts with PreCommitted, Committed, or Outdated states. /// 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 /// If clear_without_timeout is true, the parts will be deleted at once, or during the next call to
/// clearOldParts (ignoring old_parts_lifetime). /// 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 /// Removes all parts from the working set parts
/// clearOldParts. /// 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. /// 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 /// Returns old inactive parts that can be deleted. At the same time removes them from the list of parts
/// but not from the disk. /// but not from the disk.
@ -477,6 +525,13 @@ public:
/// For ATTACH/DETACH/DROP PARTITION. /// For ATTACH/DETACH/DROP PARTITION.
String getPartitionIDFromQuery(const ASTPtr & partition, const Context & context); 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; MergeTreeDataFormatVersion format_version;
@ -515,6 +570,8 @@ private:
friend class StorageMergeTree; friend class StorageMergeTree;
friend class ReplicatedMergeTreeAlterThread; friend class ReplicatedMergeTreeAlterThread;
friend class MergeTreeDataMerger; friend class MergeTreeDataMerger;
friend class StorageMergeTree;
friend class StorageReplicatedMergeTree;
bool require_part_metadata; bool require_part_metadata;
@ -587,6 +644,13 @@ private:
return {begin, end}; 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) static decltype(auto) getStateModifier(DataPartState state)
{ {
return [state] (const DataPartPtr & part) { part->state = state; }; return [state] (const DataPartPtr & part) { part->state = state; };
@ -641,7 +705,7 @@ private:
void removePartContributionToColumnSizes(const DataPartPtr & part); 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. /// 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. /// 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. /// Will check that the new part doesn't already exist and that it doesn't intersect existing part.
@ -649,7 +713,7 @@ private:
const MergeTreePartInfo & new_part_info, const MergeTreePartInfo & new_part_info,
const String & new_part_name, const String & new_part_name,
DataPartPtr & out_covering_part, 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. /// 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; bool isPrimaryOrMinMaxKeyColumnPossiblyWrappedInFunctions(const ASTPtr & node) const;

View File

@ -93,8 +93,8 @@ void MergeTreeDataMerger::FuturePart::assign(MergeTreeData::DataPartsVector part
if (parts.front()->storage.format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) 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 min_date = DayNum(std::numeric_limits<UInt16>::max());
DayNum_t max_date = DayNum_t(std::numeric_limits<UInt16>::min()); DayNum max_date = DayNum(std::numeric_limits<UInt16>::min());
for (const auto & part : parts) for (const auto & part : parts)
{ {
min_date = std::min(min_date, part->getMinDate()); min_date = std::min(min_date, part->getMinDate());

View File

@ -13,6 +13,7 @@
#include <Common/SipHash.h> #include <Common/SipHash.h>
#include <Common/escapeForFileName.h> #include <Common/escapeForFileName.h>
#include <Common/StringUtils/StringUtils.h> #include <Common/StringUtils/StringUtils.h>
#include <Common/localBackup.h>
#include <Storages/MergeTree/MergeTreeDataPart.h> #include <Storages/MergeTree/MergeTreeDataPart.h>
#include <Storages/MergeTree/MergeTreeData.h> #include <Storages/MergeTree/MergeTreeData.h>
@ -230,21 +231,21 @@ String MergeTreeDataPart::getNameWithPrefix() const
} }
DayNum_t MergeTreeDataPart::getMinDate() const DayNum MergeTreeDataPart::getMinDate() const
{ {
if (storage.minmax_idx_date_column_pos != -1) if (storage.minmax_idx_date_column_pos != -1)
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 else
return DayNum_t(); return DayNum();
} }
DayNum_t MergeTreeDataPart::getMaxDate() const DayNum MergeTreeDataPart::getMaxDate() const
{ {
if (storage.minmax_idx_date_column_pos != -1) if (storage.minmax_idx_date_column_pos != -1)
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 else
return DayNum_t(); return DayNum();
} }
@ -366,30 +367,46 @@ 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; 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, /** 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". * 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. * 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. * 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()) while (try_no < 10)
{ {
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." LOG_WARNING(storage.log, "Directory " << dst_name() << " (to detach to) is already exist."
" Will detach to directory with '_tryN' suffix."); " Will detach to directory with '_tryN' suffix.");
++try_no; ++try_no;
} }
return res;
} }
renameTo(dst_name()); 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) void MergeTreeDataPart::loadColumnsChecksumsIndexes(bool require_columns_checksums, bool check_consistency)
{ {
loadColumns(require_columns_checksums); loadColumns(require_columns_checksums);
@ -452,8 +469,8 @@ void MergeTreeDataPart::loadPartitionAndMinMaxIndex()
{ {
if (storage.format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) if (storage.format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING)
{ {
DayNum_t min_date; DayNum min_date;
DayNum_t max_date; DayNum max_date;
MergeTreePartInfo::parseMinMaxDatesFromPartName(name, min_date, max_date); MergeTreePartInfo::parseMinMaxDatesFromPartName(name, min_date, max_date);
const auto & date_lut = DateLUT::instance(); const auto & date_lut = DateLUT::instance();

View File

@ -8,6 +8,8 @@
#include <Storages/MergeTree/MergeTreeDataPartChecksum.h> #include <Storages/MergeTree/MergeTreeDataPartChecksum.h>
#include <Columns/IColumn.h> #include <Columns/IColumn.h>
#include <shared_mutex> #include <shared_mutex>
#include "../../../../contrib/poco/Foundation/include/Poco/Path.h"
#include "../../Core/Types.h"
namespace DB namespace DB
@ -63,8 +65,8 @@ struct MergeTreeDataPart
bool contains(const MergeTreeDataPart & other) const { return info.contains(other.info); } 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. /// 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 getMinDate() const;
DayNum_t getMaxDate() const; DayNum getMaxDate() const;
MergeTreeData & storage; MergeTreeData & storage;
@ -176,7 +178,7 @@ struct MergeTreeDataPart
MinMaxIndex() = default; MinMaxIndex() = default;
/// For month-based partitioning. /// 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)) : min_values(1, static_cast<UInt64>(min_date))
, max_values(1, static_cast<UInt64>(max_date)) , max_values(1, static_cast<UInt64>(max_date))
, initialized(true) , initialized(true)
@ -225,8 +227,14 @@ struct MergeTreeDataPart
/// Changes only relative_dir_name, you need to update other metadata (name, is_temp) explicitly /// 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; 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. /// Generate unique path to detach part
void renameAddPrefix(bool to_detached, const String & prefix) const; 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). /// Populates columns_to_size map (compressed size).
void accumulateColumnSizes(ColumnToSize & column_to_size) const; void accumulateColumnSizes(ColumnToSize & column_to_size) const;

View File

@ -1,5 +1,6 @@
#include "MergeTreeDataPartChecksum.h" #include "MergeTreeDataPartChecksum.h"
#include <Common/SipHash.h> #include <Common/SipHash.h>
#include <Common/hex.h>
#include <IO/ReadHelpers.h> #include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h> #include <IO/WriteHelpers.h>
#include <IO/ReadBufferFromString.h> #include <IO/ReadBufferFromString.h>
@ -272,6 +273,34 @@ bool MergeTreeDataPartChecksums::isBadChecksumsErrorCode(int code)
|| code == ErrorCodes::UNEXPECTED_FILE_IN_DATA_PART; || 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 void MinimalisticDataPartChecksums::serialize(WriteBuffer & to) const
{ {
writeString("checksums format version: 5\n", to); writeString("checksums format version: 5\n", to);
@ -331,31 +360,24 @@ void MinimalisticDataPartChecksums::computeTotalChecksums(const MergeTreeDataPar
SipHash hash_of_uncompressed_files_; SipHash hash_of_uncompressed_files_;
SipHash uncompressed_hash_of_compressed_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) for (const auto & elem : full_checksums.files)
{ {
const String & name = elem.first; const String & name = elem.first;
const auto & checksum = elem.second; 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); hash_of_all_files_.update(checksum.file_hash);
if (!checksum.is_compressed) if (!checksum.is_compressed)
{ {
++num_uncompressed_files; ++num_uncompressed_files;
update_hash(hash_of_uncompressed_files_, name); updateHash(hash_of_uncompressed_files_, name);
hash_of_uncompressed_files_.update(checksum.file_hash); hash_of_uncompressed_files_.update(checksum.file_hash);
} }
else else
{ {
++num_compressed_files; ++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); uncompressed_hash_of_compressed_files_.update(checksum.uncompressed_hash);
} }
} }

View File

@ -79,6 +79,9 @@ struct MergeTreeDataPartChecksums
/// Checksum from the set of checksums of .bin files (for deduplication). /// Checksum from the set of checksums of .bin files (for deduplication).
void computeTotalChecksumDataOnly(SipHash & hash) const; void computeTotalChecksumDataOnly(SipHash & hash) const;
/// SipHash of all all files hashes represented as hex string
String getTotalChecksumHex() const;
String getSerializedString() const; String getSerializedString() const;
static MergeTreeDataPartChecksums deserializeFrom(const String & s); static MergeTreeDataPartChecksums deserializeFrom(const String & s);
}; };

View File

@ -140,13 +140,13 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa
String part_name; String part_name;
if (data.format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) 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 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 max_date(minmax_idx.max_values[data.minmax_idx_date_column_pos].get<UInt64>());
const auto & date_lut = DateLUT::instance(); const auto & date_lut = DateLUT::instance();
DayNum_t min_month = date_lut.toFirstDayNumOfMonth(DayNum_t(min_date)); DayNum min_month = date_lut.toFirstDayNumOfMonth(DayNum(min_date));
DayNum_t max_month = date_lut.toFirstDayNumOfMonth(DayNum_t(max_date)); DayNum max_month = date_lut.toFirstDayNumOfMonth(DayNum(max_date));
if (min_month != max_month) if (min_month != max_month)
throw Exception("Logical error: part spans more than one month."); throw Exception("Logical error: part spans more than one month.");

View File

@ -78,7 +78,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 min_yyyymmdd = 0;
UInt32 max_yyyymmdd = 0; UInt32 max_yyyymmdd = 0;
@ -97,8 +97,8 @@ void MergeTreePartInfo::parseMinMaxDatesFromPartName(const String & dir_name, Da
min_date = date_lut.YYYYMMDDToDayNum(min_yyyymmdd); min_date = date_lut.YYYYMMDDToDayNum(min_yyyymmdd);
max_date = date_lut.YYYYMMDDToDayNum(max_yyyymmdd); max_date = date_lut.YYYYMMDDToDayNum(max_yyyymmdd);
DayNum_t min_month = date_lut.toFirstDayNumOfMonth(min_date); DayNum min_month = date_lut.toFirstDayNumOfMonth(min_date);
DayNum_t max_month = date_lut.toFirstDayNumOfMonth(max_date); DayNum max_month = date_lut.toFirstDayNumOfMonth(max_date);
if (min_month != max_month) if (min_month != max_month)
throw Exception("Part name " + dir_name + " contains different months", ErrorCodes::BAD_DATA_PART_NAME); throw Exception("Part name " + dir_name + " contains different months", ErrorCodes::BAD_DATA_PART_NAME);
@ -129,7 +129,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(); const auto & date_lut = DateLUT::instance();

View File

@ -52,7 +52,7 @@ struct MergeTreePartInfo
} }
String getPartName() const; 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 UInt64 getBlocksCount() const
{ {
return static_cast<UInt64>(max_block - min_block + 1); return static_cast<UInt64>(max_block - min_block + 1);
@ -62,7 +62,7 @@ struct MergeTreePartInfo
static bool tryParsePartName(const String & dir_name, MergeTreePartInfo * part_info, MergeTreeDataFormatVersion format_version); 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); static bool contains(const String & outer_part_name, const String & inner_part_name, MergeTreeDataFormatVersion format_version);
}; };

View File

@ -52,7 +52,7 @@ String MergeTreePartition::getID(const MergeTreeData & storage) const
result += '-'; result += '-';
if (typeid_cast<const DataTypeDate *>(storage.partition_key_sample.getByPosition(i).type.get())) 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 else
result += applyVisitor(to_string_visitor, value[i]); result += applyVisitor(to_string_visitor, value[i]);

View File

@ -334,7 +334,7 @@ bool MergeTreeWhereOptimizer::isPrimaryKeyAtom(const IAST * const ast) const
if ((primary_key_columns.count(first_arg_name) && isConstant(args[1])) || 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(second_arg_name) && isConstant(args[0])) ||
(primary_key_columns.count(first_arg_name) (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; return true;
} }

View File

@ -14,20 +14,23 @@ namespace DB
static const auto ALTER_ERROR_SLEEP_MS = 10 * 1000; static const auto ALTER_ERROR_SLEEP_MS = 10 * 1000;
ReplicatedMergeTreeAlterThread::ReplicatedMergeTreeAlterThread(StorageReplicatedMergeTree & storage_) ReplicatedMergeTreeAlterThread::ReplicatedMergeTreeAlterThread(StorageReplicatedMergeTree & storage_) :
: storage(storage_), storage(storage_),
log(&Logger::get(storage.database_name + "." + storage.table_name + " (StorageReplicatedMergeTree, AlterThread)")), log(&Logger::get(storage.database_name + "." + storage.table_name + " (StorageReplicatedMergeTree, AlterThread)"))
thread([this] { run(); }) {} {
task_handle = storage_.context.getSchedulePool().addTask("ReplicatedMergeTreeAlterThread", [this]{run();});
task_handle->schedule();
}
ReplicatedMergeTreeAlterThread::~ReplicatedMergeTreeAlterThread()
{
storage.context.getSchedulePool().removeTask(task_handle);
}
void ReplicatedMergeTreeAlterThread::run() void ReplicatedMergeTreeAlterThread::run()
{ {
setThreadName("ReplMTAlter");
bool force_recheck_parts = true; 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), /** We have a description of columns in ZooKeeper, common for all replicas (Example: /clickhouse/tables/02-06/visits/columns),
@ -56,14 +59,14 @@ void ReplicatedMergeTreeAlterThread::run()
auto zookeeper = storage.getZooKeeper(); auto zookeeper = storage.getZooKeeper();
zkutil::Stat stat; zkutil::Stat stat;
const String columns_str = zookeeper->get(storage.zookeeper_path + "/columns", &stat, wakeup_event); const String columns_str = zookeeper->getWatch(storage.zookeeper_path + "/columns", &stat, task_handle->getWatchCallback());
auto columns_in_zk = ColumnsDescription::parse(columns_str); auto columns_in_zk = ColumnsDescription::parse(columns_str);
bool changed_version = (stat.version != storage.columns_version); bool changed_version = (stat.version != storage.columns_version);
{ {
/// If you need to lock table structure, then suspend merges. /// If you need to lock table structure, then suspend merges.
ActionBlocker::LockHolder merge_blocker; ActionLock merge_blocker;
if (changed_version || force_recheck_parts) if (changed_version || force_recheck_parts)
merge_blocker = storage.merger.merges_blocker.cancel(); merge_blocker = storage.merger.merges_blocker.cancel();
@ -77,9 +80,9 @@ void ReplicatedMergeTreeAlterThread::run()
auto temporarily_stop_part_checks = storage.part_check_thread.temporarilyStop(); auto temporarily_stop_part_checks = storage.part_check_thread.temporarilyStop();
/// Temporarily cancel parts sending /// Temporarily cancel parts sending
ActionBlocker::LockHolder data_parts_exchange_blocker; ActionLock data_parts_exchange_blocker;
if (storage.data_parts_exchange_endpoint_holder) 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 /// Temporarily cancel part fetches
auto fetches_blocker = storage.fetcher.blocker.cancel(); auto fetches_blocker = storage.fetcher.blocker.cancel();
@ -185,29 +188,24 @@ void ReplicatedMergeTreeAlterThread::run()
/// It's important that parts and merge_blocker are destroyed before the wait. /// It's important that parts and merge_blocker are destroyed before the wait.
} }
wakeup_event->wait();
} }
catch (const zkutil::KeeperException & e) catch (const zkutil::KeeperException & e)
{ {
tryLogCurrentException(log, __PRETTY_FUNCTION__); tryLogCurrentException(log, __PRETTY_FUNCTION__);
if (e.code == ZooKeeperImpl::ZooKeeper::ZSESSIONEXPIRED) if (e.code == ZooKeeperImpl::ZooKeeper::ZSESSIONEXPIRED)
break; return;
force_recheck_parts = true; force_recheck_parts = true;
wakeup_event->tryWait(ALTER_ERROR_SLEEP_MS); task_handle->scheduleAfter(ALTER_ERROR_SLEEP_MS);
} }
catch (...) catch (...)
{ {
tryLogCurrentException(log, __PRETTY_FUNCTION__); tryLogCurrentException(log, __PRETTY_FUNCTION__);
force_recheck_parts = true; force_recheck_parts = true;
wakeup_event->tryWait(ALTER_ERROR_SLEEP_MS); task_handle->scheduleAfter(ALTER_ERROR_SLEEP_MS);
} }
} }
LOG_DEBUG(log, "Alter thread finished");
}
} }

View File

@ -1,6 +1,7 @@
#pragma once #pragma once
#include <thread> #include <thread>
#include <Common/BackgroundSchedulePool.h>
#include <Common/ZooKeeper/Types.h> #include <Common/ZooKeeper/Types.h>
#include <Core/Types.h> #include <Core/Types.h>
#include <common/logger_useful.h> #include <common/logger_useful.h>
@ -21,25 +22,14 @@ class ReplicatedMergeTreeAlterThread
{ {
public: public:
ReplicatedMergeTreeAlterThread(StorageReplicatedMergeTree & storage_); ReplicatedMergeTreeAlterThread(StorageReplicatedMergeTree & storage_);
~ReplicatedMergeTreeAlterThread();
~ReplicatedMergeTreeAlterThread()
{
need_stop = true;
wakeup_event->set();
if (thread.joinable())
thread.join();
}
private: private:
void run(); void run();
StorageReplicatedMergeTree & storage; StorageReplicatedMergeTree & storage;
Logger * log; Logger * log;
BackgroundSchedulePool::TaskHandle task_handle;
zkutil::EventPtr wakeup_event { std::make_shared<Poco::Event>() };
std::atomic<bool> need_stop { false };
std::thread thread;
}; };
} }

View File

@ -108,7 +108,7 @@ void ReplicatedMergeTreeBlockOutputStream::write(const Block & block)
last_block_is_duplicate = false; last_block_is_duplicate = false;
/// TODO Is it possible to not lock the table structure here? /// 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(); auto zookeeper = storage.getZooKeeper();
assertSessionIsNotExpired(zookeeper); assertSessionIsNotExpired(zookeeper);
@ -161,11 +161,11 @@ void ReplicatedMergeTreeBlockOutputStream::write(const Block & block)
/// Set a special error code if the block is duplicate /// Set a special error code if the block is duplicate
int error = (deduplicate && last_block_is_duplicate) ? ErrorCodes::INSERT_WAS_DEDUPLICATED : 0; 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 (...) catch (...)
{ {
PartLog::addNewPartToTheLog(storage.context, *part, watch.elapsed(), ExecutionStatus::fromCurrentException(__PRETTY_FUNCTION__)); PartLog::addNewPart(storage.context, part, watch.elapsed(), ExecutionStatus::fromCurrentException(__PRETTY_FUNCTION__));
throw; throw;
} }
} }
@ -176,7 +176,7 @@ void ReplicatedMergeTreeBlockOutputStream::writeExistingPart(MergeTreeData::Muta
{ {
last_block_is_duplicate = false; 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(); auto zookeeper = storage.getZooKeeper();
assertSessionIsNotExpired(zookeeper); assertSessionIsNotExpired(zookeeper);
@ -189,11 +189,11 @@ void ReplicatedMergeTreeBlockOutputStream::writeExistingPart(MergeTreeData::Muta
try try
{ {
commitPart(zookeeper, part, ""); commitPart(zookeeper, part, "");
PartLog::addNewPartToTheLog(storage.context, *part, watch.elapsed()); PartLog::addNewPart(storage.context, part, watch.elapsed());
} }
catch (...) catch (...)
{ {
PartLog::addNewPartToTheLog(storage.context, *part, watch.elapsed(), ExecutionStatus::fromCurrentException(__PRETTY_FUNCTION__)); PartLog::addNewPart(storage.context, part, watch.elapsed(), ExecutionStatus::fromCurrentException(__PRETTY_FUNCTION__));
throw; 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. /// 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. /// 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(); bool deduplicate_block = !block_id.empty();
String block_id_path; String block_id_path = deduplicate_block ? storage.zookeeper_path + "/blocks/" + block_id : "";
zkutil::Requests deduplication_check_ops; auto block_number_lock = storage.allocateBlockNumber(part->info.partition_id, zookeeper, block_id_path);
zkutil::Requests * deduplication_check_ops_ptr = nullptr;
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;
}
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; part->is_duplicate = true;
last_block_is_duplicate = true; last_block_is_duplicate = true;
ProfileEvents::increment(ProfileEvents::DuplicatedInsertedBlocks); ProfileEvents::increment(ProfileEvents::DuplicatedInsertedBlocks);
return; return;
} }
throw Exception("Cannot allocate block number in ZooKeeper: " + e.displayText(), ErrorCodes::KEEPER_EXCEPTION); Int64 block_number = block_number_lock->getNumber();
}
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();
/// Set part attributes according to part_number. Prepare an entry for log. /// Set part attributes according to part_number. Prepare an entry for log.
@ -277,33 +250,7 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart(zkutil::ZooKeeperPtr & zoo
/// Information about the part. /// Information about the part.
zkutil::Requests ops; zkutil::Requests ops;
if (deduplicate_block) storage.getCommitPartOps(ops, part, block_id_path);
{
/// 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));
/// Replication log. /// Replication log.
ops.emplace_back(zkutil::makeCreateRequest( ops.emplace_back(zkutil::makeCreateRequest(
@ -312,7 +259,7 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart(zkutil::ZooKeeperPtr & zoo
zkutil::CreateMode::PersistentSequential)); zkutil::CreateMode::PersistentSequential));
/// Deletes the information that the block number is used for writing. /// 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 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. * (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.
@ -362,10 +309,10 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart(zkutil::ZooKeeperPtr & zoo
if (multi_code == ZooKeeperImpl::ZooKeeper::ZOK) if (multi_code == ZooKeeperImpl::ZooKeeper::ZOK)
{ {
transaction.commit(); 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 /// 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)) else if (zkutil::isUserError(multi_code))
{ {
@ -455,5 +402,10 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart(zkutil::ZooKeeperPtr & zoo
} }
} }
void ReplicatedMergeTreeBlockOutputStream::writePrefix()
{
storage.data.throwInsertIfNeeded();
}
} }

View File

@ -26,6 +26,7 @@ public:
bool deduplicate_); bool deduplicate_);
Block getHeader() const override; Block getHeader() const override;
void writePrefix() override;
void write(const Block & block) override; void write(const Block & block) override;
/// For ATTACHing existing data on filesystem. /// For ATTACHing existing data on filesystem.

View File

@ -17,21 +17,22 @@ namespace ErrorCodes
ReplicatedMergeTreeCleanupThread::ReplicatedMergeTreeCleanupThread(StorageReplicatedMergeTree & storage_) ReplicatedMergeTreeCleanupThread::ReplicatedMergeTreeCleanupThread(StorageReplicatedMergeTree & storage_)
: storage(storage_), : storage(storage_),
log(&Logger::get(storage.database_name + "." + storage.table_name + " (StorageReplicatedMergeTree, CleanupThread)")), log(&Logger::get(storage.database_name + "." + storage.table_name + " (StorageReplicatedMergeTree, CleanupThread)"))
thread([this] { run(); })
{ {
task_handle = storage.context.getSchedulePool().addTask("ReplicatedMergeTreeCleanupThread", [this]{ run(); });
task_handle->schedule();
} }
ReplicatedMergeTreeCleanupThread::~ReplicatedMergeTreeCleanupThread()
{
storage.context.getSchedulePool().removeTask(task_handle);
}
void ReplicatedMergeTreeCleanupThread::run() void ReplicatedMergeTreeCleanupThread::run()
{ {
setThreadName("ReplMTCleanup");
const auto CLEANUP_SLEEP_MS = storage.data.settings.cleanup_delay_period * 1000 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); + std::uniform_int_distribution<UInt64>(0, storage.data.settings.cleanup_delay_period_random_add * 1000)(rng);
while (!storage.shutdown_called)
{
try try
{ {
iterate(); iterate();
@ -41,17 +42,15 @@ void ReplicatedMergeTreeCleanupThread::run()
tryLogCurrentException(log, __PRETTY_FUNCTION__); tryLogCurrentException(log, __PRETTY_FUNCTION__);
if (e.code == ZooKeeperImpl::ZooKeeper::ZSESSIONEXPIRED) if (e.code == ZooKeeperImpl::ZooKeeper::ZSESSIONEXPIRED)
break; return;
} }
catch (...) catch (...)
{ {
tryLogCurrentException(log, __PRETTY_FUNCTION__); tryLogCurrentException(log, __PRETTY_FUNCTION__);
} }
storage.cleanup_thread_event.tryWait(CLEANUP_SLEEP_MS); task_handle->scheduleAfter(CLEANUP_SLEEP_MS);
}
LOG_DEBUG(log, "Cleanup thread finished");
} }
@ -243,11 +242,4 @@ void ReplicatedMergeTreeCleanupThread::getBlocksSortedByTime(zkutil::ZooKeeper &
std::sort(timed_blocks.begin(), timed_blocks.end(), NodeWithStat::greaterByTime); std::sort(timed_blocks.begin(), timed_blocks.end(), NodeWithStat::greaterByTime);
} }
ReplicatedMergeTreeCleanupThread::~ReplicatedMergeTreeCleanupThread()
{
if (thread.joinable())
thread.join();
}
} }

View File

@ -4,6 +4,7 @@
#include <Common/ZooKeeper/Types.h> #include <Common/ZooKeeper/Types.h>
#include <Common/ZooKeeper/ZooKeeper.h> #include <Common/ZooKeeper/ZooKeeper.h>
#include <common/logger_useful.h> #include <common/logger_useful.h>
#include <Common/BackgroundSchedulePool.h>
#include <thread> #include <thread>
#include <map> #include <map>
@ -25,10 +26,12 @@ public:
~ReplicatedMergeTreeCleanupThread(); ~ReplicatedMergeTreeCleanupThread();
void schedule() { task_handle->schedule(); }
private: private:
StorageReplicatedMergeTree & storage; StorageReplicatedMergeTree & storage;
Logger * log; Logger * log;
std::thread thread; BackgroundSchedulePool::TaskHandle task_handle;
pcg64 rng; pcg64 rng;
void run(); void run();

View File

@ -46,6 +46,11 @@ void ReplicatedMergeTreeLogEntryData::writeText(WriteBuffer & out) const
<< new_part_name; << new_part_name;
break; break;
case REPLACE_RANGE:
out << typeToString(REPLACE_RANGE) << "\n";
replace_range_entry->writeText(out);
break;
default: default:
throw Exception("Unknown log entry type: " + DB::toString<int>(type), ErrorCodes::LOGICAL_ERROR); throw Exception("Unknown log entry type: " + DB::toString<int>(type), ErrorCodes::LOGICAL_ERROR);
} }
@ -113,16 +118,11 @@ void ReplicatedMergeTreeLogEntryData::readText(ReadBuffer & in)
type = CLEAR_COLUMN; type = CLEAR_COLUMN;
in >> escape >> column_name >> "\nfrom\n" >> new_part_name; 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 = REPLACE_RANGE;
type = ATTACH_PART; replace_range_entry = std::make_shared<ReplaceRangeEntry>();
String source_type; replace_range_entry->readText(in);
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;
} }
in >> "\n"; in >> "\n";
@ -132,6 +132,48 @@ void ReplicatedMergeTreeLogEntryData::readText(ReadBuffer & in)
in >> "quorum: " >> quorum >> "\n"; 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 String ReplicatedMergeTreeLogEntryData::toString() const
{ {
WriteBufferFromOwnString out; WriteBufferFromOwnString out;

View File

@ -33,24 +33,29 @@ struct ReplicatedMergeTreeLogEntryData
GET_PART, /// Get the part from another replica. GET_PART, /// Get the part from another replica.
MERGE_PARTS, /// Merge the parts. MERGE_PARTS, /// Merge the parts.
DROP_RANGE, /// Delete the parts in the specified partition in the specified number range. 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. CLEAR_COLUMN, /// Drop specific column from specified partition.
REPLACE_RANGE, /// Drop certain range of partitions and replace them by new ones
}; };
String typeToString() const static String typeToString(Type type)
{ {
switch (type) switch (type)
{ {
case ReplicatedMergeTreeLogEntryData::GET_PART: return "GET_PART"; case ReplicatedMergeTreeLogEntryData::GET_PART: return "GET_PART";
case ReplicatedMergeTreeLogEntryData::MERGE_PARTS: return "MERGE_PARTS"; case ReplicatedMergeTreeLogEntryData::MERGE_PARTS: return "MERGE_PARTS";
case ReplicatedMergeTreeLogEntryData::DROP_RANGE: return "DROP_RANGE"; case ReplicatedMergeTreeLogEntryData::DROP_RANGE: return "DROP_RANGE";
case ReplicatedMergeTreeLogEntryData::ATTACH_PART: return "ATTACH_PART";
case ReplicatedMergeTreeLogEntryData::CLEAR_COLUMN: return "CLEAR_COLUMN"; case ReplicatedMergeTreeLogEntryData::CLEAR_COLUMN: return "CLEAR_COLUMN";
case ReplicatedMergeTreeLogEntryData::REPLACE_RANGE: return "REPLACE_RANGE";
default: default:
throw Exception("Unknown log entry type: " + DB::toString<int>(type), ErrorCodes::LOGICAL_ERROR); 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 writeText(WriteBuffer & out) const;
void readText(ReadBuffer & in); void readText(ReadBuffer & in);
String toString() const; String toString() const;
@ -60,8 +65,8 @@ struct ReplicatedMergeTreeLogEntryData
Type type = EMPTY; Type type = EMPTY;
String source_replica; /// Empty string means that this entry was added to the queue immediately, and not copied from the log. 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. /// The name of resulting part for GET_PART and MERGE_PARTS
/// For DROP_RANGE, the name of a non-existent part. You need to remove all the parts covered by it. /// Part range for DROP_RANGE and CLEAR_COLUMN
String new_part_name; String new_part_name;
String block_id; /// For parts of level zero, the block identifier for deduplication (node name in /blocks/). 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'. mutable String actual_new_part_name; /// GET_PART could actually fetch a part covering 'new_part_name'.
@ -73,6 +78,41 @@ struct ReplicatedMergeTreeLogEntryData
/// For DROP_RANGE, true means that the parts need not be deleted, but moved to the `detached` directory. /// For DROP_RANGE, true means that the parts need not be deleted, but moved to the `detached` directory.
bool detach = false; 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. /// Access under queue_mutex, see ReplicatedMergeTreeQueue.
bool currently_executing = false; /// Whether the action is executing now. bool currently_executing = false; /// Whether the action is executing now.
/// These several fields are informational only (for viewing by the user using system tables). /// These several fields are informational only (for viewing by the user using system tables).

View File

@ -21,33 +21,33 @@ ReplicatedMergeTreePartCheckThread::ReplicatedMergeTreePartCheckThread(StorageRe
: storage(storage_), : storage(storage_),
log(&Logger::get(storage.database_name + "." + storage.table_name + " (StorageReplicatedMergeTree, PartCheckThread)")) 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() void ReplicatedMergeTreePartCheckThread::start()
{ {
std::lock_guard<std::mutex> lock(start_stop_mutex); std::lock_guard<std::mutex> lock(start_stop_mutex);
if (need_stop)
need_stop = false; need_stop = false;
else task_handle->activate();
thread = std::thread([this] { run(); }); task_handle->schedule();
} }
void ReplicatedMergeTreePartCheckThread::stop() 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); std::lock_guard<std::mutex> lock(start_stop_mutex);
need_stop = true; need_stop = true;
if (thread.joinable()) task_handle->deactivate();
{
wakeup_event.set();
thread.join();
need_stop = false;
} }
}
void ReplicatedMergeTreePartCheckThread::enqueuePart(const String & name, time_t delay_to_check_seconds) void ReplicatedMergeTreePartCheckThread::enqueuePart(const String & name, time_t delay_to_check_seconds)
{ {
@ -58,7 +58,7 @@ void ReplicatedMergeTreePartCheckThread::enqueuePart(const String & name, time_t
parts_queue.emplace_back(name, time(nullptr) + delay_to_check_seconds); parts_queue.emplace_back(name, time(nullptr) + delay_to_check_seconds);
parts_set.insert(name); parts_set.insert(name);
wakeup_event.set(); task_handle->schedule();
} }
@ -273,7 +273,7 @@ void ReplicatedMergeTreePartCheckThread::checkPart(const String & part_name)
storage.removePartAndEnqueueFetch(part_name); storage.removePartAndEnqueueFetch(part_name);
/// Delete part locally. /// 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)) else if (part->modification_time + MAX_AGE_OF_LOCAL_PART_THAT_WASNT_ADDED_TO_ZOOKEEPER < time(nullptr))
@ -284,7 +284,7 @@ void ReplicatedMergeTreePartCheckThread::checkPart(const String & part_name)
ProfileEvents::increment(ProfileEvents::ReplicatedPartChecksFailed); ProfileEvents::increment(ProfileEvents::ReplicatedPartChecksFailed);
LOG_ERROR(log, "Unexpected part " << part_name << " in filesystem. Removing."); LOG_ERROR(log, "Unexpected part " << part_name << " in filesystem. Removing.");
storage.data.renameAndDetachPart(part, "unexpected_"); storage.data.forgetPartAndMoveToDetached(part, "unexpected_");
} }
else else
{ {
@ -309,10 +309,9 @@ void ReplicatedMergeTreePartCheckThread::checkPart(const String & part_name)
void ReplicatedMergeTreePartCheckThread::run() void ReplicatedMergeTreePartCheckThread::run()
{ {
setThreadName("ReplMTPartCheck"); if (need_stop)
return;
while (!need_stop)
{
try try
{ {
time_t current_time = time(nullptr); time_t current_time = time(nullptr);
@ -349,22 +348,12 @@ void ReplicatedMergeTreePartCheckThread::run()
} }
if (selected == parts_queue.end()) if (selected == parts_queue.end())
{ return;
/// 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); checkPart(selected->first);
if (need_stop) if (need_stop)
break; return;
/// Remove the part from check queue. /// Remove the part from check queue.
{ {
@ -380,24 +369,23 @@ void ReplicatedMergeTreePartCheckThread::run()
parts_queue.erase(selected); parts_queue.erase(selected);
} }
} }
task_handle->schedule();
} }
catch (const zkutil::KeeperException & e) catch (const zkutil::KeeperException & e)
{ {
tryLogCurrentException(log, __PRETTY_FUNCTION__); tryLogCurrentException(log, __PRETTY_FUNCTION__);
if (e.code == ZooKeeperImpl::ZooKeeper::ZSESSIONEXPIRED) if (e.code == ZooKeeperImpl::ZooKeeper::ZSESSIONEXPIRED)
break; return;
wakeup_event.tryWait(PART_CHECK_ERROR_SLEEP_MS); task_handle->scheduleAfter(PART_CHECK_ERROR_SLEEP_MS);
} }
catch (...) catch (...)
{ {
tryLogCurrentException(log, __PRETTY_FUNCTION__); tryLogCurrentException(log, __PRETTY_FUNCTION__);
wakeup_event.tryWait(PART_CHECK_ERROR_SLEEP_MS); task_handle->scheduleAfter(PART_CHECK_ERROR_SLEEP_MS);
} }
} }
LOG_DEBUG(log, "Part check thread finished");
}
} }

Some files were not shown because too many files have changed in this diff Show More