Merge master

This commit is contained in:
alesapin 2018-12-03 12:10:06 +03:00
commit bf338b957f
218 changed files with 8136 additions and 12395 deletions

View File

@ -1,3 +1,12 @@
## ClickHouse release 18.14.17, 2018-11-30
### Исправления ошибок:
* Исправлена ситуация, при которой ODBC Bridge продолжал работу после завершения работы сервера ClickHouse. Теперь ODBC Bridge всегда завершает работу вместе с сервером. [#3642](https://github.com/yandex/ClickHouse/pull/3642)
* Исправлена синхронная вставка в `Distributed` таблицу в случае явного указания неполного списка столбцов или списка столбцов в измененном порядке. [#3673](https://github.com/yandex/ClickHouse/pull/3673)
* Исправлен race condition в `BackgroundProcessingPoolTaskInfo` возникающий при попытке выполнения задачи одновременно с инвалидацией её итератора очереди. [#3680](https://github.com/yandex/ClickHouse/pull/3680)
* Исправлен deadlock в `ParallelInputsProcessor` возникающий в случае исключения при создании потока. [#3643](https://github.com/yandex/ClickHouse/pull/3643)
* Исправлена ошибка парсинга `Engine` при создании таблицы с синтаксисом `AS table` в случае, когда `AS table` указывался после `Engine`, что приводило к игнорированию указанного движка. [#3692](https://github.com/yandex/ClickHouse/pull/3692)
## ClickHouse release 18.14.15, 2018-11-21
### Исправления ошибок:

View File

@ -25,6 +25,18 @@ endif ()
# Write compile_commands.json
set(CMAKE_EXPORT_COMPILE_COMMANDS 1)
set(PARALLEL_COMPILE_JOBS "" CACHE STRING "Define the maximum number of concurrent compilation jobs")
if (PARALLEL_COMPILE_JOBS)
set_property(GLOBAL APPEND PROPERTY JOB_POOLS compile_job_pool="${PARALLEL_COMPILE_JOBS}")
set(CMAKE_JOB_POOL_COMPILE compile_job_pool)
endif ()
set(PARALLEL_LINK_JOBS "" CACHE STRING "Define the maximum number of concurrent link jobs")
if (LLVM_PARALLEL_LINK_JOBS)
set_property(GLOBAL APPEND PROPERTY JOB_POOLS link_job_pool=${PARALLEL_LINK_JOBS})
set(CMAKE_JOB_POOL_LINK link_job_pool)
endif ()
include (cmake/find_ccache.cmake)
if (NOT CMAKE_BUILD_TYPE OR CMAKE_BUILD_TYPE STREQUAL "None")
@ -147,13 +159,6 @@ set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${COMPILER_FLAGS} -fn
set (CMAKE_C_FLAGS_RELWITHDEBINFO "${CMAKE_C_FLAGS_RELWITHDEBINFO} -O3 ${CMAKE_C_FLAGS_ADD}")
set (CMAKE_C_FLAGS_DEBUG "${CMAKE_C_FLAGS_DEBUG} -O0 -g3 -ggdb3 -fno-inline ${CMAKE_C_FLAGS_ADD}")
if (MAKE_STATIC_LIBRARIES AND NOT APPLE AND NOT (COMPILER_CLANG AND OS_FREEBSD))
set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -static-libgcc -static-libstdc++")
# Along with executables, we also build example of shared library for "library dictionary source"; and it also should be self-contained.
set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -static-libgcc -static-libstdc++")
endif ()
set(THREADS_PREFER_PTHREAD_FLAG ON)
find_package (Threads)
@ -169,7 +174,8 @@ if (OS_LINUX AND COMPILER_CLANG)
set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -stdlib=libc++") # Ok for clang6, for older can cause 'not used option' warning
set (CMAKE_CXX_FLAGS_DEBUG "${CMAKE_CXX_FLAGS_DEBUG} -D_LIBCPP_DEBUG=0") # More checks in debug build.
if (MAKE_STATIC_LIBRARIES)
link_libraries (-Wl,-Bstatic -stdlib=libc++ c++ c++abi -Wl,-Bdynamic)
execute_process (COMMAND ${CMAKE_CXX_COMPILER} --print-file-name=libclang_rt.builtins-${CMAKE_SYSTEM_PROCESSOR}.a OUTPUT_VARIABLE BUILTINS_LIB_PATH OUTPUT_STRIP_TRAILING_WHITESPACE)
link_libraries (-nodefaultlibs -Wl,-Bstatic -stdlib=libc++ c++ c++abi gcc_eh ${BUILTINS_LIB_PATH} rt -Wl,-Bdynamic dl pthread m c)
else ()
link_libraries (-stdlib=libc++ c++ c++abi)
endif ()
@ -181,6 +187,19 @@ if (OS_LINUX AND COMPILER_CLANG)
endif ()
endif ()
if (COMPILER_GCC)
set (STATIC_STDLIB_FLAGS "-static-libgcc -static-libstdc++")
else ()
set (STATIC_STDLIB_FLAGS "")
endif ()
if (MAKE_STATIC_LIBRARIES AND NOT APPLE AND NOT (COMPILER_CLANG AND OS_FREEBSD))
set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} ${STATIC_STDLIB_FLAGS}")
# Along with executables, we also build example of shared library for "library dictionary source"; and it also should be self-contained.
set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} ${STATIC_STDLIB_FLAGS}")
endif ()
if (USE_STATIC_LIBRARIES AND HAVE_NO_PIE)
set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${FLAG_NO_PIE}")
set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${FLAG_NO_PIE}")

View File

@ -2,10 +2,10 @@
set(VERSION_REVISION 54409 CACHE STRING "")
set(VERSION_MAJOR 18 CACHE STRING "")
set(VERSION_MINOR 14 CACHE STRING "")
set(VERSION_PATCH 9 CACHE STRING "")
set(VERSION_GITHASH 457f8fd495b2812940e69c15ab5b499cd863aae4 CACHE STRING "")
set(VERSION_DESCRIBE v18.14.9-testing CACHE STRING "")
set(VERSION_STRING 18.14.9 CACHE STRING "")
set(VERSION_PATCH 17 CACHE STRING "")
set(VERSION_GITHASH ac2895d769c3dcf070530dec7fcfdcf87bfa852a CACHE STRING "")
set(VERSION_DESCRIBE v18.14.17-testing CACHE STRING "")
set(VERSION_STRING 18.14.17 CACHE STRING "")
# end of autochange
set(VERSION_EXTRA "" CACHE STRING "")

View File

@ -125,12 +125,13 @@ int Server::main(const std::vector<std::string> & /*args*/)
bool has_zookeeper = config().has("zookeeper");
zkutil::ZooKeeperNodeCache main_config_zk_node_cache([&] { return global_context->getZooKeeper(); });
zkutil::EventPtr main_config_zk_changed_event = std::make_shared<Poco::Event>();
if (loaded_config.has_zk_includes)
{
auto old_configuration = loaded_config.configuration;
ConfigProcessor config_processor(config_path);
loaded_config = config_processor.loadConfigWithZooKeeperIncludes(
main_config_zk_node_cache, /* fallback_to_preprocessed = */ true);
main_config_zk_node_cache, main_config_zk_changed_event, /* fallback_to_preprocessed = */ true);
config_processor.savePreprocessedConfig(loaded_config, config().getString("path", DBMS_DEFAULT_PATH));
config().removeConfiguration(old_configuration.get());
config().add(loaded_config.configuration.duplicate(), PRIO_DEFAULT, false);
@ -305,6 +306,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
include_from_path,
config().getString("path", ""),
std::move(main_config_zk_node_cache),
main_config_zk_changed_event,
[&](ConfigurationPtr config)
{
buildLoggers(*config);
@ -327,6 +329,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
include_from_path,
config().getString("path", ""),
zkutil::ZooKeeperNodeCache([&] { return global_context->getZooKeeper(); }),
std::make_shared<Poco::Event>(),
[&](ConfigurationPtr config) { global_context->setUsersConfig(config); },
/* already_loaded = */ false);

View File

@ -231,6 +231,7 @@ void ConfigProcessor::doIncludesRecursive(
XMLDocumentPtr include_from,
Node * node,
zkutil::ZooKeeperNodeCache * zk_node_cache,
const zkutil::EventPtr & zk_changed_event,
std::unordered_set<std::string> & contributing_zk_paths)
{
if (node->nodeType() == Node::TEXT_NODE)
@ -349,12 +350,12 @@ void ConfigProcessor::doIncludesRecursive(
XMLDocumentPtr zk_document;
auto get_zk_node = [&](const std::string & name) -> const Node *
{
std::optional<std::string> contents = zk_node_cache->get(name);
if (!contents)
zkutil::ZooKeeperNodeCache::ZNode znode = zk_node_cache->get(name, zk_changed_event);
if (!znode.exists)
return nullptr;
/// Enclose contents into a fake <from_zk> tag to allow pure text substitutions.
zk_document = dom_parser.parseString("<from_zk>" + *contents + "</from_zk>");
zk_document = dom_parser.parseString("<from_zk>" + znode.contents + "</from_zk>");
return getRootNode(zk_document.get());
};
@ -380,13 +381,13 @@ void ConfigProcessor::doIncludesRecursive(
}
if (included_something)
doIncludesRecursive(config, include_from, node, zk_node_cache, contributing_zk_paths);
doIncludesRecursive(config, include_from, node, zk_node_cache, zk_changed_event, contributing_zk_paths);
else
{
NodeListPtr children = node->childNodes();
Node * child = nullptr;
for (size_t i = 0; (child = children->item(i)); ++i)
doIncludesRecursive(config, include_from, child, zk_node_cache, contributing_zk_paths);
doIncludesRecursive(config, include_from, child, zk_node_cache, zk_changed_event, contributing_zk_paths);
}
}
@ -430,7 +431,8 @@ ConfigProcessor::Files ConfigProcessor::getConfigMergeFiles(const std::string &
XMLDocumentPtr ConfigProcessor::processConfig(
bool * has_zk_includes,
zkutil::ZooKeeperNodeCache * zk_node_cache)
zkutil::ZooKeeperNodeCache * zk_node_cache,
const zkutil::EventPtr & zk_changed_event)
{
XMLDocumentPtr config = dom_parser.parse(path);
@ -460,7 +462,7 @@ XMLDocumentPtr ConfigProcessor::processConfig(
if (node)
{
/// if we include_from env or zk.
doIncludesRecursive(config, nullptr, node, zk_node_cache, contributing_zk_paths);
doIncludesRecursive(config, nullptr, node, zk_node_cache, zk_changed_event, contributing_zk_paths);
include_from_path = node->innerText();
}
else
@ -475,7 +477,7 @@ XMLDocumentPtr ConfigProcessor::processConfig(
include_from = dom_parser.parse(include_from_path);
}
doIncludesRecursive(config, include_from, getRootNode(config.get()), zk_node_cache, contributing_zk_paths);
doIncludesRecursive(config, include_from, getRootNode(config.get()), zk_node_cache, zk_changed_event, contributing_zk_paths);
}
catch (Poco::Exception & e)
{
@ -524,6 +526,7 @@ ConfigProcessor::LoadedConfig ConfigProcessor::loadConfig(bool allow_zk_includes
ConfigProcessor::LoadedConfig ConfigProcessor::loadConfigWithZooKeeperIncludes(
zkutil::ZooKeeperNodeCache & zk_node_cache,
const zkutil::EventPtr & zk_changed_event,
bool fallback_to_preprocessed)
{
XMLDocumentPtr config_xml;
@ -531,7 +534,7 @@ ConfigProcessor::LoadedConfig ConfigProcessor::loadConfigWithZooKeeperIncludes(
bool processed_successfully = false;
try
{
config_xml = processConfig(&has_zk_includes, &zk_node_cache);
config_xml = processConfig(&has_zk_includes, &zk_node_cache, zk_changed_event);
processed_successfully = true;
}
catch (const Poco::Exception & ex)

View File

@ -22,6 +22,7 @@
namespace zkutil
{
class ZooKeeperNodeCache;
using EventPtr = std::shared_ptr<Poco::Event>;
}
namespace DB
@ -61,7 +62,8 @@ public:
/// 5) (Yandex.Metrika-specific) Substitute "<layer/>" with "<layer>layer number from the hostname</layer>".
XMLDocumentPtr processConfig(
bool * has_zk_includes = nullptr,
zkutil::ZooKeeperNodeCache * zk_node_cache = nullptr);
zkutil::ZooKeeperNodeCache * zk_node_cache = nullptr,
const zkutil::EventPtr & zk_changed_event = nullptr);
/// loadConfig* functions apply processConfig and create Poco::Util::XMLConfiguration.
@ -87,6 +89,7 @@ public:
/// processing, load the configuration from the preprocessed file.
LoadedConfig loadConfigWithZooKeeperIncludes(
zkutil::ZooKeeperNodeCache & zk_node_cache,
const zkutil::EventPtr & zk_changed_event,
bool fallback_to_preprocessed = false);
/// Save preprocessed config to specified directory.
@ -134,6 +137,7 @@ private:
XMLDocumentPtr include_from,
Poco::XML::Node * node,
zkutil::ZooKeeperNodeCache * zk_node_cache,
const zkutil::EventPtr & zk_changed_event,
std::unordered_set<std::string> & contributing_zk_paths);
};

View File

@ -17,11 +17,13 @@ ConfigReloader::ConfigReloader(
const std::string & include_from_path_,
const std::string & preprocessed_dir_,
zkutil::ZooKeeperNodeCache && zk_node_cache_,
const zkutil::EventPtr & zk_changed_event_,
Updater && updater_,
bool already_loaded)
: path(path_), include_from_path(include_from_path_)
, preprocessed_dir(preprocessed_dir_)
, zk_node_cache(std::move(zk_node_cache_))
, zk_changed_event(zk_changed_event_)
, updater(std::move(updater_))
{
if (!already_loaded)
@ -40,7 +42,7 @@ ConfigReloader::~ConfigReloader()
try
{
quit = true;
zk_node_cache.getChangedEvent().set();
zk_changed_event->set();
if (thread.joinable())
thread.join();
@ -60,7 +62,7 @@ void ConfigReloader::run()
{
try
{
bool zk_changed = zk_node_cache.getChangedEvent().tryWait(std::chrono::milliseconds(reload_interval).count());
bool zk_changed = zk_changed_event->tryWait(std::chrono::milliseconds(reload_interval).count());
if (quit)
return;
@ -90,7 +92,7 @@ void ConfigReloader::reloadIfNewer(bool force, bool throw_on_error, bool fallbac
loaded_config = config_processor.loadConfig(/* allow_zk_includes = */ true);
if (loaded_config.has_zk_includes)
loaded_config = config_processor.loadConfigWithZooKeeperIncludes(
zk_node_cache, fallback_to_preprocessed);
zk_node_cache, zk_changed_event, fallback_to_preprocessed);
}
catch (...)
{

View File

@ -35,6 +35,7 @@ public:
const std::string & include_from_path,
const std::string & preprocessed_dir,
zkutil::ZooKeeperNodeCache && zk_node_cache,
const zkutil::EventPtr & zk_changed_event,
Updater && updater,
bool already_loaded);
@ -74,6 +75,7 @@ private:
std::string preprocessed_dir;
FilesChangesTracker files;
zkutil::ZooKeeperNodeCache zk_node_cache;
zkutil::EventPtr zk_changed_event = std::make_shared<Poco::Event>();
Updater updater;

View File

@ -401,6 +401,7 @@ namespace ErrorCodes
extern const int CANNOT_LINK = 424;
extern const int SYSTEM_ERROR = 425;
extern const int NULL_POINTER_DEREFERENCE = 426;
extern const int CANNOT_COMPILE_REGEXP = 427;
extern const int KEEPER_EXCEPTION = 999;
extern const int POCO_EXCEPTION = 1000;

View File

@ -1,7 +1,4 @@
#include <iostream>
#include <Poco/Exception.h>
#include <Common/Exception.h>
#include <Common/OptimizedRegularExpression.h>
@ -9,6 +6,15 @@
#define MAX_SUBPATTERNS 5
namespace DB
{
namespace ErrorCodes
{
extern const int CANNOT_COMPILE_REGEXP;
}
}
template <bool thread_safe>
void OptimizedRegularExpressionImpl<thread_safe>::analyze(
const std::string & regexp,
@ -254,11 +260,11 @@ OptimizedRegularExpressionImpl<thread_safe>::OptimizedRegularExpressionImpl(cons
/// Just three following options are supported
if (options & (~(RE_CASELESS | RE_NO_CAPTURE | RE_DOT_NL)))
throw Poco::Exception("OptimizedRegularExpression: Unsupported option.");
throw DB::Exception("OptimizedRegularExpression: Unsupported option.", DB::ErrorCodes::CANNOT_COMPILE_REGEXP);
is_case_insensitive = options & RE_CASELESS;
bool is_no_capture = options & RE_NO_CAPTURE;
bool is_dot_nl = options & RE_DOT_NL;
is_case_insensitive = options & RE_CASELESS;
bool is_no_capture = options & RE_NO_CAPTURE;
bool is_dot_nl = options & RE_DOT_NL;
number_of_subpatterns = 0;
if (!is_trivial)
@ -266,6 +272,9 @@ OptimizedRegularExpressionImpl<thread_safe>::OptimizedRegularExpressionImpl(cons
/// Compile the re2 regular expression.
typename RegexType::Options regexp_options;
/// Never write error messages to stderr. It's ignorant to do it from library code.
regexp_options.set_log_errors(false);
if (is_case_insensitive)
regexp_options.set_case_sensitive(false);
@ -274,13 +283,13 @@ OptimizedRegularExpressionImpl<thread_safe>::OptimizedRegularExpressionImpl(cons
re2 = std::make_unique<RegexType>(regexp_, regexp_options);
if (!re2->ok())
throw Poco::Exception("OptimizedRegularExpression: cannot compile re2: " + regexp_ + ", error: " + re2->error());
throw DB::Exception("OptimizedRegularExpression: cannot compile re2: " + regexp_ + ", error: " + re2->error() + ". Look at https://github.com/google/re2/wiki/Syntax for reference.", DB::ErrorCodes::CANNOT_COMPILE_REGEXP);
if (!is_no_capture)
{
number_of_subpatterns = re2->NumberOfCapturingGroups();
if (number_of_subpatterns > MAX_SUBPATTERNS)
throw Poco::Exception("OptimizedRegularExpression: too many subpatterns in regexp: " + regexp_);
throw DB::Exception("OptimizedRegularExpression: too many subpatterns in regexp: " + regexp_, DB::ErrorCodes::CANNOT_COMPILE_REGEXP);
}
}
}
@ -432,6 +441,5 @@ unsigned OptimizedRegularExpressionImpl<thread_safe>::match(const char * subject
}
}
#undef MIN_LENGTH_FOR_STRSTR
#undef MAX_SUBPATTERNS
template class OptimizedRegularExpressionImpl<true>;
template class OptimizedRegularExpressionImpl<false>;

View File

@ -45,9 +45,9 @@ class OptimizedRegularExpressionImpl
public:
enum Options
{
RE_CASELESS = 0x00000001,
RE_NO_CAPTURE = 0x00000010,
RE_DOT_NL = 0x00000100
RE_CASELESS = 0x00000001,
RE_NO_CAPTURE = 0x00000010,
RE_DOT_NL = 0x00000100
};
using Match = OptimizedRegularExpressionDetails::Match;
@ -106,5 +106,3 @@ private:
};
using OptimizedRegularExpression = OptimizedRegularExpressionImpl<true>;
#include "OptimizedRegularExpression.inl.h"

View File

@ -1,4 +1,4 @@
#include "RWLockFIFO.h"
#include "RWLock.h"
#include <Common/Stopwatch.h>
#include <Common/Exception.h>
#include <Poco/Ext/ThreadNumber.h>
@ -33,15 +33,15 @@ namespace ErrorCodes
}
class RWLockFIFO::LockHandlerImpl
class RWLockImpl::LockHandlerImpl
{
RWLockFIFOPtr parent;
RWLock parent;
GroupsContainer::iterator it_group;
ClientsContainer::iterator it_client;
ThreadToHandler::iterator it_handler;
CurrentMetrics::Increment active_client_increment;
LockHandlerImpl(RWLockFIFOPtr && parent, GroupsContainer::iterator it_group, ClientsContainer::iterator it_client);
LockHandlerImpl(RWLock && parent, GroupsContainer::iterator it_group, ClientsContainer::iterator it_client);
public:
@ -49,11 +49,11 @@ public:
~LockHandlerImpl();
friend class RWLockFIFO;
friend class RWLockImpl;
};
RWLockFIFO::LockHandler RWLockFIFO::getLock(RWLockFIFO::Type type, RWLockFIFO::Client client)
RWLockImpl::LockHandler RWLockImpl::getLock(RWLockImpl::Type type)
{
Stopwatch watch(CLOCK_MONOTONIC_COARSE);
CurrentMetrics::Increment waiting_client_increment((type == Read) ? CurrentMetrics::RWLockWaitingReaders
@ -78,15 +78,16 @@ RWLockFIFO::LockHandler RWLockFIFO::getLock(RWLockFIFO::Type type, RWLockFIFO::C
{
auto handler_ptr = it_handler->second.lock();
if (!handler_ptr)
throw Exception("Lock handler cannot be nullptr. This is a bug", ErrorCodes::LOGICAL_ERROR);
/// Lock may be released in another thread, but not yet deleted inside |~LogHandlerImpl()|
if (type != Read || handler_ptr->it_group->type != Read)
throw Exception("Attempt to acquire exclusive lock recursively", ErrorCodes::LOGICAL_ERROR);
if (handler_ptr)
{
/// XXX: it means we can't upgrade lock from read to write - with proper waiting!
if (type != Read || handler_ptr->it_group->type != Read)
throw Exception("Attempt to acquire exclusive lock recursively", ErrorCodes::LOGICAL_ERROR);
handler_ptr->it_client->info += "; " + client.info;
return handler_ptr;
return handler_ptr;
}
}
if (type == Type::Write || queue.empty() || queue.back().type == Type::Write)
@ -104,7 +105,7 @@ RWLockFIFO::LockHandler RWLockFIFO::getLock(RWLockFIFO::Type type, RWLockFIFO::C
auto & clients = it_group->clients;
try
{
it_client = clients.emplace(clients.end(), std::move(client));
it_client = clients.emplace(clients.end(), type);
}
catch (...)
{
@ -114,10 +115,6 @@ RWLockFIFO::LockHandler RWLockFIFO::getLock(RWLockFIFO::Type type, RWLockFIFO::C
throw;
}
it_client->thread_number = Poco::ThreadNumber::get();
it_client->enqueue_time = time(nullptr);
it_client->type = type;
LockHandler res(new LockHandlerImpl(shared_from_this(), it_group, it_client));
/// Insert myself (weak_ptr to the handler) to threads set to implement recursive lock
@ -128,7 +125,6 @@ RWLockFIFO::LockHandler RWLockFIFO::getLock(RWLockFIFO::Type type, RWLockFIFO::C
/// If we are not the first client in the group, a notification could be already sent
if (it_group == queue.begin())
{
it_client->start_time = it_client->enqueue_time;
finalize_metrics();
return res;
}
@ -136,30 +132,12 @@ RWLockFIFO::LockHandler RWLockFIFO::getLock(RWLockFIFO::Type type, RWLockFIFO::C
/// Wait a notification
it_group->cv.wait(lock, [&] () { return it_group == queue.begin(); });
it_client->start_time = time(nullptr);
finalize_metrics();
return res;
}
RWLockFIFO::Clients RWLockFIFO::getClientsInTheQueue() const
{
std::unique_lock<std::mutex> lock(mutex);
Clients res;
for (const auto & group : queue)
{
for (const auto & client : group.clients)
{
res.emplace_back(client);
}
}
return res;
}
RWLockFIFO::LockHandlerImpl::~LockHandlerImpl()
RWLockImpl::LockHandlerImpl::~LockHandlerImpl()
{
std::unique_lock<std::mutex> lock(parent->mutex);
@ -183,11 +161,11 @@ RWLockFIFO::LockHandlerImpl::~LockHandlerImpl()
}
RWLockFIFO::LockHandlerImpl::LockHandlerImpl(RWLockFIFOPtr && parent, RWLockFIFO::GroupsContainer::iterator it_group,
RWLockFIFO::ClientsContainer::iterator it_client)
RWLockImpl::LockHandlerImpl::LockHandlerImpl(RWLock && parent, RWLockImpl::GroupsContainer::iterator it_group,
RWLockImpl::ClientsContainer::iterator it_client)
: parent{std::move(parent)}, it_group{it_group}, it_client{it_client},
active_client_increment{(it_client->type == RWLockFIFO::Read) ? CurrentMetrics::RWLockActiveReaders
: CurrentMetrics::RWLockActiveWriters}
active_client_increment{(*it_client == RWLockImpl::Read) ? CurrentMetrics::RWLockActiveReaders
: CurrentMetrics::RWLockActiveWriters}
{}
}

68
dbms/src/Common/RWLock.h Normal file
View File

@ -0,0 +1,68 @@
#pragma once
#include <boost/core/noncopyable.hpp>
#include <list>
#include <vector>
#include <mutex>
#include <condition_variable>
#include <thread>
#include <map>
#include <string>
namespace DB
{
class RWLockImpl;
using RWLock = std::shared_ptr<RWLockImpl>;
/// Implements shared lock with FIFO service
/// Can be acquired recursively (several calls from the same thread) in Read mode
class RWLockImpl : public std::enable_shared_from_this<RWLockImpl>
{
public:
enum Type
{
Read,
Write,
};
static RWLock create() { return RWLock(new RWLockImpl); }
/// Just use LockHandler::reset() to release the lock
class LockHandlerImpl;
friend class LockHandlerImpl;
using LockHandler = std::shared_ptr<LockHandlerImpl>;
/// Waits in the queue and returns appropriate lock
LockHandler getLock(Type type);
private:
RWLockImpl() = default;
struct Group;
using GroupsContainer = std::list<Group>;
using ClientsContainer = std::list<Type>;
using ThreadToHandler = std::map<std::thread::id, std::weak_ptr<LockHandlerImpl>>;
/// Group of clients that should be executed concurrently
/// i.e. a group could contain several readers, but only one writer
struct Group
{
// FIXME: there is only redundant |type| information inside |clients|.
const Type type;
ClientsContainer clients;
std::condition_variable cv; /// all clients of the group wait group condvar
explicit Group(Type type) : type{type} {}
};
mutable std::mutex mutex;
GroupsContainer queue;
ThreadToHandler thread_to_handler;
};
}

View File

@ -1,98 +0,0 @@
#pragma once
#include <boost/core/noncopyable.hpp>
#include <list>
#include <vector>
#include <mutex>
#include <condition_variable>
#include <thread>
#include <map>
#include <string>
namespace DB
{
class RWLockFIFO;
using RWLockFIFOPtr = std::shared_ptr<RWLockFIFO>;
/// Implements shared lock with FIFO service
/// You could call it recursively (several calls from the same thread) in Read mode
class RWLockFIFO : public std::enable_shared_from_this<RWLockFIFO>
{
public:
enum Type
{
Read,
Write
};
private:
/// Client is that who wants to acquire the lock.
struct Client
{
explicit Client(const std::string & info = {}) : info{info} {}
bool isStarted() { return start_time != 0; }
/// TODO: delete extra info below if there is no need fot it already.
std::string info;
int thread_number = 0;
std::time_t enqueue_time = 0;
std::time_t start_time = 0;
Type type = Read;
};
public:
static RWLockFIFOPtr create()
{
return RWLockFIFOPtr(new RWLockFIFO);
}
/// Just use LockHandler::reset() to release the lock
class LockHandlerImpl;
friend class LockHandlerImpl;
using LockHandler = std::shared_ptr<LockHandlerImpl>;
/// Waits in the queue and returns appropriate lock
LockHandler getLock(Type type, Client client = Client{});
LockHandler getLock(Type type, const std::string & who)
{
return getLock(type, Client(who));
}
using Clients = std::vector<Client>;
/// Returns list of executing and waiting clients
Clients getClientsInTheQueue() const;
private:
RWLockFIFO() = default;
struct Group;
using GroupsContainer = std::list<Group>;
using ClientsContainer = std::list<Client>;
using ThreadToHandler = std::map<std::thread::id, std::weak_ptr<LockHandlerImpl>>;
/// Group of clients that should be executed concurrently
/// i.e. a group could contain several readers, but only one writer
struct Group
{
const Type type;
ClientsContainer clients;
std::condition_variable cv; /// all clients of the group wait group condvar
explicit Group(Type type) : type{type} {}
};
mutable std::mutex mutex;
GroupsContainer queue;
ThreadToHandler thread_to_handler;
};
}

View File

@ -5,7 +5,6 @@
#include <Common/Exception.h>
#include <Common/ShellCommand.h>
#include <common/logger_useful.h>
#include <IO/WriteBufferFromVector.h>
#include <IO/WriteHelpers.h>
#include <port/unistd.h>
#include <csignal>

View File

@ -9,7 +9,16 @@ ZooKeeperNodeCache::ZooKeeperNodeCache(GetZooKeeper get_zookeeper_)
{
}
std::optional<std::string> ZooKeeperNodeCache::get(const std::string & path)
ZooKeeperNodeCache::ZNode ZooKeeperNodeCache::get(const std::string & path, EventPtr watch_event)
{
Coordination::WatchCallback watch_callback;
if (watch_event)
watch_callback = [watch_event](const Coordination::WatchResponse &) { watch_event->set(); };
return get(path, watch_callback);
}
ZooKeeperNodeCache::ZNode ZooKeeperNodeCache::get(const std::string & path, Coordination::WatchCallback caller_watch_callback)
{
zkutil::ZooKeeperPtr zookeeper;
std::unordered_set<std::string> invalidated_paths;
@ -19,8 +28,7 @@ std::optional<std::string> ZooKeeperNodeCache::get(const std::string & path)
if (!context->zookeeper)
{
/// Possibly, there was a previous session and it has expired. Clear the cache.
nonexistent_nodes.clear();
node_cache.clear();
path_to_cached_znode.clear();
context->zookeeper = get_zookeeper();
}
@ -33,65 +41,62 @@ std::optional<std::string> ZooKeeperNodeCache::get(const std::string & path)
throw DB::Exception("Could not get znode: `" + path + "'. ZooKeeper not configured.", DB::ErrorCodes::NO_ZOOKEEPER);
for (const auto & invalidated_path : invalidated_paths)
{
nonexistent_nodes.erase(invalidated_path);
node_cache.erase(invalidated_path);
}
path_to_cached_znode.erase(invalidated_path);
if (nonexistent_nodes.count(path))
return std::nullopt;
auto cache_it = path_to_cached_znode.find(path);
if (cache_it != path_to_cached_znode.end())
return cache_it->second;
auto watch_callback = [context=context](const Coordination::WatchResponse & response)
std::weak_ptr<Context> weak_context(context);
auto watch_callback = [weak_context, caller_watch_callback](const Coordination::WatchResponse & response)
{
if (!(response.type != Coordination::SESSION || response.state == Coordination::EXPIRED_SESSION))
return;
auto owned_context = weak_context.lock();
if (!owned_context)
return;
bool changed = false;
{
std::lock_guard<std::mutex> lock(context->mutex);
std::lock_guard<std::mutex> lock(owned_context->mutex);
if (response.type != Coordination::SESSION)
changed = context->invalidated_paths.emplace(response.path).second;
changed = owned_context->invalidated_paths.emplace(response.path).second;
else if (response.state == Coordination::EXPIRED_SESSION)
{
context->zookeeper = nullptr;
context->invalidated_paths.clear();
owned_context->zookeeper = nullptr;
owned_context->invalidated_paths.clear();
changed = true;
}
}
if (changed)
context->changed_event.set();
if (changed && caller_watch_callback)
caller_watch_callback(response);
};
std::string contents;
ZNode result;
auto cache_it = node_cache.find(path);
if (cache_it != node_cache.end())
result.exists = zookeeper->tryGetWatch(path, result.contents, &result.stat, watch_callback);
if (result.exists)
{
return cache_it->second;
path_to_cached_znode.emplace(path, result);
return result;
}
if (zookeeper->tryGetWatch(path, contents, /* stat = */nullptr, watch_callback))
/// Node doesn't exist. We must set a watch on node creation (because it wasn't set by tryGetWatch).
result.exists = zookeeper->existsWatch(path, &result.stat, watch_callback);
if (!result.exists)
{
node_cache.emplace(path, contents);
return contents;
path_to_cached_znode.emplace(path, result);
return result;
}
/// Node doesn't exist. Create a watch on node creation.
nonexistent_nodes.insert(path);
if (!zookeeper->existsWatch(path, /* stat = */nullptr, watch_callback))
return std::nullopt;
/// Node was created between the two previous calls, try again. Watch is already set.
if (zookeeper->tryGet(path, contents))
{
nonexistent_nodes.erase(path);
node_cache.emplace(path, contents);
return contents;
}
return std::nullopt;
result.exists = zookeeper->tryGet(path, result.contents, &result.stat);
path_to_cached_znode.emplace(path, result);
return result;
}
}

View File

@ -22,8 +22,13 @@ namespace zkutil
/// This class allows querying the contents of ZooKeeper nodes and caching the results.
/// Watches are set for cached nodes and for nodes that were nonexistent at the time of query.
/// After a watch fires, a notification is generated for the change event.
/// After a watch fires, the callback or event that was passed by the user is notified.
///
/// NOTE: methods of this class are not thread-safe.
///
/// Intended use case: if you need one thread to watch changes in several nodes.
/// If instead you use simple a watch event for this, watches will accumulate for nodes that do not change
/// or change rarely.
class ZooKeeperNodeCache
{
public:
@ -32,17 +37,21 @@ public:
ZooKeeperNodeCache(const ZooKeeperNodeCache &) = delete;
ZooKeeperNodeCache(ZooKeeperNodeCache &&) = default;
std::optional<std::string> get(const std::string & path);
struct ZNode
{
bool exists = false;
std::string contents;
Coordination::Stat stat;
};
Poco::Event & getChangedEvent() { return context->changed_event; }
ZNode get(const std::string & path, EventPtr watch_event);
ZNode get(const std::string & path, Coordination::WatchCallback watch_callback);
private:
GetZooKeeper get_zookeeper;
struct Context
{
Poco::Event changed_event;
std::mutex mutex;
zkutil::ZooKeeperPtr zookeeper;
std::unordered_set<std::string> invalidated_paths;
@ -50,8 +59,7 @@ private:
std::shared_ptr<Context> context;
std::unordered_set<std::string> nonexistent_nodes;
std::unordered_map<std::string, std::string> node_cache;
std::unordered_map<std::string, ZNode> path_to_cached_znode;
};
}

View File

@ -4,7 +4,7 @@
#endif
#include <gtest/gtest.h>
#include <Common/RWLockFIFO.h>
#include <Common/RWLock.h>
#include <Common/Stopwatch.h>
#include <common/Types.h>
#include <common/ThreadPool.h>
@ -18,7 +18,7 @@
using namespace DB;
TEST(Common, RWLockFIFO_1)
TEST(Common, RWLock_1)
{
constexpr int cycles = 1000;
const std::vector<size_t> pool_sizes{1, 2, 4, 8};
@ -26,7 +26,7 @@ TEST(Common, RWLockFIFO_1)
static std::atomic<int> readers{0};
static std::atomic<int> writers{0};
static auto fifo_lock = RWLockFIFO::create();
static auto fifo_lock = RWLockImpl::create();
static thread_local std::random_device rd;
static thread_local pcg64 gen(rd());
@ -35,12 +35,12 @@ TEST(Common, RWLockFIFO_1)
{
for (int i = 0; i < cycles; ++i)
{
auto type = (std::uniform_int_distribution<>(0, 9)(gen) >= round) ? RWLockFIFO::Read : RWLockFIFO::Write;
auto type = (std::uniform_int_distribution<>(0, 9)(gen) >= round) ? RWLockImpl::Read : RWLockImpl::Write;
auto sleep_for = std::chrono::duration<int, std::micro>(std::uniform_int_distribution<>(1, 100)(gen));
auto lock = fifo_lock->getLock(type, "RW");
auto lock = fifo_lock->getLock(type);
if (type == RWLockFIFO::Write)
if (type == RWLockImpl::Write)
{
++writers;
@ -85,11 +85,11 @@ TEST(Common, RWLockFIFO_1)
}
}
TEST(Common, RWLockFIFO_Recursive)
TEST(Common, RWLock_Recursive)
{
constexpr auto cycles = 10000;
static auto fifo_lock = RWLockFIFO::create();
static auto fifo_lock = RWLockImpl::create();
static thread_local std::random_device rd;
static thread_local pcg64 gen(rd());
@ -98,7 +98,7 @@ TEST(Common, RWLockFIFO_Recursive)
{
for (int i = 0; i < 2 * cycles; ++i)
{
auto lock = fifo_lock->getLock(RWLockFIFO::Write);
auto lock = fifo_lock->getLock(RWLockImpl::Write);
auto sleep_for = std::chrono::duration<int, std::micro>(std::uniform_int_distribution<>(1, 100)(gen));
std::this_thread::sleep_for(sleep_for);
@ -109,17 +109,17 @@ TEST(Common, RWLockFIFO_Recursive)
{
for (int i = 0; i < cycles; ++i)
{
auto lock1 = fifo_lock->getLock(RWLockFIFO::Read);
auto lock1 = fifo_lock->getLock(RWLockImpl::Read);
auto sleep_for = std::chrono::duration<int, std::micro>(std::uniform_int_distribution<>(1, 100)(gen));
std::this_thread::sleep_for(sleep_for);
auto lock2 = fifo_lock->getLock(RWLockFIFO::Read);
auto lock2 = fifo_lock->getLock(RWLockImpl::Read);
EXPECT_ANY_THROW({fifo_lock->getLock(RWLockFIFO::Write);});
EXPECT_ANY_THROW({fifo_lock->getLock(RWLockImpl::Write);});
}
fifo_lock->getLock(RWLockFIFO::Write);
fifo_lock->getLock(RWLockImpl::Write);
});
t1.join();
@ -127,12 +127,12 @@ TEST(Common, RWLockFIFO_Recursive)
}
TEST(Common, RWLockFIFO_PerfTest_Readers)
TEST(Common, RWLock_PerfTest_Readers)
{
constexpr int cycles = 100000; // 100k
const std::vector<size_t> pool_sizes{1, 2, 4, 8};
static auto fifo_lock = RWLockFIFO::create();
static auto fifo_lock = RWLockImpl::create();
for (auto pool_size : pool_sizes)
{
@ -142,7 +142,7 @@ TEST(Common, RWLockFIFO_PerfTest_Readers)
{
for (auto i = 0; i < cycles; ++i)
{
auto lock = fifo_lock->getLock(RWLockFIFO::Read);
auto lock = fifo_lock->getLock(RWLockImpl::Read);
}
};

View File

@ -84,6 +84,19 @@ size_t NamesAndTypesList::sizeOfDifference(const NamesAndTypesList & rhs) const
return (std::unique(vector.begin(), vector.end()) - vector.begin()) * 2 - size() - rhs.size();
}
void NamesAndTypesList::getDifference(const NamesAndTypesList & rhs, NamesAndTypesList & deleted, NamesAndTypesList & added) const
{
NamesAndTypes lhs_vector(begin(), end());
std::sort(lhs_vector.begin(), lhs_vector.end());
NamesAndTypes rhs_vector(rhs.begin(), rhs.end());
std::sort(rhs_vector.begin(), rhs_vector.end());
std::set_difference(lhs_vector.begin(), lhs_vector.end(), rhs_vector.begin(), rhs_vector.end(),
std::back_inserter(deleted));
std::set_difference(rhs_vector.begin(), rhs_vector.end(), lhs_vector.begin(), lhs_vector.end(),
std::back_inserter(added));
}
Names NamesAndTypesList::getNames() const
{
Names res;

View File

@ -58,6 +58,9 @@ public:
/// (in other words, the added and deleted columns are counted once, the columns that changed the type - twice).
size_t sizeOfDifference(const NamesAndTypesList & rhs) const;
/// If an element changes type, it is present both in deleted (with the old type) and in added (with the new type).
void getDifference(const NamesAndTypesList & rhs, NamesAndTypesList & deleted, NamesAndTypesList & added) const;
Names getNames() const;
DataTypes getTypes() const;

View File

@ -27,7 +27,7 @@ Block ExpressionBlockInputStream::getTotals()
Block ExpressionBlockInputStream::getHeader() const
{
Block res = children.back()->getHeader();
expression->execute(res);
expression->execute(res, true);
return res;
}

View File

@ -20,7 +20,7 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream(
* Although now any insertion into the table is done via PushingToViewsBlockOutputStream,
* but it's clear that here is not the best place for this functionality.
*/
addTableLock(storage->lockStructure(true, __PRETTY_FUNCTION__));
addTableLock(storage->lockStructure(true));
/// 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
@ -45,7 +45,7 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream(
auto & materialized_view = dynamic_cast<const StorageMaterializedView &>(*dependent_table);
if (StoragePtr inner_table = materialized_view.tryGetTargetTable())
addTableLock(inner_table->lockStructure(true, __PRETTY_FUNCTION__));
addTableLock(inner_table->lockStructure(true));
auto query = materialized_view.getInnerQuery();
BlockOutputStreamPtr out = std::make_shared<PushingToViewsBlockOutputStream>(

View File

@ -246,6 +246,11 @@ void registerDictionarySourceXDBC(DictionarySourceFactory & factory)
BridgeHelperPtr bridge = std::make_shared<XDBCBridgeHelper<ODBCBridgeMixin>>(context, context.getSettings().http_receive_timeout, config.getString(config_prefix + ".odbc.connection_string"));
return std::make_unique<XDBCDictionarySource>(dict_struct, config, config_prefix + ".odbc", sample_block, context, bridge);
#else
(void)dict_struct;
(void)config;
(void)config_prefix;
(void)sample_block;
(void)context;
throw Exception {"Dictionary source of type `odbc` is disabled because poco library was built without ODBC support.",
ErrorCodes::SUPPORT_IS_DISABLED};
#endif

View File

@ -42,7 +42,7 @@ struct EmptyImpl
size_t size = data.size() / n;
for (size_t i = 0; i < size; ++i)
res[i] = negative ^ (0 == memcmp(&data[i * size], empty_chars.data(), n));
res[i] = negative ^ (0 == memcmp(&data[i * n], empty_chars.data(), n));
}
static void array(const ColumnString::Offsets & offsets, PaddedPODArray<UInt8> & res)

View File

@ -0,0 +1,69 @@
#pragma once
#include <Common/config.h>
#include <Functions/IFunction.h>
#include <DataTypes/Native.h>
namespace DB
{
template <bool null_is_false>
class FunctionIfBase : public IFunction
{
#if USE_EMBEDDED_COMPILER
public:
bool isCompilableImpl(const DataTypes & types) const override
{
for (const auto & type : types)
if (!isCompilableType(removeNullable(type)))
return false;
return true;
}
llvm::Value * compileImpl(llvm::IRBuilderBase & builder, const DataTypes & types, ValuePlaceholders values) const override
{
auto & b = static_cast<llvm::IRBuilder<> &>(builder);
auto type = getReturnTypeImpl(types);
llvm::Value * null = nullptr;
if (!null_is_false && type->isNullable())
null = b.CreateInsertValue(llvm::Constant::getNullValue(toNativeType(b, type)), b.getTrue(), {1});
auto * head = b.GetInsertBlock();
auto * join = llvm::BasicBlock::Create(head->getContext(), "", head->getParent());
std::vector<std::pair<llvm::BasicBlock *, llvm::Value *>> returns;
for (size_t i = 0; i + 1 < types.size(); i += 2)
{
auto * then = llvm::BasicBlock::Create(head->getContext(), "", head->getParent());
auto * next = llvm::BasicBlock::Create(head->getContext(), "", head->getParent());
auto * cond = values[i]();
if (!null_is_false && types[i]->isNullable())
{
auto * nonnull = llvm::BasicBlock::Create(head->getContext(), "", head->getParent());
returns.emplace_back(b.GetInsertBlock(), null);
b.CreateCondBr(b.CreateExtractValue(cond, {1}), join, nonnull);
b.SetInsertPoint(nonnull);
b.CreateCondBr(nativeBoolCast(b, removeNullable(types[i]), b.CreateExtractValue(cond, {0})), then, next);
}
else
{
b.CreateCondBr(nativeBoolCast(b, types[i], cond), then, next);
}
b.SetInsertPoint(then);
auto * value = nativeCast(b, types[i + 1], values[i + 1](), type);
returns.emplace_back(b.GetInsertBlock(), value);
b.CreateBr(join);
b.SetInsertPoint(next);
}
auto * value = nativeCast(b, types.back(), values.back()(), type);
returns.emplace_back(b.GetInsertBlock(), value);
b.CreateBr(join);
b.SetInsertPoint(join);
auto * phi = b.CreatePHI(toNativeType(b, type), returns.size());
for (const auto & r : returns)
phi->addIncoming(r.second, r.first);
return phi;
}
#endif
};
}

View File

@ -1,6 +1,5 @@
#pragma once
#include <common/preciseExp10.h>
#include <Core/callOnTypeIndex.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypesDecimal.h>
@ -19,16 +18,15 @@
#if USE_VECTORCLASS
#ifdef __clang__
#pragma clang diagnostic push
#pragma clang diagnostic ignored "-Wshift-negative-value"
#pragma clang diagnostic push
#pragma clang diagnostic ignored "-Wshift-negative-value"
#endif
#include <vectorf128.h> // Y_IGNORE
#include <vectormath_exp.h> // Y_IGNORE
#include <vectormath_trig.h> // Y_IGNORE
#ifdef __clang__
#pragma clang diagnostic pop
#pragma clang diagnostic pop
#endif
#endif
@ -41,169 +39,6 @@ namespace ErrorCodes
extern const int ILLEGAL_COLUMN;
}
template <typename Impl>
class FunctionMathNullaryConstFloat64 : public IFunction
{
public:
static constexpr auto name = Impl::name;
static FunctionPtr create(const Context &) { return std::make_shared<FunctionMathNullaryConstFloat64>(); }
private:
String getName() const override { return name; }
size_t getNumberOfArguments() const override { return 0; }
DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override
{
return std::make_shared<DataTypeFloat64>();
}
void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override
{
block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst(input_rows_count, Impl::value);
}
};
template <typename Impl>
class FunctionMathUnaryFloat64 : public IFunction
{
public:
static constexpr auto name = Impl::name;
static FunctionPtr create(const Context &) { return std::make_shared<FunctionMathUnaryFloat64>(); }
static_assert(Impl::rows_per_iteration > 0, "Impl must process at least one row per iteration");
private:
String getName() const override { return name; }
size_t getNumberOfArguments() const override { return 1; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
const auto & arg = arguments.front();
if (!isNumber(arg) && !isDecimal(arg))
throw Exception{"Illegal type " + arg->getName() + " of argument of function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
return std::make_shared<DataTypeFloat64>();
}
template <typename T>
static void executeInIterations(const T * src_data, Float64 * dst_data, size_t size)
{
const size_t rows_remaining = size % Impl::rows_per_iteration;
const size_t rows_size = size - rows_remaining;
for (size_t i = 0; i < rows_size; i += Impl::rows_per_iteration)
Impl::execute(&src_data[i], &dst_data[i]);
if (rows_remaining != 0)
{
T src_remaining[Impl::rows_per_iteration];
memcpy(src_remaining, &src_data[rows_size], rows_remaining * sizeof(T));
memset(src_remaining + rows_remaining, 0, (Impl::rows_per_iteration - rows_remaining) * sizeof(T));
Float64 dst_remaining[Impl::rows_per_iteration];
Impl::execute(src_remaining, dst_remaining);
memcpy(&dst_data[rows_size], dst_remaining, rows_remaining * sizeof(Float64));
}
}
template <typename T>
static bool execute(Block & block, const ColumnVector<T> * col, const size_t result)
{
const auto & src_data = col->getData();
const size_t size = src_data.size();
auto dst = ColumnVector<Float64>::create();
auto & dst_data = dst->getData();
dst_data.resize(size);
executeInIterations(src_data.data(), dst_data.data(), size);
block.getByPosition(result).column = std::move(dst);
return true;
}
template <typename T>
static bool execute(Block & block, const ColumnDecimal<T> * col, const size_t result)
{
const auto & src_data = col->getData();
const size_t size = src_data.size();
UInt32 scale = src_data.getScale();
auto dst = ColumnVector<Float64>::create();
auto & dst_data = dst->getData();
dst_data.resize(size);
for (size_t i = 0; i < size; ++i)
dst_data[i] = convertFromDecimal<DataTypeDecimal<T>, DataTypeNumber<Float64>>(src_data[i], scale);
executeInIterations(dst_data.data(), dst_data.data(), size);
block.getByPosition(result).column = std::move(dst);
return true;
}
bool useDefaultImplementationForConstants() const override { return true; }
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
const ColumnWithTypeAndName & col = block.getByPosition(arguments[0]);
auto call = [&](const auto & types) -> bool
{
using Types = std::decay_t<decltype(types)>;
using Type = typename Types::RightType;
using ColVecType = std::conditional_t<IsDecimalNumber<Type>, ColumnDecimal<Type>, ColumnVector<Type>>;
const auto col_vec = checkAndGetColumn<ColVecType>(col.column.get());
return execute<Type>(block, col_vec, result);
};
if (!callOnBasicType<void, true, true, true, false>(col.type->getTypeId(), call))
throw Exception{"Illegal column " + col.column->getName() + " of argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN};
}
};
template <typename Name, Float64(Function)(Float64)>
struct UnaryFunctionPlain
{
static constexpr auto name = Name::name;
static constexpr auto rows_per_iteration = 1;
template <typename T>
static void execute(const T * src, Float64 * dst)
{
dst[0] = static_cast<Float64>(Function(static_cast<Float64>(src[0])));
}
};
#if USE_VECTORCLASS
template <typename Name, Vec2d(Function)(const Vec2d &)>
struct UnaryFunctionVectorized
{
static constexpr auto name = Name::name;
static constexpr auto rows_per_iteration = 2;
template <typename T>
static void execute(const T * src, Float64 * dst)
{
const auto result = Function(Vec2d(src[0], src[1]));
result.store(dst);
}
};
#else
#define UnaryFunctionVectorized UnaryFunctionPlain
#endif
template <typename Impl>
class FunctionMathBinaryFloat64 : public IFunction
@ -427,66 +262,4 @@ struct BinaryFunctionVectorized
#endif
struct EImpl
{
static constexpr auto name = "e";
static const double value; /// See .cpp
};
struct PiImpl
{
static constexpr auto name = "pi";
static const double value;
};
struct ExpName { static constexpr auto name = "exp"; };
struct LogName { static constexpr auto name = "log"; };
struct Exp2Name { static constexpr auto name = "exp2"; };
struct Log2Name { static constexpr auto name = "log2"; };
struct Exp10Name { static constexpr auto name = "exp10"; };
struct Log10Name { static constexpr auto name = "log10"; };
struct SqrtName { static constexpr auto name = "sqrt"; };
struct CbrtName { static constexpr auto name = "cbrt"; };
struct SinName { static constexpr auto name = "sin"; };
struct CosName { static constexpr auto name = "cos"; };
struct TanName { static constexpr auto name = "tan"; };
struct AsinName { static constexpr auto name = "asin"; };
struct AcosName { static constexpr auto name = "acos"; };
struct AtanName { static constexpr auto name = "atan"; };
struct ErfName { static constexpr auto name = "erf"; };
struct ErfcName { static constexpr auto name = "erfc"; };
struct LGammaName { static constexpr auto name = "lgamma"; };
struct TGammaName { static constexpr auto name = "tgamma"; };
struct PowName { static constexpr auto name = "pow"; };
using FunctionE = FunctionMathNullaryConstFloat64<EImpl>;
using FunctionPi = FunctionMathNullaryConstFloat64<PiImpl>;
using FunctionExp = FunctionMathUnaryFloat64<UnaryFunctionVectorized<ExpName, exp>>;
using FunctionLog = FunctionMathUnaryFloat64<UnaryFunctionVectorized<LogName, log>>;
using FunctionExp2 = FunctionMathUnaryFloat64<UnaryFunctionVectorized<Exp2Name, exp2>>;
using FunctionLog2 = FunctionMathUnaryFloat64<UnaryFunctionVectorized<Log2Name, log2>>;
using FunctionExp10 = FunctionMathUnaryFloat64<UnaryFunctionVectorized<Exp10Name,
#if USE_VECTORCLASS
exp10
#else
preciseExp10
#endif
>>;
using FunctionLog10 = FunctionMathUnaryFloat64<UnaryFunctionVectorized<Log10Name, log10>>;
using FunctionSqrt = FunctionMathUnaryFloat64<UnaryFunctionVectorized<SqrtName, sqrt>>;
using FunctionCbrt = FunctionMathUnaryFloat64<UnaryFunctionVectorized<CbrtName, cbrt>>;
using FunctionSin = FunctionMathUnaryFloat64<UnaryFunctionVectorized<SinName, sin>>;
using FunctionCos = FunctionMathUnaryFloat64<UnaryFunctionVectorized<CosName, cos>>;
using FunctionTan = FunctionMathUnaryFloat64<UnaryFunctionVectorized<TanName, tan>>;
using FunctionAsin = FunctionMathUnaryFloat64<UnaryFunctionVectorized<AsinName, asin>>;
using FunctionAcos = FunctionMathUnaryFloat64<UnaryFunctionVectorized<AcosName, acos>>;
using FunctionAtan = FunctionMathUnaryFloat64<UnaryFunctionVectorized<AtanName, atan>>;
using FunctionErf = FunctionMathUnaryFloat64<UnaryFunctionPlain<ErfName, std::erf>>;
using FunctionErfc = FunctionMathUnaryFloat64<UnaryFunctionPlain<ErfcName, std::erfc>>;
using FunctionLGamma = FunctionMathUnaryFloat64<UnaryFunctionPlain<LGammaName, std::lgamma>>;
using FunctionTGamma = FunctionMathUnaryFloat64<UnaryFunctionPlain<TGammaName, std::tgamma>>;
using FunctionPow = FunctionMathBinaryFloat64<BinaryFunctionVectorized<PowName, pow>>;
}

View File

@ -0,0 +1,34 @@
#pragma once
#include <DataTypes/DataTypesNumber.h>
#include <Columns/ColumnsNumber.h>
#include <Functions/IFunction.h>
namespace DB
{
template <typename Impl>
class FunctionMathConstFloat64 : public IFunction
{
public:
static constexpr auto name = Impl::name;
static FunctionPtr create(const Context &) { return std::make_shared<FunctionMathConstFloat64>(); }
private:
String getName() const override { return name; }
size_t getNumberOfArguments() const override { return 0; }
DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override
{
return std::make_shared<DataTypeFloat64>();
}
void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override
{
block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst(input_rows_count, Impl::value);
}
};
}

View File

@ -0,0 +1,182 @@
#pragma once
#include <Core/callOnTypeIndex.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypesDecimal.h>
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnDecimal.h>
#include <Functions/IFunction.h>
#include <Functions/FunctionHelpers.h>
#include <Common/config.h>
/** More efficient implementations of mathematical functions are possible when using a separate library.
* Disabled due to licence compatibility limitations.
* To enable: download http://www.agner.org/optimize/vectorclass.zip and unpack to contrib/vectorclass
* Then rebuild with -DENABLE_VECTORCLASS=1
*/
#if USE_VECTORCLASS
#ifdef __clang__
#pragma clang diagnostic push
#pragma clang diagnostic ignored "-Wshift-negative-value"
#endif
#include <vectorf128.h> // Y_IGNORE
#include <vectormath_exp.h> // Y_IGNORE
#include <vectormath_trig.h> // Y_IGNORE
#ifdef __clang__
#pragma clang diagnostic pop
#endif
#endif
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_COLUMN;
}
template <typename Impl>
class FunctionMathUnaryFloat64 : public IFunction
{
public:
static constexpr auto name = Impl::name;
static FunctionPtr create(const Context &) { return std::make_shared<FunctionMathUnaryFloat64>(); }
static_assert(Impl::rows_per_iteration > 0, "Impl must process at least one row per iteration");
private:
String getName() const override { return name; }
size_t getNumberOfArguments() const override { return 1; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
const auto & arg = arguments.front();
if (!isNumber(arg) && !isDecimal(arg))
throw Exception{"Illegal type " + arg->getName() + " of argument of function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
return std::make_shared<DataTypeFloat64>();
}
template <typename T>
static void executeInIterations(const T * src_data, Float64 * dst_data, size_t size)
{
const size_t rows_remaining = size % Impl::rows_per_iteration;
const size_t rows_size = size - rows_remaining;
for (size_t i = 0; i < rows_size; i += Impl::rows_per_iteration)
Impl::execute(&src_data[i], &dst_data[i]);
if (rows_remaining != 0)
{
T src_remaining[Impl::rows_per_iteration];
memcpy(src_remaining, &src_data[rows_size], rows_remaining * sizeof(T));
memset(src_remaining + rows_remaining, 0, (Impl::rows_per_iteration - rows_remaining) * sizeof(T));
Float64 dst_remaining[Impl::rows_per_iteration];
Impl::execute(src_remaining, dst_remaining);
memcpy(&dst_data[rows_size], dst_remaining, rows_remaining * sizeof(Float64));
}
}
template <typename T>
static bool execute(Block & block, const ColumnVector<T> * col, const size_t result)
{
const auto & src_data = col->getData();
const size_t size = src_data.size();
auto dst = ColumnVector<Float64>::create();
auto & dst_data = dst->getData();
dst_data.resize(size);
executeInIterations(src_data.data(), dst_data.data(), size);
block.getByPosition(result).column = std::move(dst);
return true;
}
template <typename T>
static bool execute(Block & block, const ColumnDecimal<T> * col, const size_t result)
{
const auto & src_data = col->getData();
const size_t size = src_data.size();
UInt32 scale = src_data.getScale();
auto dst = ColumnVector<Float64>::create();
auto & dst_data = dst->getData();
dst_data.resize(size);
for (size_t i = 0; i < size; ++i)
dst_data[i] = convertFromDecimal<DataTypeDecimal<T>, DataTypeNumber<Float64>>(src_data[i], scale);
executeInIterations(dst_data.data(), dst_data.data(), size);
block.getByPosition(result).column = std::move(dst);
return true;
}
bool useDefaultImplementationForConstants() const override { return true; }
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
const ColumnWithTypeAndName & col = block.getByPosition(arguments[0]);
auto call = [&](const auto & types) -> bool
{
using Types = std::decay_t<decltype(types)>;
using Type = typename Types::RightType;
using ColVecType = std::conditional_t<IsDecimalNumber<Type>, ColumnDecimal<Type>, ColumnVector<Type>>;
const auto col_vec = checkAndGetColumn<ColVecType>(col.column.get());
return execute<Type>(block, col_vec, result);
};
if (!callOnBasicType<void, true, true, true, false>(col.type->getTypeId(), call))
throw Exception{"Illegal column " + col.column->getName() + " of argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN};
}
};
template <typename Name, Float64(Function)(Float64)>
struct UnaryFunctionPlain
{
static constexpr auto name = Name::name;
static constexpr auto rows_per_iteration = 1;
template <typename T>
static void execute(const T * src, Float64 * dst)
{
dst[0] = static_cast<Float64>(Function(static_cast<Float64>(src[0])));
}
};
#if USE_VECTORCLASS
template <typename Name, Vec2d(Function)(const Vec2d &)>
struct UnaryFunctionVectorized
{
static constexpr auto name = Name::name;
static constexpr auto rows_per_iteration = 2;
template <typename T>
static void execute(const T * src, Float64 * dst)
{
const auto result = Function(Vec2d(src[0], src[1]));
result.store(dst);
}
};
#else
#define UnaryFunctionVectorized UnaryFunctionPlain
#endif
}

View File

@ -23,7 +23,6 @@ void registerFunctionsCoding(FunctionFactory & factory)
factory.registerFunction<FunctionMACStringTo<ParseOUIImpl>>();
factory.registerFunction<FunctionUUIDNumToString>();
factory.registerFunction<FunctionUUIDStringToNum>();
factory.registerFunction<FunctionGenerateUUIDv4>();
factory.registerFunction<FunctionHex>();
factory.registerFunction<FunctionUnhex>();
factory.registerFunction<FunctionBitmaskToArray>();

View File

@ -17,7 +17,6 @@
#include <Columns/ColumnArray.h>
#include <Columns/ColumnConst.h>
#include <Functions/IFunction.h>
#include <Functions/FunctionsRandom.h>
#include <Functions/FunctionHelpers.h>
#include <arpa/inet.h>
@ -1093,47 +1092,6 @@ public:
}
};
class FunctionGenerateUUIDv4 : public IFunction
{
public:
static constexpr auto name = "generateUUIDv4";
static FunctionPtr create(const Context &) { return std::make_shared<FunctionGenerateUUIDv4>(); }
String getName() const override
{
return name;
}
size_t getNumberOfArguments() const override { return 0; }
DataTypePtr getReturnTypeImpl(const DataTypes &) const override
{
return std::make_shared<DataTypeUUID>();
}
bool isDeterministic() const override { return false; }
void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override
{
auto col_res = ColumnVector<UInt128>::create();
typename ColumnVector<UInt128>::Container & vec_to = col_res->getData();
size_t size = input_rows_count;
vec_to.resize(size);
Rand64Impl::execute(reinterpret_cast<UInt64 *>(vec_to.data()), vec_to.size() * 2);
for (UInt128 & uuid: vec_to)
{
/** https://tools.ietf.org/html/rfc4122#section-4.4
*/
uuid.low = (uuid.low & 0xffffffffffff0fffull) | 0x0000000000004000ull;
uuid.high = (uuid.high & 0x3fffffffffffffffull) | 0x8000000000000000ull;
}
block.getByPosition(result).column = std::move(col_res);
}
};
class FunctionHex : public IFunction
{

View File

@ -1,337 +0,0 @@
#include <Functions/FunctionsConditional.h>
#include <Functions/FunctionsTransform.h>
#include <Functions/FunctionFactory.h>
#include <Columns/ColumnNullable.h>
#include <Columns/ColumnConst.h>
#include <DataTypes/getLeastSupertype.h>
#include <Interpreters/castColumn.h>
#include <vector>
namespace DB
{
namespace ErrorCodes
{
extern const int TOO_LESS_ARGUMENTS_FOR_FUNCTION;
}
void registerFunctionsConditional(FunctionFactory & factory)
{
factory.registerFunction<FunctionIf>();
factory.registerFunction<FunctionMultiIf>();
factory.registerFunction<FunctionCaseWithExpression>();
/// These are obsolete function names.
factory.registerFunction<FunctionCaseWithExpression>("caseWithExpr");
factory.registerFunction<FunctionMultiIf>("caseWithoutExpr");
factory.registerFunction<FunctionMultiIf>("caseWithoutExpression");
}
/// Implementation of FunctionMultiIf.
FunctionPtr FunctionMultiIf::create(const Context & context)
{
return std::make_shared<FunctionMultiIf>(context);
}
String FunctionMultiIf::getName() const
{
return name;
}
void FunctionMultiIf::executeImpl(Block & block, const ColumnNumbers & args, size_t result, size_t input_rows_count)
{
/** We will gather values from columns in branches to result column,
* depending on values of conditions.
*/
struct Instruction
{
const IColumn * condition = nullptr;
const IColumn * source = nullptr;
bool condition_always_true = false;
bool condition_is_nullable = false;
bool source_is_constant = false;
};
std::vector<Instruction> instructions;
instructions.reserve(args.size() / 2 + 1);
Columns converted_columns_holder;
converted_columns_holder.reserve(instructions.size());
const DataTypePtr & return_type = block.getByPosition(result).type;
for (size_t i = 0; i < args.size(); i += 2)
{
Instruction instruction;
size_t source_idx = i + 1;
if (source_idx == args.size())
{
/// The last, "else" branch can be treated as a branch with always true condition "else if (true)".
--source_idx;
instruction.condition_always_true = true;
}
else
{
const ColumnWithTypeAndName & cond_col = block.getByPosition(args[i]);
/// We skip branches that are always false.
/// If we encounter a branch that is always true, we can finish.
if (cond_col.column->onlyNull())
continue;
if (cond_col.column->isColumnConst())
{
Field value = typeid_cast<const ColumnConst &>(*cond_col.column).getField();
if (value.isNull())
continue;
if (value.get<UInt64>() == 0)
continue;
instruction.condition_always_true = true;
}
else
{
if (cond_col.column->isColumnNullable())
instruction.condition_is_nullable = true;
instruction.condition = cond_col.column.get();
}
}
const ColumnWithTypeAndName & source_col = block.getByPosition(args[source_idx]);
if (source_col.type->equals(*return_type))
{
instruction.source = source_col.column.get();
}
else
{
/// Cast all columns to result type.
converted_columns_holder.emplace_back(castColumn(source_col, return_type, context));
instruction.source = converted_columns_holder.back().get();
}
if (instruction.source && instruction.source->isColumnConst())
instruction.source_is_constant = true;
instructions.emplace_back(std::move(instruction));
if (instructions.back().condition_always_true)
break;
}
size_t rows = input_rows_count;
MutableColumnPtr res = return_type->createColumn();
for (size_t i = 0; i < rows; ++i)
{
for (const auto & instruction : instructions)
{
bool insert = false;
if (instruction.condition_always_true)
insert = true;
else if (!instruction.condition_is_nullable)
insert = static_cast<const ColumnUInt8 &>(*instruction.condition).getData()[i];
else
{
const ColumnNullable & condition_nullable = static_cast<const ColumnNullable &>(*instruction.condition);
const ColumnUInt8 & condition_nested = static_cast<const ColumnUInt8 &>(condition_nullable.getNestedColumn());
const NullMap & condition_null_map = condition_nullable.getNullMapData();
insert = !condition_null_map[i] && condition_nested.getData()[i];
}
if (insert)
{
if (!instruction.source_is_constant)
res->insertFrom(*instruction.source, i);
else
res->insertFrom(static_cast<const ColumnConst &>(*instruction.source).getDataColumn(), 0);
break;
}
}
}
block.getByPosition(result).column = std::move(res);
}
DataTypePtr FunctionMultiIf::getReturnTypeImpl(const DataTypes & args) const
{
/// Arguments are the following: cond1, then1, cond2, then2, ... condN, thenN, else.
auto for_conditions = [&args](auto && f)
{
size_t conditions_end = args.size() - 1;
for (size_t i = 0; i < conditions_end; i += 2)
f(args[i]);
};
auto for_branches = [&args](auto && f)
{
size_t branches_end = args.size();
for (size_t i = 1; i < branches_end; i += 2)
f(args[i]);
f(args.back());
};
if (!(args.size() >= 3 && args.size() % 2 == 1))
throw Exception{"Invalid number of arguments for function " + getName(),
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
/// Conditions must be UInt8, Nullable(UInt8) or Null. If one of conditions is Nullable, the result is also Nullable.
bool have_nullable_condition = false;
for_conditions([&](const DataTypePtr & arg)
{
const IDataType * nested_type;
if (arg->isNullable())
{
have_nullable_condition = true;
if (arg->onlyNull())
return;
const DataTypeNullable & nullable_type = static_cast<const DataTypeNullable &>(*arg);
nested_type = nullable_type.getNestedType().get();
}
else
{
nested_type = arg.get();
}
if (!WhichDataType(nested_type).isUInt8())
throw Exception{"Illegal type " + arg->getName() + " of argument (condition) "
"of function " + getName() + ". Must be UInt8.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
});
DataTypes types_of_branches;
types_of_branches.reserve(args.size() / 2 + 1);
for_branches([&](const DataTypePtr & arg)
{
types_of_branches.emplace_back(arg);
});
DataTypePtr common_type_of_branches = getLeastSupertype(types_of_branches);
return have_nullable_condition
? makeNullable(common_type_of_branches)
: common_type_of_branches;
}
FunctionPtr FunctionCaseWithExpression::create(const Context & context_)
{
return std::make_shared<FunctionCaseWithExpression>(context_);
}
FunctionCaseWithExpression::FunctionCaseWithExpression(const Context & context_)
: context{context_}
{
}
String FunctionCaseWithExpression::getName() const
{
return name;
}
DataTypePtr FunctionCaseWithExpression::getReturnTypeImpl(const DataTypes & args) const
{
if (!args.size())
throw Exception{"Function " + getName() + " expects at least 1 arguments",
ErrorCodes::TOO_LESS_ARGUMENTS_FOR_FUNCTION};
/// See the comments in executeImpl() to understand why we actually have to
/// get the return type of a transform function.
/// Get the types of the arrays that we pass to the transform function.
DataTypes src_array_types;
DataTypes dst_array_types;
for (size_t i = 1; i < args.size() - 1; ++i)
((i % 2) ? src_array_types : dst_array_types).push_back(args[i]);
DataTypePtr src_array_type = std::make_shared<DataTypeArray>(getLeastSupertype(src_array_types));
DataTypePtr dst_array_type = std::make_shared<DataTypeArray>(getLeastSupertype(dst_array_types));
/// Finally get the return type of the transform function.
FunctionTransform fun_transform;
ColumnsWithTypeAndName transform_args = {{nullptr, args.front(), {}}, {nullptr, src_array_type, {}},
{nullptr, dst_array_type, {}}, {nullptr, args.back(), {}}};
return fun_transform.getReturnType(transform_args);
}
void FunctionCaseWithExpression::executeImpl(Block & block, const ColumnNumbers & args, size_t result, size_t input_rows_count)
{
if (!args.size())
throw Exception{"Function " + getName() + " expects at least 1 arguments",
ErrorCodes::TOO_LESS_ARGUMENTS_FOR_FUNCTION};
/// In the following code, we turn the construction:
/// CASE expr WHEN val[0] THEN branch[0] ... WHEN val[N-1] then branch[N-1] ELSE branchN
/// into the construction transform(expr, src, dest, branchN)
/// where:
/// src = [val[0], val[1], ..., val[N-1]]
/// dst = [branch[0], ..., branch[N-1]]
/// then we perform it.
/// Create the arrays required by the transform function.
ColumnNumbers src_array_args;
ColumnsWithTypeAndName src_array_elems;
DataTypes src_array_types;
ColumnNumbers dst_array_args;
ColumnsWithTypeAndName dst_array_elems;
DataTypes dst_array_types;
for (size_t i = 1; i < (args.size() - 1); ++i)
{
if (i % 2)
{
src_array_args.push_back(args[i]);
src_array_elems.push_back(block.getByPosition(args[i]));
src_array_types.push_back(block.getByPosition(args[i]).type);
}
else
{
dst_array_args.push_back(args[i]);
dst_array_elems.push_back(block.getByPosition(args[i]));
dst_array_types.push_back(block.getByPosition(args[i]).type);
}
}
DataTypePtr src_array_type = std::make_shared<DataTypeArray>(getLeastSupertype(src_array_types));
DataTypePtr dst_array_type = std::make_shared<DataTypeArray>(getLeastSupertype(dst_array_types));
Block temp_block = block;
size_t src_array_pos = temp_block.columns();
temp_block.insert({nullptr, src_array_type, ""});
size_t dst_array_pos = temp_block.columns();
temp_block.insert({nullptr, dst_array_type, ""});
auto fun_array = FunctionFactory::instance().get("array", context);
fun_array->build(src_array_elems)->execute(temp_block, src_array_args, src_array_pos, input_rows_count);
fun_array->build(dst_array_elems)->execute(temp_block, dst_array_args, dst_array_pos, input_rows_count);
/// Execute transform.
FunctionTransform fun_transform;
ColumnNumbers transform_args{args.front(), src_array_pos, dst_array_pos, args.back()};
fun_transform.execute(temp_block, transform_args, result, input_rows_count);
/// Put the result into the original block.
block.getByPosition(result).column = std::move(temp_block.getByPosition(result).column);
}
}

View File

@ -312,8 +312,7 @@ struct ConvertImpl<FromDataType, std::enable_if_t<!std::is_same_v<FromDataType,
offsets_to[i] = write_buffer.count();
}
data_to.resize(write_buffer.count());
write_buffer.finish();
block.getByPosition(result).column = std::move(col_to);
}
else
@ -353,7 +352,7 @@ struct ConvertImplGenericToString
offsets_to[i] = write_buffer.count();
}
data_to.resize(write_buffer.count());
write_buffer.finish();
block.getByPosition(result).column = std::move(col_to);
}
};

View File

@ -108,8 +108,8 @@ private:
writeChar(0, buf_to);
offsets_to[i] = buf_to.count();
}
data_to.resize(buf_to.count());
buf_to.finish();
block.getByPosition(result).column = std::move(col_to);
}
else
@ -187,8 +187,8 @@ private:
writeChar(0, buf_to);
offsets_to[i] = buf_to.count();
}
data_to.resize(buf_to.count());
buf_to.finish();
block.getByPosition(result).column = std::move(col_to);
return true;
}

View File

@ -1,39 +0,0 @@
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionsMath.h>
namespace DB
{
const double EImpl::value = 2.7182818284590452353602874713526624977572470;
const double PiImpl::value = 3.1415926535897932384626433832795028841971693;
void registerFunctionsMath(FunctionFactory & factory)
{
factory.registerFunction<FunctionE>();
factory.registerFunction<FunctionPi>(FunctionFactory::CaseInsensitive);
factory.registerFunction<FunctionExp>(FunctionFactory::CaseInsensitive);
factory.registerFunction<FunctionLog>(FunctionFactory::CaseInsensitive);
factory.registerFunction<FunctionExp2>();
factory.registerFunction<FunctionLog2>(FunctionFactory::CaseInsensitive);
factory.registerFunction<FunctionExp10>();
factory.registerFunction<FunctionLog10>(FunctionFactory::CaseInsensitive);
factory.registerFunction<FunctionSqrt>(FunctionFactory::CaseInsensitive);
factory.registerFunction<FunctionCbrt>();
factory.registerFunction<FunctionErf>();
factory.registerFunction<FunctionErfc>();
factory.registerFunction<FunctionLGamma>();
factory.registerFunction<FunctionTGamma>();
factory.registerFunction<FunctionSin>(FunctionFactory::CaseInsensitive);
factory.registerFunction<FunctionCos>(FunctionFactory::CaseInsensitive);
factory.registerFunction<FunctionTan>(FunctionFactory::CaseInsensitive);
factory.registerFunction<FunctionAsin>(FunctionFactory::CaseInsensitive);
factory.registerFunction<FunctionAcos>(FunctionFactory::CaseInsensitive);
factory.registerFunction<FunctionAtan>(FunctionFactory::CaseInsensitive);
factory.registerFunction<FunctionPow>(FunctionFactory::CaseInsensitive);
factory.registerAlias("power", "pow", FunctionFactory::CaseInsensitive);
factory.registerAlias("ln", "log", FunctionFactory::CaseInsensitive);
}
}

View File

@ -34,7 +34,7 @@ public:
return std::const_pointer_cast<FunctionExpression>(shared_from_this());
}
void execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
void execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/, bool) override
{
Block expr_block;
for (size_t i = 0; i < arguments.size(); ++i)
@ -118,7 +118,7 @@ public:
return std::const_pointer_cast<FunctionCapture>(shared_from_this());
}
void execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
void execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count, bool) override
{
ColumnsWithTypeAndName columns;
columns.reserve(arguments.size());

View File

@ -1,24 +1,72 @@
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionsRandom.h>
#include <Common/HashTable/Hash.h>
#include <Common/randomSeed.h>
#include <common/unaligned.h>
namespace DB
{
namespace detail
namespace
{
/// NOTE Probably
/// http://www.pcg-random.org/
/// or http://www.math.sci.hiroshima-u.ac.jp/~m-mat/MT/SFMT/
/// or http://docs.yeppp.info/c/group__yep_random___w_e_l_l1024a.html
/// could go better.
struct LinearCongruentialGenerator
{
/// Constants from `man lrand48_r`.
static constexpr UInt64 a = 0x5DEECE66D;
static constexpr UInt64 c = 0xB;
/// And this is from `head -c8 /dev/urandom | xxd -p`
UInt64 current = 0x09826f4a081cee35ULL;
LinearCongruentialGenerator() {}
LinearCongruentialGenerator(UInt64 value) : current(value) {}
void seed(UInt64 value)
{
current = value;
}
UInt32 next()
{
current = current * a + c;
return current >> 16;
}
};
void seed(LinearCongruentialGenerator & generator, intptr_t additional_seed)
{
generator.seed(intHash64(randomSeed() ^ intHash64(additional_seed)));
}
}
void registerFunctionsRandom(FunctionFactory & factory)
void RandImpl::execute(char * output, size_t size)
{
factory.registerFunction<FunctionRand>();
factory.registerFunction<FunctionRand64>();
factory.registerFunction<FunctionRandConstant>();
LinearCongruentialGenerator generator0;
LinearCongruentialGenerator generator1;
LinearCongruentialGenerator generator2;
LinearCongruentialGenerator generator3;
seed(generator0, 0xfb4121280b2ab902ULL + reinterpret_cast<intptr_t>(output));
seed(generator1, 0x0121cf76df39c673ULL + reinterpret_cast<intptr_t>(output));
seed(generator2, 0x17ae86e3a19a602fULL + reinterpret_cast<intptr_t>(output));
seed(generator3, 0x8b6e16da7e06d622ULL + reinterpret_cast<intptr_t>(output));
for (const char * end = output + size; output < end; output += 16)
{
unalignedStore(output, generator0.next());
unalignedStore(output + 4, generator1.next());
unalignedStore(output + 8, generator2.next());
unalignedStore(output + 12, generator3.next());
}
/// It is guaranteed (by PaddedPODArray) that we can overwrite up to 15 bytes after end.
}
}

View File

@ -2,12 +2,8 @@
#include <DataTypes/DataTypesNumber.h>
#include <Columns/ColumnVector.h>
#include <Columns/ColumnConst.h>
#include <Functions/IFunction.h>
#include <Functions/FunctionHelpers.h>
#include <IO/WriteHelpers.h>
#include <Common/HashTable/Hash.h>
#include <Common/randomSeed.h>
namespace DB
@ -38,95 +34,16 @@ namespace ErrorCodes
* This means that the timer must be of sufficient resolution to give different values to each block.
*/
namespace detail
{
/// NOTE Probably
/// http://www.pcg-random.org/
/// or http://www.math.sci.hiroshima-u.ac.jp/~m-mat/MT/SFMT/
/// or http://docs.yeppp.info/c/group__yep_random___w_e_l_l1024a.html
/// could go better.
struct LinearCongruentialGenerator
{
/// Constants from `man lrand48_r`.
static constexpr UInt64 a = 0x5DEECE66D;
static constexpr UInt64 c = 0xB;
/// And this is from `head -c8 /dev/urandom | xxd -p`
UInt64 current = 0x09826f4a081cee35ULL;
LinearCongruentialGenerator() {}
LinearCongruentialGenerator(UInt64 value) : current(value) {}
void seed(UInt64 value)
{
current = value;
}
UInt32 next()
{
current = current * a + c;
return current >> 16;
}
};
void seed(LinearCongruentialGenerator & generator, intptr_t additional_seed);
}
struct RandImpl
{
using ReturnType = UInt32;
static void execute(ReturnType * output, size_t size)
{
detail::LinearCongruentialGenerator generator0;
detail::LinearCongruentialGenerator generator1;
detail::LinearCongruentialGenerator generator2;
detail::LinearCongruentialGenerator generator3;
detail::seed(generator0, 0xfb4121280b2ab902ULL + reinterpret_cast<intptr_t>(output));
detail::seed(generator1, 0x0121cf76df39c673ULL + reinterpret_cast<intptr_t>(output));
detail::seed(generator2, 0x17ae86e3a19a602fULL + reinterpret_cast<intptr_t>(output));
detail::seed(generator3, 0x8b6e16da7e06d622ULL + reinterpret_cast<intptr_t>(output));
ReturnType * pos = output;
ReturnType * end = pos + size;
ReturnType * end4 = pos + size / 4 * 4;
while (pos < end4)
{
pos[0] = generator0.next();
pos[1] = generator1.next();
pos[2] = generator2.next();
pos[3] = generator3.next();
pos += 4;
}
while (pos < end)
{
pos[0] = generator0.next();
++pos;
}
}
};
struct Rand64Impl
{
using ReturnType = UInt64;
static void execute(ReturnType * output, size_t size)
{
RandImpl::execute(reinterpret_cast<RandImpl::ReturnType *>(output), size * 2);
}
/// Fill memory with random data. The memory region must be 15-bytes padded.
static void execute(char * output, size_t size);
};
template <typename Impl, typename Name>
template <typename ToType, typename Name>
class FunctionRandom : public IFunction
{
private:
using ToType = typename Impl::ReturnType;
public:
static constexpr auto name = Name::name;
static FunctionPtr create(const Context &) { return std::make_shared<FunctionRandom>(); }
@ -147,7 +64,7 @@ public:
+ toString(arguments.size()) + ", should be 0 or 1.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
return std::make_shared<DataTypeNumber<typename Impl::ReturnType>>();
return std::make_shared<DataTypeNumber<ToType>>();
}
void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override
@ -157,67 +74,10 @@ public:
size_t size = input_rows_count;
vec_to.resize(size);
Impl::execute(vec_to.data(), vec_to.size());
RandImpl::execute(reinterpret_cast<char *>(vec_to.data()), vec_to.size() * sizeof(ToType));
block.getByPosition(result).column = std::move(col_to);
}
};
template <typename Impl, typename Name>
class FunctionRandomConstant : public IFunction
{
private:
using ToType = typename Impl::ReturnType;
/// The value is one for different blocks.
bool is_initialized = false;
ToType value;
public:
static constexpr auto name = Name::name;
static FunctionPtr create(const Context &) { return std::make_shared<FunctionRandomConstant>(); }
String getName() const override
{
return name;
}
bool isVariadic() const override { return true; }
size_t getNumberOfArguments() const override { return 0; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() > 1)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
+ toString(arguments.size()) + ", should be 0 or 1.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
return std::make_shared<DataTypeNumber<typename Impl::ReturnType>>();
}
void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override
{
if (!is_initialized)
{
is_initialized = true;
typename ColumnVector<ToType>::Container vec_to(1);
Impl::execute(vec_to.data(), vec_to.size());
value = vec_to[0];
}
block.getByPosition(result).column = DataTypeNumber<ToType>().createColumnConst(input_rows_count, toField(value));
}
};
struct NameRand { static constexpr auto name = "rand"; };
struct NameRand64 { static constexpr auto name = "rand64"; };
struct NameRandConstant { static constexpr auto name = "randConstant"; };
using FunctionRand = FunctionRandom<RandImpl, NameRand> ;
using FunctionRand64 = FunctionRandom<Rand64Impl, NameRand64>;
using FunctionRandConstant = FunctionRandomConstant<RandImpl, NameRandConstant>;
}

View File

@ -1,12 +0,0 @@
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionsTransform.h>
namespace DB
{
void registerFunctionsTransform(FunctionFactory & factory)
{
factory.registerFunction<FunctionTransform>();
}
}

View File

@ -267,7 +267,7 @@ bool allArgumentsAreConstants(const Block & block, const ColumnNumbers & args)
}
bool PreparedFunctionImpl::defaultImplementationForConstantArguments(Block & block, const ColumnNumbers & args, size_t result,
size_t input_rows_count)
size_t input_rows_count, bool dry_run)
{
ColumnNumbers arguments_to_remain_constants = getArgumentsThatAreAlwaysConstant();
@ -312,7 +312,7 @@ bool PreparedFunctionImpl::defaultImplementationForConstantArguments(Block & blo
for (size_t i = 0; i < arguments_size; ++i)
temporary_argument_numbers[i] = i;
executeWithoutLowCardinalityColumns(temporary_block, temporary_argument_numbers, arguments_size, temporary_block.rows());
executeWithoutLowCardinalityColumns(temporary_block, temporary_argument_numbers, arguments_size, temporary_block.rows(), dry_run);
ColumnPtr result_column;
/// extremely rare case, when we have function with completely const arguments
@ -328,7 +328,7 @@ bool PreparedFunctionImpl::defaultImplementationForConstantArguments(Block & blo
bool PreparedFunctionImpl::defaultImplementationForNulls(Block & block, const ColumnNumbers & args, size_t result,
size_t input_rows_count)
size_t input_rows_count, bool dry_run)
{
if (args.empty() || !useDefaultImplementationForNulls())
return false;
@ -344,7 +344,7 @@ bool PreparedFunctionImpl::defaultImplementationForNulls(Block & block, const Co
if (null_presence.has_nullable)
{
Block temporary_block = createBlockWithNestedColumns(block, args, result);
executeWithoutLowCardinalityColumns(temporary_block, args, result, temporary_block.rows());
executeWithoutLowCardinalityColumns(temporary_block, args, result, temporary_block.rows(), dry_run);
block.getByPosition(result).column = wrapInNullable(temporary_block.getByPosition(result).column, block, args,
result, input_rows_count);
return true;
@ -353,15 +353,19 @@ bool PreparedFunctionImpl::defaultImplementationForNulls(Block & block, const Co
return false;
}
void PreparedFunctionImpl::executeWithoutLowCardinalityColumns(Block & block, const ColumnNumbers & args, size_t result, size_t input_rows_count)
void PreparedFunctionImpl::executeWithoutLowCardinalityColumns(Block & block, const ColumnNumbers & args, size_t result,
size_t input_rows_count, bool dry_run)
{
if (defaultImplementationForConstantArguments(block, args, result, input_rows_count))
if (defaultImplementationForConstantArguments(block, args, result, input_rows_count, dry_run))
return;
if (defaultImplementationForNulls(block, args, result, input_rows_count))
if (defaultImplementationForNulls(block, args, result, input_rows_count, dry_run))
return;
executeImpl(block, args, result, input_rows_count);
if (dry_run)
executeImplDryRun(block, args, result, input_rows_count);
else
executeImpl(block, args, result, input_rows_count);
}
static const ColumnLowCardinality * findLowCardinalityArgument(const Block & block, const ColumnNumbers & args)
@ -441,7 +445,7 @@ static void convertLowCardinalityColumnsToFull(Block & block, const ColumnNumber
}
}
void PreparedFunctionImpl::execute(Block & block, const ColumnNumbers & args, size_t result, size_t input_rows_count)
void PreparedFunctionImpl::execute(Block & block, const ColumnNumbers & args, size_t result, size_t input_rows_count, bool dry_run)
{
if (useDefaultImplementationForLowCardinalityColumns())
{
@ -477,7 +481,7 @@ void PreparedFunctionImpl::execute(Block & block, const ColumnNumbers & args, si
ColumnPtr indexes = replaceLowCardinalityColumnsByNestedAndGetDictionaryIndexes(
block_without_low_cardinality, args, can_be_executed_on_default_arguments);
executeWithoutLowCardinalityColumns(block_without_low_cardinality, args, result, block_without_low_cardinality.rows());
executeWithoutLowCardinalityColumns(block_without_low_cardinality, args, result, block_without_low_cardinality.rows(), dry_run);
auto & keys = block_without_low_cardinality.safeGetByPosition(result).column;
if (auto full_column = keys->convertToFullColumnIfConst())
@ -511,12 +515,12 @@ void PreparedFunctionImpl::execute(Block & block, const ColumnNumbers & args, si
else
{
convertLowCardinalityColumnsToFull(block_without_low_cardinality, args);
executeWithoutLowCardinalityColumns(block_without_low_cardinality, args, result, input_rows_count);
executeWithoutLowCardinalityColumns(block_without_low_cardinality, args, result, input_rows_count, dry_run);
res.column = block_without_low_cardinality.safeGetByPosition(result).column;
}
}
else
executeWithoutLowCardinalityColumns(block, args, result, input_rows_count);
executeWithoutLowCardinalityColumns(block, args, result, input_rows_count, dry_run);
}
void FunctionBuilderImpl::checkNumberOfArguments(size_t number_of_arguments) const

View File

@ -40,7 +40,7 @@ public:
/// Get the main function name.
virtual String getName() const = 0;
virtual void execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) = 0;
virtual void execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count, bool dry_run) = 0;
};
using PreparedFunctionPtr = std::shared_ptr<IPreparedFunction>;
@ -52,7 +52,7 @@ using PreparedFunctionLowCardinalityResultCachePtr = std::shared_ptr<PreparedFun
class PreparedFunctionImpl : public IPreparedFunction
{
public:
void execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) final;
void execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count, bool dry_run = false) final;
/// Create cache which will be used to store result of function executed on LowCardinality column.
/// Only for default LowCardinality implementation.
@ -61,6 +61,10 @@ public:
protected:
virtual void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) = 0;
virtual void executeImplDryRun(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count)
{
executeImpl(block, arguments, result, input_rows_count);
}
/** Default implementation in presence of Nullable arguments or NULL constants as arguments is the following:
* if some of arguments are NULL constants then return NULL constant,
@ -94,11 +98,11 @@ protected:
private:
bool defaultImplementationForNulls(Block & block, const ColumnNumbers & args, size_t result,
size_t input_rows_count);
size_t input_rows_count, bool dry_run);
bool defaultImplementationForConstantArguments(Block & block, const ColumnNumbers & args, size_t result,
size_t input_rows_count);
size_t input_rows_count, bool dry_run);
void executeWithoutLowCardinalityColumns(Block & block, const ColumnNumbers & arguments, size_t result,
size_t input_rows_count);
size_t input_rows_count, bool dry_run);
/// Cache is created by function createLowCardinalityResultCache()
PreparedFunctionLowCardinalityResultCachePtr low_cardinality_result_cache;
@ -123,9 +127,9 @@ public:
virtual PreparedFunctionPtr prepare(const Block & sample_block, const ColumnNumbers & arguments, size_t result) const = 0;
/// TODO: make const
virtual void execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count)
virtual void execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count, bool dry_run = false)
{
return prepare(block, arguments, result)->execute(block, arguments, result, input_rows_count);
return prepare(block, arguments, result)->execute(block, arguments, result, input_rows_count, dry_run);
}
#if USE_EMBEDDED_COMPILER
@ -330,6 +334,7 @@ public:
bool canBeExecutedOnLowCardinalityDictionary() const override { return isDeterministicInScopeOfQuery(); }
using PreparedFunctionImpl::execute;
using PreparedFunctionImpl::executeImplDryRun;
using FunctionBuilderImpl::getReturnTypeImpl;
using FunctionBuilderImpl::getLambdaArgumentTypesImpl;
using FunctionBuilderImpl::getReturnType;
@ -404,6 +409,10 @@ protected:
{
return function->executeImpl(block, arguments, result, input_rows_count);
}
void executeImplDryRun(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) final
{
return function->executeImplDryRun(block, arguments, result, input_rows_count);
}
bool useDefaultImplementationForNulls() const final { return function->useDefaultImplementationForNulls(); }
bool useDefaultImplementationForConstants() const final { return function->useDefaultImplementationForConstants(); }
bool useDefaultImplementationForLowCardinalityColumns() const final { return function->useDefaultImplementationForLowCardinalityColumns(); }

View File

@ -0,0 +1,15 @@
#include <Functions/FunctionMathUnaryFloat64.h>
#include <Functions/FunctionFactory.h>
namespace DB
{
struct AcosName { static constexpr auto name = "acos"; };
using FunctionAcos = FunctionMathUnaryFloat64<UnaryFunctionVectorized<AcosName, acos>>;
void registerFunctionAcos(FunctionFactory & factory)
{
factory.registerFunction<FunctionAcos>(FunctionFactory::CaseInsensitive);
}
}

View File

@ -0,0 +1,15 @@
#include <Functions/FunctionMathUnaryFloat64.h>
#include <Functions/FunctionFactory.h>
namespace DB
{
struct AsinName { static constexpr auto name = "asin"; };
using FunctionAsin = FunctionMathUnaryFloat64<UnaryFunctionVectorized<AsinName, asin>>;
void registerFunctionAsin(FunctionFactory & factory)
{
factory.registerFunction<FunctionAsin>(FunctionFactory::CaseInsensitive);
}
}

View File

@ -0,0 +1,15 @@
#include <Functions/FunctionMathUnaryFloat64.h>
#include <Functions/FunctionFactory.h>
namespace DB
{
struct AtanName { static constexpr auto name = "atan"; };
using FunctionAtan = FunctionMathUnaryFloat64<UnaryFunctionVectorized<AtanName, atan>>;
void registerFunctionAtan(FunctionFactory & factory)
{
factory.registerFunction<FunctionAtan>(FunctionFactory::CaseInsensitive);
}
}

View File

@ -0,0 +1,128 @@
#include <Functions/FunctionFactory.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/getLeastSupertype.h>
#include <ext/map.h>
namespace DB
{
namespace ErrorCodes
{
extern const int TOO_LESS_ARGUMENTS_FOR_FUNCTION;
}
/// Implements the CASE construction when it is
/// provided an expression. Users should not call this function.
class FunctionCaseWithExpression : public IFunction
{
public:
static constexpr auto name = "caseWithExpression";
static FunctionPtr create(const Context & context_) { return std::make_shared<FunctionCaseWithExpression>(context_); }
public:
FunctionCaseWithExpression(const Context & context_) : context(context_) {}
bool isVariadic() const override { return true; }
size_t getNumberOfArguments() const override { return 0; }
String getName() const override { return name; }
DataTypePtr getReturnTypeImpl(const DataTypes & args) const override
{
if (!args.size())
throw Exception{"Function " + getName() + " expects at least 1 arguments",
ErrorCodes::TOO_LESS_ARGUMENTS_FOR_FUNCTION};
/// See the comments in executeImpl() to understand why we actually have to
/// get the return type of a transform function.
/// Get the types of the arrays that we pass to the transform function.
DataTypes dst_array_types;
for (size_t i = 2; i < args.size() - 1; i += 2)
dst_array_types.push_back(args[i]);
return getLeastSupertype(dst_array_types);
}
void executeImpl(Block & block, const ColumnNumbers & args, size_t result, size_t input_rows_count) override
{
if (!args.size())
throw Exception{"Function " + getName() + " expects at least 1 argument",
ErrorCodes::TOO_LESS_ARGUMENTS_FOR_FUNCTION};
/// In the following code, we turn the construction:
/// CASE expr WHEN val[0] THEN branch[0] ... WHEN val[N-1] then branch[N-1] ELSE branchN
/// into the construction transform(expr, src, dest, branchN)
/// where:
/// src = [val[0], val[1], ..., val[N-1]]
/// dst = [branch[0], ..., branch[N-1]]
/// then we perform it.
/// Create the arrays required by the transform function.
ColumnNumbers src_array_args;
ColumnsWithTypeAndName src_array_elems;
DataTypes src_array_types;
ColumnNumbers dst_array_args;
ColumnsWithTypeAndName dst_array_elems;
DataTypes dst_array_types;
for (size_t i = 1; i < (args.size() - 1); ++i)
{
if (i % 2)
{
src_array_args.push_back(args[i]);
src_array_elems.push_back(block.getByPosition(args[i]));
src_array_types.push_back(block.getByPosition(args[i]).type);
}
else
{
dst_array_args.push_back(args[i]);
dst_array_elems.push_back(block.getByPosition(args[i]));
dst_array_types.push_back(block.getByPosition(args[i]).type);
}
}
DataTypePtr src_array_type = std::make_shared<DataTypeArray>(getLeastSupertype(src_array_types));
DataTypePtr dst_array_type = std::make_shared<DataTypeArray>(getLeastSupertype(dst_array_types));
Block temp_block = block;
size_t src_array_pos = temp_block.columns();
temp_block.insert({nullptr, src_array_type, ""});
size_t dst_array_pos = temp_block.columns();
temp_block.insert({nullptr, dst_array_type, ""});
auto fun_array = FunctionFactory::instance().get("array", context);
fun_array->build(src_array_elems)->execute(temp_block, src_array_args, src_array_pos, input_rows_count);
fun_array->build(dst_array_elems)->execute(temp_block, dst_array_args, dst_array_pos, input_rows_count);
/// Execute transform.
ColumnNumbers transform_args{args.front(), src_array_pos, dst_array_pos, args.back()};
FunctionFactory::instance().get("transform", context)->build(
ext::map<ColumnsWithTypeAndName>(transform_args, [&](auto i){ return temp_block.getByPosition(i); }))
->execute(temp_block, transform_args, result, input_rows_count);
/// Put the result into the original block.
block.getByPosition(result).column = std::move(temp_block.getByPosition(result).column);
}
private:
const Context & context;
};
void registerFunctionCaseWithExpression(FunctionFactory & factory)
{
factory.registerFunction<FunctionCaseWithExpression>();
/// These are obsolete function names.
factory.registerFunction<FunctionCaseWithExpression>("caseWithExpr");
}
}

View File

@ -0,0 +1,15 @@
#include <Functions/FunctionMathUnaryFloat64.h>
#include <Functions/FunctionFactory.h>
namespace DB
{
struct CbrtName { static constexpr auto name = "cbrt"; };
using FunctionCbrt = FunctionMathUnaryFloat64<UnaryFunctionVectorized<CbrtName, cbrt>>;
void registerFunctionCbrt(FunctionFactory & factory)
{
factory.registerFunction<FunctionCbrt>();
}
}

View File

@ -0,0 +1,15 @@
#include <Functions/FunctionMathUnaryFloat64.h>
#include <Functions/FunctionFactory.h>
namespace DB
{
struct CosName { static constexpr auto name = "cos"; };
using FunctionCos = FunctionMathUnaryFloat64<UnaryFunctionVectorized<CosName, cos>>;
void registerFunctionCos(FunctionFactory & factory)
{
factory.registerFunction<FunctionCos>(FunctionFactory::CaseInsensitive);
}
}

20
dbms/src/Functions/e.cpp Normal file
View File

@ -0,0 +1,20 @@
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionMathConstFloat64.h>
namespace DB
{
struct EImpl
{
static constexpr auto name = "e";
static constexpr double value = 2.7182818284590452353602874713526624977572470;
};
using FunctionE = FunctionMathConstFloat64<EImpl>;
void registerFunctionE(FunctionFactory & factory)
{
factory.registerFunction<FunctionE>();
}
}

View File

@ -0,0 +1,15 @@
#include <Functions/FunctionMathUnaryFloat64.h>
#include <Functions/FunctionFactory.h>
namespace DB
{
struct ErfName { static constexpr auto name = "erf"; };
using FunctionErf = FunctionMathUnaryFloat64<UnaryFunctionPlain<ErfName, std::erf>>;
void registerFunctionErf(FunctionFactory & factory)
{
factory.registerFunction<FunctionErf>();
}
}

View File

@ -0,0 +1,15 @@
#include <Functions/FunctionMathUnaryFloat64.h>
#include <Functions/FunctionFactory.h>
namespace DB
{
struct ErfcName { static constexpr auto name = "erfc"; };
using FunctionErfc = FunctionMathUnaryFloat64<UnaryFunctionPlain<ErfcName, std::erfc>>;
void registerFunctionErfc(FunctionFactory & factory)
{
factory.registerFunction<FunctionErfc>();
}
}

View File

@ -0,0 +1,15 @@
#include <Functions/FunctionMathUnaryFloat64.h>
#include <Functions/FunctionFactory.h>
namespace DB
{
struct ExpName { static constexpr auto name = "exp"; };
using FunctionExp = FunctionMathUnaryFloat64<UnaryFunctionVectorized<ExpName, exp>>;
void registerFunctionExp(FunctionFactory & factory)
{
factory.registerFunction<FunctionExp>(FunctionFactory::CaseInsensitive);
}
}

View File

@ -0,0 +1,23 @@
#include <Functions/FunctionMathUnaryFloat64.h>
#include <Functions/FunctionFactory.h>
#include <common/preciseExp10.h>
namespace DB
{
struct Exp10Name { static constexpr auto name = "exp10"; };
using FunctionExp10 = FunctionMathUnaryFloat64<UnaryFunctionVectorized<Exp10Name,
#if USE_VECTORCLASS
exp10
#else
preciseExp10
#endif
>>;
void registerFunctionExp10(FunctionFactory & factory)
{
factory.registerFunction<FunctionExp10>();
}
}

View File

@ -0,0 +1,15 @@
#include <Functions/FunctionMathUnaryFloat64.h>
#include <Functions/FunctionFactory.h>
namespace DB
{
struct Exp2Name { static constexpr auto name = "exp2"; };
using FunctionExp2 = FunctionMathUnaryFloat64<UnaryFunctionVectorized<Exp2Name, exp2>>;
void registerFunctionExp2(FunctionFactory & factory)
{
factory.registerFunction<FunctionExp2>();
}
}

View File

@ -0,0 +1,57 @@
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionsRandom.h>
#include <DataTypes/DataTypeUUID.h>
namespace DB
{
class FunctionGenerateUUIDv4 : public IFunction
{
public:
static constexpr auto name = "generateUUIDv4";
static FunctionPtr create(const Context &) { return std::make_shared<FunctionGenerateUUIDv4>(); }
String getName() const override
{
return name;
}
size_t getNumberOfArguments() const override { return 0; }
DataTypePtr getReturnTypeImpl(const DataTypes &) const override
{
return std::make_shared<DataTypeUUID>();
}
bool isDeterministic() const override { return false; }
void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override
{
auto col_res = ColumnVector<UInt128>::create();
typename ColumnVector<UInt128>::Container & vec_to = col_res->getData();
size_t size = input_rows_count;
vec_to.resize(size);
RandImpl::execute(reinterpret_cast<char *>(vec_to.data()), vec_to.size() * sizeof(UInt128));
for (UInt128 & uuid: vec_to)
{
/** https://tools.ietf.org/html/rfc4122#section-4.4
*/
uuid.low = (uuid.low & 0xffffffffffff0fffull) | 0x0000000000004000ull;
uuid.high = (uuid.high & 0x3fffffffffffffffull) | 0x8000000000000000ull;
}
block.getByPosition(result).column = std::move(col_res);
}
};
void registerFunctionGenerateUUIDv4(FunctionFactory & factory)
{
factory.registerFunction<FunctionGenerateUUIDv4>();
}
}

View File

@ -1,5 +1,3 @@
#pragma once
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypesDecimal.h>
#include <DataTypes/DataTypeArray.h>
@ -7,7 +5,6 @@
#include <DataTypes/DataTypeFixedString.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/Native.h>
#include <Columns/ColumnVector.h>
#include <Columns/ColumnDecimal.h>
#include <Columns/ColumnString.h>
@ -23,6 +20,8 @@
#include <DataTypes/getLeastSupertype.h>
#include <Functions/GatherUtils/GatherUtils.h>
#include <Functions/GatherUtils/Algorithms.h>
#include <Functions/FunctionIfBase.h>
#include <Functions/FunctionFactory.h>
namespace DB
@ -165,64 +164,6 @@ public:
};
template <bool null_is_false>
class FunctionIfBase : public IFunction
{
#if USE_EMBEDDED_COMPILER
public:
bool isCompilableImpl(const DataTypes & types) const override
{
for (const auto & type : types)
if (!isCompilableType(removeNullable(type)))
return false;
return true;
}
llvm::Value * compileImpl(llvm::IRBuilderBase & builder, const DataTypes & types, ValuePlaceholders values) const override
{
auto & b = static_cast<llvm::IRBuilder<> &>(builder);
auto type = getReturnTypeImpl(types);
llvm::Value * null = nullptr;
if (!null_is_false && type->isNullable())
null = b.CreateInsertValue(llvm::Constant::getNullValue(toNativeType(b, type)), b.getTrue(), {1});
auto * head = b.GetInsertBlock();
auto * join = llvm::BasicBlock::Create(head->getContext(), "", head->getParent());
std::vector<std::pair<llvm::BasicBlock *, llvm::Value *>> returns;
for (size_t i = 0; i + 1 < types.size(); i += 2)
{
auto * then = llvm::BasicBlock::Create(head->getContext(), "", head->getParent());
auto * next = llvm::BasicBlock::Create(head->getContext(), "", head->getParent());
auto * cond = values[i]();
if (!null_is_false && types[i]->isNullable())
{
auto * nonnull = llvm::BasicBlock::Create(head->getContext(), "", head->getParent());
returns.emplace_back(b.GetInsertBlock(), null);
b.CreateCondBr(b.CreateExtractValue(cond, {1}), join, nonnull);
b.SetInsertPoint(nonnull);
b.CreateCondBr(nativeBoolCast(b, removeNullable(types[i]), b.CreateExtractValue(cond, {0})), then, next);
}
else
{
b.CreateCondBr(nativeBoolCast(b, types[i], cond), then, next);
}
b.SetInsertPoint(then);
auto * value = nativeCast(b, types[i + 1], values[i + 1](), type);
returns.emplace_back(b.GetInsertBlock(), value);
b.CreateBr(join);
b.SetInsertPoint(next);
}
auto * value = nativeCast(b, types.back(), values.back()(), type);
returns.emplace_back(b.GetInsertBlock(), value);
b.CreateBr(join);
b.SetInsertPoint(join);
auto * phi = b.CreatePHI(toNativeType(b, type), returns.size());
for (const auto & r : returns)
phi->addIncoming(r.second, r.first);
return phi;
}
#endif
};
class FunctionIf : public FunctionIfBase</*null_is_false=*/false>
{
public:
@ -993,60 +934,9 @@ public:
}
};
/// Function multiIf, which generalizes the function if.
///
/// Syntax: multiIf(cond_1, then_1, ..., cond_N, then_N, else)
/// where N >= 1.
///
/// For all 1 <= i <= N, "cond_i" has type UInt8.
/// Types of all the branches "then_i" and "else" are either of the following:
/// - numeric types for which there exists a common type;
/// - dates;
/// - dates with time;
/// - strings;
/// - arrays of such types.
///
/// Additionally the arguments, conditions or branches, support nullable types
/// and the NULL value, with a NULL condition treated as false.
class FunctionMultiIf final : public FunctionIfBase</*null_is_false=*/true>
void registerFunctionIf(FunctionFactory & factory)
{
public:
static constexpr auto name = "multiIf";
static FunctionPtr create(const Context & context);
FunctionMultiIf(const Context & context) : context(context) {}
public:
String getName() const override;
bool isVariadic() const override { return true; }
size_t getNumberOfArguments() const override { return 0; }
bool useDefaultImplementationForNulls() const override { return false; }
DataTypePtr getReturnTypeImpl(const DataTypes & args) const override;
void executeImpl(Block & block, const ColumnNumbers & args, size_t result, size_t input_rows_count) override;
private:
const Context & context;
};
/// Implements the CASE construction when it is
/// provided an expression. Users should not call this function.
class FunctionCaseWithExpression : public IFunction
{
public:
static constexpr auto name = "caseWithExpression";
static FunctionPtr create(const Context & context_);
public:
FunctionCaseWithExpression(const Context & context_);
bool isVariadic() const override { return true; }
size_t getNumberOfArguments() const override { return 0; }
String getName() const override;
DataTypePtr getReturnTypeImpl(const DataTypes & args) const override;
void executeImpl(Block & block, const ColumnNumbers & args, size_t result, size_t input_rows_count) override;
private:
const Context & context;
};
factory.registerFunction<FunctionIf>();
}
}

View File

@ -0,0 +1,15 @@
#include <Functions/FunctionMathUnaryFloat64.h>
#include <Functions/FunctionFactory.h>
namespace DB
{
struct LGammaName { static constexpr auto name = "lgamma"; };
using FunctionLGamma = FunctionMathUnaryFloat64<UnaryFunctionPlain<LGammaName, std::lgamma>>;
void registerFunctionLGamma(FunctionFactory & factory)
{
factory.registerFunction<FunctionLGamma>();
}
}

View File

@ -0,0 +1,16 @@
#include <Functions/FunctionMathUnaryFloat64.h>
#include <Functions/FunctionFactory.h>
namespace DB
{
struct LogName { static constexpr auto name = "log"; };
using FunctionLog = FunctionMathUnaryFloat64<UnaryFunctionVectorized<LogName, log>>;
void registerFunctionLog(FunctionFactory & factory)
{
factory.registerFunction<FunctionLog>(FunctionFactory::CaseInsensitive);
factory.registerAlias("ln", "log", FunctionFactory::CaseInsensitive);
}
}

View File

@ -0,0 +1,15 @@
#include <Functions/FunctionMathUnaryFloat64.h>
#include <Functions/FunctionFactory.h>
namespace DB
{
struct Log10Name { static constexpr auto name = "log10"; };
using FunctionLog10 = FunctionMathUnaryFloat64<UnaryFunctionVectorized<Log10Name, log10>>;
void registerFunctionLog10(FunctionFactory & factory)
{
factory.registerFunction<FunctionLog10>(FunctionFactory::CaseInsensitive);
}
}

View File

@ -0,0 +1,15 @@
#include <Functions/FunctionMathUnaryFloat64.h>
#include <Functions/FunctionFactory.h>
namespace DB
{
struct Log2Name { static constexpr auto name = "log2"; };
using FunctionLog2 = FunctionMathUnaryFloat64<UnaryFunctionVectorized<Log2Name, log2>>;
void registerFunctionLog2(FunctionFactory & factory)
{
factory.registerFunction<FunctionLog2>(FunctionFactory::CaseInsensitive);
}
}

View File

@ -0,0 +1,249 @@
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionIfBase.h>
#include <Columns/ColumnNullable.h>
#include <Columns/ColumnConst.h>
#include <Columns/ColumnsNumber.h>
#include <Interpreters/castColumn.h>
#include <Common/typeid_cast.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/getLeastSupertype.h>
namespace DB
{
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
}
/// Function multiIf, which generalizes the function if.
///
/// Syntax: multiIf(cond_1, then_1, ..., cond_N, then_N, else)
/// where N >= 1.
///
/// For all 1 <= i <= N, "cond_i" has type UInt8.
/// Types of all the branches "then_i" and "else" are either of the following:
/// - numeric types for which there exists a common type;
/// - dates;
/// - dates with time;
/// - strings;
/// - arrays of such types.
///
/// Additionally the arguments, conditions or branches, support nullable types
/// and the NULL value, with a NULL condition treated as false.
class FunctionMultiIf final : public FunctionIfBase</*null_is_false=*/true>
{
public:
static constexpr auto name = "multiIf";
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionMultiIf>(context); }
FunctionMultiIf(const Context & context) : context(context) {}
public:
String getName() const override { return name; }
bool isVariadic() const override { return true; }
size_t getNumberOfArguments() const override { return 0; }
bool useDefaultImplementationForNulls() const override { return false; }
DataTypePtr getReturnTypeImpl(const DataTypes & args) const override
{
/// Arguments are the following: cond1, then1, cond2, then2, ... condN, thenN, else.
auto for_conditions = [&args](auto && f)
{
size_t conditions_end = args.size() - 1;
for (size_t i = 0; i < conditions_end; i += 2)
f(args[i]);
};
auto for_branches = [&args](auto && f)
{
size_t branches_end = args.size();
for (size_t i = 1; i < branches_end; i += 2)
f(args[i]);
f(args.back());
};
if (!(args.size() >= 3 && args.size() % 2 == 1))
throw Exception{"Invalid number of arguments for function " + getName(),
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
/// Conditions must be UInt8, Nullable(UInt8) or Null. If one of conditions is Nullable, the result is also Nullable.
bool have_nullable_condition = false;
for_conditions([&](const DataTypePtr & arg)
{
const IDataType * nested_type;
if (arg->isNullable())
{
have_nullable_condition = true;
if (arg->onlyNull())
return;
const DataTypeNullable & nullable_type = static_cast<const DataTypeNullable &>(*arg);
nested_type = nullable_type.getNestedType().get();
}
else
{
nested_type = arg.get();
}
if (!WhichDataType(nested_type).isUInt8())
throw Exception{"Illegal type " + arg->getName() + " of argument (condition) "
"of function " + getName() + ". Must be UInt8.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
});
DataTypes types_of_branches;
types_of_branches.reserve(args.size() / 2 + 1);
for_branches([&](const DataTypePtr & arg)
{
types_of_branches.emplace_back(arg);
});
DataTypePtr common_type_of_branches = getLeastSupertype(types_of_branches);
return have_nullable_condition
? makeNullable(common_type_of_branches)
: common_type_of_branches;
}
void executeImpl(Block & block, const ColumnNumbers & args, size_t result, size_t input_rows_count) override
{
/** We will gather values from columns in branches to result column,
* depending on values of conditions.
*/
struct Instruction
{
const IColumn * condition = nullptr;
const IColumn * source = nullptr;
bool condition_always_true = false;
bool condition_is_nullable = false;
bool source_is_constant = false;
};
std::vector<Instruction> instructions;
instructions.reserve(args.size() / 2 + 1);
Columns converted_columns_holder;
converted_columns_holder.reserve(instructions.size());
const DataTypePtr & return_type = block.getByPosition(result).type;
for (size_t i = 0; i < args.size(); i += 2)
{
Instruction instruction;
size_t source_idx = i + 1;
if (source_idx == args.size())
{
/// The last, "else" branch can be treated as a branch with always true condition "else if (true)".
--source_idx;
instruction.condition_always_true = true;
}
else
{
const ColumnWithTypeAndName & cond_col = block.getByPosition(args[i]);
/// We skip branches that are always false.
/// If we encounter a branch that is always true, we can finish.
if (cond_col.column->onlyNull())
continue;
if (cond_col.column->isColumnConst())
{
Field value = typeid_cast<const ColumnConst &>(*cond_col.column).getField();
if (value.isNull())
continue;
if (value.get<UInt64>() == 0)
continue;
instruction.condition_always_true = true;
}
else
{
if (cond_col.column->isColumnNullable())
instruction.condition_is_nullable = true;
instruction.condition = cond_col.column.get();
}
}
const ColumnWithTypeAndName & source_col = block.getByPosition(args[source_idx]);
if (source_col.type->equals(*return_type))
{
instruction.source = source_col.column.get();
}
else
{
/// Cast all columns to result type.
converted_columns_holder.emplace_back(castColumn(source_col, return_type, context));
instruction.source = converted_columns_holder.back().get();
}
if (instruction.source && instruction.source->isColumnConst())
instruction.source_is_constant = true;
instructions.emplace_back(std::move(instruction));
if (instructions.back().condition_always_true)
break;
}
size_t rows = input_rows_count;
MutableColumnPtr res = return_type->createColumn();
for (size_t i = 0; i < rows; ++i)
{
for (const auto & instruction : instructions)
{
bool insert = false;
if (instruction.condition_always_true)
insert = true;
else if (!instruction.condition_is_nullable)
insert = static_cast<const ColumnUInt8 &>(*instruction.condition).getData()[i];
else
{
const ColumnNullable & condition_nullable = static_cast<const ColumnNullable &>(*instruction.condition);
const ColumnUInt8 & condition_nested = static_cast<const ColumnUInt8 &>(condition_nullable.getNestedColumn());
const NullMap & condition_null_map = condition_nullable.getNullMapData();
insert = !condition_null_map[i] && condition_nested.getData()[i];
}
if (insert)
{
if (!instruction.source_is_constant)
res->insertFrom(*instruction.source, i);
else
res->insertFrom(static_cast<const ColumnConst &>(*instruction.source).getDataColumn(), 0);
break;
}
}
}
block.getByPosition(result).column = std::move(res);
}
private:
const Context & context;
};
void registerFunctionMultiIf(FunctionFactory & factory)
{
factory.registerFunction<FunctionMultiIf>();
/// These are obsolete function names.
factory.registerFunction<FunctionMultiIf>("caseWithoutExpr");
factory.registerFunction<FunctionMultiIf>("caseWithoutExpression");
}
}

20
dbms/src/Functions/pi.cpp Normal file
View File

@ -0,0 +1,20 @@
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionMathConstFloat64.h>
namespace DB
{
struct PiImpl
{
static constexpr auto name = "pi";
static constexpr double value = 3.1415926535897932384626433832795028841971693;
};
using FunctionPi = FunctionMathConstFloat64<PiImpl>;
void registerFunctionPi(FunctionFactory & factory)
{
factory.registerFunction<FunctionPi>(FunctionFactory::CaseInsensitive);
}
}

View File

@ -0,0 +1,16 @@
#include <Functions/FunctionMathBinaryFloat64.h>
#include <Functions/FunctionFactory.h>
namespace DB
{
struct PowName { static constexpr auto name = "pow"; };
using FunctionPow = FunctionMathBinaryFloat64<BinaryFunctionVectorized<PowName, pow>>;
void registerFunctionPow(FunctionFactory & factory)
{
factory.registerFunction<FunctionPow>(FunctionFactory::CaseInsensitive);
factory.registerAlias("power", "pow", FunctionFactory::CaseInsensitive);
}
}

View File

@ -0,0 +1,16 @@
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionsRandom.h>
namespace DB
{
struct NameRand { static constexpr auto name = "rand"; };
using FunctionRand = FunctionRandom<UInt32, NameRand>;
void registerFunctionRand(FunctionFactory & factory)
{
factory.registerFunction<FunctionRand>();
}
}

View File

@ -0,0 +1,17 @@
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionsRandom.h>
namespace DB
{
struct NameRand64 { static constexpr auto name = "rand64"; };
using FunctionRand64 = FunctionRandom<UInt64, NameRand64>;
void registerFunctionRand64(FunctionFactory & factory)
{
factory.registerFunction<FunctionRand64>();
}
}

View File

@ -0,0 +1,62 @@
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionHelpers.h>
#include <Functions/FunctionsRandom.h>
namespace DB
{
template <typename ToType, typename Name>
class FunctionRandomConstant : public IFunction
{
private:
/// The value is one for different blocks.
bool is_initialized = false;
ToType value;
public:
static constexpr auto name = Name::name;
static FunctionPtr create(const Context &) { return std::make_shared<FunctionRandomConstant>(); }
String getName() const override
{
return name;
}
bool isVariadic() const override { return true; }
size_t getNumberOfArguments() const override { return 0; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() > 1)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
+ toString(arguments.size()) + ", should be 0 or 1.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
return std::make_shared<DataTypeNumber<ToType>>();
}
void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override
{
if (!is_initialized)
{
is_initialized = true;
typename ColumnVector<ToType>::Container vec_to(1);
RandImpl::execute(reinterpret_cast<char *>(vec_to.data()), sizeof(ToType));
value = vec_to[0];
}
block.getByPosition(result).column = DataTypeNumber<ToType>().createColumnConst(input_rows_count, toField(value));
}
};
struct NameRandConstant { static constexpr auto name = "randConstant"; };
using FunctionRandConstant = FunctionRandomConstant<UInt32, NameRandConstant>;
void registerFunctionRandConstant(FunctionFactory & factory)
{
factory.registerFunction<FunctionRandConstant>();
}
}

View File

@ -1,14 +0,0 @@
#include "register@FUNCTION@.h"
#include <Functions/FunctionFactory.h>
#include <Functions/Functions@FUNCTION_AREA@.h>
namespace DB
{
void register@FUNCTION@(FunctionFactory & factory)
{
factory.registerFunction<@FUNCTION@>();
}
}

View File

@ -1,10 +0,0 @@
#pragma once
namespace DB
{
class FunctionFactory;
void register@FUNCTION@(FunctionFactory & factory);
}

View File

@ -36,10 +36,10 @@ void registerFunctionsStringSearch(FunctionFactory &);
void registerFunctionsURL(FunctionFactory &);
void registerFunctionsVisitParam(FunctionFactory &);
void registerFunctionsMath(FunctionFactory &);
void registerFunctionsTransform(FunctionFactory &);
void registerFunctionsGeo(FunctionFactory &);
void registerFunctionsNull(FunctionFactory &);
void registerFunctionsFindCluster(FunctionFactory &);
void registerFunctionTransform(FunctionFactory &);
#if USE_ICU
void registerFunctionConvertCharset(FunctionFactory &);
@ -75,10 +75,10 @@ void registerFunctions()
registerFunctionsURL(factory);
registerFunctionsVisitParam(factory);
registerFunctionsMath(factory);
registerFunctionsTransform(factory);
registerFunctionsGeo(factory);
registerFunctionsNull(factory);
registerFunctionsFindCluster(factory);
registerFunctionTransform(factory);
#if USE_ICU
registerFunctionConvertCharset(factory);

View File

@ -0,0 +1,20 @@
namespace DB
{
class FunctionFactory;
void registerFunctionIf(FunctionFactory & factory);
void registerFunctionMultiIf(FunctionFactory & factory);
void registerFunctionCaseWithExpression(FunctionFactory & factory);
void registerFunctionsConditional(FunctionFactory & factory)
{
registerFunctionIf(factory);
registerFunctionMultiIf(factory);
registerFunctionCaseWithExpression(factory);
}
}

View File

@ -0,0 +1,54 @@
namespace DB
{
class FunctionFactory;
void registerFunctionE(FunctionFactory & factory);
void registerFunctionPi(FunctionFactory & factory);
void registerFunctionExp(FunctionFactory & factory);
void registerFunctionLog(FunctionFactory & factory);
void registerFunctionExp2(FunctionFactory & factory);
void registerFunctionLog2(FunctionFactory & factory);
void registerFunctionExp10(FunctionFactory & factory);
void registerFunctionLog10(FunctionFactory & factory);
void registerFunctionSqrt(FunctionFactory & factory);
void registerFunctionCbrt(FunctionFactory & factory);
void registerFunctionErf(FunctionFactory & factory);
void registerFunctionErfc(FunctionFactory & factory);
void registerFunctionLGamma(FunctionFactory & factory);
void registerFunctionTGamma(FunctionFactory & factory);
void registerFunctionSin(FunctionFactory & factory);
void registerFunctionCos(FunctionFactory & factory);
void registerFunctionTan(FunctionFactory & factory);
void registerFunctionAsin(FunctionFactory & factory);
void registerFunctionAcos(FunctionFactory & factory);
void registerFunctionAtan(FunctionFactory & factory);
void registerFunctionPow(FunctionFactory & factory);
void registerFunctionsMath(FunctionFactory & factory)
{
registerFunctionE(factory);
registerFunctionPi(factory);
registerFunctionExp(factory);
registerFunctionLog(factory);
registerFunctionExp2(factory);
registerFunctionLog2(factory);
registerFunctionExp10(factory);
registerFunctionLog10(factory);
registerFunctionSqrt(factory);
registerFunctionCbrt(factory);
registerFunctionErf(factory);
registerFunctionErfc(factory);
registerFunctionLGamma(factory);
registerFunctionTGamma(factory);
registerFunctionSin(factory);
registerFunctionCos(factory);
registerFunctionTan(factory);
registerFunctionAsin(factory);
registerFunctionAcos(factory);
registerFunctionAtan(factory);
registerFunctionPow(factory);
}
}

View File

@ -0,0 +1,21 @@
namespace DB
{
class FunctionFactory;
void registerFunctionRand(FunctionFactory & factory);
void registerFunctionRand64(FunctionFactory & factory);
void registerFunctionRandConstant(FunctionFactory & factory);
void registerFunctionGenerateUUIDv4(FunctionFactory & factory);
void registerFunctionsRandom(FunctionFactory & factory)
{
registerFunctionRand(factory);
registerFunctionRand64(factory);
registerFunctionRandConstant(factory);
registerFunctionGenerateUUIDv4(factory);
}
}

View File

@ -1,14 +0,0 @@
#include <Functions/FunctionFactory.h>
#include <Functions/Functions@FUNCTION_AREA@.h>
@REGISTER_HEADERS@
namespace DB
{
void registerFunctions@FUNCTION_AREA@(FunctionFactory & factory)
{
@REGISTER_FUNCTIONS@
}
}

View File

@ -44,6 +44,12 @@ public:
return std::make_shared<DataTypeUInt64>();
}
void executeImplDryRun(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override
{
auto column = ColumnUInt64::create(input_rows_count);
block.getByPosition(result).column = std::move(column);
}
void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override
{
size_t current_row_number = rows.fetch_add(input_rows_count);

View File

@ -0,0 +1,15 @@
#include <Functions/FunctionMathUnaryFloat64.h>
#include <Functions/FunctionFactory.h>
namespace DB
{
struct SinName { static constexpr auto name = "sin"; };
using FunctionSin = FunctionMathUnaryFloat64<UnaryFunctionVectorized<SinName, sin>>;
void registerFunctionSin(FunctionFactory & factory)
{
factory.registerFunction<FunctionSin>(FunctionFactory::CaseInsensitive);
}
}

View File

@ -0,0 +1,15 @@
#include <Functions/FunctionMathUnaryFloat64.h>
#include <Functions/FunctionFactory.h>
namespace DB
{
struct SqrtName { static constexpr auto name = "sqrt"; };
using FunctionSqrt = FunctionMathUnaryFloat64<UnaryFunctionVectorized<SqrtName, sqrt>>;
void registerFunctionSqrt(FunctionFactory & factory)
{
factory.registerFunction<FunctionSqrt>(FunctionFactory::CaseInsensitive);
}
}

View File

@ -0,0 +1,15 @@
#include <Functions/FunctionMathUnaryFloat64.h>
#include <Functions/FunctionFactory.h>
namespace DB
{
struct TanName { static constexpr auto name = "tan"; };
using FunctionTan = FunctionMathUnaryFloat64<UnaryFunctionVectorized<TanName, tan>>;
void registerFunctionTan(FunctionFactory & factory)
{
factory.registerFunction<FunctionTan>(FunctionFactory::CaseInsensitive);
}
}

View File

@ -0,0 +1,15 @@
#include <Functions/FunctionMathUnaryFloat64.h>
#include <Functions/FunctionFactory.h>
namespace DB
{
struct TGammaName { static constexpr auto name = "tgamma"; };
using FunctionTGamma = FunctionMathUnaryFloat64<UnaryFunctionPlain<TGammaName, std::tgamma>>;
void registerFunctionTGamma(FunctionFactory & factory)
{
factory.registerFunction<FunctionTGamma>();
}
}

View File

@ -8,6 +8,7 @@
#include <Functions/FunctionHelpers.h>
#include <Functions/extractTimeZoneFromFunctionArguments.h>
#include <IO/WriteHelpers.h>
namespace DB
{
@ -20,20 +21,19 @@ namespace ErrorCodes
/** timeSlots(StartTime, Duration)
* - for the time interval beginning at `StartTime` and continuing `Duration` seconds,
* returns an array of time points, consisting of rounding down to half an hour of points from this interval.
* returns an array of time points, consisting of rounding down to half an hour (default; or another value) of points from this interval.
* For example, timeSlots(toDateTime('2012-01-01 12:20:00'), 600) = [toDateTime('2012-01-01 12:00:00'), toDateTime('2012-01-01 12:30:00')].
* This is necessary to search for hits that are part of the corresponding visit.
*
* This is obsolete function. It was developed for Yandex.Metrica, but no longer used.
* This is obsolete function. It was developed for Yandex.Metrica, but no longer used in Yandex.
* But this function was adopted by wider audience.
*/
template <typename DurationType>
struct TimeSlotsImpl
{
static constexpr UInt32 TIME_SLOT_SIZE = 1800;
static void vector_vector(
const PaddedPODArray<UInt32> & starts, const PaddedPODArray<DurationType> & durations,
const PaddedPODArray<UInt32> & starts, const PaddedPODArray<DurationType> & durations, UInt32 time_slot_size,
PaddedPODArray<UInt32> & result_values, ColumnArray::Offsets & result_offsets)
{
size_t size = starts.size();
@ -44,9 +44,9 @@ struct TimeSlotsImpl
ColumnArray::Offset current_offset = 0;
for (size_t i = 0; i < size; ++i)
{
for (UInt32 value = starts[i] / TIME_SLOT_SIZE; value <= (starts[i] + durations[i]) / TIME_SLOT_SIZE; ++value)
for (UInt32 value = starts[i] / time_slot_size, end = (starts[i] + durations[i]) / time_slot_size; value <= end; ++value)
{
result_values.push_back(value * TIME_SLOT_SIZE);
result_values.push_back(value * time_slot_size);
++current_offset;
}
@ -55,7 +55,7 @@ struct TimeSlotsImpl
}
static void vector_constant(
const PaddedPODArray<UInt32> & starts, DurationType duration,
const PaddedPODArray<UInt32> & starts, DurationType duration, UInt32 time_slot_size,
PaddedPODArray<UInt32> & result_values, ColumnArray::Offsets & result_offsets)
{
size_t size = starts.size();
@ -66,9 +66,9 @@ struct TimeSlotsImpl
ColumnArray::Offset current_offset = 0;
for (size_t i = 0; i < size; ++i)
{
for (UInt32 value = starts[i] / TIME_SLOT_SIZE; value <= (starts[i] + duration) / TIME_SLOT_SIZE; ++value)
for (UInt32 value = starts[i] / time_slot_size, end = (starts[i] + duration) / time_slot_size; value <= end; ++value)
{
result_values.push_back(value * TIME_SLOT_SIZE);
result_values.push_back(value * time_slot_size);
++current_offset;
}
@ -77,7 +77,7 @@ struct TimeSlotsImpl
}
static void constant_vector(
UInt32 start, const PaddedPODArray<DurationType> & durations,
UInt32 start, const PaddedPODArray<DurationType> & durations, UInt32 time_slot_size,
PaddedPODArray<UInt32> & result_values, ColumnArray::Offsets & result_offsets)
{
size_t size = durations.size();
@ -88,23 +88,15 @@ struct TimeSlotsImpl
ColumnArray::Offset current_offset = 0;
for (size_t i = 0; i < size; ++i)
{
for (UInt32 value = start / TIME_SLOT_SIZE; value <= (start + durations[i]) / TIME_SLOT_SIZE; ++value)
for (UInt32 value = start / time_slot_size, end = (start + durations[i]) / time_slot_size; value <= end; ++value)
{
result_values.push_back(value * TIME_SLOT_SIZE);
result_values.push_back(value * time_slot_size);
++current_offset;
}
result_offsets[i] = current_offset;
}
}
static void constant_constant(
UInt32 start, DurationType duration,
Array & result)
{
for (UInt32 value = start / TIME_SLOT_SIZE; value <= (start + duration) / TIME_SLOT_SIZE; ++value)
result.push_back(value * TIME_SLOT_SIZE);
}
};
@ -112,6 +104,7 @@ class FunctionTimeSlots : public IFunction
{
public:
static constexpr auto name = "timeSlots";
static constexpr UInt32 TIME_SLOT_SIZE = 1800;
static FunctionPtr create(const Context &) { return std::make_shared<FunctionTimeSlots>(); }
String getName() const override
@ -119,10 +112,18 @@ public:
return name;
}
size_t getNumberOfArguments() const override { return 2; }
bool isVariadic() const override { return true; }
size_t getNumberOfArguments() const override { return 0; }
bool useDefaultImplementationForConstants() const override { return true; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {2}; }
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
{
if (arguments.size() != 2 && arguments.size() != 3)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
+ toString(arguments.size()) + ", should be 2 or 3",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
if (!WhichDataType(arguments[0].type).isDateTime())
throw Exception("Illegal type " + arguments[0].type->getName() + " of first argument of function " + getName() + ". Must be DateTime.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
@ -131,12 +132,16 @@ public:
throw Exception("Illegal type " + arguments[1].type->getName() + " of second argument of function " + getName() + ". Must be UInt32.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
if (arguments.size() == 3 && !WhichDataType(arguments[2].type).isNativeUInt())
throw Exception("Illegal type " + arguments[2].type->getName() + " of third argument of function " + getName() + ". Must be UInt32.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
/// If time zone is specified for source data type, attach it to the resulting type.
/// Note that there is no explicit time zone argument for this function (we specify 2 as an argument number with explicit time zone).
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeDateTime>(extractTimeZoneNameFromFunctionArguments(arguments, 2, 0)));
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeDateTime>(extractTimeZoneNameFromFunctionArguments(arguments, 3, 0)));
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t) override
{
auto starts = checkAndGetColumn<ColumnUInt32>(block.getByPosition(arguments[0]).column.get());
auto const_starts = checkAndGetColumnConst<ColumnUInt32>(block.getByPosition(arguments[0]).column.get());
@ -147,30 +152,37 @@ public:
auto res = ColumnArray::create(ColumnUInt32::create());
ColumnUInt32::Container & res_values = typeid_cast<ColumnUInt32 &>(res->getData()).getData();
auto time_slot_size = TIME_SLOT_SIZE;
if (arguments.size() == 3)
{
auto time_slot_column = checkAndGetColumn<ColumnConst>(block.getByPosition(arguments[2]).column.get());
if (!time_slot_column)
throw Exception("Third argument for function " + getName() + " must be constant UInt32", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
if (time_slot_size = time_slot_column->getValue<UInt32>(); time_slot_size == 0)
throw Exception("Third argument for function " + getName() + " must be greater than zero", ErrorCodes::ILLEGAL_COLUMN);
}
if (starts && durations)
{
TimeSlotsImpl<UInt32>::vector_vector(starts->getData(), durations->getData(), res_values, res->getOffsets());
TimeSlotsImpl<UInt32>::vector_vector(starts->getData(), durations->getData(), time_slot_size, res_values, res->getOffsets());
block.getByPosition(result).column = std::move(res);
}
else if (starts && const_durations)
{
TimeSlotsImpl<UInt32>::vector_constant(starts->getData(), const_durations->getValue<UInt32>(), res_values, res->getOffsets());
TimeSlotsImpl<UInt32>::vector_constant(starts->getData(), const_durations->getValue<UInt32>(), time_slot_size, res_values, res->getOffsets());
block.getByPosition(result).column = std::move(res);
}
else if (const_starts && durations)
{
TimeSlotsImpl<UInt32>::constant_vector(const_starts->getValue<UInt32>(), durations->getData(), res_values, res->getOffsets());
TimeSlotsImpl<UInt32>::constant_vector(const_starts->getValue<UInt32>(), durations->getData(), time_slot_size, res_values, res->getOffsets());
block.getByPosition(result).column = std::move(res);
}
else if (const_starts && const_durations)
{
Array const_res;
TimeSlotsImpl<UInt32>::constant_constant(const_starts->getValue<UInt32>(), const_durations->getValue<UInt32>(), const_res);
block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst(input_rows_count, const_res);
}
else
throw Exception("Illegal columns " + block.getByPosition(arguments[0]).column->getName()
+ ", " + block.getByPosition(arguments[1]).column->getName()
+ ", " + block.getByPosition(arguments[2]).column->getName()
+ " of arguments of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}

View File

@ -1,5 +1,3 @@
#pragma once
#include <mutex>
#include <Common/FieldVisitors.h>
#include <DataTypes/DataTypesNumber.h>
@ -15,6 +13,7 @@
#include <Common/typeid_cast.h>
#include <Functions/IFunction.h>
#include <Functions/FunctionHelpers.h>
#include <Functions/FunctionFactory.h>
#include <DataTypes/getLeastSupertype.h>
@ -836,4 +835,9 @@ private:
}
};
void registerFunctionTransform(FunctionFactory & factory)
{
factory.registerFunction<FunctionTransform>();
}
}

View File

@ -8,6 +8,7 @@ namespace DB
{
/** Allows to read from another ReadBuffer no more than the specified number of bytes.
* Note that the nested ReadBuffer may read slightly more data internally to fill its buffer.
*/
class LimitReadBuffer : public ReadBuffer
{

View File

@ -1,10 +1,7 @@
#pragma once
#include <string>
#include <IO/WriteBuffer.h>
#define WRITE_BUFFER_FROM_STRING_INITIAL_SIZE_IF_EMPTY 32
#include <IO/WriteBufferFromVector.h>
namespace DB
@ -13,41 +10,7 @@ namespace DB
/** Writes the data to a string.
* Note: before using the resulting string, destroy this object.
*/
class WriteBufferFromString : public WriteBuffer
{
private:
std::string & s;
void nextImpl() override
{
size_t old_size = s.size();
s.resize(old_size * 2);
internal_buffer = Buffer(reinterpret_cast<Position>(&s[old_size]), reinterpret_cast<Position>(&s[s.size()]));
working_buffer = internal_buffer;
}
protected:
void finish()
{
s.resize(count());
}
public:
WriteBufferFromString(std::string & s_)
: WriteBuffer(reinterpret_cast<Position>(s_.data()), s_.size()), s(s_)
{
if (s.empty())
{
s.resize(WRITE_BUFFER_FROM_STRING_INITIAL_SIZE_IF_EMPTY);
set(reinterpret_cast<Position>(s.data()), s.size());
}
}
~WriteBufferFromString() override
{
finish();
}
};
using WriteBufferFromString = WriteBufferFromVector<std::string>;
namespace detail

View File

@ -5,23 +5,33 @@
#include <IO/WriteBuffer.h>
#define WRITE_BUFFER_FROM_VECTOR_INITIAL_SIZE_IF_EMPTY 32
namespace DB
{
/** Initialized by vector. Writes data to it. When the vector is finished, it doubles its size.
* CharType - char or unsigned char.
namespace ErrorCodes
{
extern const int CANNOT_WRITE_AFTER_END_OF_BUFFER;
}
/** Writes data to existing std::vector or similar type. When not enough space, it doubles vector size.
*
* In destructor, vector is cutted to the size of written data.
* You can call to 'finish' to resize earlier.
*
* The vector should live until this object is destroyed or until the 'finish' method is called.
*/
template <typename VectorType = std::vector<char>>
template <typename VectorType>
class WriteBufferFromVector : public WriteBuffer
{
private:
VectorType & vector;
bool is_finished = false;
void nextImpl() override
{
if (is_finished)
throw Exception("WriteBufferFromVector is finished", ErrorCodes::CANNOT_WRITE_AFTER_END_OF_BUFFER);
size_t old_size = vector.size();
vector.resize(old_size * 2);
internal_buffer = Buffer(reinterpret_cast<Position>(&vector[old_size]), reinterpret_cast<Position>(vector.data() + vector.size()));
@ -34,10 +44,31 @@ public:
{
if (vector.empty())
{
vector.resize(WRITE_BUFFER_FROM_VECTOR_INITIAL_SIZE_IF_EMPTY);
static constexpr size_t initial_size = 32;
vector.resize(initial_size);
set(reinterpret_cast<Position>(vector.data()), vector.size());
}
}
void finish()
{
if (is_finished)
return;
is_finished = true;
vector.resize(
((position() - reinterpret_cast<Position>(vector.data()))
+ sizeof(typename VectorType::value_type) - 1) /// Align up.
/ sizeof(typename VectorType::value_type));
/// Prevent further writes.
set(nullptr, 0);
}
~WriteBufferFromVector() override
{
if (!is_finished)
finish();
}
};
}

View File

@ -61,7 +61,7 @@ int main(int argc, char ** argv)
formatted.reserve(n * 21);
{
DB::WriteBufferFromVector<> wb(formatted);
DB::WriteBufferFromVector wb(formatted);
// DB::CompressedWriteBuffer wb2(wb1);
// DB::AsynchronousWriteBuffer wb(wb2);
Stopwatch watch;

View File

@ -204,7 +204,7 @@ void ExpressionAction::prepare(Block & sample_block, const Settings & settings)
/// so we don't want to unfold non deterministic functions
if (all_const && function_base->isSuitableForConstantFolding() && (!compile_expressions || function_base->isDeterministic()))
{
function->execute(sample_block, arguments, result_position, sample_block.rows());
function->execute(sample_block, arguments, result_position, sample_block.rows(), true);
/// If the result is not a constant, just in case, we will consider the result as unknown.
ColumnWithTypeAndName & col = sample_block.safeGetByPosition(result_position);
@ -325,7 +325,7 @@ void ExpressionAction::prepare(Block & sample_block, const Settings & settings)
}
void ExpressionAction::execute(Block & block) const
void ExpressionAction::execute(Block & block, bool dry_run) const
{
size_t input_rows_count = block.rows();
@ -355,7 +355,7 @@ void ExpressionAction::execute(Block & block) const
ProfileEvents::increment(ProfileEvents::FunctionExecute);
if (is_function_compiled)
ProfileEvents::increment(ProfileEvents::CompiledFunctionExecute);
function->execute(block, arguments, num_columns_without_result, input_rows_count);
function->execute(block, arguments, num_columns_without_result, input_rows_count, dry_run);
break;
}
@ -383,7 +383,7 @@ void ExpressionAction::execute(Block & block) const
Block tmp_block{src_col, {{}, src_col.type, {}}};
function_builder->build({src_col})->execute(tmp_block, {0}, 1, src_col.column->size());
function_builder->build({src_col})->execute(tmp_block, {0}, 1, src_col.column->size(), dry_run);
non_empty_array_columns[name] = tmp_block.safeGetByPosition(1).column;
}
@ -492,7 +492,7 @@ void ExpressionAction::execute(Block & block) const
void ExpressionAction::executeOnTotals(Block & block) const
{
if (type != JOIN)
execute(block);
execute(block, false);
else
join->joinTotals(block);
}
@ -704,11 +704,11 @@ bool ExpressionActions::popUnusedArrayJoin(const Names & required_columns, Expre
return true;
}
void ExpressionActions::execute(Block & block) const
void ExpressionActions::execute(Block & block, bool dry_run) const
{
for (const auto & action : actions)
{
action.execute(block);
action.execute(block, dry_run);
checkLimits(block);
}
}

View File

@ -136,7 +136,7 @@ private:
friend class ExpressionActions;
void prepare(Block & sample_block, const Settings & settings);
void execute(Block & block) const;
void execute(Block & block, bool dry_run) const;
void executeOnTotals(Block & block) const;
};
@ -217,7 +217,7 @@ public:
const NamesAndTypesList & getRequiredColumnsWithTypes() const { return input_columns; }
/// Execute the expression on the block. The block must contain all the columns returned by getRequiredColumns.
void execute(Block & block) const;
void execute(Block & block, bool dry_run = false) const;
/** Execute the expression on the block of total values.
* Almost the same as `execute`. The difference is only when JOIN is executed.

View File

@ -684,30 +684,14 @@ bool ExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, bool only_ty
return true;
}
bool ExpressionAnalyzer::appendPrewhere(ExpressionActionsChain & chain, bool only_types,
const ASTPtr & sampling_expression, const ASTPtr & primary_expression)
bool ExpressionAnalyzer::appendPrewhere(
ExpressionActionsChain & chain, bool only_types, const Names & additional_required_columns)
{
assertSelect();
if (!select_query->prewhere_expression)
return false;
Names additional_required_mergetree_columns;
if (sampling_expression)
{
auto ast = sampling_expression;
auto syntax_result = SyntaxAnalyzer(context, storage).analyze(ast, {});
additional_required_mergetree_columns = ExpressionAnalyzer(ast, syntax_result, context).getRequiredSourceColumns();
}
if (primary_expression)
{
auto ast = primary_expression;
auto syntax_result = SyntaxAnalyzer(context, storage).analyze(ast, {});
auto required_primary_columns = ExpressionAnalyzer(ast, syntax_result, context).getRequiredSourceColumns();
additional_required_mergetree_columns.insert(additional_required_mergetree_columns.end(),
required_primary_columns.begin(), required_primary_columns.end());
}
initChain(chain, source_columns);
auto & step = chain.getLastStep();
getRootActions(select_query->prewhere_expression, only_types, step.actions);
@ -725,7 +709,7 @@ bool ExpressionAnalyzer::appendPrewhere(ExpressionActionsChain & chain, bool onl
/// Add required columns to required output in order not to remove them after prewhere execution.
/// TODO: add sampling and final execution to common chain.
for (const auto & column : additional_required_mergetree_columns)
for (const auto & column : additional_required_columns)
{
if (required_source_columns.count(column))
{

View File

@ -174,9 +174,8 @@ public:
bool appendArrayJoin(ExpressionActionsChain & chain, bool only_types);
bool appendJoin(ExpressionActionsChain & chain, bool only_types);
/// remove_filter is set in ExpressionActionsChain::finalize();
/// sampling_expression and primary_expression are needed in order to not remove columns are used in it.
bool appendPrewhere(ExpressionActionsChain & chain, bool only_types,
const ASTPtr & sampling_expression, const ASTPtr & primary_expression);
/// Columns in `additional_required_columns` will not be removed (they can be used for e.g. sampling or FINAL modifier).
bool appendPrewhere(ExpressionActionsChain & chain, bool only_types, const Names & additional_required_columns);
bool appendWhere(ExpressionActionsChain & chain, bool only_types);
bool appendGroupBy(ExpressionActionsChain & chain, bool only_types);
void appendAggregateFunctionsArguments(ExpressionActionsChain & chain, bool only_types);

View File

@ -103,7 +103,7 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create)
const ASTStorage & storage = *create.storage;
const ASTFunction & engine = *storage.engine;
/// Currently, there are no database engines, that support any arguments.
if (engine.arguments || engine.parameters || storage.partition_by || storage.order_by || storage.sample_by || storage.settings)
if (engine.arguments || engine.parameters || storage.partition_by || storage.primary_key || storage.order_by || storage.sample_by || storage.settings)
{
std::stringstream ostr;
formatAST(storage, ostr, false, false);
@ -547,7 +547,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create)
if (!as_table_name.empty())
{
as_storage = context.getTable(as_database_name, as_table_name);
as_storage_lock = as_storage->lockStructure(false, __PRETTY_FUNCTION__);
as_storage_lock = as_storage->lockStructure(false);
}
/// Set and retrieve list of columns.

View File

@ -102,7 +102,7 @@ BlockInputStreamPtr InterpreterDescribeQuery::executeImpl()
table = context.getTable(database_name, table_name);
}
auto table_lock = table->lockStructure(false, __PRETTY_FUNCTION__);
auto table_lock = table->lockStructure(false);
columns = table->getColumns().getAll();
column_defaults = table->getColumns().defaults;
column_comments = table->getColumns().comments;

View File

@ -69,7 +69,7 @@ BlockIO InterpreterDropQuery::executeToTable(String & database_name_, String & t
{
database_and_table.second->shutdown();
/// If table was already dropped by anyone, an exception will be thrown
auto table_lock = database_and_table.second->lockForAlter(__PRETTY_FUNCTION__);
auto table_lock = database_and_table.second->lockForAlter();
/// Drop table from memory, don't touch data and metadata
database_and_table.first->detachTable(database_and_table.second->getTableName());
}
@ -78,7 +78,7 @@ BlockIO InterpreterDropQuery::executeToTable(String & database_name_, String & t
database_and_table.second->checkTableCanBeDropped();
/// If table was already dropped by anyone, an exception will be thrown
auto table_lock = database_and_table.second->lockForAlter(__PRETTY_FUNCTION__);
auto table_lock = database_and_table.second->lockForAlter();
/// Drop table data, don't touch metadata
database_and_table.second->truncate(query_ptr);
}
@ -88,8 +88,10 @@ BlockIO InterpreterDropQuery::executeToTable(String & database_name_, String & t
database_and_table.second->shutdown();
/// If table was already dropped by anyone, an exception will be thrown
auto table_lock = database_and_table.second->lockForAlter(__PRETTY_FUNCTION__);
auto table_lock = database_and_table.second->lockForAlter();
/// Delete table metadata and table itself from memory
database_and_table.first->removeTable(context, database_and_table.second->getTableName());
/// Delete table data
database_and_table.second->drop();
@ -124,7 +126,7 @@ BlockIO InterpreterDropQuery::executeToTemporaryTable(String & table_name, ASTDr
if (kind == ASTDropQuery::Kind::Truncate)
{
/// If table was already dropped by anyone, an exception will be thrown
auto table_lock = table->lockForAlter(__PRETTY_FUNCTION__);
auto table_lock = table->lockForAlter();
/// Drop table data, don't touch metadata
table->truncate(query_ptr);
}
@ -133,7 +135,7 @@ BlockIO InterpreterDropQuery::executeToTemporaryTable(String & table_name, ASTDr
context_handle.tryRemoveExternalTable(table_name);
table->shutdown();
/// If table was already dropped by anyone, an exception will be thrown
auto table_lock = table->lockForAlter(__PRETTY_FUNCTION__);
auto table_lock = table->lockForAlter();
/// Delete table data
table->drop();
table->is_dropped = true;

View File

@ -92,7 +92,7 @@ BlockIO InterpreterInsertQuery::execute()
checkAccess(query);
StoragePtr table = getTable(query);
auto table_lock = table->lockStructure(true, __PRETTY_FUNCTION__);
auto table_lock = table->lockStructure(true);
/// We create a pipeline of several streams, into which we will write data.
BlockOutputStreamPtr out;

View File

@ -23,7 +23,7 @@ BlockIO InterpreterOptimizeQuery::execute()
return executeDDLQueryOnCluster(query_ptr, context, {ast.database});
StoragePtr table = context.getTable(ast.database, ast.table);
auto table_lock = table->lockStructure(true, __PRETTY_FUNCTION__);
auto table_lock = table->lockStructure(true);
table->optimize(query_ptr, ast.partition, ast.final, ast.deduplicate, context);
return {};
}

View File

@ -101,7 +101,7 @@ BlockIO InterpreterRenameQuery::execute()
for (const auto & names : unique_tables_from)
if (auto table = context.tryGetTable(names.database_name, names.table_name))
locks.emplace_back(table->lockForAlter(__PRETTY_FUNCTION__));
locks.emplace_back(table->lockForAlter());
/** All tables are locked. If there are more than one rename in chain,
* we need to hold global lock while doing all renames. Order matters to avoid deadlocks.

View File

@ -182,7 +182,7 @@ InterpreterSelectQuery::InterpreterSelectQuery(
}
if (storage)
table_lock = storage->lockStructure(false, __PRETTY_FUNCTION__);
table_lock = storage->lockStructure(false);
syntax_analyzer_result = SyntaxAnalyzer(context, storage)
.analyze(query_ptr, source_header.getNamesAndTypesList(), required_result_column_names, subquery_depth);
@ -339,9 +339,23 @@ InterpreterSelectQuery::AnalysisResult InterpreterSelectQuery::analyzeExpression
{
ExpressionActionsChain chain(context);
ASTPtr sampling_expression = (storage && query.sample_size()) ? storage->getSamplingExpression() : nullptr;
ASTPtr primary_expression = (storage && query.final()) ? storage->getPrimaryExpression() : nullptr;
if (query_analyzer->appendPrewhere(chain, !res.first_stage, sampling_expression, primary_expression))
Names additional_required_columns_after_prewhere;
if (storage && query.sample_size())
{
Names columns_for_sampling = storage->getColumnsRequiredForSampling();
additional_required_columns_after_prewhere.insert(additional_required_columns_after_prewhere.end(),
columns_for_sampling.begin(), columns_for_sampling.end());
}
if (storage && query.final())
{
Names columns_for_final = storage->getColumnsRequiredForFinal();
additional_required_columns_after_prewhere.insert(additional_required_columns_after_prewhere.end(),
columns_for_final.begin(), columns_for_final.end());
}
if (query_analyzer->appendPrewhere(chain, !res.first_stage, additional_required_columns_after_prewhere))
{
has_prewhere = true;

View File

@ -238,7 +238,7 @@ StoragePtr InterpreterSystemQuery::tryRestartReplica(const String & database_nam
table->shutdown();
/// If table was already dropped by anyone, an exception will be thrown
auto table_lock = table->lockForAlter(__PRETTY_FUNCTION__);
auto table_lock = table->lockForAlter();
create_ast = system_context.getCreateTableQuery(database_name, table_name);
database->detachTable(table_name);

View File

@ -960,7 +960,7 @@ struct AdderNonJoined;
template <typename Mapped>
struct AdderNonJoined<ASTTableJoin::Strictness::Any, Mapped>
{
static void add(const Mapped & mapped,
static void add(const Mapped & mapped, size_t & rows_added,
size_t num_columns_left, MutableColumns & columns_left,
size_t num_columns_right, MutableColumns & columns_right)
{
@ -969,13 +969,15 @@ struct AdderNonJoined<ASTTableJoin::Strictness::Any, Mapped>
for (size_t j = 0; j < num_columns_right; ++j)
columns_right[j]->insertFrom(*mapped.block->getByPosition(j).column.get(), mapped.row_num);
++rows_added;
}
};
template <typename Mapped>
struct AdderNonJoined<ASTTableJoin::Strictness::All, Mapped>
{
static void add(const Mapped & mapped,
static void add(const Mapped & mapped, size_t & rows_added,
size_t num_columns_left, MutableColumns & columns_left,
size_t num_columns_right, MutableColumns & columns_right)
{
@ -986,6 +988,8 @@ struct AdderNonJoined<ASTTableJoin::Strictness::All, Mapped>
for (size_t j = 0; j < num_columns_right; ++j)
columns_right[j]->insertFrom(*current->block->getByPosition(j).column.get(), current->row_num);
++rows_added;
}
}
};
@ -1155,10 +1159,9 @@ private:
if (it->second.getUsed())
continue;
AdderNonJoined<STRICTNESS, typename Map::mapped_type>::add(it->second, num_columns_left, columns_left, num_columns_right, columns_right);
AdderNonJoined<STRICTNESS, typename Map::mapped_type>::add(it->second, rows_added, num_columns_left, columns_left, num_columns_right, columns_right);
++rows_added;
if (rows_added == max_block_size)
if (rows_added >= max_block_size)
{
++it;
break;

View File

@ -99,24 +99,22 @@ static NameSet getKeyColumns(const StoragePtr & storage)
NameSet key_columns;
if (merge_tree_data->partition_expr)
for (const String & col : merge_tree_data->partition_expr->getRequiredColumns())
if (merge_tree_data->partition_key_expr)
for (const String & col : merge_tree_data->partition_key_expr->getRequiredColumns())
key_columns.insert(col);
auto primary_expr = merge_tree_data->getPrimaryExpression();
if (primary_expr)
for (const String & col : primary_expr->getRequiredColumns())
key_columns.insert(col);
/// We don't process sampling_expression separately because it must be among the primary key columns.
auto secondary_sort_expr = merge_tree_data->getSecondarySortExpression();
if (secondary_sort_expr)
for (const String & col : secondary_sort_expr->getRequiredColumns())
auto sorting_key_expr = merge_tree_data->sorting_key_expr;
if (sorting_key_expr)
for (const String & col : sorting_key_expr->getRequiredColumns())
key_columns.insert(col);
/// We don't process sample_by_ast separately because it must be among the primary key columns.
if (!merge_tree_data->merging_params.sign_column.empty())
key_columns.insert(merge_tree_data->merging_params.sign_column);
if (!merge_tree_data->merging_params.version_column.empty())
key_columns.insert(merge_tree_data->merging_params.version_column);
return key_columns;
}

View File

@ -3,6 +3,9 @@
#include <IO/ConcatReadBuffer.h>
#include <IO/WriteBufferFromFile.h>
#include <IO/WriteBufferFromVector.h>
#include <IO/LimitReadBuffer.h>
#include <IO/copyData.h>
#include <DataStreams/BlockIO.h>
#include <DataStreams/copyData.h>
@ -445,8 +448,11 @@ void executeQuery(
else
{
/// If not - copy enough data into 'parse_buf'.
parse_buf.resize(max_query_size + 1);
parse_buf.resize(istr.read(parse_buf.data(), max_query_size + 1));
WriteBufferFromVector<PODArray<char>> out(parse_buf);
LimitReadBuffer limit(istr, max_query_size + 1, false);
copyData(limit, out);
out.finish();
begin = parse_buf.data();
end = begin + parse_buf.size();
}

View File

@ -30,6 +30,11 @@ ASTPtr ASTAlterCommand::clone() const
res->primary_key = primary_key->clone();
res->children.push_back(res->primary_key);
}
if (order_by)
{
res->order_by = order_by->clone();
res->children.push_back(res->order_by);
}
if (partition)
{
res->partition = partition->clone();
@ -80,9 +85,12 @@ void ASTAlterCommand::formatImpl(
else if (type == ASTAlterCommand::MODIFY_PRIMARY_KEY)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "MODIFY PRIMARY KEY " << (settings.hilite ? hilite_none : "");
settings.ostr << "(";
primary_key->formatImpl(settings, state, frame);
settings.ostr << ")";
}
else if (type == ASTAlterCommand::MODIFY_ORDER_BY)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "MODIFY ORDER BY " << (settings.hilite ? hilite_none : "");
order_by->formatImpl(settings, state, frame);
}
else if (type == ASTAlterCommand::DROP_PARTITION)
{

View File

@ -25,8 +25,9 @@ public:
ADD_COLUMN,
DROP_COLUMN,
MODIFY_COLUMN,
MODIFY_PRIMARY_KEY,
COMMENT_COLUMN,
MODIFY_PRIMARY_KEY,
MODIFY_ORDER_BY,
DROP_PARTITION,
ATTACH_PARTITION,
@ -58,6 +59,10 @@ public:
*/
ASTPtr primary_key;
/** For MODIFY ORDER BY
*/
ASTPtr order_by;
/** Used in DROP PARTITION and ATTACH PARTITION FROM queries.
* The value or ID of the partition is stored here.
*/

View File

@ -16,6 +16,7 @@ class ASTStorage : public IAST
public:
ASTFunction * engine = nullptr;
IAST * partition_by = nullptr;
IAST * primary_key = nullptr;
IAST * order_by = nullptr;
IAST * sample_by = nullptr;
ASTSetQuery * settings = nullptr;
@ -31,6 +32,8 @@ public:
res->set(res->engine, engine->clone());
if (partition_by)
res->set(res->partition_by, partition_by->clone());
if (primary_key)
res->set(res->primary_key, primary_key->clone());
if (order_by)
res->set(res->order_by, order_by->clone());
if (sample_by)
@ -53,6 +56,11 @@ public:
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << "PARTITION BY " << (s.hilite ? hilite_none : "");
partition_by->formatImpl(s, state, frame);
}
if (primary_key)
{
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << "PRIMARY KEY " << (s.hilite ? hilite_none : "");
primary_key->formatImpl(s, state, frame);
}
if (order_by)
{
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << "ORDER BY " << (s.hilite ? hilite_none : "");

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