mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-04 05:22:17 +00:00
Changes required for auto-sync with Arcadia
This commit is contained in:
parent
871d3f2aa0
commit
e230632645
@ -11,6 +11,10 @@ using Int16 = int16_t;
|
|||||||
using Int32 = int32_t;
|
using Int32 = int32_t;
|
||||||
using Int64 = int64_t;
|
using Int64 = int64_t;
|
||||||
|
|
||||||
|
#if __cplusplus <= 201703L
|
||||||
|
using char8_t = unsigned char;
|
||||||
|
#endif
|
||||||
|
|
||||||
using UInt8 = char8_t;
|
using UInt8 = char8_t;
|
||||||
using UInt16 = uint16_t;
|
using UInt16 = uint16_t;
|
||||||
using UInt32 = uint32_t;
|
using UInt32 = uint32_t;
|
||||||
|
@ -1,12 +1,47 @@
|
|||||||
LIBRARY()
|
LIBRARY()
|
||||||
|
|
||||||
|
ADDINCL(
|
||||||
|
GLOBAL clickhouse/base
|
||||||
|
contrib/libs/cctz/include
|
||||||
|
)
|
||||||
|
|
||||||
|
CFLAGS (GLOBAL -DARCADIA_BUILD)
|
||||||
|
|
||||||
|
IF (OS_DARWIN)
|
||||||
|
CFLAGS (GLOBAL -DOS_DARWIN)
|
||||||
|
ELSEIF (OS_FREEBSD)
|
||||||
|
CFLAGS (GLOBAL -DOS_FREEBSD)
|
||||||
|
ELSEIF (OS_LINUX)
|
||||||
|
CFLAGS (GLOBAL -DOS_LINUX)
|
||||||
|
ENDIF ()
|
||||||
|
|
||||||
PEERDIR(
|
PEERDIR(
|
||||||
|
contrib/libs/cctz/src
|
||||||
|
contrib/libs/cxxsupp/libcxx-filesystem
|
||||||
|
contrib/libs/poco/Net
|
||||||
contrib/libs/poco/Util
|
contrib/libs/poco/Util
|
||||||
|
contrib/restricted/boost
|
||||||
|
contrib/restricted/cityhash-1.0.2
|
||||||
)
|
)
|
||||||
|
|
||||||
SRCS(
|
SRCS(
|
||||||
argsToConfig.cpp
|
argsToConfig.cpp
|
||||||
coverage.cpp
|
coverage.cpp
|
||||||
|
DateLUT.cpp
|
||||||
|
DateLUTImpl.cpp
|
||||||
|
demangle.cpp
|
||||||
|
getFQDNOrHostName.cpp
|
||||||
|
getMemoryAmount.cpp
|
||||||
|
getThreadId.cpp
|
||||||
|
JSON.cpp
|
||||||
|
LineReader.cpp
|
||||||
|
mremap.cpp
|
||||||
|
phdr_cache.cpp
|
||||||
|
preciseExp10.c
|
||||||
|
setTerminalEcho.cpp
|
||||||
|
shift10.cpp
|
||||||
|
sleep.cpp
|
||||||
|
terminalColors.cpp
|
||||||
)
|
)
|
||||||
|
|
||||||
END()
|
END()
|
||||||
|
@ -50,11 +50,13 @@
|
|||||||
#include <Common/getMultipleKeysFromConfig.h>
|
#include <Common/getMultipleKeysFromConfig.h>
|
||||||
#include <Common/ClickHouseRevision.h>
|
#include <Common/ClickHouseRevision.h>
|
||||||
#include <Common/Config/ConfigProcessor.h>
|
#include <Common/Config/ConfigProcessor.h>
|
||||||
#include <Common/config_version.h>
|
|
||||||
|
|
||||||
#ifdef __APPLE__
|
#if !defined(ARCADIA_BUILD)
|
||||||
// ucontext is not available without _XOPEN_SOURCE
|
# include <Common/config_version.h>
|
||||||
#define _XOPEN_SOURCE 700
|
#endif
|
||||||
|
|
||||||
|
#if defined(OS_DARWIN)
|
||||||
|
# define _XOPEN_SOURCE 700 // ucontext is not available without _XOPEN_SOURCE
|
||||||
#endif
|
#endif
|
||||||
#include <ucontext.h>
|
#include <ucontext.h>
|
||||||
|
|
||||||
@ -410,7 +412,7 @@ std::string BaseDaemon::getDefaultCorePath() const
|
|||||||
|
|
||||||
void BaseDaemon::closeFDs()
|
void BaseDaemon::closeFDs()
|
||||||
{
|
{
|
||||||
#if defined(__FreeBSD__) || (defined(__APPLE__) && defined(__MACH__))
|
#if defined(OS_FREEBSD) || defined(OS_DARWIN)
|
||||||
Poco::File proc_path{"/dev/fd"};
|
Poco::File proc_path{"/dev/fd"};
|
||||||
#else
|
#else
|
||||||
Poco::File proc_path{"/proc/self/fd"};
|
Poco::File proc_path{"/proc/self/fd"};
|
||||||
@ -430,7 +432,7 @@ void BaseDaemon::closeFDs()
|
|||||||
else
|
else
|
||||||
{
|
{
|
||||||
int max_fd = -1;
|
int max_fd = -1;
|
||||||
#ifdef _SC_OPEN_MAX
|
#if defined(_SC_OPEN_MAX)
|
||||||
max_fd = sysconf(_SC_OPEN_MAX);
|
max_fd = sysconf(_SC_OPEN_MAX);
|
||||||
if (max_fd == -1)
|
if (max_fd == -1)
|
||||||
#endif
|
#endif
|
||||||
@ -448,7 +450,7 @@ namespace
|
|||||||
/// the maximum is 1000, and chromium uses 300 for its tab processes. Ignore
|
/// the maximum is 1000, and chromium uses 300 for its tab processes. Ignore
|
||||||
/// whatever errors that occur, because it's just a debugging aid and we don't
|
/// whatever errors that occur, because it's just a debugging aid and we don't
|
||||||
/// care if it breaks.
|
/// care if it breaks.
|
||||||
#if defined(__linux__) && !defined(NDEBUG)
|
#if defined(OS_LINUX) && !defined(NDEBUG)
|
||||||
void debugIncreaseOOMScore()
|
void debugIncreaseOOMScore()
|
||||||
{
|
{
|
||||||
const std::string new_score = "555";
|
const std::string new_score = "555";
|
||||||
|
14
base/daemon/ya.make
Normal file
14
base/daemon/ya.make
Normal file
@ -0,0 +1,14 @@
|
|||||||
|
LIBRARY()
|
||||||
|
|
||||||
|
NO_COMPILER_WARNINGS()
|
||||||
|
|
||||||
|
PEERDIR(
|
||||||
|
clickhouse/src/Common
|
||||||
|
)
|
||||||
|
|
||||||
|
SRCS(
|
||||||
|
BaseDaemon.cpp
|
||||||
|
GraphiteWriter.cpp
|
||||||
|
)
|
||||||
|
|
||||||
|
END()
|
15
base/loggers/ya.make
Normal file
15
base/loggers/ya.make
Normal file
@ -0,0 +1,15 @@
|
|||||||
|
LIBRARY()
|
||||||
|
|
||||||
|
PEERDIR(
|
||||||
|
clickhouse/src/Common
|
||||||
|
)
|
||||||
|
|
||||||
|
SRCS(
|
||||||
|
ExtendedLogChannel.cpp
|
||||||
|
Loggers.cpp
|
||||||
|
OwnFormattingChannel.cpp
|
||||||
|
OwnPatternFormatter.cpp
|
||||||
|
OwnSplitChannel.cpp
|
||||||
|
)
|
||||||
|
|
||||||
|
END()
|
5
base/pcg-random/ya.make
Normal file
5
base/pcg-random/ya.make
Normal file
@ -0,0 +1,5 @@
|
|||||||
|
LIBRARY()
|
||||||
|
|
||||||
|
ADDINCL (GLOBAL clickhouse/base/pcg-random)
|
||||||
|
|
||||||
|
END()
|
9
base/widechar_width/ya.make
Normal file
9
base/widechar_width/ya.make
Normal file
@ -0,0 +1,9 @@
|
|||||||
|
LIBRARY()
|
||||||
|
|
||||||
|
ADDINCL(GLOBAL clickhouse/base/widechar_width)
|
||||||
|
|
||||||
|
SRCS(
|
||||||
|
widechar_width.cpp
|
||||||
|
)
|
||||||
|
|
||||||
|
END()
|
@ -1,3 +1,7 @@
|
|||||||
RECURSE(
|
RECURSE(
|
||||||
common
|
common
|
||||||
|
daemon
|
||||||
|
loggers
|
||||||
|
pcg-random
|
||||||
|
widechar_width
|
||||||
)
|
)
|
||||||
|
@ -15,7 +15,6 @@
|
|||||||
#include <common/getFQDNOrHostName.h>
|
#include <common/getFQDNOrHostName.h>
|
||||||
#include <Common/CurrentThread.h>
|
#include <Common/CurrentThread.h>
|
||||||
#include <Common/setThreadName.h>
|
#include <Common/setThreadName.h>
|
||||||
#include <Common/config.h>
|
|
||||||
#include <Common/SettingsChanges.h>
|
#include <Common/SettingsChanges.h>
|
||||||
#include <Disks/DiskSpaceMonitor.h>
|
#include <Disks/DiskSpaceMonitor.h>
|
||||||
#include <Compression/CompressedReadBuffer.h>
|
#include <Compression/CompressedReadBuffer.h>
|
||||||
@ -36,6 +35,11 @@
|
|||||||
#include <Common/typeid_cast.h>
|
#include <Common/typeid_cast.h>
|
||||||
#include <Poco/Net/HTTPStream.h>
|
#include <Poco/Net/HTTPStream.h>
|
||||||
|
|
||||||
|
#if !defined(ARCADIA_BUILD)
|
||||||
|
# include <Common/config.h>
|
||||||
|
#endif
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
|
@ -1,10 +1,8 @@
|
|||||||
#include <Common/config.h>
|
|
||||||
|
|
||||||
#include "MySQLHandler.h"
|
#include "MySQLHandler.h"
|
||||||
|
|
||||||
#include <limits>
|
#include <limits>
|
||||||
#include <ext/scope_guard.h>
|
#include <ext/scope_guard.h>
|
||||||
#include <Columns/ColumnVector.h>
|
#include <Columns/ColumnVector.h>
|
||||||
#include <Common/config_version.h>
|
|
||||||
#include <Common/NetException.h>
|
#include <Common/NetException.h>
|
||||||
#include <Common/OpenSSLHelpers.h>
|
#include <Common/OpenSSLHelpers.h>
|
||||||
#include <Core/MySQLProtocol.h>
|
#include <Core/MySQLProtocol.h>
|
||||||
@ -18,11 +16,15 @@
|
|||||||
#include <boost/algorithm/string/replace.hpp>
|
#include <boost/algorithm/string/replace.hpp>
|
||||||
#include <regex>
|
#include <regex>
|
||||||
|
|
||||||
|
#if !defined(ARCADIA_BUILD)
|
||||||
|
# include <Common/config_version.h>
|
||||||
|
#endif
|
||||||
|
|
||||||
#if USE_POCO_NETSSL
|
#if USE_POCO_NETSSL
|
||||||
#include <Poco/Net/SecureStreamSocket.h>
|
# include <Poco/Crypto/CipherFactory.h>
|
||||||
#include <Poco/Net/SSLManager.h>
|
# include <Poco/Crypto/RSAKey.h>
|
||||||
#include <Poco/Crypto/CipherFactory.h>
|
# include <Poco/Net/SSLManager.h>
|
||||||
#include <Poco/Crypto/RSAKey.h>
|
# include <Poco/Net/SecureStreamSocket.h>
|
||||||
#endif
|
#endif
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
|
@ -1,13 +1,17 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
#include <Common/config.h>
|
|
||||||
#include <Poco/Net/TCPServerConnection.h>
|
#include <Poco/Net/TCPServerConnection.h>
|
||||||
#include <common/getFQDNOrHostName.h>
|
#include <common/getFQDNOrHostName.h>
|
||||||
#include <Common/CurrentMetrics.h>
|
#include <Common/CurrentMetrics.h>
|
||||||
#include <Core/MySQLProtocol.h>
|
#include <Core/MySQLProtocol.h>
|
||||||
#include "IServer.h"
|
#include "IServer.h"
|
||||||
|
|
||||||
|
#if !defined(ARCADIA_BUILD)
|
||||||
|
# include <Common/config.h>
|
||||||
|
#endif
|
||||||
|
|
||||||
#if USE_POCO_NETSSL
|
#if USE_POCO_NETSSL
|
||||||
#include <Poco/Net/SecureStreamSocket.h>
|
# include <Poco/Net/SecureStreamSocket.h>
|
||||||
#endif
|
#endif
|
||||||
|
|
||||||
namespace CurrentMetrics
|
namespace CurrentMetrics
|
||||||
|
@ -1,11 +1,15 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include <Common/config.h>
|
|
||||||
#include <Poco/Net/TCPServerConnectionFactory.h>
|
#include <Poco/Net/TCPServerConnectionFactory.h>
|
||||||
#include <atomic>
|
#include <atomic>
|
||||||
#include "IServer.h"
|
#include "IServer.h"
|
||||||
|
|
||||||
|
#if !defined(ARCADIA_BUILD)
|
||||||
|
# include <Common/config.h>
|
||||||
|
#endif
|
||||||
|
|
||||||
#if USE_SSL
|
#if USE_SSL
|
||||||
#include <openssl/rsa.h>
|
# include <openssl/rsa.h>
|
||||||
#endif
|
#endif
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
|
@ -15,7 +15,6 @@
|
|||||||
#include <ext/scope_guard.h>
|
#include <ext/scope_guard.h>
|
||||||
#include <common/logger_useful.h>
|
#include <common/logger_useful.h>
|
||||||
#include <common/phdr_cache.h>
|
#include <common/phdr_cache.h>
|
||||||
#include <common/config_common.h>
|
|
||||||
#include <common/ErrorHandlers.h>
|
#include <common/ErrorHandlers.h>
|
||||||
#include <common/getMemoryAmount.h>
|
#include <common/getMemoryAmount.h>
|
||||||
#include <common/coverage.h>
|
#include <common/coverage.h>
|
||||||
@ -26,7 +25,6 @@
|
|||||||
#include <Common/StringUtils/StringUtils.h>
|
#include <Common/StringUtils/StringUtils.h>
|
||||||
#include <Common/ZooKeeper/ZooKeeper.h>
|
#include <Common/ZooKeeper/ZooKeeper.h>
|
||||||
#include <Common/ZooKeeper/ZooKeeperNodeCache.h>
|
#include <Common/ZooKeeper/ZooKeeperNodeCache.h>
|
||||||
#include "config_core.h"
|
|
||||||
#include <common/getFQDNOrHostName.h>
|
#include <common/getFQDNOrHostName.h>
|
||||||
#include <Common/getMultipleKeysFromConfig.h>
|
#include <Common/getMultipleKeysFromConfig.h>
|
||||||
#include <Common/getNumberOfPhysicalCPUCores.h>
|
#include <Common/getNumberOfPhysicalCPUCores.h>
|
||||||
@ -59,19 +57,24 @@
|
|||||||
#include "MetricsTransmitter.h"
|
#include "MetricsTransmitter.h"
|
||||||
#include <Common/StatusFile.h>
|
#include <Common/StatusFile.h>
|
||||||
#include "TCPHandlerFactory.h"
|
#include "TCPHandlerFactory.h"
|
||||||
#include "Common/config_version.h"
|
|
||||||
#include <Common/SensitiveDataMasker.h>
|
#include <Common/SensitiveDataMasker.h>
|
||||||
#include <Common/ThreadFuzzer.h>
|
#include <Common/ThreadFuzzer.h>
|
||||||
#include "MySQLHandlerFactory.h"
|
#include "MySQLHandlerFactory.h"
|
||||||
|
|
||||||
|
#if !defined(ARCADIA_BUILD)
|
||||||
|
# include <common/config_common.h>
|
||||||
|
# include "config_core.h"
|
||||||
|
# include "Common/config_version.h"
|
||||||
|
#endif
|
||||||
|
|
||||||
#if defined(OS_LINUX)
|
#if defined(OS_LINUX)
|
||||||
#include <Common/hasLinuxCapability.h>
|
# include <sys/mman.h>
|
||||||
#include <sys/mman.h>
|
# include <Common/hasLinuxCapability.h>
|
||||||
#endif
|
#endif
|
||||||
|
|
||||||
#if USE_POCO_NETSSL
|
#if USE_POCO_NETSSL
|
||||||
#include <Poco/Net/Context.h>
|
# include <Poco/Net/Context.h>
|
||||||
#include <Poco/Net/SecureServerSocket.h>
|
# include <Poco/Net/SecureServerSocket.h>
|
||||||
#endif
|
#endif
|
||||||
|
|
||||||
namespace CurrentMetrics
|
namespace CurrentMetrics
|
||||||
@ -248,7 +251,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
|||||||
|
|
||||||
const auto memory_amount = getMemoryAmount();
|
const auto memory_amount = getMemoryAmount();
|
||||||
|
|
||||||
#if defined(__linux__)
|
#if defined(OS_LINUX)
|
||||||
std::string executable_path = getExecutablePath();
|
std::string executable_path = getExecutablePath();
|
||||||
if (executable_path.empty())
|
if (executable_path.empty())
|
||||||
executable_path = "/usr/bin/clickhouse"; /// It is used for information messages.
|
executable_path = "/usr/bin/clickhouse"; /// It is used for information messages.
|
||||||
@ -631,7 +634,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
|||||||
dns_cache_updater = std::make_unique<DNSCacheUpdater>(*global_context, config().getInt("dns_cache_update_period", 15));
|
dns_cache_updater = std::make_unique<DNSCacheUpdater>(*global_context, config().getInt("dns_cache_update_period", 15));
|
||||||
}
|
}
|
||||||
|
|
||||||
#if defined(__linux__)
|
#if defined(OS_LINUX)
|
||||||
if (!TaskStatsInfoGetter::checkPermissions())
|
if (!TaskStatsInfoGetter::checkPermissions())
|
||||||
{
|
{
|
||||||
LOG_INFO(log, "It looks like the process has no CAP_NET_ADMIN capability, 'taskstats' performance statistics will be disabled."
|
LOG_INFO(log, "It looks like the process has no CAP_NET_ADMIN capability, 'taskstats' performance statistics will be disabled."
|
||||||
|
@ -6,7 +6,6 @@
|
|||||||
#include <Common/Stopwatch.h>
|
#include <Common/Stopwatch.h>
|
||||||
#include <Common/NetException.h>
|
#include <Common/NetException.h>
|
||||||
#include <Common/setThreadName.h>
|
#include <Common/setThreadName.h>
|
||||||
#include <Common/config_version.h>
|
|
||||||
#include <IO/Progress.h>
|
#include <IO/Progress.h>
|
||||||
#include <Compression/CompressedReadBuffer.h>
|
#include <Compression/CompressedReadBuffer.h>
|
||||||
#include <Compression/CompressedWriteBuffer.h>
|
#include <Compression/CompressedWriteBuffer.h>
|
||||||
@ -33,6 +32,10 @@
|
|||||||
|
|
||||||
#include "TCPHandler.h"
|
#include "TCPHandler.h"
|
||||||
|
|
||||||
|
#if !defined(ARCADIA_BUILD)
|
||||||
|
# include <Common/config_version.h>
|
||||||
|
#endif
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
30
programs/server/ya.make
Normal file
30
programs/server/ya.make
Normal file
@ -0,0 +1,30 @@
|
|||||||
|
PROGRAM(clickhouse-server)
|
||||||
|
|
||||||
|
PEERDIR(
|
||||||
|
clickhouse/base/common
|
||||||
|
clickhouse/base/daemon
|
||||||
|
clickhouse/base/loggers
|
||||||
|
clickhouse/src
|
||||||
|
contrib/libs/poco/NetSSL_OpenSSL
|
||||||
|
)
|
||||||
|
|
||||||
|
SRCS(
|
||||||
|
clickhouse-server.cpp
|
||||||
|
|
||||||
|
HTTPHandler.cpp
|
||||||
|
HTTPHandlerFactory.cpp
|
||||||
|
InterserverIOHTTPHandler.cpp
|
||||||
|
MetricsTransmitter.cpp
|
||||||
|
MySQLHandler.cpp
|
||||||
|
MySQLHandlerFactory.cpp
|
||||||
|
NotFoundHandler.cpp
|
||||||
|
PingRequestHandler.cpp
|
||||||
|
PrometheusMetricsWriter.cpp
|
||||||
|
PrometheusRequestHandler.cpp
|
||||||
|
ReplicasStatusHandler.cpp
|
||||||
|
RootRequestHandler.cpp
|
||||||
|
Server.cpp
|
||||||
|
TCPHandler.cpp
|
||||||
|
)
|
||||||
|
|
||||||
|
END()
|
3
programs/ya.make
Normal file
3
programs/ya.make
Normal file
@ -0,0 +1,3 @@
|
|||||||
|
RECURSE(
|
||||||
|
server
|
||||||
|
)
|
@ -253,7 +253,7 @@ private:
|
|||||||
}
|
}
|
||||||
else
|
else
|
||||||
{
|
{
|
||||||
if (nodes.contains(keyword))
|
if (nodes.count(keyword))
|
||||||
throw Exception(keyword + " declared twice", ErrorCodes::LOGICAL_ERROR);
|
throw Exception(keyword + " declared twice", ErrorCodes::LOGICAL_ERROR);
|
||||||
node = std::make_unique<Node>(keyword, node_type);
|
node = std::make_unique<Node>(keyword, node_type);
|
||||||
nodes[node->keyword] = node.get();
|
nodes[node->keyword] = node.get();
|
||||||
@ -279,7 +279,7 @@ private:
|
|||||||
{
|
{
|
||||||
auto parent_node = std::make_unique<Node>(parent_keyword);
|
auto parent_node = std::make_unique<Node>(parent_keyword);
|
||||||
it_parent = nodes.emplace(parent_node->keyword, parent_node.get()).first;
|
it_parent = nodes.emplace(parent_node->keyword, parent_node.get()).first;
|
||||||
assert(!owned_nodes.contains(parent_node->keyword));
|
assert(!owned_nodes.count(parent_node->keyword));
|
||||||
std::string_view parent_keyword_as_string_view = parent_node->keyword;
|
std::string_view parent_keyword_as_string_view = parent_node->keyword;
|
||||||
owned_nodes[parent_keyword_as_string_view] = std::move(parent_node);
|
owned_nodes[parent_keyword_as_string_view] = std::move(parent_node);
|
||||||
}
|
}
|
||||||
@ -299,9 +299,9 @@ private:
|
|||||||
|
|
||||||
#undef MAKE_ACCESS_FLAGS_TO_KEYWORD_TREE_NODE
|
#undef MAKE_ACCESS_FLAGS_TO_KEYWORD_TREE_NODE
|
||||||
|
|
||||||
if (!owned_nodes.contains("NONE"))
|
if (!owned_nodes.count("NONE"))
|
||||||
throw Exception("'NONE' not declared", ErrorCodes::LOGICAL_ERROR);
|
throw Exception("'NONE' not declared", ErrorCodes::LOGICAL_ERROR);
|
||||||
if (!owned_nodes.contains("ALL"))
|
if (!owned_nodes.count("ALL"))
|
||||||
throw Exception("'ALL' not declared", ErrorCodes::LOGICAL_ERROR);
|
throw Exception("'ALL' not declared", ErrorCodes::LOGICAL_ERROR);
|
||||||
|
|
||||||
flags_to_keyword_tree = std::move(owned_nodes["ALL"]);
|
flags_to_keyword_tree = std::move(owned_nodes["ALL"]);
|
||||||
|
@ -147,9 +147,9 @@ void ContextAccess::setUser(const UserPtr & user_) const
|
|||||||
current_roles.reserve(params.current_roles.size());
|
current_roles.reserve(params.current_roles.size());
|
||||||
for (const auto & id : params.current_roles)
|
for (const auto & id : params.current_roles)
|
||||||
{
|
{
|
||||||
if (user->granted_roles.contains(id))
|
if (user->granted_roles.count(id))
|
||||||
current_roles.push_back(id);
|
current_roles.push_back(id);
|
||||||
if (user->granted_roles_with_admin_option.contains(id))
|
if (user->granted_roles_with_admin_option.count(id))
|
||||||
current_roles_with_admin_option.push_back(id);
|
current_roles_with_admin_option.push_back(id);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -358,7 +358,7 @@ void ContextAccess::checkAdminOption(const UUID & role_id) const
|
|||||||
return;
|
return;
|
||||||
|
|
||||||
auto roles_with_admin_option_loaded = roles_with_admin_option.load();
|
auto roles_with_admin_option_loaded = roles_with_admin_option.load();
|
||||||
if (roles_with_admin_option_loaded && roles_with_admin_option_loaded->contains(role_id))
|
if (roles_with_admin_option_loaded && roles_with_admin_option_loaded->count(role_id))
|
||||||
return;
|
return;
|
||||||
|
|
||||||
std::optional<String> role_name = manager->readName(role_id);
|
std::optional<String> role_name = manager->readName(role_id);
|
||||||
|
@ -560,7 +560,7 @@ std::vector<UUID> DiskAccessStorage::findAllImpl(std::type_index type) const
|
|||||||
bool DiskAccessStorage::existsImpl(const UUID & id) const
|
bool DiskAccessStorage::existsImpl(const UUID & id) const
|
||||||
{
|
{
|
||||||
std::lock_guard lock{mutex};
|
std::lock_guard lock{mutex};
|
||||||
return id_to_entry_map.contains(id);
|
return id_to_entry_map.count(id);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@ -709,7 +709,7 @@ void DiskAccessStorage::updateNoLock(const UUID & id, const UpdateFunc & update_
|
|||||||
if (name_changed)
|
if (name_changed)
|
||||||
{
|
{
|
||||||
const auto & name_to_id_map = name_to_id_maps.at(type);
|
const auto & name_to_id_map = name_to_id_maps.at(type);
|
||||||
if (name_to_id_map.contains(new_name))
|
if (name_to_id_map.count(new_name))
|
||||||
throwNameCollisionCannotRename(type, String{old_name}, new_name);
|
throwNameCollisionCannotRename(type, String{old_name}, new_name);
|
||||||
scheduleWriteLists(type);
|
scheduleWriteLists(type);
|
||||||
}
|
}
|
||||||
|
@ -253,44 +253,44 @@ void ExtendedRoleSet::add(const boost::container::flat_set<UUID> & ids_)
|
|||||||
|
|
||||||
bool ExtendedRoleSet::match(const UUID & id) const
|
bool ExtendedRoleSet::match(const UUID & id) const
|
||||||
{
|
{
|
||||||
return (all || ids.contains(id)) && !except_ids.contains(id);
|
return (all || ids.count(id)) && !except_ids.count(id);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
bool ExtendedRoleSet::match(const UUID & user_id, const std::vector<UUID> & enabled_roles) const
|
bool ExtendedRoleSet::match(const UUID & user_id, const std::vector<UUID> & enabled_roles) const
|
||||||
{
|
{
|
||||||
if (!all && !ids.contains(user_id))
|
if (!all && !ids.count(user_id))
|
||||||
{
|
{
|
||||||
bool found_enabled_role = std::any_of(
|
bool found_enabled_role = std::any_of(
|
||||||
enabled_roles.begin(), enabled_roles.end(), [this](const UUID & enabled_role) { return ids.contains(enabled_role); });
|
enabled_roles.begin(), enabled_roles.end(), [this](const UUID & enabled_role) { return ids.count(enabled_role); });
|
||||||
if (!found_enabled_role)
|
if (!found_enabled_role)
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
|
||||||
if (except_ids.contains(user_id))
|
if (except_ids.count(user_id))
|
||||||
return false;
|
return false;
|
||||||
|
|
||||||
bool in_except_list = std::any_of(
|
bool in_except_list = std::any_of(
|
||||||
enabled_roles.begin(), enabled_roles.end(), [this](const UUID & enabled_role) { return except_ids.contains(enabled_role); });
|
enabled_roles.begin(), enabled_roles.end(), [this](const UUID & enabled_role) { return except_ids.count(enabled_role); });
|
||||||
return !in_except_list;
|
return !in_except_list;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
bool ExtendedRoleSet::match(const UUID & user_id, const boost::container::flat_set<UUID> & enabled_roles) const
|
bool ExtendedRoleSet::match(const UUID & user_id, const boost::container::flat_set<UUID> & enabled_roles) const
|
||||||
{
|
{
|
||||||
if (!all && !ids.contains(user_id))
|
if (!all && !ids.count(user_id))
|
||||||
{
|
{
|
||||||
bool found_enabled_role = std::any_of(
|
bool found_enabled_role = std::any_of(
|
||||||
enabled_roles.begin(), enabled_roles.end(), [this](const UUID & enabled_role) { return ids.contains(enabled_role); });
|
enabled_roles.begin(), enabled_roles.end(), [this](const UUID & enabled_role) { return ids.count(enabled_role); });
|
||||||
if (!found_enabled_role)
|
if (!found_enabled_role)
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
|
||||||
if (except_ids.contains(user_id))
|
if (except_ids.count(user_id))
|
||||||
return false;
|
return false;
|
||||||
|
|
||||||
bool in_except_list = std::any_of(
|
bool in_except_list = std::any_of(
|
||||||
enabled_roles.begin(), enabled_roles.end(), [this](const UUID & enabled_role) { return except_ids.contains(enabled_role); });
|
enabled_roles.begin(), enabled_roles.end(), [this](const UUID & enabled_role) { return except_ids.count(enabled_role); });
|
||||||
return !in_except_list;
|
return !in_except_list;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -250,16 +250,18 @@ void QuotaCache::quotaRemoved(const UUID & quota_id)
|
|||||||
void QuotaCache::chooseQuotaToConsume()
|
void QuotaCache::chooseQuotaToConsume()
|
||||||
{
|
{
|
||||||
/// `mutex` is already locked.
|
/// `mutex` is already locked.
|
||||||
std::erase_if(
|
|
||||||
enabled_quotas,
|
for (auto i = enabled_quotas.begin(), e = enabled_quotas.end(); i != e;)
|
||||||
[&](const std::pair<EnabledQuota::Params, std::weak_ptr<EnabledQuota>> & pr)
|
{
|
||||||
|
auto elem = i->second.lock();
|
||||||
|
if (!elem)
|
||||||
|
i = enabled_quotas.erase(i);
|
||||||
|
else
|
||||||
{
|
{
|
||||||
auto elem = pr.second.lock();
|
|
||||||
if (!elem)
|
|
||||||
return true; // remove from the `enabled_quotas` list.
|
|
||||||
chooseQuotaToConsumeFor(*elem);
|
chooseQuotaToConsumeFor(*elem);
|
||||||
return false; // keep in the `enabled_quotas` list.
|
++i;
|
||||||
});
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
void QuotaCache::chooseQuotaToConsumeFor(EnabledQuota & enabled)
|
void QuotaCache::chooseQuotaToConsumeFor(EnabledQuota & enabled)
|
||||||
|
@ -103,16 +103,17 @@ void RoleCache::collectRolesInfo()
|
|||||||
{
|
{
|
||||||
/// `mutex` is already locked.
|
/// `mutex` is already locked.
|
||||||
|
|
||||||
std::erase_if(
|
for (auto i = enabled_roles.begin(), e = enabled_roles.end(); i != e;)
|
||||||
enabled_roles,
|
{
|
||||||
[&](const std::pair<EnabledRoles::Params, std::weak_ptr<EnabledRoles>> & pr)
|
auto elem = i->second.lock();
|
||||||
|
if (!elem)
|
||||||
|
i = enabled_roles.erase(i);
|
||||||
|
else
|
||||||
{
|
{
|
||||||
auto elem = pr.second.lock();
|
|
||||||
if (!elem)
|
|
||||||
return true; // remove from the `enabled_roles` map.
|
|
||||||
collectRolesInfoFor(*elem);
|
collectRolesInfoFor(*elem);
|
||||||
return false; // keep in the `enabled_roles` map.
|
++i;
|
||||||
});
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@ -178,16 +178,17 @@ void RowPolicyCache::rowPolicyRemoved(const UUID & policy_id)
|
|||||||
void RowPolicyCache::mixConditions()
|
void RowPolicyCache::mixConditions()
|
||||||
{
|
{
|
||||||
/// `mutex` is already locked.
|
/// `mutex` is already locked.
|
||||||
std::erase_if(
|
for (auto i = enabled_row_policies.begin(), e = enabled_row_policies.end(); i != e;)
|
||||||
enabled_row_policies,
|
{
|
||||||
[&](const std::pair<EnabledRowPolicies::Params, std::weak_ptr<EnabledRowPolicies>> & pr)
|
auto elem = i->second.lock();
|
||||||
|
if (!elem)
|
||||||
|
i = enabled_row_policies.erase(i);
|
||||||
|
else
|
||||||
{
|
{
|
||||||
auto elem = pr.second.lock();
|
|
||||||
if (!elem)
|
|
||||||
return true; // remove from the `enabled_row_policies` map.
|
|
||||||
mixConditionsFor(*elem);
|
mixConditionsFor(*elem);
|
||||||
return false; // keep in the `enabled_row_policies` map.
|
++i;
|
||||||
});
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@ -104,16 +104,17 @@ void SettingsProfilesCache::setDefaultProfileName(const String & default_profile
|
|||||||
void SettingsProfilesCache::mergeSettingsAndConstraints()
|
void SettingsProfilesCache::mergeSettingsAndConstraints()
|
||||||
{
|
{
|
||||||
/// `mutex` is already locked.
|
/// `mutex` is already locked.
|
||||||
std::erase_if(
|
for (auto i = enabled_settings.begin(), e = enabled_settings.end(); i != e;)
|
||||||
enabled_settings,
|
{
|
||||||
[&](const std::pair<EnabledSettings::Params, std::weak_ptr<EnabledSettings>> & pr)
|
auto enabled = i->second.lock();
|
||||||
|
if (!enabled)
|
||||||
|
i = enabled_settings.erase(i);
|
||||||
|
else
|
||||||
{
|
{
|
||||||
auto enabled = pr.second.lock();
|
|
||||||
if (!enabled)
|
|
||||||
return true; // remove from the `enabled_settings` list.
|
|
||||||
mergeSettingsAndConstraintsFor(*enabled);
|
mergeSettingsAndConstraintsFor(*enabled);
|
||||||
return false; // keep in the `enabled_settings` list.
|
++i;
|
||||||
});
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@ -161,7 +162,7 @@ void SettingsProfilesCache::substituteProfiles(SettingsProfileElements & element
|
|||||||
|
|
||||||
auto parent_profile_id = *element.parent_profile;
|
auto parent_profile_id = *element.parent_profile;
|
||||||
element.parent_profile.reset();
|
element.parent_profile.reset();
|
||||||
if (already_substituted.contains(parent_profile_id))
|
if (already_substituted.count(parent_profile_id))
|
||||||
continue;
|
continue;
|
||||||
|
|
||||||
already_substituted.insert(parent_profile_id);
|
already_substituted.insert(parent_profile_id);
|
||||||
|
40
src/Access/ya.make
Normal file
40
src/Access/ya.make
Normal file
@ -0,0 +1,40 @@
|
|||||||
|
LIBRARY()
|
||||||
|
|
||||||
|
PEERDIR(
|
||||||
|
clickhouse/src/Common
|
||||||
|
)
|
||||||
|
|
||||||
|
SRCS(
|
||||||
|
AccessControlManager.cpp
|
||||||
|
AccessRights.cpp
|
||||||
|
AccessRightsElement.cpp
|
||||||
|
AllowedClientHosts.cpp
|
||||||
|
Authentication.cpp
|
||||||
|
ContextAccess.cpp
|
||||||
|
DiskAccessStorage.cpp
|
||||||
|
EnabledQuota.cpp
|
||||||
|
EnabledRoles.cpp
|
||||||
|
EnabledRolesInfo.cpp
|
||||||
|
EnabledRowPolicies.cpp
|
||||||
|
EnabledSettings.cpp
|
||||||
|
ExtendedRoleSet.cpp
|
||||||
|
IAccessEntity.cpp
|
||||||
|
IAccessStorage.cpp
|
||||||
|
MemoryAccessStorage.cpp
|
||||||
|
MultipleAccessStorage.cpp
|
||||||
|
Quota.cpp
|
||||||
|
QuotaCache.cpp
|
||||||
|
QuotaUsageInfo.cpp
|
||||||
|
Role.cpp
|
||||||
|
RoleCache.cpp
|
||||||
|
RowPolicy.cpp
|
||||||
|
RowPolicyCache.cpp
|
||||||
|
SettingsConstraints.cpp
|
||||||
|
SettingsProfile.cpp
|
||||||
|
SettingsProfileElement.cpp
|
||||||
|
SettingsProfilesCache.cpp
|
||||||
|
User.cpp
|
||||||
|
UsersConfigAccessStorage.cpp
|
||||||
|
)
|
||||||
|
|
||||||
|
END()
|
@ -32,7 +32,9 @@ void registerAggregateFunctions()
|
|||||||
registerAggregateFunctionUniqUpTo(factory);
|
registerAggregateFunctionUniqUpTo(factory);
|
||||||
registerAggregateFunctionTopK(factory);
|
registerAggregateFunctionTopK(factory);
|
||||||
registerAggregateFunctionsBitwise(factory);
|
registerAggregateFunctionsBitwise(factory);
|
||||||
|
#if !defined(ARCADIA_BUILD)
|
||||||
registerAggregateFunctionsBitmap(factory);
|
registerAggregateFunctionsBitmap(factory);
|
||||||
|
#endif
|
||||||
registerAggregateFunctionsMaxIntersections(factory);
|
registerAggregateFunctionsMaxIntersections(factory);
|
||||||
registerAggregateFunctionHistogram(factory);
|
registerAggregateFunctionHistogram(factory);
|
||||||
registerAggregateFunctionRetention(factory);
|
registerAggregateFunctionRetention(factory);
|
||||||
|
54
src/AggregateFunctions/ya.make
Normal file
54
src/AggregateFunctions/ya.make
Normal file
@ -0,0 +1,54 @@
|
|||||||
|
LIBRARY()
|
||||||
|
|
||||||
|
PEERDIR(
|
||||||
|
clickhouse/src/Common
|
||||||
|
)
|
||||||
|
|
||||||
|
SRCS(
|
||||||
|
AggregateFunctionAggThrow.cpp
|
||||||
|
AggregateFunctionArray.cpp
|
||||||
|
AggregateFunctionAvg.cpp
|
||||||
|
AggregateFunctionAvgWeighted.cpp
|
||||||
|
AggregateFunctionBitwise.cpp
|
||||||
|
AggregateFunctionBoundingRatio.cpp
|
||||||
|
AggregateFunctionCategoricalInformationValue.cpp
|
||||||
|
AggregateFunctionCombinatorFactory.cpp
|
||||||
|
AggregateFunctionCount.cpp
|
||||||
|
AggregateFunctionEntropy.cpp
|
||||||
|
AggregateFunctionFactory.cpp
|
||||||
|
AggregateFunctionForEach.cpp
|
||||||
|
AggregateFunctionGroupArray.cpp
|
||||||
|
AggregateFunctionGroupArrayInsertAt.cpp
|
||||||
|
AggregateFunctionGroupArrayMoving.cpp
|
||||||
|
AggregateFunctionGroupUniqArray.cpp
|
||||||
|
AggregateFunctionHistogram.cpp
|
||||||
|
AggregateFunctionIf.cpp
|
||||||
|
AggregateFunctionMaxIntersections.cpp
|
||||||
|
AggregateFunctionMerge.cpp
|
||||||
|
AggregateFunctionMinMaxAny.cpp
|
||||||
|
AggregateFunctionMLMethod.cpp
|
||||||
|
AggregateFunctionNull.cpp
|
||||||
|
AggregateFunctionOrFill.cpp
|
||||||
|
AggregateFunctionQuantile.cpp
|
||||||
|
AggregateFunctionResample.cpp
|
||||||
|
AggregateFunctionRetention.cpp
|
||||||
|
AggregateFunctionSequenceMatch.cpp
|
||||||
|
AggregateFunctionSimpleLinearRegression.cpp
|
||||||
|
AggregateFunctionState.cpp
|
||||||
|
AggregateFunctionStatistics.cpp
|
||||||
|
AggregateFunctionStatisticsSimple.cpp
|
||||||
|
AggregateFunctionSum.cpp
|
||||||
|
AggregateFunctionSumMap.cpp
|
||||||
|
AggregateFunctionTimeSeriesGroupSum.cpp
|
||||||
|
AggregateFunctionTopK.cpp
|
||||||
|
AggregateFunctionUniq.cpp
|
||||||
|
AggregateFunctionUniqCombined.cpp
|
||||||
|
AggregateFunctionUniqUpTo.cpp
|
||||||
|
AggregateFunctionWindowFunnel.cpp
|
||||||
|
parseAggregateFunctionParameters.cpp
|
||||||
|
registerAggregateFunctions.cpp
|
||||||
|
UniqCombinedBiasData.cpp
|
||||||
|
UniqVariadicHash.cpp
|
||||||
|
)
|
||||||
|
|
||||||
|
END()
|
@ -19,16 +19,19 @@
|
|||||||
#include <Common/CurrentMetrics.h>
|
#include <Common/CurrentMetrics.h>
|
||||||
#include <Common/DNSResolver.h>
|
#include <Common/DNSResolver.h>
|
||||||
#include <Common/StringUtils/StringUtils.h>
|
#include <Common/StringUtils/StringUtils.h>
|
||||||
#include <Common/config_version.h>
|
|
||||||
#include <Interpreters/ClientInfo.h>
|
#include <Interpreters/ClientInfo.h>
|
||||||
#include <Compression/CompressionFactory.h>
|
#include <Compression/CompressionFactory.h>
|
||||||
#include <Processors/Pipe.h>
|
#include <Processors/Pipe.h>
|
||||||
#include <Processors/ISink.h>
|
#include <Processors/ISink.h>
|
||||||
#include <Processors/Executors/PipelineExecutor.h>
|
#include <Processors/Executors/PipelineExecutor.h>
|
||||||
|
|
||||||
#include <Common/config.h>
|
#if !defined(ARCADIA_BUILD)
|
||||||
|
# include <Common/config_version.h>
|
||||||
|
# include <Common/config.h>
|
||||||
|
#endif
|
||||||
|
|
||||||
#if USE_POCO_NETSSL
|
#if USE_POCO_NETSSL
|
||||||
#include <Poco/Net/SecureStreamSocket.h>
|
# include <Poco/Net/SecureStreamSocket.h>
|
||||||
#endif
|
#endif
|
||||||
|
|
||||||
namespace CurrentMetrics
|
namespace CurrentMetrics
|
||||||
|
15
src/Client/ya.make
Normal file
15
src/Client/ya.make
Normal file
@ -0,0 +1,15 @@
|
|||||||
|
LIBRARY()
|
||||||
|
|
||||||
|
PEERDIR(
|
||||||
|
clickhouse/src/Common
|
||||||
|
contrib/libs/poco/NetSSL_OpenSSL
|
||||||
|
)
|
||||||
|
|
||||||
|
SRCS(
|
||||||
|
Connection.cpp
|
||||||
|
ConnectionPoolWithFailover.cpp
|
||||||
|
MultiplexedConnections.cpp
|
||||||
|
TimeoutSetter.cpp
|
||||||
|
)
|
||||||
|
|
||||||
|
END()
|
@ -1,17 +1,19 @@
|
|||||||
#include <Columns/Collator.h>
|
#include <Columns/Collator.h>
|
||||||
|
|
||||||
#include "config_core.h"
|
#if !defined(ARCADIA_BUILD)
|
||||||
|
# include "config_core.h"
|
||||||
|
#endif
|
||||||
|
|
||||||
#if USE_ICU
|
#if USE_ICU
|
||||||
#include <unicode/ucol.h>
|
# include <unicode/locid.h>
|
||||||
#include <unicode/unistr.h>
|
# include <unicode/ucnv.h>
|
||||||
#include <unicode/locid.h>
|
# include <unicode/ucol.h>
|
||||||
#include <unicode/ucnv.h>
|
# include <unicode/unistr.h>
|
||||||
#else
|
#else
|
||||||
#ifdef __clang__
|
# if defined(__clang__)
|
||||||
#pragma clang diagnostic ignored "-Wunused-private-field"
|
# pragma clang diagnostic ignored "-Wunused-private-field"
|
||||||
#pragma clang diagnostic ignored "-Wmissing-noreturn"
|
# pragma clang diagnostic ignored "-Wmissing-noreturn"
|
||||||
#endif
|
# endif
|
||||||
#endif
|
#endif
|
||||||
|
|
||||||
#include <Common/Exception.h>
|
#include <Common/Exception.h>
|
||||||
|
34
src/Columns/ya.make
Normal file
34
src/Columns/ya.make
Normal file
@ -0,0 +1,34 @@
|
|||||||
|
LIBRARY()
|
||||||
|
|
||||||
|
ADDINCL(
|
||||||
|
contrib/libs/icu/common
|
||||||
|
contrib/libs/icu/i18n
|
||||||
|
contrib/libs/pdqsort
|
||||||
|
)
|
||||||
|
|
||||||
|
PEERDIR(
|
||||||
|
clickhouse/src/Common
|
||||||
|
contrib/libs/icu
|
||||||
|
contrib/libs/pdqsort
|
||||||
|
)
|
||||||
|
|
||||||
|
SRCS(
|
||||||
|
Collator.cpp
|
||||||
|
ColumnAggregateFunction.cpp
|
||||||
|
ColumnArray.cpp
|
||||||
|
ColumnConst.cpp
|
||||||
|
ColumnDecimal.cpp
|
||||||
|
ColumnFixedString.cpp
|
||||||
|
ColumnFunction.cpp
|
||||||
|
ColumnLowCardinality.cpp
|
||||||
|
ColumnNullable.cpp
|
||||||
|
ColumnsCommon.cpp
|
||||||
|
ColumnString.cpp
|
||||||
|
ColumnTuple.cpp
|
||||||
|
ColumnVector.cpp
|
||||||
|
FilterDescription.cpp
|
||||||
|
getLeastSuperColumn.cpp
|
||||||
|
IColumn.cpp
|
||||||
|
)
|
||||||
|
|
||||||
|
END()
|
@ -1,5 +1,8 @@
|
|||||||
#include <Common/ClickHouseRevision.h>
|
#include <Common/ClickHouseRevision.h>
|
||||||
#include <Common/config_version.h>
|
|
||||||
|
#if !defined(ARCADIA_BUILD)
|
||||||
|
# include <Common/config_version.h>
|
||||||
|
#endif
|
||||||
|
|
||||||
namespace ClickHouseRevision
|
namespace ClickHouseRevision
|
||||||
{
|
{
|
||||||
|
@ -8,11 +8,14 @@
|
|||||||
#include <IO/Operators.h>
|
#include <IO/Operators.h>
|
||||||
#include <IO/ReadBufferFromString.h>
|
#include <IO/ReadBufferFromString.h>
|
||||||
#include <common/demangle.h>
|
#include <common/demangle.h>
|
||||||
#include <Common/config_version.h>
|
|
||||||
#include <Common/formatReadable.h>
|
#include <Common/formatReadable.h>
|
||||||
#include <Common/filesystemHelpers.h>
|
#include <Common/filesystemHelpers.h>
|
||||||
#include <filesystem>
|
#include <filesystem>
|
||||||
|
|
||||||
|
#if !defined(ARCADIA_BUILD)
|
||||||
|
# include <Common/config_version.h>
|
||||||
|
#endif
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
|
@ -1,8 +1,11 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
#include <Common/config.h>
|
|
||||||
#if USE_SSL
|
|
||||||
|
|
||||||
#include <Core/Types.h>
|
#if !defined(ARCADIA_BUILD)
|
||||||
|
# include <Common/config.h>
|
||||||
|
#endif
|
||||||
|
|
||||||
|
#if USE_SSL
|
||||||
|
# include <Core/Types.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
|
@ -5,12 +5,16 @@
|
|||||||
#include <memory>
|
#include <memory>
|
||||||
#include <optional>
|
#include <optional>
|
||||||
#include <Common/StringSearcher.h>
|
#include <Common/StringSearcher.h>
|
||||||
#include <Common/config.h>
|
|
||||||
#include <re2/re2.h>
|
#include <re2/re2.h>
|
||||||
|
|
||||||
|
#if !defined(ARCADIA_BUILD)
|
||||||
|
# include <Common/config.h>
|
||||||
|
#endif
|
||||||
|
|
||||||
#if USE_RE2_ST
|
#if USE_RE2_ST
|
||||||
#include <re2_st/re2.h>
|
# include <re2_st/re2.h>
|
||||||
#else
|
#else
|
||||||
#define re2_st re2
|
# define re2_st re2
|
||||||
#endif
|
#endif
|
||||||
|
|
||||||
|
|
||||||
|
@ -6,7 +6,6 @@
|
|||||||
#include <Common/TraceCollector.h>
|
#include <Common/TraceCollector.h>
|
||||||
#include <Common/thread_local_rng.h>
|
#include <Common/thread_local_rng.h>
|
||||||
#include <common/StringRef.h>
|
#include <common/StringRef.h>
|
||||||
#include <common/config_common.h>
|
|
||||||
#include <common/logger_useful.h>
|
#include <common/logger_useful.h>
|
||||||
#include <common/phdr_cache.h>
|
#include <common/phdr_cache.h>
|
||||||
|
|
||||||
|
@ -1,11 +1,14 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include <Core/Types.h>
|
#include <Core/Types.h>
|
||||||
#include <Common/config.h>
|
|
||||||
#include <common/config_common.h>
|
|
||||||
#include <signal.h>
|
#include <signal.h>
|
||||||
#include <time.h>
|
#include <time.h>
|
||||||
|
|
||||||
|
#if !defined(ARCADIA_BUILD)
|
||||||
|
# include <Common/config.h>
|
||||||
|
# include <common/config_common.h>
|
||||||
|
#endif
|
||||||
|
|
||||||
|
|
||||||
namespace Poco
|
namespace Poco
|
||||||
{
|
{
|
||||||
|
@ -3,7 +3,6 @@
|
|||||||
#include <Common/Dwarf.h>
|
#include <Common/Dwarf.h>
|
||||||
#include <Common/Elf.h>
|
#include <Common/Elf.h>
|
||||||
#include <Common/SymbolIndex.h>
|
#include <Common/SymbolIndex.h>
|
||||||
#include <Common/config.h>
|
|
||||||
#include <Common/MemorySanitizer.h>
|
#include <Common/MemorySanitizer.h>
|
||||||
#include <common/SimpleCache.h>
|
#include <common/SimpleCache.h>
|
||||||
#include <common/demangle.h>
|
#include <common/demangle.h>
|
||||||
@ -14,8 +13,12 @@
|
|||||||
#include <sstream>
|
#include <sstream>
|
||||||
#include <unordered_map>
|
#include <unordered_map>
|
||||||
|
|
||||||
|
#if !defined(ARCADIA_BUILD)
|
||||||
|
# include <Common/config.h>
|
||||||
|
#endif
|
||||||
|
|
||||||
#if USE_UNWIND
|
#if USE_UNWIND
|
||||||
# include <libunwind.h>
|
# include <libunwind.h>
|
||||||
#endif
|
#endif
|
||||||
|
|
||||||
std::string signalToErrorMessage(int sig, const siginfo_t & info, const ucontext_t & context)
|
std::string signalToErrorMessage(int sig, const siginfo_t & info, const ucontext_t & context)
|
||||||
|
@ -13,10 +13,13 @@
|
|||||||
#include <Poco/URI.h>
|
#include <Poco/URI.h>
|
||||||
#include <Poco/Util/AbstractConfiguration.h>
|
#include <Poco/Util/AbstractConfiguration.h>
|
||||||
#include <Common/ShellCommand.h>
|
#include <Common/ShellCommand.h>
|
||||||
#include <Common/config.h>
|
|
||||||
#include <common/logger_useful.h>
|
#include <common/logger_useful.h>
|
||||||
#include <ext/range.h>
|
#include <ext/range.h>
|
||||||
|
|
||||||
|
#if !defined(ARCADIA_BUILD)
|
||||||
|
# include <Common/config.h>
|
||||||
|
#endif
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
|
@ -1,11 +1,16 @@
|
|||||||
#include <Common/getNumberOfPhysicalCPUCores.h>
|
#include <Common/getNumberOfPhysicalCPUCores.h>
|
||||||
#include <thread>
|
#include <thread>
|
||||||
|
|
||||||
#include <Common/config.h>
|
#if !defined(ARCADIA_BUILD)
|
||||||
|
# include <Common/config.h>
|
||||||
|
#else
|
||||||
|
# include <libcpuid/libcpuid.h>
|
||||||
|
#endif
|
||||||
|
|
||||||
#if USE_CPUID
|
#if USE_CPUID
|
||||||
# include <libcpuid/libcpuid.h>
|
# include <libcpuid/libcpuid.h>
|
||||||
#elif USE_CPUINFO
|
#elif USE_CPUINFO
|
||||||
# include <cpuinfo.h>
|
# include <cpuinfo.h>
|
||||||
#endif
|
#endif
|
||||||
|
|
||||||
|
|
||||||
|
@ -1,4 +1,3 @@
|
|||||||
#include <common/config_common.h>
|
|
||||||
#include <common/memory.h>
|
#include <common/memory.h>
|
||||||
#include <Common/MemoryTracker.h>
|
#include <Common/MemoryTracker.h>
|
||||||
|
|
||||||
@ -14,14 +13,13 @@
|
|||||||
/// Replace default new/delete with memory tracking versions.
|
/// Replace default new/delete with memory tracking versions.
|
||||||
/// @sa https://en.cppreference.com/w/cpp/memory/new/operator_new
|
/// @sa https://en.cppreference.com/w/cpp/memory/new/operator_new
|
||||||
/// https://en.cppreference.com/w/cpp/memory/new/operator_delete
|
/// https://en.cppreference.com/w/cpp/memory/new/operator_delete
|
||||||
#if !UNBUNDLED
|
|
||||||
|
|
||||||
namespace Memory
|
namespace Memory
|
||||||
{
|
{
|
||||||
|
|
||||||
inline ALWAYS_INLINE void trackMemory(std::size_t size)
|
inline ALWAYS_INLINE void trackMemory(std::size_t size)
|
||||||
{
|
{
|
||||||
#if USE_JEMALLOC
|
#if USE_JEMALLOC && JEMALLOC_VERSION_MAJOR >= 5
|
||||||
/// The nallocx() function allocates no memory, but it performs the same size computation as the mallocx() function
|
/// The nallocx() function allocates no memory, but it performs the same size computation as the mallocx() function
|
||||||
/// @note je_mallocx() != je_malloc(). It's expected they don't differ much in allocation logic.
|
/// @note je_mallocx() != je_malloc(). It's expected they don't differ much in allocation logic.
|
||||||
if (likely(size != 0))
|
if (likely(size != 0))
|
||||||
@ -49,18 +47,18 @@ inline ALWAYS_INLINE void untrackMemory(void * ptr [[maybe_unused]], std::size_t
|
|||||||
{
|
{
|
||||||
try
|
try
|
||||||
{
|
{
|
||||||
#if USE_JEMALLOC
|
#if USE_JEMALLOC && JEMALLOC_VERSION_MAJOR >= 5
|
||||||
/// @note It's also possible to use je_malloc_usable_size() here.
|
/// @note It's also possible to use je_malloc_usable_size() here.
|
||||||
if (likely(ptr != nullptr))
|
if (likely(ptr != nullptr))
|
||||||
CurrentMemoryTracker::free(sallocx(ptr, 0));
|
CurrentMemoryTracker::free(sallocx(ptr, 0));
|
||||||
#else
|
#else
|
||||||
if (size)
|
if (size)
|
||||||
CurrentMemoryTracker::free(size);
|
CurrentMemoryTracker::free(size);
|
||||||
# ifdef _GNU_SOURCE
|
# if defined(_GNU_SOURCE)
|
||||||
/// It's innaccurate resource free for sanitizers. malloc_usable_size() result is greater or equal to allocated size.
|
/// It's innaccurate resource free for sanitizers. malloc_usable_size() result is greater or equal to allocated size.
|
||||||
else
|
else
|
||||||
CurrentMemoryTracker::free(malloc_usable_size(ptr));
|
CurrentMemoryTracker::free(malloc_usable_size(ptr));
|
||||||
# endif
|
# endif
|
||||||
#endif
|
#endif
|
||||||
}
|
}
|
||||||
catch (...)
|
catch (...)
|
||||||
@ -130,26 +128,3 @@ void operator delete[](void * ptr, std::size_t size) noexcept
|
|||||||
Memory::untrackMemory(ptr, size);
|
Memory::untrackMemory(ptr, size);
|
||||||
Memory::deleteSized(ptr, size);
|
Memory::deleteSized(ptr, size);
|
||||||
}
|
}
|
||||||
|
|
||||||
#else
|
|
||||||
|
|
||||||
/// new
|
|
||||||
|
|
||||||
void * operator new(std::size_t size) { return Memory::newImpl(size); }
|
|
||||||
void * operator new[](std::size_t size) { return Memory::newImpl(size); }
|
|
||||||
|
|
||||||
void * operator new(std::size_t size, const std::nothrow_t &) noexcept { return Memory::newNoExept(size); }
|
|
||||||
void * operator new[](std::size_t size, const std::nothrow_t &) noexcept { return Memory::newNoExept(size); }
|
|
||||||
|
|
||||||
/// delete
|
|
||||||
|
|
||||||
void operator delete(void * ptr) noexcept { Memory::deleteImpl(ptr); }
|
|
||||||
void operator delete[](void * ptr) noexcept { Memory::deleteImpl(ptr); }
|
|
||||||
|
|
||||||
void operator delete(void * ptr, const std::nothrow_t &) noexcept { Memory::deleteImpl(ptr); }
|
|
||||||
void operator delete[](void * ptr, const std::nothrow_t &) noexcept { Memory::deleteImpl(ptr); }
|
|
||||||
|
|
||||||
void operator delete(void * ptr, std::size_t size) noexcept { Memory::deleteSized(ptr, size); }
|
|
||||||
void operator delete[](void * ptr, std::size_t size) noexcept { Memory::deleteSized(ptr, size); }
|
|
||||||
|
|
||||||
#endif
|
|
||||||
|
109
src/Common/ya.make
Normal file
109
src/Common/ya.make
Normal file
@ -0,0 +1,109 @@
|
|||||||
|
LIBRARY()
|
||||||
|
|
||||||
|
ADDINCL (
|
||||||
|
GLOBAL clickhouse/src
|
||||||
|
contrib/libs/libcpuid
|
||||||
|
contrib/libs/libunwind/include
|
||||||
|
GLOBAL contrib/restricted/ryu
|
||||||
|
)
|
||||||
|
|
||||||
|
PEERDIR(
|
||||||
|
clickhouse/base/common
|
||||||
|
clickhouse/base/pcg-random
|
||||||
|
clickhouse/base/widechar_width
|
||||||
|
contrib/libs/libcpuid/libcpuid
|
||||||
|
contrib/libs/openssl
|
||||||
|
contrib/libs/re2
|
||||||
|
contrib/restricted/ryu
|
||||||
|
)
|
||||||
|
|
||||||
|
# TODO: stub for config_version.h
|
||||||
|
CFLAGS (GLOBAL -DDBMS_NAME=\"ClickHouse\")
|
||||||
|
CFLAGS (GLOBAL -DDBMS_VERSION_MAJOR=0)
|
||||||
|
CFLAGS (GLOBAL -DDBMS_VERSION_MINOR=0)
|
||||||
|
CFLAGS (GLOBAL -DDBMS_VERSION_PATCH=0)
|
||||||
|
CFLAGS (GLOBAL -DVERSION_FULL=\"Clickhouse\")
|
||||||
|
CFLAGS (GLOBAL -DVERSION_INTEGER=0)
|
||||||
|
CFLAGS (GLOBAL -DVERSION_NAME=\"Clickhouse\")
|
||||||
|
CFLAGS (GLOBAL -DVERSION_OFFICIAL=\"\\\(arcadia\\\)\")
|
||||||
|
CFLAGS (GLOBAL -DVERSION_REVISION=0)
|
||||||
|
CFLAGS (GLOBAL -DVERSION_STRING=\"Unknown\")
|
||||||
|
|
||||||
|
SRCS(
|
||||||
|
ActionLock.cpp
|
||||||
|
AlignedBuffer.cpp
|
||||||
|
checkStackSize.cpp
|
||||||
|
ClickHouseRevision.cpp
|
||||||
|
Config/AbstractConfigurationComparison.cpp
|
||||||
|
Config/ConfigProcessor.cpp
|
||||||
|
Config/configReadClient.cpp
|
||||||
|
Config/ConfigReloader.cpp
|
||||||
|
createHardLink.cpp
|
||||||
|
CurrentMetrics.cpp
|
||||||
|
CurrentThread.cpp
|
||||||
|
DNSResolver.cpp
|
||||||
|
Dwarf.cpp
|
||||||
|
Elf.cpp
|
||||||
|
ErrorCodes.cpp
|
||||||
|
escapeForFileName.cpp
|
||||||
|
Exception.cpp
|
||||||
|
ExternalLoaderStatus.cpp
|
||||||
|
FieldVisitors.cpp
|
||||||
|
FileChecker.cpp
|
||||||
|
filesystemHelpers.cpp
|
||||||
|
formatIPv6.cpp
|
||||||
|
formatReadable.cpp
|
||||||
|
getExecutablePath.cpp
|
||||||
|
getMultipleKeysFromConfig.cpp
|
||||||
|
getNumberOfPhysicalCPUCores.cpp
|
||||||
|
hasLinuxCapability.cpp
|
||||||
|
hex.cpp
|
||||||
|
IntervalKind.cpp
|
||||||
|
IPv6ToBinary.cpp
|
||||||
|
isLocalAddress.cpp
|
||||||
|
Macros.cpp
|
||||||
|
malloc.cpp
|
||||||
|
MemoryTracker.cpp
|
||||||
|
new_delete.cpp
|
||||||
|
OptimizedRegularExpression.cpp
|
||||||
|
parseAddress.cpp
|
||||||
|
parseGlobs.cpp
|
||||||
|
parseRemoteDescription.cpp
|
||||||
|
PipeFDs.cpp
|
||||||
|
PODArray.cpp
|
||||||
|
ProfileEvents.cpp
|
||||||
|
QueryProfiler.cpp
|
||||||
|
quoteString.cpp
|
||||||
|
randomSeed.cpp
|
||||||
|
RemoteHostFilter.cpp
|
||||||
|
RWLock.cpp
|
||||||
|
SensitiveDataMasker.cpp
|
||||||
|
setThreadName.cpp
|
||||||
|
SharedLibrary.cpp
|
||||||
|
ShellCommand.cpp
|
||||||
|
StackTrace.cpp
|
||||||
|
StatusFile.cpp
|
||||||
|
StatusInfo.cpp
|
||||||
|
Stopwatch.cpp
|
||||||
|
StringUtils/StringUtils.cpp
|
||||||
|
StudentTTest.cpp
|
||||||
|
SymbolIndex.cpp
|
||||||
|
TaskStatsInfoGetter.cpp
|
||||||
|
TerminalSize.cpp
|
||||||
|
thread_local_rng.cpp
|
||||||
|
ThreadFuzzer.cpp
|
||||||
|
ThreadPool.cpp
|
||||||
|
ThreadStatus.cpp
|
||||||
|
TraceCollector.cpp
|
||||||
|
UTF8Helpers.cpp
|
||||||
|
WeakHash.cpp
|
||||||
|
ZooKeeper/IKeeper.cpp
|
||||||
|
ZooKeeper/Lock.cpp
|
||||||
|
ZooKeeper/TestKeeper.cpp
|
||||||
|
ZooKeeper/ZooKeeper.cpp
|
||||||
|
ZooKeeper/ZooKeeperHolder.cpp
|
||||||
|
ZooKeeper/ZooKeeperImpl.cpp
|
||||||
|
ZooKeeper/ZooKeeperNodeCache.cpp
|
||||||
|
)
|
||||||
|
|
||||||
|
END()
|
33
src/Compression/ya.make
Normal file
33
src/Compression/ya.make
Normal file
@ -0,0 +1,33 @@
|
|||||||
|
LIBRARY()
|
||||||
|
|
||||||
|
ADDINCL(
|
||||||
|
contrib/libs/lz4
|
||||||
|
contrib/libs/zstd
|
||||||
|
)
|
||||||
|
|
||||||
|
PEERDIR(
|
||||||
|
clickhouse/src/Common
|
||||||
|
contrib/libs/lz4
|
||||||
|
contrib/libs/zstd
|
||||||
|
)
|
||||||
|
|
||||||
|
SRCS(
|
||||||
|
CachedCompressedReadBuffer.cpp
|
||||||
|
CompressedReadBuffer.cpp
|
||||||
|
CompressedReadBufferBase.cpp
|
||||||
|
CompressedReadBufferFromFile.cpp
|
||||||
|
CompressedWriteBuffer.cpp
|
||||||
|
CompressionCodecDelta.cpp
|
||||||
|
CompressionCodecDoubleDelta.cpp
|
||||||
|
CompressionCodecGorilla.cpp
|
||||||
|
CompressionCodecLZ4.cpp
|
||||||
|
CompressionCodecMultiple.cpp
|
||||||
|
CompressionCodecNone.cpp
|
||||||
|
CompressionCodecT64.cpp
|
||||||
|
CompressionCodecZSTD.cpp
|
||||||
|
CompressionFactory.cpp
|
||||||
|
ICompressionCodec.cpp
|
||||||
|
LZ4_decompress_faster.cpp
|
||||||
|
)
|
||||||
|
|
||||||
|
END()
|
@ -157,7 +157,9 @@ private:
|
|||||||
template <> struct NearestFieldTypeImpl<char> { using Type = std::conditional_t<is_signed_v<char>, Int64, UInt64>; };
|
template <> struct NearestFieldTypeImpl<char> { using Type = std::conditional_t<is_signed_v<char>, Int64, UInt64>; };
|
||||||
template <> struct NearestFieldTypeImpl<signed char> { using Type = Int64; };
|
template <> struct NearestFieldTypeImpl<signed char> { using Type = Int64; };
|
||||||
template <> struct NearestFieldTypeImpl<unsigned char> { using Type = UInt64; };
|
template <> struct NearestFieldTypeImpl<unsigned char> { using Type = UInt64; };
|
||||||
|
#if __cplusplus > 201703L
|
||||||
template <> struct NearestFieldTypeImpl<char8_t> { using Type = UInt64; };
|
template <> struct NearestFieldTypeImpl<char8_t> { using Type = UInt64; };
|
||||||
|
#endif
|
||||||
|
|
||||||
template <> struct NearestFieldTypeImpl<UInt16> { using Type = UInt64; };
|
template <> struct NearestFieldTypeImpl<UInt16> { using Type = UInt64; };
|
||||||
template <> struct NearestFieldTypeImpl<UInt32> { using Type = UInt64; };
|
template <> struct NearestFieldTypeImpl<UInt32> { using Type = UInt64; };
|
||||||
|
@ -22,10 +22,14 @@
|
|||||||
#include <Poco/Net/StreamSocket.h>
|
#include <Poco/Net/StreamSocket.h>
|
||||||
#include <Poco/RandomStream.h>
|
#include <Poco/RandomStream.h>
|
||||||
#include <Poco/SHA1Engine.h>
|
#include <Poco/SHA1Engine.h>
|
||||||
#include "config_core.h"
|
|
||||||
|
#if !defined(ARCADIA_BUILD)
|
||||||
|
# include "config_core.h"
|
||||||
|
#endif
|
||||||
|
|
||||||
#if USE_SSL
|
#if USE_SSL
|
||||||
#include <openssl/pem.h>
|
# include <openssl/pem.h>
|
||||||
#include <openssl/rsa.h>
|
# include <openssl/rsa.h>
|
||||||
#endif
|
#endif
|
||||||
|
|
||||||
/// Implementation of MySQL wire protocol.
|
/// Implementation of MySQL wire protocol.
|
||||||
|
@ -137,8 +137,12 @@ NamesAndTypesList NamesAndTypesList::filter(const Names & names) const
|
|||||||
|
|
||||||
NamesAndTypesList NamesAndTypesList::addTypes(const Names & names) const
|
NamesAndTypesList NamesAndTypesList::addTypes(const Names & names) const
|
||||||
{
|
{
|
||||||
/// NOTE It's better to make a map in `IStorage` than to create it here every time again.
|
/// NOTE: It's better to make a map in `IStorage` than to create it here every time again.
|
||||||
::google::dense_hash_map<StringRef, const DataTypePtr *, StringRefHash> types;
|
#if !defined(ARCADIA_BUILD)
|
||||||
|
google::dense_hash_map<StringRef, const DataTypePtr *, StringRefHash> types;
|
||||||
|
#else
|
||||||
|
google::sparsehash::dense_hash_map<StringRef, const DataTypePtr *, StringRefHash> types;
|
||||||
|
#endif
|
||||||
types.set_empty_key(StringRef());
|
types.set_empty_key(StringRef());
|
||||||
|
|
||||||
for (const NameAndTypePair & column : *this)
|
for (const NameAndTypePair & column : *this)
|
||||||
|
24
src/Core/ya.make
Normal file
24
src/Core/ya.make
Normal file
@ -0,0 +1,24 @@
|
|||||||
|
LIBRARY()
|
||||||
|
|
||||||
|
PEERDIR(
|
||||||
|
clickhouse/src/Common
|
||||||
|
contrib/libs/sparsehash
|
||||||
|
contrib/restricted/boost/libs
|
||||||
|
)
|
||||||
|
|
||||||
|
SRCS(
|
||||||
|
BackgroundSchedulePool.cpp
|
||||||
|
Block.cpp
|
||||||
|
BlockInfo.cpp
|
||||||
|
ColumnWithTypeAndName.cpp
|
||||||
|
ExternalResultDescription.cpp
|
||||||
|
ExternalTable.cpp
|
||||||
|
Field.cpp
|
||||||
|
iostream_debug_helpers.cpp
|
||||||
|
MySQLProtocol.cpp
|
||||||
|
NamesAndTypes.cpp
|
||||||
|
Settings.cpp
|
||||||
|
SettingsCollection.cpp
|
||||||
|
)
|
||||||
|
|
||||||
|
END()
|
71
src/DataStreams/ya.make
Normal file
71
src/DataStreams/ya.make
Normal file
@ -0,0 +1,71 @@
|
|||||||
|
LIBRARY()
|
||||||
|
|
||||||
|
PEERDIR(
|
||||||
|
clickhouse/src/Common
|
||||||
|
)
|
||||||
|
|
||||||
|
NO_COMPILER_WARNINGS()
|
||||||
|
|
||||||
|
SRCS(
|
||||||
|
AddingDefaultBlockOutputStream.cpp
|
||||||
|
AddingDefaultsBlockInputStream.cpp
|
||||||
|
AggregatingBlockInputStream.cpp
|
||||||
|
AggregatingSortedBlockInputStream.cpp
|
||||||
|
AsynchronousBlockInputStream.cpp
|
||||||
|
BlockIO.cpp
|
||||||
|
BlockStreamProfileInfo.cpp
|
||||||
|
CheckConstraintsBlockOutputStream.cpp
|
||||||
|
CheckSortedBlockInputStream.cpp
|
||||||
|
CollapsingFinalBlockInputStream.cpp
|
||||||
|
CollapsingSortedBlockInputStream.cpp
|
||||||
|
ColumnGathererStream.cpp
|
||||||
|
ConvertingBlockInputStream.cpp
|
||||||
|
copyData.cpp
|
||||||
|
CountingBlockOutputStream.cpp
|
||||||
|
CreatingSetsBlockInputStream.cpp
|
||||||
|
CubeBlockInputStream.cpp
|
||||||
|
DistinctBlockInputStream.cpp
|
||||||
|
DistinctSortedBlockInputStream.cpp
|
||||||
|
ExecutionSpeedLimits.cpp
|
||||||
|
ExpressionBlockInputStream.cpp
|
||||||
|
FillingBlockInputStream.cpp
|
||||||
|
FilterBlockInputStream.cpp
|
||||||
|
FilterColumnsBlockInputStream.cpp
|
||||||
|
finalizeBlock.cpp
|
||||||
|
FinishSortingBlockInputStream.cpp
|
||||||
|
GraphiteRollupSortedBlockInputStream.cpp
|
||||||
|
IBlockInputStream.cpp
|
||||||
|
InputStreamFromASTInsertQuery.cpp
|
||||||
|
InternalTextLogsRowOutputStream.cpp
|
||||||
|
LimitBlockInputStream.cpp
|
||||||
|
LimitByBlockInputStream.cpp
|
||||||
|
materializeBlock.cpp
|
||||||
|
MaterializingBlockInputStream.cpp
|
||||||
|
MergeSortingBlockInputStream.cpp
|
||||||
|
MergingAggregatedBlockInputStream.cpp
|
||||||
|
MergingAggregatedMemoryEfficientBlockInputStream.cpp
|
||||||
|
MergingSortedBlockInputStream.cpp
|
||||||
|
narrowBlockInputStreams.cpp
|
||||||
|
NativeBlockInputStream.cpp
|
||||||
|
NativeBlockOutputStream.cpp
|
||||||
|
ParallelAggregatingBlockInputStream.cpp
|
||||||
|
ParallelParsingBlockInputStream.cpp
|
||||||
|
PartialSortingBlockInputStream.cpp
|
||||||
|
processConstants.cpp
|
||||||
|
PushingToViewsBlockOutputStream.cpp
|
||||||
|
RemoteBlockInputStream.cpp
|
||||||
|
RemoteBlockOutputStream.cpp
|
||||||
|
ReplacingSortedBlockInputStream.cpp
|
||||||
|
ReverseBlockInputStream.cpp
|
||||||
|
RollupBlockInputStream.cpp
|
||||||
|
SizeLimits.cpp
|
||||||
|
SquashingBlockInputStream.cpp
|
||||||
|
SquashingBlockOutputStream.cpp
|
||||||
|
SquashingTransform.cpp
|
||||||
|
SummingSortedBlockInputStream.cpp
|
||||||
|
TotalsHavingBlockInputStream.cpp
|
||||||
|
TTLBlockInputStream.cpp
|
||||||
|
VersionedCollapsingSortedBlockInputStream.cpp
|
||||||
|
)
|
||||||
|
|
||||||
|
END()
|
@ -1,23 +1,25 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include "config_core.h"
|
#if !defined(ARCADIA_BUILD)
|
||||||
|
# include "config_core.h"
|
||||||
|
#endif
|
||||||
|
|
||||||
#if USE_EMBEDDED_COMPILER
|
#if USE_EMBEDDED_COMPILER
|
||||||
|
# include <DataTypes/DataTypeDate.h>
|
||||||
|
# include <DataTypes/DataTypeDateTime.h>
|
||||||
|
# include <DataTypes/DataTypeFixedString.h>
|
||||||
|
# include <DataTypes/DataTypeInterval.h>
|
||||||
|
# include <DataTypes/DataTypeNullable.h>
|
||||||
|
# include <DataTypes/DataTypeUUID.h>
|
||||||
|
# include <DataTypes/DataTypesNumber.h>
|
||||||
|
# include <Common/typeid_cast.h>
|
||||||
|
|
||||||
#include <Common/typeid_cast.h>
|
# pragma GCC diagnostic push
|
||||||
#include <DataTypes/DataTypeDate.h>
|
# pragma GCC diagnostic ignored "-Wunused-parameter"
|
||||||
#include <DataTypes/DataTypeDateTime.h>
|
|
||||||
#include <DataTypes/DataTypeFixedString.h>
|
|
||||||
#include <DataTypes/DataTypeInterval.h>
|
|
||||||
#include <DataTypes/DataTypeNullable.h>
|
|
||||||
#include <DataTypes/DataTypesNumber.h>
|
|
||||||
#include <DataTypes/DataTypeUUID.h>
|
|
||||||
|
|
||||||
#pragma GCC diagnostic push
|
# include <llvm/IR/IRBuilder.h>
|
||||||
#pragma GCC diagnostic ignored "-Wunused-parameter"
|
|
||||||
|
|
||||||
#include <llvm/IR/IRBuilder.h>
|
# pragma GCC diagnostic pop
|
||||||
|
|
||||||
#pragma GCC diagnostic pop
|
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
|
41
src/DataTypes/ya.make
Normal file
41
src/DataTypes/ya.make
Normal file
@ -0,0 +1,41 @@
|
|||||||
|
LIBRARY()
|
||||||
|
|
||||||
|
PEERDIR(
|
||||||
|
clickhouse/src/Common
|
||||||
|
clickhouse/src/Formats
|
||||||
|
)
|
||||||
|
|
||||||
|
SRCS(
|
||||||
|
convertMySQLDataType.cpp
|
||||||
|
DataTypeAggregateFunction.cpp
|
||||||
|
DataTypeArray.cpp
|
||||||
|
DataTypeCustomIPv4AndIPv6.cpp
|
||||||
|
DataTypeCustomSimpleAggregateFunction.cpp
|
||||||
|
DataTypeCustomSimpleTextSerialization.cpp
|
||||||
|
DataTypeDate.cpp
|
||||||
|
DataTypeDateTime.cpp
|
||||||
|
DataTypeDateTime64.cpp
|
||||||
|
DataTypeDecimalBase.cpp
|
||||||
|
DataTypeEnum.cpp
|
||||||
|
DataTypeFactory.cpp
|
||||||
|
DataTypeFixedString.cpp
|
||||||
|
DataTypeFunction.cpp
|
||||||
|
DataTypeInterval.cpp
|
||||||
|
DataTypeLowCardinality.cpp
|
||||||
|
DataTypeLowCardinalityHelpers.cpp
|
||||||
|
DataTypeNothing.cpp
|
||||||
|
DataTypeNullable.cpp
|
||||||
|
DataTypeNumberBase.cpp
|
||||||
|
DataTypesDecimal.cpp
|
||||||
|
DataTypesNumber.cpp
|
||||||
|
DataTypeString.cpp
|
||||||
|
DataTypeTuple.cpp
|
||||||
|
DataTypeUUID.cpp
|
||||||
|
FieldToDataType.cpp
|
||||||
|
getLeastSupertype.cpp
|
||||||
|
getMostSubtype.cpp
|
||||||
|
IDataType.cpp
|
||||||
|
NestedUtils.cpp
|
||||||
|
)
|
||||||
|
|
||||||
|
END()
|
@ -9,15 +9,16 @@
|
|||||||
#include <Parsers/ASTCreateQuery.h>
|
#include <Parsers/ASTCreateQuery.h>
|
||||||
#include <Parsers/ASTFunction.h>
|
#include <Parsers/ASTFunction.h>
|
||||||
#include <Common/parseAddress.h>
|
#include <Common/parseAddress.h>
|
||||||
#include "config_core.h"
|
|
||||||
#include "DatabaseFactory.h"
|
#include "DatabaseFactory.h"
|
||||||
#include <Poco/File.h>
|
#include <Poco/File.h>
|
||||||
|
|
||||||
|
#if !defined(ARCADIA_BUILD)
|
||||||
|
# include "config_core.h"
|
||||||
|
#endif
|
||||||
|
|
||||||
#if USE_MYSQL
|
#if USE_MYSQL
|
||||||
|
# include <Databases/DatabaseMySQL.h>
|
||||||
#include <Databases/DatabaseMySQL.h>
|
# include <Interpreters/evaluateConstantExpression.h>
|
||||||
#include <Interpreters/evaluateConstantExpression.h>
|
|
||||||
|
|
||||||
#endif
|
#endif
|
||||||
|
|
||||||
|
|
||||||
|
@ -1,29 +1,30 @@
|
|||||||
#include "config_core.h"
|
#if !defined(ARCADIA_BUILD)
|
||||||
|
# include "config_core.h"
|
||||||
|
#endif
|
||||||
|
|
||||||
#if USE_MYSQL
|
#if USE_MYSQL
|
||||||
|
# include <string>
|
||||||
|
# include <Core/SettingsCollection.h>
|
||||||
|
# include <DataTypes/DataTypeDateTime.h>
|
||||||
|
# include <DataTypes/DataTypeNullable.h>
|
||||||
|
# include <DataTypes/DataTypeString.h>
|
||||||
|
# include <DataTypes/DataTypesNumber.h>
|
||||||
|
# include <DataTypes/convertMySQLDataType.h>
|
||||||
|
# include <Databases/DatabaseMySQL.h>
|
||||||
|
# include <Formats/MySQLBlockInputStream.h>
|
||||||
|
# include <IO/Operators.h>
|
||||||
|
# include <Parsers/ASTCreateQuery.h>
|
||||||
|
# include <Parsers/ASTFunction.h>
|
||||||
|
# include <Parsers/ParserCreateQuery.h>
|
||||||
|
# include <Parsers/parseQuery.h>
|
||||||
|
# include <Parsers/queryToString.h>
|
||||||
|
# include <Storages/StorageMySQL.h>
|
||||||
|
# include <Common/escapeForFileName.h>
|
||||||
|
# include <Common/parseAddress.h>
|
||||||
|
# include <Common/setThreadName.h>
|
||||||
|
|
||||||
#include <string>
|
# include <Poco/DirectoryIterator.h>
|
||||||
#include <Databases/DatabaseMySQL.h>
|
# include <Poco/File.h>
|
||||||
#include <Common/parseAddress.h>
|
|
||||||
#include <Core/SettingsCollection.h>
|
|
||||||
#include <IO/Operators.h>
|
|
||||||
#include <Formats/MySQLBlockInputStream.h>
|
|
||||||
#include <DataTypes/DataTypeString.h>
|
|
||||||
#include <DataTypes/DataTypesNumber.h>
|
|
||||||
#include <DataTypes/DataTypeDateTime.h>
|
|
||||||
#include <DataTypes/DataTypeNullable.h>
|
|
||||||
#include <Storages/StorageMySQL.h>
|
|
||||||
#include <Parsers/ASTFunction.h>
|
|
||||||
#include <Parsers/ParserCreateQuery.h>
|
|
||||||
#include <Parsers/parseQuery.h>
|
|
||||||
#include <Common/setThreadName.h>
|
|
||||||
#include <Common/escapeForFileName.h>
|
|
||||||
#include <Parsers/queryToString.h>
|
|
||||||
#include <Parsers/ASTCreateQuery.h>
|
|
||||||
#include <DataTypes/convertMySQLDataType.h>
|
|
||||||
|
|
||||||
#include <Poco/File.h>
|
|
||||||
#include <Poco/DirectoryIterator.h>
|
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
|
19
src/Databases/ya.make
Normal file
19
src/Databases/ya.make
Normal file
@ -0,0 +1,19 @@
|
|||||||
|
LIBRARY()
|
||||||
|
|
||||||
|
PEERDIR(
|
||||||
|
clickhouse/src/Common
|
||||||
|
)
|
||||||
|
|
||||||
|
SRCS(
|
||||||
|
DatabaseDictionary.cpp
|
||||||
|
DatabaseFactory.cpp
|
||||||
|
DatabaseLazy.cpp
|
||||||
|
DatabaseMemory.cpp
|
||||||
|
DatabaseMySQL.cpp
|
||||||
|
DatabaseOnDisk.cpp
|
||||||
|
DatabaseOrdinary.cpp
|
||||||
|
DatabasesCommon.cpp
|
||||||
|
DatabaseWithDictionaries.cpp
|
||||||
|
)
|
||||||
|
|
||||||
|
END()
|
@ -1,14 +1,7 @@
|
|||||||
include(${ClickHouse_SOURCE_DIR}/cmake/dbms_glob_sources.cmake)
|
include(${ClickHouse_SOURCE_DIR}/cmake/dbms_glob_sources.cmake)
|
||||||
include(${ClickHouse_SOURCE_DIR}/cmake/generate_code.cmake)
|
|
||||||
|
|
||||||
add_headers_and_sources(clickhouse_dictionaries .)
|
add_headers_and_sources(clickhouse_dictionaries .)
|
||||||
|
|
||||||
generate_code(ComplexKeyCacheDictionary_generate1 UInt8 UInt16 UInt32 UInt64 UInt128 Int8 Int16 Int32 Int64 Float32 Float64 Decimal32 Decimal64 Decimal128)
|
|
||||||
generate_code(ComplexKeyCacheDictionary_generate2 UInt8 UInt16 UInt32 UInt64 UInt128 Int8 Int16 Int32 Int64 Float32 Float64 Decimal32 Decimal64 Decimal128)
|
|
||||||
generate_code(ComplexKeyCacheDictionary_generate3 UInt8 UInt16 UInt32 UInt64 UInt128 Int8 Int16 Int32 Int64 Float32 Float64 Decimal32 Decimal64 Decimal128)
|
|
||||||
generate_code(CacheDictionary_generate1 UInt8 UInt16 UInt32 UInt64 UInt128 Int8 Int16 Int32 Int64 Float32 Float64 Decimal32 Decimal64 Decimal128)
|
|
||||||
generate_code(CacheDictionary_generate2 UInt8 UInt16 UInt32 UInt64 UInt128 Int8 Int16 Int32 Int64 Float32 Float64 Decimal32 Decimal64 Decimal128)
|
|
||||||
generate_code(CacheDictionary_generate3 UInt8 UInt16 UInt32 UInt64 UInt128 Int8 Int16 Int32 Int64 Float32 Float64 Decimal32 Decimal64 Decimal128)
|
|
||||||
add_headers_and_sources(clickhouse_dictionaries ${CMAKE_CURRENT_BINARY_DIR}/generated/)
|
add_headers_and_sources(clickhouse_dictionaries ${CMAKE_CURRENT_BINARY_DIR}/generated/)
|
||||||
|
|
||||||
list(REMOVE_ITEM clickhouse_dictionaries_sources DictionaryFactory.cpp DictionarySourceFactory.cpp DictionaryStructure.cpp getDictionaryConfigurationFromAST.cpp)
|
list(REMOVE_ITEM clickhouse_dictionaries_sources DictionaryFactory.cpp DictionarySourceFactory.cpp DictionaryStructure.cpp getDictionaryConfigurationFromAST.cpp)
|
||||||
|
32
src/Dictionaries/CacheDictionary_generate1.cpp
Normal file
32
src/Dictionaries/CacheDictionary_generate1.cpp
Normal file
@ -0,0 +1,32 @@
|
|||||||
|
#include <Dictionaries/CacheDictionary.h>
|
||||||
|
#include <Dictionaries/CacheDictionary.inc.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
#define DEFINE(TYPE) \
|
||||||
|
void CacheDictionary::get##TYPE(const std::string & attribute_name, const PaddedPODArray<Key> & ids, ResultArrayType<TYPE> & out) \
|
||||||
|
const \
|
||||||
|
{ \
|
||||||
|
auto & attribute = getAttribute(attribute_name); \
|
||||||
|
checkAttributeType(name, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \
|
||||||
|
const auto null_value = std::get<TYPE>(attribute.null_values); \
|
||||||
|
getItemsNumberImpl<TYPE, TYPE>(attribute, ids, out, [&](const size_t) { return null_value; }); \
|
||||||
|
}
|
||||||
|
|
||||||
|
DEFINE(UInt8)
|
||||||
|
DEFINE(UInt16)
|
||||||
|
DEFINE(UInt32)
|
||||||
|
DEFINE(UInt64)
|
||||||
|
DEFINE(UInt128)
|
||||||
|
DEFINE(Int8)
|
||||||
|
DEFINE(Int16)
|
||||||
|
DEFINE(Int32)
|
||||||
|
DEFINE(Int64)
|
||||||
|
DEFINE(Float32)
|
||||||
|
DEFINE(Float64)
|
||||||
|
DEFINE(Decimal32)
|
||||||
|
DEFINE(Decimal64)
|
||||||
|
DEFINE(Decimal128)
|
||||||
|
|
||||||
|
#undef DEFINE
|
||||||
|
}
|
@ -1,22 +0,0 @@
|
|||||||
#include <Dictionaries/CacheDictionary.h>
|
|
||||||
#include <Dictionaries/CacheDictionary.inc.h>
|
|
||||||
|
|
||||||
namespace DB
|
|
||||||
{
|
|
||||||
namespace ErrorCodes
|
|
||||||
{
|
|
||||||
extern const int TYPE_MISMATCH;
|
|
||||||
}
|
|
||||||
|
|
||||||
using TYPE = @NAME@;
|
|
||||||
void CacheDictionary::get@NAME@(const std::string & attribute_name, const PaddedPODArray<Key> & ids, ResultArrayType<TYPE> & out) const
|
|
||||||
{
|
|
||||||
auto & attribute = getAttribute(attribute_name);
|
|
||||||
checkAttributeType(name, attribute_name, attribute.type, AttributeUnderlyingType::ut@NAME@);
|
|
||||||
|
|
||||||
const auto null_value = std::get<TYPE>(attribute.null_values);
|
|
||||||
|
|
||||||
getItemsNumberImpl<TYPE, TYPE>(attribute, ids, out, [&](const size_t) { return null_value; });
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
|
34
src/Dictionaries/CacheDictionary_generate2.cpp
Normal file
34
src/Dictionaries/CacheDictionary_generate2.cpp
Normal file
@ -0,0 +1,34 @@
|
|||||||
|
#include <Dictionaries/CacheDictionary.h>
|
||||||
|
#include <Dictionaries/CacheDictionary.inc.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
#define DEFINE(TYPE) \
|
||||||
|
void CacheDictionary::get##TYPE( \
|
||||||
|
const std::string & attribute_name, \
|
||||||
|
const PaddedPODArray<Key> & ids, \
|
||||||
|
const PaddedPODArray<TYPE> & def, \
|
||||||
|
ResultArrayType<TYPE> & out) const \
|
||||||
|
{ \
|
||||||
|
auto & attribute = getAttribute(attribute_name); \
|
||||||
|
checkAttributeType(name, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \
|
||||||
|
getItemsNumberImpl<TYPE, TYPE>(attribute, ids, out, [&](const size_t row) { return def[row]; }); \
|
||||||
|
}
|
||||||
|
|
||||||
|
DEFINE(UInt8)
|
||||||
|
DEFINE(UInt16)
|
||||||
|
DEFINE(UInt32)
|
||||||
|
DEFINE(UInt64)
|
||||||
|
DEFINE(UInt128)
|
||||||
|
DEFINE(Int8)
|
||||||
|
DEFINE(Int16)
|
||||||
|
DEFINE(Int32)
|
||||||
|
DEFINE(Int64)
|
||||||
|
DEFINE(Float32)
|
||||||
|
DEFINE(Float64)
|
||||||
|
DEFINE(Decimal32)
|
||||||
|
DEFINE(Decimal64)
|
||||||
|
DEFINE(Decimal128)
|
||||||
|
|
||||||
|
#undef DEFINE
|
||||||
|
}
|
@ -1,23 +0,0 @@
|
|||||||
#include <Dictionaries/CacheDictionary.h>
|
|
||||||
#include <Dictionaries/CacheDictionary.inc.h>
|
|
||||||
|
|
||||||
namespace DB
|
|
||||||
{
|
|
||||||
namespace ErrorCodes
|
|
||||||
{
|
|
||||||
extern const int TYPE_MISMATCH;
|
|
||||||
}
|
|
||||||
|
|
||||||
using TYPE = @NAME@;
|
|
||||||
void CacheDictionary::get@NAME@(const std::string & attribute_name,
|
|
||||||
const PaddedPODArray<Key> & ids,
|
|
||||||
const PaddedPODArray<TYPE> & def,
|
|
||||||
ResultArrayType<TYPE> & out) const
|
|
||||||
{
|
|
||||||
auto & attribute = getAttribute(attribute_name);
|
|
||||||
checkAttributeType(name, attribute_name, attribute.type, AttributeUnderlyingType::ut@NAME@);
|
|
||||||
|
|
||||||
getItemsNumberImpl<TYPE, TYPE>(attribute, ids, out, [&](const size_t row) { return def[row]; });
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
|
31
src/Dictionaries/CacheDictionary_generate3.cpp
Normal file
31
src/Dictionaries/CacheDictionary_generate3.cpp
Normal file
@ -0,0 +1,31 @@
|
|||||||
|
#include <Dictionaries/CacheDictionary.h>
|
||||||
|
#include <Dictionaries/CacheDictionary.inc.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
#define DEFINE(TYPE) \
|
||||||
|
void CacheDictionary::get##TYPE( \
|
||||||
|
const std::string & attribute_name, const PaddedPODArray<Key> & ids, const TYPE def, ResultArrayType<TYPE> & out) const \
|
||||||
|
{ \
|
||||||
|
auto & attribute = getAttribute(attribute_name); \
|
||||||
|
checkAttributeType(name, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \
|
||||||
|
getItemsNumberImpl<TYPE, TYPE>(attribute, ids, out, [&](const size_t) { return def; }); \
|
||||||
|
}
|
||||||
|
|
||||||
|
DEFINE(UInt8)
|
||||||
|
DEFINE(UInt16)
|
||||||
|
DEFINE(UInt32)
|
||||||
|
DEFINE(UInt64)
|
||||||
|
DEFINE(UInt128)
|
||||||
|
DEFINE(Int8)
|
||||||
|
DEFINE(Int16)
|
||||||
|
DEFINE(Int32)
|
||||||
|
DEFINE(Int64)
|
||||||
|
DEFINE(Float32)
|
||||||
|
DEFINE(Float64)
|
||||||
|
DEFINE(Decimal32)
|
||||||
|
DEFINE(Decimal64)
|
||||||
|
DEFINE(Decimal128)
|
||||||
|
|
||||||
|
#undef DEFINE
|
||||||
|
}
|
@ -1,20 +0,0 @@
|
|||||||
#include <Dictionaries/CacheDictionary.h>
|
|
||||||
#include <Dictionaries/CacheDictionary.inc.h>
|
|
||||||
|
|
||||||
namespace DB
|
|
||||||
{
|
|
||||||
namespace ErrorCodes
|
|
||||||
{
|
|
||||||
extern const int TYPE_MISMATCH;
|
|
||||||
}
|
|
||||||
|
|
||||||
using TYPE = @NAME@;
|
|
||||||
void CacheDictionary::get@NAME@(const std::string & attribute_name, const PaddedPODArray<Key> & ids, const TYPE def, ResultArrayType<TYPE> & out) const
|
|
||||||
{
|
|
||||||
auto & attribute = getAttribute(attribute_name);
|
|
||||||
checkAttributeType(name, attribute_name, attribute.type, AttributeUnderlyingType::ut@NAME@);
|
|
||||||
|
|
||||||
getItemsNumberImpl<TYPE, TYPE>(attribute, ids, out, [&](const size_t) { return def; });
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
|
32
src/Dictionaries/ComplexKeyCacheDictionary_generate1.cpp
Normal file
32
src/Dictionaries/ComplexKeyCacheDictionary_generate1.cpp
Normal file
@ -0,0 +1,32 @@
|
|||||||
|
#include <Dictionaries/ComplexKeyCacheDictionary.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
#define DEFINE(TYPE) \
|
||||||
|
void ComplexKeyCacheDictionary::get##TYPE( \
|
||||||
|
const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types, ResultArrayType<TYPE> & out) const \
|
||||||
|
{ \
|
||||||
|
dict_struct.validateKeyTypes(key_types); \
|
||||||
|
auto & attribute = getAttribute(attribute_name); \
|
||||||
|
checkAttributeType(name, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \
|
||||||
|
const auto null_value = std::get<TYPE>(attribute.null_values); \
|
||||||
|
getItemsNumberImpl<TYPE, TYPE>(attribute, key_columns, out, [&](const size_t) { return null_value; }); \
|
||||||
|
}
|
||||||
|
|
||||||
|
DEFINE(UInt8)
|
||||||
|
DEFINE(UInt16)
|
||||||
|
DEFINE(UInt32)
|
||||||
|
DEFINE(UInt64)
|
||||||
|
DEFINE(UInt128)
|
||||||
|
DEFINE(Int8)
|
||||||
|
DEFINE(Int16)
|
||||||
|
DEFINE(Int32)
|
||||||
|
DEFINE(Int64)
|
||||||
|
DEFINE(Float32)
|
||||||
|
DEFINE(Float64)
|
||||||
|
DEFINE(Decimal32)
|
||||||
|
DEFINE(Decimal64)
|
||||||
|
DEFINE(Decimal128)
|
||||||
|
|
||||||
|
#undef DEFINE
|
||||||
|
}
|
@ -1,22 +0,0 @@
|
|||||||
#include <Dictionaries/ComplexKeyCacheDictionary.h>
|
|
||||||
|
|
||||||
namespace DB
|
|
||||||
{
|
|
||||||
namespace ErrorCodes
|
|
||||||
{
|
|
||||||
extern const int TYPE_MISMATCH;
|
|
||||||
}
|
|
||||||
|
|
||||||
using TYPE = @NAME@;
|
|
||||||
void ComplexKeyCacheDictionary::get@NAME@(const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types, ResultArrayType<TYPE> & out) const
|
|
||||||
{
|
|
||||||
dict_struct.validateKeyTypes(key_types);
|
|
||||||
|
|
||||||
auto & attribute = getAttribute(attribute_name);
|
|
||||||
checkAttributeType(name, attribute_name, attribute.type, AttributeUnderlyingType::ut@NAME@);
|
|
||||||
|
|
||||||
const auto null_value = std::get<TYPE>(attribute.null_values);
|
|
||||||
|
|
||||||
getItemsNumberImpl<TYPE, TYPE>(attribute, key_columns, out, [&](const size_t) { return null_value; });
|
|
||||||
}
|
|
||||||
}
|
|
35
src/Dictionaries/ComplexKeyCacheDictionary_generate2.cpp
Normal file
35
src/Dictionaries/ComplexKeyCacheDictionary_generate2.cpp
Normal file
@ -0,0 +1,35 @@
|
|||||||
|
#include <Dictionaries/ComplexKeyCacheDictionary.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
#define DEFINE(TYPE) \
|
||||||
|
void ComplexKeyCacheDictionary::get##TYPE( \
|
||||||
|
const std::string & attribute_name, \
|
||||||
|
const Columns & key_columns, \
|
||||||
|
const DataTypes & key_types, \
|
||||||
|
const PaddedPODArray<TYPE> & def, \
|
||||||
|
ResultArrayType<TYPE> & out) const \
|
||||||
|
{ \
|
||||||
|
dict_struct.validateKeyTypes(key_types); \
|
||||||
|
auto & attribute = getAttribute(attribute_name); \
|
||||||
|
checkAttributeType(name, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \
|
||||||
|
getItemsNumberImpl<TYPE, TYPE>(attribute, key_columns, out, [&](const size_t row) { return def[row]; }); \
|
||||||
|
}
|
||||||
|
|
||||||
|
DEFINE(UInt8)
|
||||||
|
DEFINE(UInt16)
|
||||||
|
DEFINE(UInt32)
|
||||||
|
DEFINE(UInt64)
|
||||||
|
DEFINE(UInt128)
|
||||||
|
DEFINE(Int8)
|
||||||
|
DEFINE(Int16)
|
||||||
|
DEFINE(Int32)
|
||||||
|
DEFINE(Int64)
|
||||||
|
DEFINE(Float32)
|
||||||
|
DEFINE(Float64)
|
||||||
|
DEFINE(Decimal32)
|
||||||
|
DEFINE(Decimal64)
|
||||||
|
DEFINE(Decimal128)
|
||||||
|
|
||||||
|
#undef DEFINE
|
||||||
|
}
|
@ -1,25 +0,0 @@
|
|||||||
#include <Dictionaries/ComplexKeyCacheDictionary.h>
|
|
||||||
|
|
||||||
namespace DB
|
|
||||||
{
|
|
||||||
namespace ErrorCodes
|
|
||||||
{
|
|
||||||
extern const int TYPE_MISMATCH;
|
|
||||||
}
|
|
||||||
|
|
||||||
using TYPE = @NAME@;
|
|
||||||
|
|
||||||
void ComplexKeyCacheDictionary::get@NAME@(const std::string & attribute_name,
|
|
||||||
const Columns & key_columns,
|
|
||||||
const DataTypes & key_types,
|
|
||||||
const PaddedPODArray<TYPE> & def,
|
|
||||||
ResultArrayType<TYPE> & out) const
|
|
||||||
{
|
|
||||||
dict_struct.validateKeyTypes(key_types);
|
|
||||||
|
|
||||||
auto & attribute = getAttribute(attribute_name);
|
|
||||||
checkAttributeType(name, attribute_name, attribute.type, AttributeUnderlyingType::ut@NAME@);
|
|
||||||
|
|
||||||
getItemsNumberImpl<TYPE, TYPE>(attribute, key_columns, out, [&](const size_t row) { return def[row]; });
|
|
||||||
}
|
|
||||||
}
|
|
35
src/Dictionaries/ComplexKeyCacheDictionary_generate3.cpp
Normal file
35
src/Dictionaries/ComplexKeyCacheDictionary_generate3.cpp
Normal file
@ -0,0 +1,35 @@
|
|||||||
|
#include <Dictionaries/ComplexKeyCacheDictionary.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
#define DEFINE(TYPE) \
|
||||||
|
void ComplexKeyCacheDictionary::get##TYPE( \
|
||||||
|
const std::string & attribute_name, \
|
||||||
|
const Columns & key_columns, \
|
||||||
|
const DataTypes & key_types, \
|
||||||
|
const TYPE def, \
|
||||||
|
ResultArrayType<TYPE> & out) const \
|
||||||
|
{ \
|
||||||
|
dict_struct.validateKeyTypes(key_types); \
|
||||||
|
auto & attribute = getAttribute(attribute_name); \
|
||||||
|
checkAttributeType(name, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \
|
||||||
|
getItemsNumberImpl<TYPE, TYPE>(attribute, key_columns, out, [&](const size_t) { return def; }); \
|
||||||
|
}
|
||||||
|
|
||||||
|
DEFINE(UInt8)
|
||||||
|
DEFINE(UInt16)
|
||||||
|
DEFINE(UInt32)
|
||||||
|
DEFINE(UInt64)
|
||||||
|
DEFINE(UInt128)
|
||||||
|
DEFINE(Int8)
|
||||||
|
DEFINE(Int16)
|
||||||
|
DEFINE(Int32)
|
||||||
|
DEFINE(Int64)
|
||||||
|
DEFINE(Float32)
|
||||||
|
DEFINE(Float64)
|
||||||
|
DEFINE(Decimal32)
|
||||||
|
DEFINE(Decimal64)
|
||||||
|
DEFINE(Decimal128)
|
||||||
|
|
||||||
|
#undef DEFINE
|
||||||
|
}
|
@ -1,25 +0,0 @@
|
|||||||
#include <Dictionaries/ComplexKeyCacheDictionary.h>
|
|
||||||
|
|
||||||
namespace DB
|
|
||||||
{
|
|
||||||
namespace ErrorCodes
|
|
||||||
{
|
|
||||||
extern const int TYPE_MISMATCH;
|
|
||||||
}
|
|
||||||
|
|
||||||
using TYPE = @NAME@;
|
|
||||||
|
|
||||||
void ComplexKeyCacheDictionary::get@NAME@(const std::string & attribute_name,
|
|
||||||
const Columns & key_columns,
|
|
||||||
const DataTypes & key_types,
|
|
||||||
const TYPE def,
|
|
||||||
ResultArrayType<TYPE> & out) const
|
|
||||||
{
|
|
||||||
dict_struct.validateKeyTypes(key_types);
|
|
||||||
|
|
||||||
auto & attribute = getAttribute(attribute_name);
|
|
||||||
checkAttributeType(name, attribute_name, attribute.type, AttributeUnderlyingType::ut@NAME@);
|
|
||||||
|
|
||||||
getItemsNumberImpl<TYPE, TYPE>(attribute, key_columns, out, [&](const size_t) { return def; });
|
|
||||||
}
|
|
||||||
}
|
|
@ -156,8 +156,14 @@ private:
|
|||||||
template <typename Value>
|
template <typename Value>
|
||||||
using CollectionPtrType = std::unique_ptr<CollectionType<Value>>;
|
using CollectionPtrType = std::unique_ptr<CollectionType<Value>>;
|
||||||
|
|
||||||
|
#if !defined(ARCADIA_BUILD)
|
||||||
template <typename Value>
|
template <typename Value>
|
||||||
using SparseCollectionType = google::sparse_hash_map<UInt64, Value, DefaultHash<UInt64>>;
|
using SparseCollectionType = google::sparse_hash_map<UInt64, Value, DefaultHash<UInt64>>;
|
||||||
|
#else
|
||||||
|
template <typename Value>
|
||||||
|
using SparseCollectionType = google::sparsehash::sparse_hash_map<UInt64, Value, DefaultHash<UInt64>>;
|
||||||
|
#endif
|
||||||
|
|
||||||
template <typename Value>
|
template <typename Value>
|
||||||
using SparseCollectionPtrType = std::unique_ptr<SparseCollectionType<Value>>;
|
using SparseCollectionPtrType = std::unique_ptr<SparseCollectionType<Value>>;
|
||||||
|
|
||||||
|
@ -1,25 +1,27 @@
|
|||||||
#include "config_core.h"
|
#if !defined(ARCADIA_BUILD)
|
||||||
|
# include "config_core.h"
|
||||||
|
#endif
|
||||||
|
|
||||||
#if USE_POCO_MONGODB
|
#if USE_POCO_MONGODB
|
||||||
|
# include <sstream>
|
||||||
|
# include <string>
|
||||||
|
# include <vector>
|
||||||
|
|
||||||
#include <sstream>
|
# include <Poco/MongoDB/Connection.h>
|
||||||
#include <string>
|
# include <Poco/MongoDB/Cursor.h>
|
||||||
#include <vector>
|
# include <Poco/MongoDB/Element.h>
|
||||||
|
# include <Poco/MongoDB/ObjectId.h>
|
||||||
|
|
||||||
#include <Poco/MongoDB/Connection.h>
|
# include <Columns/ColumnNullable.h>
|
||||||
#include <Poco/MongoDB/Cursor.h>
|
# include <Columns/ColumnString.h>
|
||||||
#include <Poco/MongoDB/Element.h>
|
# include <Columns/ColumnsNumber.h>
|
||||||
#include <Poco/MongoDB/ObjectId.h>
|
# include <IO/ReadHelpers.h>
|
||||||
|
# include <IO/WriteHelpers.h>
|
||||||
#include <Columns/ColumnNullable.h>
|
# include <Common/FieldVisitors.h>
|
||||||
#include <Columns/ColumnString.h>
|
# include <Common/assert_cast.h>
|
||||||
#include <Columns/ColumnsNumber.h>
|
# include <ext/range.h>
|
||||||
#include <IO/ReadHelpers.h>
|
# include "DictionaryStructure.h"
|
||||||
#include <IO/WriteHelpers.h>
|
# include "MongoDBBlockInputStream.h"
|
||||||
#include <Common/FieldVisitors.h>
|
|
||||||
#include <Common/assert_cast.h>
|
|
||||||
#include <ext/range.h>
|
|
||||||
#include "DictionaryStructure.h"
|
|
||||||
#include "MongoDBBlockInputStream.h"
|
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
|
@ -1,9 +1,12 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include <Core/Block.h>
|
#include <Core/Block.h>
|
||||||
#include "config_core.h"
|
|
||||||
#if USE_POCO_MONGODB
|
|
||||||
|
|
||||||
|
#if !defined(ARCADIA_BUILD)
|
||||||
|
# include "config_core.h"
|
||||||
|
#endif
|
||||||
|
|
||||||
|
#if USE_POCO_MONGODB
|
||||||
# include "DictionaryStructure.h"
|
# include "DictionaryStructure.h"
|
||||||
# include "IDictionarySource.h"
|
# include "IDictionarySource.h"
|
||||||
|
|
||||||
@ -91,4 +94,5 @@ private:
|
|||||||
};
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
#endif
|
#endif
|
||||||
|
@ -1,8 +1,8 @@
|
|||||||
#include "MySQLDictionarySource.h"
|
#include "MySQLDictionarySource.h"
|
||||||
|
|
||||||
#include <Poco/Util/AbstractConfiguration.h>
|
#include <Poco/Util/AbstractConfiguration.h>
|
||||||
#include "DictionarySourceFactory.h"
|
#include "DictionarySourceFactory.h"
|
||||||
#include "DictionaryStructure.h"
|
#include "DictionaryStructure.h"
|
||||||
#include "config_core.h"
|
|
||||||
#include "registerDictionaries.h"
|
#include "registerDictionaries.h"
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
|
@ -2,9 +2,11 @@
|
|||||||
|
|
||||||
#include <Core/Block.h>
|
#include <Core/Block.h>
|
||||||
|
|
||||||
#include "config_core.h"
|
#if !defined(ARCADIA_BUILD)
|
||||||
#if USE_MYSQL
|
# include "config_core.h"
|
||||||
|
#endif
|
||||||
|
|
||||||
|
#if USE_MYSQL
|
||||||
# include <common/LocalDateTime.h>
|
# include <common/LocalDateTime.h>
|
||||||
# include <mysqlxx/PoolWithFailover.h>
|
# include <mysqlxx/PoolWithFailover.h>
|
||||||
# include "DictionaryStructure.h"
|
# include "DictionaryStructure.h"
|
||||||
|
@ -1,8 +1,11 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include "config_core.h"
|
|
||||||
#include <Core/Block.h>
|
#include <Core/Block.h>
|
||||||
|
|
||||||
|
#if !defined(ARCADIA_BUILD)
|
||||||
|
# include "config_core.h"
|
||||||
|
#endif
|
||||||
|
|
||||||
#if USE_POCO_REDIS
|
#if USE_POCO_REDIS
|
||||||
# include <Core/ExternalResultDescription.h>
|
# include <Core/ExternalResultDescription.h>
|
||||||
# include <DataStreams/IBlockInputStream.h>
|
# include <DataStreams/IBlockInputStream.h>
|
||||||
|
@ -1,10 +1,12 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include "config_core.h"
|
|
||||||
#include <Core/Block.h>
|
#include <Core/Block.h>
|
||||||
|
|
||||||
#if USE_POCO_REDIS
|
#if !defined(ARCADIA_BUILD)
|
||||||
|
# include "config_core.h"
|
||||||
|
#endif
|
||||||
|
|
||||||
|
#if USE_POCO_REDIS
|
||||||
# include "DictionaryStructure.h"
|
# include "DictionaryStructure.h"
|
||||||
# include "IDictionarySource.h"
|
# include "IDictionarySource.h"
|
||||||
|
|
||||||
@ -101,4 +103,5 @@ namespace ErrorCodes
|
|||||||
};
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
#endif
|
#endif
|
||||||
|
@ -16,9 +16,12 @@
|
|||||||
#include "DictionaryStructure.h"
|
#include "DictionaryStructure.h"
|
||||||
#include "readInvalidateQuery.h"
|
#include "readInvalidateQuery.h"
|
||||||
|
|
||||||
#include <Common/config.h>
|
|
||||||
#include "registerDictionaries.h"
|
#include "registerDictionaries.h"
|
||||||
|
|
||||||
|
#if !defined(ARCADIA_BUILD)
|
||||||
|
# include <Common/config.h>
|
||||||
|
#endif
|
||||||
|
|
||||||
#if USE_POCO_SQLODBC || USE_POCO_DATAODBC
|
#if USE_POCO_SQLODBC || USE_POCO_DATAODBC
|
||||||
# include <Poco/Data/ODBC/Connector.h>
|
# include <Poco/Data/ODBC/Connector.h>
|
||||||
#endif
|
#endif
|
||||||
|
@ -25,7 +25,9 @@ void registerDictionaries()
|
|||||||
registerDictionaryRangeHashed(factory);
|
registerDictionaryRangeHashed(factory);
|
||||||
registerDictionaryComplexKeyHashed(factory);
|
registerDictionaryComplexKeyHashed(factory);
|
||||||
registerDictionaryComplexKeyCache(factory);
|
registerDictionaryComplexKeyCache(factory);
|
||||||
|
#if !defined(ARCADIA_BUILD)
|
||||||
registerDictionaryTrie(factory);
|
registerDictionaryTrie(factory);
|
||||||
|
#endif
|
||||||
registerDictionaryFlat(factory);
|
registerDictionaryFlat(factory);
|
||||||
registerDictionaryHashed(factory);
|
registerDictionaryHashed(factory);
|
||||||
registerDictionaryCache(factory);
|
registerDictionaryCache(factory);
|
||||||
|
63
src/Dictionaries/ya.make
Normal file
63
src/Dictionaries/ya.make
Normal file
@ -0,0 +1,63 @@
|
|||||||
|
LIBRARY()
|
||||||
|
|
||||||
|
PEERDIR(
|
||||||
|
clickhouse/src/Common
|
||||||
|
contrib/libs/poco/Data
|
||||||
|
contrib/libs/poco/Data/ODBC
|
||||||
|
contrib/libs/poco/MongoDB
|
||||||
|
contrib/libs/poco/Redis
|
||||||
|
contrib/libs/sparsehash
|
||||||
|
)
|
||||||
|
|
||||||
|
NO_COMPILER_WARNINGS()
|
||||||
|
|
||||||
|
SRCS(
|
||||||
|
CacheDictionary_generate1.cpp
|
||||||
|
CacheDictionary_generate2.cpp
|
||||||
|
CacheDictionary_generate3.cpp
|
||||||
|
CacheDictionary.cpp
|
||||||
|
ClickHouseDictionarySource.cpp
|
||||||
|
ComplexKeyCacheDictionary_createAttributeWithType.cpp
|
||||||
|
ComplexKeyCacheDictionary_generate1.cpp
|
||||||
|
ComplexKeyCacheDictionary_generate2.cpp
|
||||||
|
ComplexKeyCacheDictionary_generate3.cpp
|
||||||
|
ComplexKeyCacheDictionary_setAttributeValue.cpp
|
||||||
|
ComplexKeyCacheDictionary_setDefaultAttributeValue.cpp
|
||||||
|
ComplexKeyCacheDictionary.cpp
|
||||||
|
ComplexKeyHashedDictionary.cpp
|
||||||
|
DictionaryBlockInputStreamBase.cpp
|
||||||
|
DictionaryFactory.cpp
|
||||||
|
DictionarySourceFactory.cpp
|
||||||
|
DictionarySourceHelpers.cpp
|
||||||
|
DictionaryStructure.cpp
|
||||||
|
Embedded/GeodataProviders/HierarchiesProvider.cpp
|
||||||
|
Embedded/GeodataProviders/HierarchyFormatReader.cpp
|
||||||
|
Embedded/GeodataProviders/NamesFormatReader.cpp
|
||||||
|
Embedded/GeodataProviders/NamesProvider.cpp
|
||||||
|
Embedded/GeoDictionariesLoader.cpp
|
||||||
|
Embedded/RegionsHierarchies.cpp
|
||||||
|
Embedded/RegionsHierarchy.cpp
|
||||||
|
Embedded/RegionsNames.cpp
|
||||||
|
ExecutableDictionarySource.cpp
|
||||||
|
ExternalQueryBuilder.cpp
|
||||||
|
FileDictionarySource.cpp
|
||||||
|
FlatDictionary.cpp
|
||||||
|
getDictionaryConfigurationFromAST.cpp
|
||||||
|
HashedDictionary.cpp
|
||||||
|
HTTPDictionarySource.cpp
|
||||||
|
LibraryDictionarySource.cpp
|
||||||
|
LibraryDictionarySourceExternal.cpp
|
||||||
|
MongoDBBlockInputStream.cpp
|
||||||
|
MongoDBDictionarySource.cpp
|
||||||
|
MySQLDictionarySource.cpp
|
||||||
|
PolygonDictionary.cpp
|
||||||
|
RangeHashedDictionary.cpp
|
||||||
|
readInvalidateQuery.cpp
|
||||||
|
RedisBlockInputStream.cpp
|
||||||
|
RedisDictionarySource.cpp
|
||||||
|
registerDictionaries.cpp
|
||||||
|
writeParenthesisedString.cpp
|
||||||
|
XDBCDictionarySource.cpp
|
||||||
|
)
|
||||||
|
|
||||||
|
END()
|
@ -1,6 +1,8 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include <Common/config.h>
|
#if !defined(ARCADIA_BUILD)
|
||||||
|
# include <Common/config.h>
|
||||||
|
#endif
|
||||||
|
|
||||||
#if USE_AWS_S3
|
#if USE_AWS_S3
|
||||||
# include "DiskFactory.h"
|
# include "DiskFactory.h"
|
||||||
|
@ -2,7 +2,9 @@
|
|||||||
|
|
||||||
#include "DiskFactory.h"
|
#include "DiskFactory.h"
|
||||||
|
|
||||||
#include <Common/config.h>
|
#if !defined(ARCADIA_BUILD)
|
||||||
|
# include <Common/config.h>
|
||||||
|
#endif
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
17
src/Disks/ya.make
Normal file
17
src/Disks/ya.make
Normal file
@ -0,0 +1,17 @@
|
|||||||
|
LIBRARY()
|
||||||
|
|
||||||
|
PEERDIR(
|
||||||
|
clickhouse/src/Common
|
||||||
|
)
|
||||||
|
|
||||||
|
SRCS(
|
||||||
|
DiskFactory.cpp
|
||||||
|
DiskLocal.cpp
|
||||||
|
DiskMemory.cpp
|
||||||
|
DiskS3.cpp
|
||||||
|
DiskSpaceMonitor.cpp
|
||||||
|
IDisk.cpp
|
||||||
|
registerDisks.cpp
|
||||||
|
)
|
||||||
|
|
||||||
|
END()
|
@ -1,12 +1,12 @@
|
|||||||
|
#include <Formats/FormatFactory.h>
|
||||||
|
|
||||||
#include <algorithm>
|
#include <algorithm>
|
||||||
#include <Common/config.h>
|
|
||||||
#include <Common/Exception.h>
|
#include <Common/Exception.h>
|
||||||
#include <Interpreters/Context.h>
|
#include <Interpreters/Context.h>
|
||||||
#include <Core/Settings.h>
|
#include <Core/Settings.h>
|
||||||
#include <DataStreams/MaterializingBlockOutputStream.h>
|
#include <DataStreams/MaterializingBlockOutputStream.h>
|
||||||
#include <DataStreams/ParallelParsingBlockInputStream.h>
|
#include <DataStreams/ParallelParsingBlockInputStream.h>
|
||||||
#include <Formats/FormatSettings.h>
|
#include <Formats/FormatSettings.h>
|
||||||
#include <Formats/FormatFactory.h>
|
|
||||||
#include <Processors/Formats/IRowInputFormat.h>
|
#include <Processors/Formats/IRowInputFormat.h>
|
||||||
#include <Processors/Formats/InputStreamFromInputFormat.h>
|
#include <Processors/Formats/InputStreamFromInputFormat.h>
|
||||||
#include <Processors/Formats/OutputStreamToOutputFormat.h>
|
#include <Processors/Formats/OutputStreamToOutputFormat.h>
|
||||||
@ -16,6 +16,10 @@
|
|||||||
#include <Processors/Formats/Impl/MySQLOutputFormat.h>
|
#include <Processors/Formats/Impl/MySQLOutputFormat.h>
|
||||||
#include <Poco/URI.h>
|
#include <Poco/URI.h>
|
||||||
|
|
||||||
|
#if !defined(ARCADIA_BUILD)
|
||||||
|
# include <Common/config.h>
|
||||||
|
#endif
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
@ -347,12 +351,14 @@ FormatFactory::FormatFactory()
|
|||||||
registerOutputFormatProcessorJSONCompactEachRow(*this);
|
registerOutputFormatProcessorJSONCompactEachRow(*this);
|
||||||
registerInputFormatProcessorProtobuf(*this);
|
registerInputFormatProcessorProtobuf(*this);
|
||||||
registerOutputFormatProcessorProtobuf(*this);
|
registerOutputFormatProcessorProtobuf(*this);
|
||||||
|
#if !defined(ARCADIA_BUILD)
|
||||||
registerInputFormatProcessorCapnProto(*this);
|
registerInputFormatProcessorCapnProto(*this);
|
||||||
registerInputFormatProcessorORC(*this);
|
registerInputFormatProcessorORC(*this);
|
||||||
registerInputFormatProcessorParquet(*this);
|
registerInputFormatProcessorParquet(*this);
|
||||||
registerOutputFormatProcessorParquet(*this);
|
registerOutputFormatProcessorParquet(*this);
|
||||||
registerInputFormatProcessorAvro(*this);
|
registerInputFormatProcessorAvro(*this);
|
||||||
registerOutputFormatProcessorAvro(*this);
|
registerOutputFormatProcessorAvro(*this);
|
||||||
|
#endif
|
||||||
registerInputFormatProcessorTemplate(*this);
|
registerInputFormatProcessorTemplate(*this);
|
||||||
registerOutputFormatProcessorTemplate(*this);
|
registerOutputFormatProcessorTemplate(*this);
|
||||||
registerInputFormatProcessorRegexp(*this);
|
registerInputFormatProcessorRegexp(*this);
|
||||||
|
@ -1,15 +1,17 @@
|
|||||||
#include "config_core.h"
|
#if !defined(ARCADIA_BUILD)
|
||||||
#if USE_MYSQL
|
# include "config_core.h"
|
||||||
|
#endif
|
||||||
|
|
||||||
#include <vector>
|
#if USE_MYSQL
|
||||||
#include <Columns/ColumnNullable.h>
|
# include <vector>
|
||||||
#include <Columns/ColumnString.h>
|
# include <Columns/ColumnNullable.h>
|
||||||
#include <Columns/ColumnsNumber.h>
|
# include <Columns/ColumnString.h>
|
||||||
#include <Common/assert_cast.h>
|
# include <Columns/ColumnsNumber.h>
|
||||||
#include <IO/ReadHelpers.h>
|
# include <IO/ReadHelpers.h>
|
||||||
#include <IO/WriteHelpers.h>
|
# include <IO/WriteHelpers.h>
|
||||||
#include <ext/range.h>
|
# include <Common/assert_cast.h>
|
||||||
#include "MySQLBlockInputStream.h"
|
# include <ext/range.h>
|
||||||
|
# include "MySQLBlockInputStream.h"
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
|
@ -1,15 +1,17 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include "config_formats.h"
|
#if !defined(ARCADIA_BUILD)
|
||||||
#if USE_PROTOBUF
|
# include "config_formats.h"
|
||||||
|
#endif
|
||||||
|
|
||||||
#include <memory>
|
#if USE_PROTOBUF
|
||||||
#include <unordered_map>
|
# include <memory>
|
||||||
#include <vector>
|
# include <unordered_map>
|
||||||
#include <Core/Types.h>
|
# include <vector>
|
||||||
#include <boost/blank.hpp>
|
# include <Core/Types.h>
|
||||||
#include <google/protobuf/descriptor.h>
|
# include <boost/blank.hpp>
|
||||||
#include <google/protobuf/descriptor.pb.h>
|
# include <google/protobuf/descriptor.h>
|
||||||
|
# include <google/protobuf/descriptor.pb.h>
|
||||||
|
|
||||||
namespace google
|
namespace google
|
||||||
{
|
{
|
||||||
@ -190,4 +192,5 @@ namespace ProtobufColumnMatcher
|
|||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
#endif
|
#endif
|
||||||
|
@ -1,16 +1,14 @@
|
|||||||
#include "config_formats.h"
|
|
||||||
#if USE_PROTOBUF
|
|
||||||
|
|
||||||
#include "ProtobufReader.h"
|
#include "ProtobufReader.h"
|
||||||
|
|
||||||
#include <AggregateFunctions/IAggregateFunction.h>
|
#if USE_PROTOBUF
|
||||||
#include <boost/numeric/conversion/cast.hpp>
|
# include <optional>
|
||||||
#include <DataTypes/DataTypesDecimal.h>
|
# include <AggregateFunctions/IAggregateFunction.h>
|
||||||
#include <IO/ReadBufferFromString.h>
|
# include <DataTypes/DataTypesDecimal.h>
|
||||||
#include <IO/ReadHelpers.h>
|
# include <IO/ReadBufferFromString.h>
|
||||||
#include <IO/WriteBufferFromVector.h>
|
# include <IO/ReadHelpers.h>
|
||||||
#include <IO/WriteHelpers.h>
|
# include <IO/WriteBufferFromVector.h>
|
||||||
#include <optional>
|
# include <IO/WriteHelpers.h>
|
||||||
|
# include <boost/numeric/conversion/cast.hpp>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
@ -272,25 +270,25 @@ UInt64 ProtobufReader::SimpleReader::continueReadingVarint(UInt64 first_byte)
|
|||||||
UInt64 result = (first_byte & ~static_cast<UInt64>(0x80));
|
UInt64 result = (first_byte & ~static_cast<UInt64>(0x80));
|
||||||
char c;
|
char c;
|
||||||
|
|
||||||
#define PROTOBUF_READER_READ_VARINT_BYTE(byteNo) \
|
# define PROTOBUF_READER_READ_VARINT_BYTE(byteNo) \
|
||||||
do \
|
do \
|
||||||
{ \
|
|
||||||
in.readStrict(c); \
|
|
||||||
++cursor; \
|
|
||||||
if constexpr ((byteNo) < 10) \
|
|
||||||
{ \
|
{ \
|
||||||
result |= static_cast<UInt64>(static_cast<UInt8>(c)) << (7 * ((byteNo) - 1)); \
|
in.readStrict(c); \
|
||||||
if (likely(!(c & 0x80))) \
|
++cursor; \
|
||||||
return result; \
|
if constexpr ((byteNo) < 10) \
|
||||||
} \
|
{ \
|
||||||
else \
|
result |= static_cast<UInt64>(static_cast<UInt8>(c)) << (7 * ((byteNo)-1)); \
|
||||||
{ \
|
if (likely(!(c & 0x80))) \
|
||||||
if (likely(c == 1)) \
|
return result; \
|
||||||
return result; \
|
} \
|
||||||
} \
|
else \
|
||||||
if constexpr ((byteNo) < 9) \
|
{ \
|
||||||
result &= ~(static_cast<UInt64>(0x80) << (7 * ((byteNo) - 1))); \
|
if (likely(c == 1)) \
|
||||||
} while (false)
|
return result; \
|
||||||
|
} \
|
||||||
|
if constexpr ((byteNo) < 9) \
|
||||||
|
result &= ~(static_cast<UInt64>(0x80) << (7 * ((byteNo)-1))); \
|
||||||
|
} while (false)
|
||||||
|
|
||||||
PROTOBUF_READER_READ_VARINT_BYTE(2);
|
PROTOBUF_READER_READ_VARINT_BYTE(2);
|
||||||
PROTOBUF_READER_READ_VARINT_BYTE(3);
|
PROTOBUF_READER_READ_VARINT_BYTE(3);
|
||||||
@ -302,7 +300,7 @@ UInt64 ProtobufReader::SimpleReader::continueReadingVarint(UInt64 first_byte)
|
|||||||
PROTOBUF_READER_READ_VARINT_BYTE(9);
|
PROTOBUF_READER_READ_VARINT_BYTE(9);
|
||||||
PROTOBUF_READER_READ_VARINT_BYTE(10);
|
PROTOBUF_READER_READ_VARINT_BYTE(10);
|
||||||
|
|
||||||
#undef PROTOBUF_READER_READ_VARINT_BYTE
|
# undef PROTOBUF_READER_READ_VARINT_BYTE
|
||||||
|
|
||||||
throwUnknownFormat();
|
throwUnknownFormat();
|
||||||
}
|
}
|
||||||
@ -311,22 +309,22 @@ void ProtobufReader::SimpleReader::ignoreVarint()
|
|||||||
{
|
{
|
||||||
char c;
|
char c;
|
||||||
|
|
||||||
#define PROTOBUF_READER_IGNORE_VARINT_BYTE(byteNo) \
|
# define PROTOBUF_READER_IGNORE_VARINT_BYTE(byteNo) \
|
||||||
do \
|
do \
|
||||||
{ \
|
|
||||||
in.readStrict(c); \
|
|
||||||
++cursor; \
|
|
||||||
if constexpr ((byteNo) < 10) \
|
|
||||||
{ \
|
{ \
|
||||||
if (likely(!(c & 0x80))) \
|
in.readStrict(c); \
|
||||||
return; \
|
++cursor; \
|
||||||
} \
|
if constexpr ((byteNo) < 10) \
|
||||||
else \
|
{ \
|
||||||
{ \
|
if (likely(!(c & 0x80))) \
|
||||||
if (likely(c == 1)) \
|
return; \
|
||||||
return; \
|
} \
|
||||||
} \
|
else \
|
||||||
} while (false)
|
{ \
|
||||||
|
if (likely(c == 1)) \
|
||||||
|
return; \
|
||||||
|
} \
|
||||||
|
} while (false)
|
||||||
|
|
||||||
PROTOBUF_READER_IGNORE_VARINT_BYTE(1);
|
PROTOBUF_READER_IGNORE_VARINT_BYTE(1);
|
||||||
PROTOBUF_READER_IGNORE_VARINT_BYTE(2);
|
PROTOBUF_READER_IGNORE_VARINT_BYTE(2);
|
||||||
@ -338,7 +336,8 @@ void ProtobufReader::SimpleReader::ignoreVarint()
|
|||||||
PROTOBUF_READER_IGNORE_VARINT_BYTE(8);
|
PROTOBUF_READER_IGNORE_VARINT_BYTE(8);
|
||||||
PROTOBUF_READER_IGNORE_VARINT_BYTE(9);
|
PROTOBUF_READER_IGNORE_VARINT_BYTE(9);
|
||||||
PROTOBUF_READER_IGNORE_VARINT_BYTE(10);
|
PROTOBUF_READER_IGNORE_VARINT_BYTE(10);
|
||||||
#undef PROTOBUF_READER_IGNORE_VARINT_BYTE
|
|
||||||
|
# undef PROTOBUF_READER_IGNORE_VARINT_BYTE
|
||||||
|
|
||||||
throwUnknownFormat();
|
throwUnknownFormat();
|
||||||
}
|
}
|
||||||
@ -694,16 +693,17 @@ private:
|
|||||||
std::optional<std::unordered_map<StringRef, Int16>> enum_name_to_value_map;
|
std::optional<std::unordered_map<StringRef, Int16>> enum_name_to_value_map;
|
||||||
};
|
};
|
||||||
|
|
||||||
#define PROTOBUF_READER_CREATE_CONVERTER_SPECIALIZATION_FOR_STRINGS(field_type_id) \
|
# define PROTOBUF_READER_CREATE_CONVERTER_SPECIALIZATION_FOR_STRINGS(field_type_id) \
|
||||||
template <> \
|
template <> \
|
||||||
std::unique_ptr<ProtobufReader::IConverter> ProtobufReader::createConverter<field_type_id>( \
|
std::unique_ptr<ProtobufReader::IConverter> ProtobufReader::createConverter<field_type_id>( \
|
||||||
const google::protobuf::FieldDescriptor * field) \
|
const google::protobuf::FieldDescriptor * field) \
|
||||||
{ \
|
{ \
|
||||||
return std::make_unique<ConverterFromString>(simple_reader, field); \
|
return std::make_unique<ConverterFromString>(simple_reader, field); \
|
||||||
}
|
}
|
||||||
PROTOBUF_READER_CREATE_CONVERTER_SPECIALIZATION_FOR_STRINGS(google::protobuf::FieldDescriptor::TYPE_STRING)
|
PROTOBUF_READER_CREATE_CONVERTER_SPECIALIZATION_FOR_STRINGS(google::protobuf::FieldDescriptor::TYPE_STRING)
|
||||||
PROTOBUF_READER_CREATE_CONVERTER_SPECIALIZATION_FOR_STRINGS(google::protobuf::FieldDescriptor::TYPE_BYTES)
|
PROTOBUF_READER_CREATE_CONVERTER_SPECIALIZATION_FOR_STRINGS(google::protobuf::FieldDescriptor::TYPE_BYTES)
|
||||||
#undef PROTOBUF_READER_CREATE_CONVERTER_SPECIALIZATION_FOR_STRINGS
|
|
||||||
|
# undef PROTOBUF_READER_CREATE_CONVERTER_SPECIALIZATION_FOR_STRINGS
|
||||||
|
|
||||||
|
|
||||||
template <int field_type_id, typename FromType>
|
template <int field_type_id, typename FromType>
|
||||||
@ -850,13 +850,14 @@ private:
|
|||||||
std::optional<std::unordered_set<Int16>> set_of_enum_values;
|
std::optional<std::unordered_set<Int16>> set_of_enum_values;
|
||||||
};
|
};
|
||||||
|
|
||||||
#define PROTOBUF_READER_CREATE_CONVERTER_SPECIALIZATION_FOR_NUMBERS(field_type_id, field_type) \
|
# define PROTOBUF_READER_CREATE_CONVERTER_SPECIALIZATION_FOR_NUMBERS(field_type_id, field_type) \
|
||||||
template <> \
|
template <> \
|
||||||
std::unique_ptr<ProtobufReader::IConverter> ProtobufReader::createConverter<field_type_id>( \
|
std::unique_ptr<ProtobufReader::IConverter> ProtobufReader::createConverter<field_type_id>( \
|
||||||
const google::protobuf::FieldDescriptor * field) \
|
const google::protobuf::FieldDescriptor * field) \
|
||||||
{ \
|
{ \
|
||||||
return std::make_unique<ConverterFromNumber<field_type_id, field_type>>(simple_reader, field); /* NOLINT */ \
|
return std::make_unique<ConverterFromNumber<field_type_id, field_type>>(simple_reader, field); /* NOLINT */ \
|
||||||
}
|
}
|
||||||
|
|
||||||
PROTOBUF_READER_CREATE_CONVERTER_SPECIALIZATION_FOR_NUMBERS(google::protobuf::FieldDescriptor::TYPE_INT32, Int64);
|
PROTOBUF_READER_CREATE_CONVERTER_SPECIALIZATION_FOR_NUMBERS(google::protobuf::FieldDescriptor::TYPE_INT32, Int64);
|
||||||
PROTOBUF_READER_CREATE_CONVERTER_SPECIALIZATION_FOR_NUMBERS(google::protobuf::FieldDescriptor::TYPE_SINT32, Int64);
|
PROTOBUF_READER_CREATE_CONVERTER_SPECIALIZATION_FOR_NUMBERS(google::protobuf::FieldDescriptor::TYPE_SINT32, Int64);
|
||||||
PROTOBUF_READER_CREATE_CONVERTER_SPECIALIZATION_FOR_NUMBERS(google::protobuf::FieldDescriptor::TYPE_UINT32, UInt64);
|
PROTOBUF_READER_CREATE_CONVERTER_SPECIALIZATION_FOR_NUMBERS(google::protobuf::FieldDescriptor::TYPE_UINT32, UInt64);
|
||||||
@ -869,7 +870,8 @@ PROTOBUF_READER_CREATE_CONVERTER_SPECIALIZATION_FOR_NUMBERS(google::protobuf::Fi
|
|||||||
PROTOBUF_READER_CREATE_CONVERTER_SPECIALIZATION_FOR_NUMBERS(google::protobuf::FieldDescriptor::TYPE_SFIXED64, Int64);
|
PROTOBUF_READER_CREATE_CONVERTER_SPECIALIZATION_FOR_NUMBERS(google::protobuf::FieldDescriptor::TYPE_SFIXED64, Int64);
|
||||||
PROTOBUF_READER_CREATE_CONVERTER_SPECIALIZATION_FOR_NUMBERS(google::protobuf::FieldDescriptor::TYPE_FLOAT, float);
|
PROTOBUF_READER_CREATE_CONVERTER_SPECIALIZATION_FOR_NUMBERS(google::protobuf::FieldDescriptor::TYPE_FLOAT, float);
|
||||||
PROTOBUF_READER_CREATE_CONVERTER_SPECIALIZATION_FOR_NUMBERS(google::protobuf::FieldDescriptor::TYPE_DOUBLE, double);
|
PROTOBUF_READER_CREATE_CONVERTER_SPECIALIZATION_FOR_NUMBERS(google::protobuf::FieldDescriptor::TYPE_DOUBLE, double);
|
||||||
#undef PROTOBUF_READER_CREATE_CONVERTER_SPECIALIZATION_FOR_NUMBERS
|
|
||||||
|
# undef PROTOBUF_READER_CREATE_CONVERTER_SPECIALIZATION_FOR_NUMBERS
|
||||||
|
|
||||||
|
|
||||||
class ProtobufReader::ConverterFromBool : public ConverterBaseImpl
|
class ProtobufReader::ConverterFromBool : public ConverterBaseImpl
|
||||||
@ -1073,10 +1075,10 @@ void ProtobufReader::setTraitsDataAfterMatchingColumns(Message * message)
|
|||||||
}
|
}
|
||||||
switch (field.field_descriptor->type())
|
switch (field.field_descriptor->type())
|
||||||
{
|
{
|
||||||
#define PROTOBUF_READER_CONVERTER_CREATING_CASE(field_type_id) \
|
# define PROTOBUF_READER_CONVERTER_CREATING_CASE(field_type_id) \
|
||||||
case field_type_id: \
|
case field_type_id: \
|
||||||
field.data.converter = createConverter<field_type_id>(field.field_descriptor); \
|
field.data.converter = createConverter<field_type_id>(field.field_descriptor); \
|
||||||
break
|
break
|
||||||
PROTOBUF_READER_CONVERTER_CREATING_CASE(google::protobuf::FieldDescriptor::TYPE_STRING);
|
PROTOBUF_READER_CONVERTER_CREATING_CASE(google::protobuf::FieldDescriptor::TYPE_STRING);
|
||||||
PROTOBUF_READER_CONVERTER_CREATING_CASE(google::protobuf::FieldDescriptor::TYPE_BYTES);
|
PROTOBUF_READER_CONVERTER_CREATING_CASE(google::protobuf::FieldDescriptor::TYPE_BYTES);
|
||||||
PROTOBUF_READER_CONVERTER_CREATING_CASE(google::protobuf::FieldDescriptor::TYPE_INT32);
|
PROTOBUF_READER_CONVERTER_CREATING_CASE(google::protobuf::FieldDescriptor::TYPE_INT32);
|
||||||
@ -1093,8 +1095,9 @@ void ProtobufReader::setTraitsDataAfterMatchingColumns(Message * message)
|
|||||||
PROTOBUF_READER_CONVERTER_CREATING_CASE(google::protobuf::FieldDescriptor::TYPE_DOUBLE);
|
PROTOBUF_READER_CONVERTER_CREATING_CASE(google::protobuf::FieldDescriptor::TYPE_DOUBLE);
|
||||||
PROTOBUF_READER_CONVERTER_CREATING_CASE(google::protobuf::FieldDescriptor::TYPE_BOOL);
|
PROTOBUF_READER_CONVERTER_CREATING_CASE(google::protobuf::FieldDescriptor::TYPE_BOOL);
|
||||||
PROTOBUF_READER_CONVERTER_CREATING_CASE(google::protobuf::FieldDescriptor::TYPE_ENUM);
|
PROTOBUF_READER_CONVERTER_CREATING_CASE(google::protobuf::FieldDescriptor::TYPE_ENUM);
|
||||||
#undef PROTOBUF_READER_CONVERTER_CREATING_CASE
|
# undef PROTOBUF_READER_CONVERTER_CREATING_CASE
|
||||||
default: __builtin_unreachable();
|
default:
|
||||||
|
__builtin_unreachable();
|
||||||
}
|
}
|
||||||
message->data.field_number_to_field_map.emplace(field.field_number, &field);
|
message->data.field_number_to_field_map.emplace(field.field_number, &field);
|
||||||
}
|
}
|
||||||
@ -1171,4 +1174,5 @@ bool ProtobufReader::readColumnIndex(size_t & column_index)
|
|||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
#endif
|
#endif
|
||||||
|
@ -5,13 +5,15 @@
|
|||||||
#include <Common/UInt128.h>
|
#include <Common/UInt128.h>
|
||||||
#include <Core/UUID.h>
|
#include <Core/UUID.h>
|
||||||
|
|
||||||
#include "config_formats.h"
|
#if !defined(ARCADIA_BUILD)
|
||||||
#if USE_PROTOBUF
|
# include "config_formats.h"
|
||||||
|
#endif
|
||||||
|
|
||||||
#include <boost/noncopyable.hpp>
|
#if USE_PROTOBUF
|
||||||
#include "ProtobufColumnMatcher.h"
|
# include <memory>
|
||||||
#include <IO/ReadBuffer.h>
|
# include <IO/ReadBuffer.h>
|
||||||
#include <memory>
|
# include <boost/noncopyable.hpp>
|
||||||
|
# include "ProtobufColumnMatcher.h"
|
||||||
|
|
||||||
namespace google
|
namespace google
|
||||||
{
|
{
|
||||||
|
@ -1,10 +1,12 @@
|
|||||||
#include "config_formats.h"
|
#if !defined(ARCADIA_BUILD)
|
||||||
#if USE_PROTOBUF
|
# include "config_formats.h"
|
||||||
|
#endif
|
||||||
|
|
||||||
#include <Formats/FormatSchemaInfo.h>
|
#if USE_PROTOBUF
|
||||||
#include <Formats/ProtobufSchemas.h>
|
# include <Formats/FormatSchemaInfo.h>
|
||||||
#include <google/protobuf/compiler/importer.h>
|
# include <Formats/ProtobufSchemas.h>
|
||||||
#include <Common/Exception.h>
|
# include <google/protobuf/compiler/importer.h>
|
||||||
|
# include <Common/Exception.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
|
@ -1,18 +1,16 @@
|
|||||||
#include "config_formats.h"
|
|
||||||
#if USE_PROTOBUF
|
|
||||||
|
|
||||||
#include "ProtobufWriter.h"
|
#include "ProtobufWriter.h"
|
||||||
|
|
||||||
#include <cassert>
|
#if USE_PROTOBUF
|
||||||
#include <optional>
|
# include <cassert>
|
||||||
#include <math.h>
|
# include <optional>
|
||||||
#include <AggregateFunctions/IAggregateFunction.h>
|
# include <math.h>
|
||||||
#include <DataTypes/DataTypesDecimal.h>
|
# include <AggregateFunctions/IAggregateFunction.h>
|
||||||
#include <boost/numeric/conversion/cast.hpp>
|
# include <DataTypes/DataTypesDecimal.h>
|
||||||
#include <google/protobuf/descriptor.h>
|
# include <IO/ReadHelpers.h>
|
||||||
#include <google/protobuf/descriptor.pb.h>
|
# include <IO/WriteHelpers.h>
|
||||||
#include <IO/ReadHelpers.h>
|
# include <boost/numeric/conversion/cast.hpp>
|
||||||
#include <IO/WriteHelpers.h>
|
# include <google/protobuf/descriptor.h>
|
||||||
|
# include <google/protobuf/descriptor.pb.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
@ -496,19 +494,19 @@ private:
|
|||||||
std::optional<std::unordered_map<Int16, String>> enum_value_to_name_map;
|
std::optional<std::unordered_map<Int16, String>> enum_value_to_name_map;
|
||||||
};
|
};
|
||||||
|
|
||||||
#define PROTOBUF_WRITER_CREATE_CONVERTER_SPECIALIZATION_FOR_STRINGS(field_type_id) \
|
# define PROTOBUF_WRITER_CREATE_CONVERTER_SPECIALIZATION_FOR_STRINGS(field_type_id) \
|
||||||
template <> \
|
template <> \
|
||||||
std::unique_ptr<ProtobufWriter::IConverter> ProtobufWriter::createConverter<field_type_id>( \
|
std::unique_ptr<ProtobufWriter::IConverter> ProtobufWriter::createConverter<field_type_id>( \
|
||||||
const google::protobuf::FieldDescriptor * field) \
|
const google::protobuf::FieldDescriptor * field) \
|
||||||
{ \
|
{ \
|
||||||
if (shouldSkipNullValue(field)) \
|
if (shouldSkipNullValue(field)) \
|
||||||
return std::make_unique<ConverterToString<true>>(simple_writer, field); \
|
return std::make_unique<ConverterToString<true>>(simple_writer, field); \
|
||||||
else \
|
else \
|
||||||
return std::make_unique<ConverterToString<false>>(simple_writer, field); \
|
return std::make_unique<ConverterToString<false>>(simple_writer, field); \
|
||||||
}
|
}
|
||||||
PROTOBUF_WRITER_CREATE_CONVERTER_SPECIALIZATION_FOR_STRINGS(google::protobuf::FieldDescriptor::TYPE_STRING)
|
PROTOBUF_WRITER_CREATE_CONVERTER_SPECIALIZATION_FOR_STRINGS(google::protobuf::FieldDescriptor::TYPE_STRING)
|
||||||
PROTOBUF_WRITER_CREATE_CONVERTER_SPECIALIZATION_FOR_STRINGS(google::protobuf::FieldDescriptor::TYPE_BYTES)
|
PROTOBUF_WRITER_CREATE_CONVERTER_SPECIALIZATION_FOR_STRINGS(google::protobuf::FieldDescriptor::TYPE_BYTES)
|
||||||
#undef PROTOBUF_WRITER_CREATE_CONVERTER_SPECIALIZATION_FOR_STRINGS
|
# undef PROTOBUF_WRITER_CREATE_CONVERTER_SPECIALIZATION_FOR_STRINGS
|
||||||
|
|
||||||
|
|
||||||
template <int field_type_id, typename ToType, bool skip_null_value, bool pack_repeated>
|
template <int field_type_id, typename ToType, bool skip_null_value, bool pack_repeated>
|
||||||
@ -606,18 +604,19 @@ private:
|
|||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
#define PROTOBUF_WRITER_CREATE_CONVERTER_SPECIALIZATION_FOR_NUMBERS(field_type_id, field_type) \
|
# define PROTOBUF_WRITER_CREATE_CONVERTER_SPECIALIZATION_FOR_NUMBERS(field_type_id, field_type) \
|
||||||
template <> \
|
template <> \
|
||||||
std::unique_ptr<ProtobufWriter::IConverter> ProtobufWriter::createConverter<field_type_id>( \
|
std::unique_ptr<ProtobufWriter::IConverter> ProtobufWriter::createConverter<field_type_id>( \
|
||||||
const google::protobuf::FieldDescriptor * field) \
|
const google::protobuf::FieldDescriptor * field) \
|
||||||
{ \
|
{ \
|
||||||
if (shouldSkipNullValue(field)) \
|
if (shouldSkipNullValue(field)) \
|
||||||
return std::make_unique<ConverterToNumber<field_type_id, field_type, true, false>>(simple_writer, field); \
|
return std::make_unique<ConverterToNumber<field_type_id, field_type, true, false>>(simple_writer, field); \
|
||||||
else if (shouldPackRepeated(field)) \
|
else if (shouldPackRepeated(field)) \
|
||||||
return std::make_unique<ConverterToNumber<field_type_id, field_type, false, true>>(simple_writer, field); \
|
return std::make_unique<ConverterToNumber<field_type_id, field_type, false, true>>(simple_writer, field); \
|
||||||
else \
|
else \
|
||||||
return std::make_unique<ConverterToNumber<field_type_id, field_type, false, false>>(simple_writer, field); \
|
return std::make_unique<ConverterToNumber<field_type_id, field_type, false, false>>(simple_writer, field); \
|
||||||
}
|
}
|
||||||
|
|
||||||
PROTOBUF_WRITER_CREATE_CONVERTER_SPECIALIZATION_FOR_NUMBERS(google::protobuf::FieldDescriptor::TYPE_INT32, Int32);
|
PROTOBUF_WRITER_CREATE_CONVERTER_SPECIALIZATION_FOR_NUMBERS(google::protobuf::FieldDescriptor::TYPE_INT32, Int32);
|
||||||
PROTOBUF_WRITER_CREATE_CONVERTER_SPECIALIZATION_FOR_NUMBERS(google::protobuf::FieldDescriptor::TYPE_SINT32, Int32);
|
PROTOBUF_WRITER_CREATE_CONVERTER_SPECIALIZATION_FOR_NUMBERS(google::protobuf::FieldDescriptor::TYPE_SINT32, Int32);
|
||||||
PROTOBUF_WRITER_CREATE_CONVERTER_SPECIALIZATION_FOR_NUMBERS(google::protobuf::FieldDescriptor::TYPE_UINT32, UInt32);
|
PROTOBUF_WRITER_CREATE_CONVERTER_SPECIALIZATION_FOR_NUMBERS(google::protobuf::FieldDescriptor::TYPE_UINT32, UInt32);
|
||||||
@ -630,7 +629,7 @@ PROTOBUF_WRITER_CREATE_CONVERTER_SPECIALIZATION_FOR_NUMBERS(google::protobuf::Fi
|
|||||||
PROTOBUF_WRITER_CREATE_CONVERTER_SPECIALIZATION_FOR_NUMBERS(google::protobuf::FieldDescriptor::TYPE_SFIXED64, Int64);
|
PROTOBUF_WRITER_CREATE_CONVERTER_SPECIALIZATION_FOR_NUMBERS(google::protobuf::FieldDescriptor::TYPE_SFIXED64, Int64);
|
||||||
PROTOBUF_WRITER_CREATE_CONVERTER_SPECIALIZATION_FOR_NUMBERS(google::protobuf::FieldDescriptor::TYPE_FLOAT, float);
|
PROTOBUF_WRITER_CREATE_CONVERTER_SPECIALIZATION_FOR_NUMBERS(google::protobuf::FieldDescriptor::TYPE_FLOAT, float);
|
||||||
PROTOBUF_WRITER_CREATE_CONVERTER_SPECIALIZATION_FOR_NUMBERS(google::protobuf::FieldDescriptor::TYPE_DOUBLE, double);
|
PROTOBUF_WRITER_CREATE_CONVERTER_SPECIALIZATION_FOR_NUMBERS(google::protobuf::FieldDescriptor::TYPE_DOUBLE, double);
|
||||||
#undef PROTOBUF_WRITER_CREATE_CONVERTER_SPECIALIZATION_FOR_NUMBERS
|
# undef PROTOBUF_WRITER_CREATE_CONVERTER_SPECIALIZATION_FOR_NUMBERS
|
||||||
|
|
||||||
|
|
||||||
template <bool skip_null_value, bool pack_repeated>
|
template <bool skip_null_value, bool pack_repeated>
|
||||||
@ -871,10 +870,10 @@ void ProtobufWriter::setTraitsDataAfterMatchingColumns(Message * message)
|
|||||||
}
|
}
|
||||||
switch (field.field_descriptor->type())
|
switch (field.field_descriptor->type())
|
||||||
{
|
{
|
||||||
#define PROTOBUF_WRITER_CONVERTER_CREATING_CASE(field_type_id) \
|
# define PROTOBUF_WRITER_CONVERTER_CREATING_CASE(field_type_id) \
|
||||||
case field_type_id: \
|
case field_type_id: \
|
||||||
field.data.converter = createConverter<field_type_id>(field.field_descriptor); \
|
field.data.converter = createConverter<field_type_id>(field.field_descriptor); \
|
||||||
break
|
break
|
||||||
PROTOBUF_WRITER_CONVERTER_CREATING_CASE(google::protobuf::FieldDescriptor::TYPE_STRING);
|
PROTOBUF_WRITER_CONVERTER_CREATING_CASE(google::protobuf::FieldDescriptor::TYPE_STRING);
|
||||||
PROTOBUF_WRITER_CONVERTER_CREATING_CASE(google::protobuf::FieldDescriptor::TYPE_BYTES);
|
PROTOBUF_WRITER_CONVERTER_CREATING_CASE(google::protobuf::FieldDescriptor::TYPE_BYTES);
|
||||||
PROTOBUF_WRITER_CONVERTER_CREATING_CASE(google::protobuf::FieldDescriptor::TYPE_INT32);
|
PROTOBUF_WRITER_CONVERTER_CREATING_CASE(google::protobuf::FieldDescriptor::TYPE_INT32);
|
||||||
@ -891,7 +890,7 @@ void ProtobufWriter::setTraitsDataAfterMatchingColumns(Message * message)
|
|||||||
PROTOBUF_WRITER_CONVERTER_CREATING_CASE(google::protobuf::FieldDescriptor::TYPE_DOUBLE);
|
PROTOBUF_WRITER_CONVERTER_CREATING_CASE(google::protobuf::FieldDescriptor::TYPE_DOUBLE);
|
||||||
PROTOBUF_WRITER_CONVERTER_CREATING_CASE(google::protobuf::FieldDescriptor::TYPE_BOOL);
|
PROTOBUF_WRITER_CONVERTER_CREATING_CASE(google::protobuf::FieldDescriptor::TYPE_BOOL);
|
||||||
PROTOBUF_WRITER_CONVERTER_CREATING_CASE(google::protobuf::FieldDescriptor::TYPE_ENUM);
|
PROTOBUF_WRITER_CONVERTER_CREATING_CASE(google::protobuf::FieldDescriptor::TYPE_ENUM);
|
||||||
#undef PROTOBUF_WRITER_CONVERTER_CREATING_CASE
|
# undef PROTOBUF_WRITER_CONVERTER_CREATING_CASE
|
||||||
default:
|
default:
|
||||||
throw Exception(
|
throw Exception(
|
||||||
String("Protobuf type '") + field.field_descriptor->type_name() + "' isn't supported", ErrorCodes::NOT_IMPLEMENTED);
|
String("Protobuf type '") + field.field_descriptor->type_name() + "' isn't supported", ErrorCodes::NOT_IMPLEMENTED);
|
||||||
@ -991,4 +990,5 @@ void ProtobufWriter::setNestedMessageNeedsRepeat()
|
|||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
#endif
|
#endif
|
||||||
|
@ -3,17 +3,17 @@
|
|||||||
#include <Core/UUID.h>
|
#include <Core/UUID.h>
|
||||||
#include <Common/UInt128.h>
|
#include <Common/UInt128.h>
|
||||||
#include <common/DayNum.h>
|
#include <common/DayNum.h>
|
||||||
|
|
||||||
#include "config_formats.h"
|
|
||||||
|
|
||||||
#include <memory>
|
#include <memory>
|
||||||
|
|
||||||
#if USE_PROTOBUF
|
#if !defined(ARCADIA_BUILD)
|
||||||
|
# include "config_formats.h"
|
||||||
|
#endif
|
||||||
|
|
||||||
#include "ProtobufColumnMatcher.h"
|
#if USE_PROTOBUF
|
||||||
#include <IO/WriteBufferFromString.h>
|
# include <IO/WriteBufferFromString.h>
|
||||||
#include <boost/noncopyable.hpp>
|
# include <boost/noncopyable.hpp>
|
||||||
#include <Common/PODArray.h>
|
# include <Common/PODArray.h>
|
||||||
|
# include "ProtobufColumnMatcher.h"
|
||||||
|
|
||||||
|
|
||||||
namespace google
|
namespace google
|
||||||
@ -233,6 +233,8 @@ private:
|
|||||||
}
|
}
|
||||||
|
|
||||||
#else
|
#else
|
||||||
|
# include <common/StringRef.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
24
src/Formats/ya.make
Normal file
24
src/Formats/ya.make
Normal file
@ -0,0 +1,24 @@
|
|||||||
|
LIBRARY()
|
||||||
|
|
||||||
|
PEERDIR(
|
||||||
|
clickhouse/src/Common
|
||||||
|
contrib/libs/protobuf_std
|
||||||
|
)
|
||||||
|
|
||||||
|
SRCS(
|
||||||
|
FormatFactory.cpp
|
||||||
|
FormatSchemaInfo.cpp
|
||||||
|
IRowInputStream.cpp
|
||||||
|
IRowOutputStream.cpp
|
||||||
|
MySQLBlockInputStream.cpp
|
||||||
|
NativeFormat.cpp
|
||||||
|
NullFormat.cpp
|
||||||
|
ParsedTemplateFormatString.cpp
|
||||||
|
ProtobufColumnMatcher.cpp
|
||||||
|
ProtobufReader.cpp
|
||||||
|
ProtobufSchemas.cpp
|
||||||
|
ProtobufWriter.cpp
|
||||||
|
verbosePrintString.cpp
|
||||||
|
)
|
||||||
|
|
||||||
|
END()
|
@ -2,9 +2,11 @@
|
|||||||
|
|
||||||
#include <type_traits>
|
#include <type_traits>
|
||||||
#include <Common/Exception.h>
|
#include <Common/Exception.h>
|
||||||
#include <Common/config.h>
|
|
||||||
#include <DataTypes/NumberTraits.h>
|
#include <DataTypes/NumberTraits.h>
|
||||||
|
|
||||||
|
#if !defined(ARCADIA_BUILD)
|
||||||
|
# include <Common/config.h>
|
||||||
|
#endif
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
@ -1,13 +1,16 @@
|
|||||||
#include "config_functions.h"
|
#if !defined(ARCADIA_BUILD)
|
||||||
|
# include "config_functions.h"
|
||||||
|
#endif
|
||||||
|
|
||||||
#if USE_BASE64
|
#if USE_BASE64
|
||||||
#include <Columns/ColumnConst.h>
|
# include <Columns/ColumnConst.h>
|
||||||
#include <Columns/ColumnString.h>
|
# include <Columns/ColumnString.h>
|
||||||
#include <DataTypes/DataTypeString.h>
|
# include <DataTypes/DataTypeString.h>
|
||||||
#include <Functions/FunctionFactory.h>
|
# include <Functions/FunctionFactory.h>
|
||||||
#include <Functions/FunctionHelpers.h>
|
# include <Functions/FunctionHelpers.h>
|
||||||
#include <Functions/GatherUtils/Algorithms.h>
|
# include <Functions/GatherUtils/Algorithms.h>
|
||||||
#include <IO/WriteHelpers.h>
|
# include <IO/WriteHelpers.h>
|
||||||
#include <turbob64.h>
|
# include <turbob64.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
|
@ -27,13 +27,16 @@
|
|||||||
#include "FunctionFactory.h"
|
#include "FunctionFactory.h"
|
||||||
#include <Common/typeid_cast.h>
|
#include <Common/typeid_cast.h>
|
||||||
#include <Common/assert_cast.h>
|
#include <Common/assert_cast.h>
|
||||||
#include <Common/config.h>
|
|
||||||
|
#if !defined(ARCADIA_BUILD)
|
||||||
|
# include <Common/config.h>
|
||||||
|
#endif
|
||||||
|
|
||||||
#if USE_EMBEDDED_COMPILER
|
#if USE_EMBEDDED_COMPILER
|
||||||
#pragma GCC diagnostic push
|
# pragma GCC diagnostic push
|
||||||
#pragma GCC diagnostic ignored "-Wunused-parameter"
|
# pragma GCC diagnostic ignored "-Wunused-parameter"
|
||||||
#include <llvm/IR/IRBuilder.h>
|
# include <llvm/IR/IRBuilder.h>
|
||||||
#pragma GCC diagnostic pop
|
# pragma GCC diagnostic pop
|
||||||
#endif
|
#endif
|
||||||
|
|
||||||
|
|
||||||
|
@ -1,9 +1,11 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include <Common/config.h>
|
|
||||||
#include <Functions/IFunctionImpl.h>
|
#include <Functions/IFunctionImpl.h>
|
||||||
#include <DataTypes/Native.h>
|
#include <DataTypes/Native.h>
|
||||||
|
|
||||||
|
#if !defined(ARCADIA_BUILD)
|
||||||
|
# include <Common/config.h>
|
||||||
|
#endif
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
@ -8,7 +8,10 @@
|
|||||||
#include <Columns/ColumnConst.h>
|
#include <Columns/ColumnConst.h>
|
||||||
#include <Functions/IFunctionImpl.h>
|
#include <Functions/IFunctionImpl.h>
|
||||||
#include <Functions/FunctionHelpers.h>
|
#include <Functions/FunctionHelpers.h>
|
||||||
#include "config_functions.h"
|
|
||||||
|
#if !defined(ARCADIA_BUILD)
|
||||||
|
# include "config_functions.h"
|
||||||
|
#endif
|
||||||
|
|
||||||
/** More efficient implementations of mathematical functions are possible when using a separate library.
|
/** More efficient implementations of mathematical functions are possible when using a separate library.
|
||||||
* Disabled due to license compatibility limitations.
|
* Disabled due to license compatibility limitations.
|
||||||
|
@ -7,7 +7,10 @@
|
|||||||
#include <Columns/ColumnDecimal.h>
|
#include <Columns/ColumnDecimal.h>
|
||||||
#include <Functions/IFunctionImpl.h>
|
#include <Functions/IFunctionImpl.h>
|
||||||
#include <Functions/FunctionHelpers.h>
|
#include <Functions/FunctionHelpers.h>
|
||||||
#include "config_functions.h"
|
|
||||||
|
#if !defined(ARCADIA_BUILD)
|
||||||
|
# include "config_functions.h"
|
||||||
|
#endif
|
||||||
|
|
||||||
/** More efficient implementations of mathematical functions are possible when using a separate library.
|
/** More efficient implementations of mathematical functions are possible when using a separate library.
|
||||||
* Disabled due to license compatibility limitations.
|
* Disabled due to license compatibility limitations.
|
||||||
@ -20,7 +23,7 @@
|
|||||||
* Enabled by default.
|
* Enabled by default.
|
||||||
*/
|
*/
|
||||||
#if USE_FASTOPS
|
#if USE_FASTOPS
|
||||||
#include <fastops/fastops.h>
|
# include <fastops/fastops.h>
|
||||||
#endif
|
#endif
|
||||||
|
|
||||||
|
|
||||||
|
@ -10,13 +10,16 @@
|
|||||||
#include <Functions/IFunctionImpl.h>
|
#include <Functions/IFunctionImpl.h>
|
||||||
#include <Functions/FunctionHelpers.h>
|
#include <Functions/FunctionHelpers.h>
|
||||||
#include <Functions/castTypeToEither.h>
|
#include <Functions/castTypeToEither.h>
|
||||||
#include <Common/config.h>
|
|
||||||
|
#if !defined(ARCADIA_BUILD)
|
||||||
|
# include <Common/config.h>
|
||||||
|
#endif
|
||||||
|
|
||||||
#if USE_EMBEDDED_COMPILER
|
#if USE_EMBEDDED_COMPILER
|
||||||
#pragma GCC diagnostic push
|
# pragma GCC diagnostic push
|
||||||
#pragma GCC diagnostic ignored "-Wunused-parameter"
|
# pragma GCC diagnostic ignored "-Wunused-parameter"
|
||||||
#include <llvm/IR/IRBuilder.h>
|
# include <llvm/IR/IRBuilder.h>
|
||||||
#pragma GCC diagnostic pop
|
# pragma GCC diagnostic pop
|
||||||
#endif
|
#endif
|
||||||
|
|
||||||
|
|
||||||
|
@ -15,9 +15,12 @@
|
|||||||
#include <Dictionaries/Embedded/RegionsHierarchies.h>
|
#include <Dictionaries/Embedded/RegionsHierarchies.h>
|
||||||
#include <Dictionaries/Embedded/RegionsNames.h>
|
#include <Dictionaries/Embedded/RegionsNames.h>
|
||||||
#include <IO/WriteHelpers.h>
|
#include <IO/WriteHelpers.h>
|
||||||
#include <Common/config.h>
|
|
||||||
#include <Common/typeid_cast.h>
|
#include <Common/typeid_cast.h>
|
||||||
|
|
||||||
|
#if !defined(ARCADIA_BUILD)
|
||||||
|
# include <Common/config.h>
|
||||||
|
#endif
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
@ -135,7 +135,9 @@ private:
|
|||||||
!executeDispatchSimple<CacheDictionary>(block, arguments, result, dict_ptr) &&
|
!executeDispatchSimple<CacheDictionary>(block, arguments, result, dict_ptr) &&
|
||||||
!executeDispatchComplex<ComplexKeyHashedDictionary>(block, arguments, result, dict_ptr) &&
|
!executeDispatchComplex<ComplexKeyHashedDictionary>(block, arguments, result, dict_ptr) &&
|
||||||
!executeDispatchComplex<ComplexKeyCacheDictionary>(block, arguments, result, dict_ptr) &&
|
!executeDispatchComplex<ComplexKeyCacheDictionary>(block, arguments, result, dict_ptr) &&
|
||||||
|
#if !defined(ARCADIA_BUILD)
|
||||||
!executeDispatchComplex<TrieDictionary>(block, arguments, result, dict_ptr) &&
|
!executeDispatchComplex<TrieDictionary>(block, arguments, result, dict_ptr) &&
|
||||||
|
#endif
|
||||||
!executeDispatchComplex<SimplePolygonDictionary>(block, arguments, result, dict_ptr))
|
!executeDispatchComplex<SimplePolygonDictionary>(block, arguments, result, dict_ptr))
|
||||||
throw Exception{"Unsupported dictionary type " + dict_ptr->getTypeName(), ErrorCodes::UNKNOWN_TYPE};
|
throw Exception{"Unsupported dictionary type " + dict_ptr->getTypeName(), ErrorCodes::UNKNOWN_TYPE};
|
||||||
}
|
}
|
||||||
@ -306,7 +308,9 @@ private:
|
|||||||
!executeDispatch<CacheDictionary>(block, arguments, result, dict_ptr) &&
|
!executeDispatch<CacheDictionary>(block, arguments, result, dict_ptr) &&
|
||||||
!executeDispatchComplex<ComplexKeyHashedDictionary>(block, arguments, result, dict_ptr) &&
|
!executeDispatchComplex<ComplexKeyHashedDictionary>(block, arguments, result, dict_ptr) &&
|
||||||
!executeDispatchComplex<ComplexKeyCacheDictionary>(block, arguments, result, dict_ptr) &&
|
!executeDispatchComplex<ComplexKeyCacheDictionary>(block, arguments, result, dict_ptr) &&
|
||||||
|
#if !defined(ARCADIA_BUILD)
|
||||||
!executeDispatchComplex<TrieDictionary>(block, arguments, result, dict_ptr) &&
|
!executeDispatchComplex<TrieDictionary>(block, arguments, result, dict_ptr) &&
|
||||||
|
#endif
|
||||||
!executeDispatchComplex<SimplePolygonDictionary>(block, arguments, result, dict_ptr) &&
|
!executeDispatchComplex<SimplePolygonDictionary>(block, arguments, result, dict_ptr) &&
|
||||||
!executeDispatchRange<RangeHashedDictionary>(block, arguments, result, dict_ptr))
|
!executeDispatchRange<RangeHashedDictionary>(block, arguments, result, dict_ptr))
|
||||||
throw Exception{"Unsupported dictionary type " + dict_ptr->getTypeName(), ErrorCodes::UNKNOWN_TYPE};
|
throw Exception{"Unsupported dictionary type " + dict_ptr->getTypeName(), ErrorCodes::UNKNOWN_TYPE};
|
||||||
@ -488,8 +492,10 @@ private:
|
|||||||
!executeDispatch<CacheDictionary>(block, arguments, result, dict_ptr) &&
|
!executeDispatch<CacheDictionary>(block, arguments, result, dict_ptr) &&
|
||||||
!executeDispatchComplex<ComplexKeyHashedDictionary>(block, arguments, result, dict_ptr) &&
|
!executeDispatchComplex<ComplexKeyHashedDictionary>(block, arguments, result, dict_ptr) &&
|
||||||
!executeDispatchComplex<ComplexKeyCacheDictionary>(block, arguments, result, dict_ptr) &&
|
!executeDispatchComplex<ComplexKeyCacheDictionary>(block, arguments, result, dict_ptr) &&
|
||||||
!executeDispatchComplex<SimplePolygonDictionary>(block, arguments, result, dict_ptr) &&
|
#if !defined(ARCADIA_BUILD)
|
||||||
!executeDispatchComplex<TrieDictionary>(block, arguments, result, dict_ptr))
|
!executeDispatchComplex<TrieDictionary>(block, arguments, result, dict_ptr) &&
|
||||||
|
#endif
|
||||||
|
!executeDispatchComplex<SimplePolygonDictionary>(block, arguments, result, dict_ptr))
|
||||||
throw Exception{"Unsupported dictionary type " + dict_ptr->getTypeName(), ErrorCodes::UNKNOWN_TYPE};
|
throw Exception{"Unsupported dictionary type " + dict_ptr->getTypeName(), ErrorCodes::UNKNOWN_TYPE};
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -826,7 +832,9 @@ private:
|
|||||||
!executeDispatch<CacheDictionary>(block, arguments, result, dict_ptr) &&
|
!executeDispatch<CacheDictionary>(block, arguments, result, dict_ptr) &&
|
||||||
!executeDispatchComplex<ComplexKeyHashedDictionary>(block, arguments, result, dict_ptr) &&
|
!executeDispatchComplex<ComplexKeyHashedDictionary>(block, arguments, result, dict_ptr) &&
|
||||||
!executeDispatchComplex<ComplexKeyCacheDictionary>(block, arguments, result, dict_ptr) &&
|
!executeDispatchComplex<ComplexKeyCacheDictionary>(block, arguments, result, dict_ptr) &&
|
||||||
|
#if !defined(ARCADIA_BUILD)
|
||||||
!executeDispatchComplex<TrieDictionary>(block, arguments, result, dict_ptr) &&
|
!executeDispatchComplex<TrieDictionary>(block, arguments, result, dict_ptr) &&
|
||||||
|
#endif
|
||||||
!executeDispatchComplex<SimplePolygonDictionary>(block, arguments, result, dict_ptr) &&
|
!executeDispatchComplex<SimplePolygonDictionary>(block, arguments, result, dict_ptr) &&
|
||||||
!executeDispatchRange<RangeHashedDictionary>(block, arguments, result, dict_ptr))
|
!executeDispatchRange<RangeHashedDictionary>(block, arguments, result, dict_ptr))
|
||||||
throw Exception{"Unsupported dictionary type " + dict_ptr->getTypeName(), ErrorCodes::UNKNOWN_TYPE};
|
throw Exception{"Unsupported dictionary type " + dict_ptr->getTypeName(), ErrorCodes::UNKNOWN_TYPE};
|
||||||
@ -1086,8 +1094,10 @@ private:
|
|||||||
!executeDispatch<CacheDictionary>(block, arguments, result, dict_ptr) &&
|
!executeDispatch<CacheDictionary>(block, arguments, result, dict_ptr) &&
|
||||||
!executeDispatchComplex<ComplexKeyHashedDictionary>(block, arguments, result, dict_ptr) &&
|
!executeDispatchComplex<ComplexKeyHashedDictionary>(block, arguments, result, dict_ptr) &&
|
||||||
!executeDispatchComplex<ComplexKeyCacheDictionary>(block, arguments, result, dict_ptr) &&
|
!executeDispatchComplex<ComplexKeyCacheDictionary>(block, arguments, result, dict_ptr) &&
|
||||||
!executeDispatchComplex<SimplePolygonDictionary>(block, arguments, result, dict_ptr) &&
|
#if !defined(ARCADIA_BUILD)
|
||||||
!executeDispatchComplex<TrieDictionary>(block, arguments, result, dict_ptr))
|
!executeDispatchComplex<TrieDictionary>(block, arguments, result, dict_ptr) &&
|
||||||
|
#endif
|
||||||
|
!executeDispatchComplex<SimplePolygonDictionary>(block, arguments, result, dict_ptr))
|
||||||
throw Exception{"Unsupported dictionary type " + dict_ptr->getTypeName(), ErrorCodes::UNKNOWN_TYPE};
|
throw Exception{"Unsupported dictionary type " + dict_ptr->getTypeName(), ErrorCodes::UNKNOWN_TYPE};
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -26,12 +26,14 @@ void registerFunctionsHashing(FunctionFactory & factory)
|
|||||||
factory.registerFunction<FunctionJavaHash>();
|
factory.registerFunction<FunctionJavaHash>();
|
||||||
factory.registerFunction<FunctionJavaHashUTF16LE>();
|
factory.registerFunction<FunctionJavaHashUTF16LE>();
|
||||||
factory.registerFunction<FunctionHiveHash>();
|
factory.registerFunction<FunctionHiveHash>();
|
||||||
|
#if !defined(ARCADIA_BUILD)
|
||||||
factory.registerFunction<FunctionMurmurHash2_32>();
|
factory.registerFunction<FunctionMurmurHash2_32>();
|
||||||
factory.registerFunction<FunctionMurmurHash2_64>();
|
factory.registerFunction<FunctionMurmurHash2_64>();
|
||||||
factory.registerFunction<FunctionMurmurHash3_32>();
|
factory.registerFunction<FunctionMurmurHash3_32>();
|
||||||
factory.registerFunction<FunctionMurmurHash3_64>();
|
factory.registerFunction<FunctionMurmurHash3_64>();
|
||||||
factory.registerFunction<FunctionMurmurHash3_128>();
|
factory.registerFunction<FunctionMurmurHash3_128>();
|
||||||
factory.registerFunction<FunctionGccMurmurHash>();
|
factory.registerFunction<FunctionGccMurmurHash>();
|
||||||
|
#endif
|
||||||
|
|
||||||
#if USE_XXHASH
|
#if USE_XXHASH
|
||||||
factory.registerFunction<FunctionXxHash32>();
|
factory.registerFunction<FunctionXxHash32>();
|
||||||
|
@ -3,22 +3,24 @@
|
|||||||
#include <city.h>
|
#include <city.h>
|
||||||
#include <farmhash.h>
|
#include <farmhash.h>
|
||||||
#include <metrohash.h>
|
#include <metrohash.h>
|
||||||
#include <murmurhash2.h>
|
#if !defined(ARCADIA_BUILD)
|
||||||
#include <murmurhash3.h>
|
# include <murmurhash2.h>
|
||||||
|
# include <murmurhash3.h>
|
||||||
|
# include "config_functions.h"
|
||||||
|
# include "config_core.h"
|
||||||
|
#endif
|
||||||
|
|
||||||
#include <Common/SipHash.h>
|
#include <Common/SipHash.h>
|
||||||
#include <Common/typeid_cast.h>
|
#include <Common/typeid_cast.h>
|
||||||
#include <Common/HashTable/Hash.h>
|
#include <Common/HashTable/Hash.h>
|
||||||
|
|
||||||
#include "config_functions.h"
|
|
||||||
#if USE_XXHASH
|
#if USE_XXHASH
|
||||||
# include <xxhash.h>
|
# include <xxhash.h>
|
||||||
#endif
|
#endif
|
||||||
|
|
||||||
#include "config_core.h"
|
|
||||||
#if USE_SSL
|
#if USE_SSL
|
||||||
# include <openssl/md5.h>
|
# include <openssl/md5.h>
|
||||||
# include <openssl/sha.h>
|
# include <openssl/sha.h>
|
||||||
#endif
|
#endif
|
||||||
|
|
||||||
#include <Poco/ByteOrder.h>
|
#include <Poco/ByteOrder.h>
|
||||||
@ -219,7 +221,7 @@ struct SipHash128Impl
|
|||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
|
#if !defined(ARCADIA_BUILD)
|
||||||
/** Why we need MurmurHash2?
|
/** Why we need MurmurHash2?
|
||||||
* MurmurHash2 is an outdated hash function, superseded by MurmurHash3 and subsequently by CityHash, xxHash, HighwayHash.
|
* MurmurHash2 is an outdated hash function, superseded by MurmurHash3 and subsequently by CityHash, xxHash, HighwayHash.
|
||||||
* Usually there is no reason to use MurmurHash.
|
* Usually there is no reason to use MurmurHash.
|
||||||
@ -331,6 +333,18 @@ struct MurmurHash3Impl64
|
|||||||
static constexpr bool use_int_hash_for_pods = false;
|
static constexpr bool use_int_hash_for_pods = false;
|
||||||
};
|
};
|
||||||
|
|
||||||
|
struct MurmurHash3Impl128
|
||||||
|
{
|
||||||
|
static constexpr auto name = "murmurHash3_128";
|
||||||
|
enum { length = 16 };
|
||||||
|
|
||||||
|
static void apply(const char * begin, const size_t size, unsigned char * out_char_data)
|
||||||
|
{
|
||||||
|
MurmurHash3_x64_128(begin, size, 0, out_char_data);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
#endif
|
||||||
|
|
||||||
/// http://hg.openjdk.java.net/jdk8u/jdk8u/jdk/file/478a4add975b/src/share/classes/java/lang/String.java#l1452
|
/// http://hg.openjdk.java.net/jdk8u/jdk8u/jdk/file/478a4add975b/src/share/classes/java/lang/String.java#l1452
|
||||||
/// Care should be taken to do all calculation in unsigned integers (to avoid undefined behaviour on overflow)
|
/// Care should be taken to do all calculation in unsigned integers (to avoid undefined behaviour on overflow)
|
||||||
/// but obtain the same result as it is done in singed integers with two's complement arithmetic.
|
/// but obtain the same result as it is done in singed integers with two's complement arithmetic.
|
||||||
@ -411,17 +425,6 @@ struct HiveHashImpl
|
|||||||
static constexpr bool use_int_hash_for_pods = false;
|
static constexpr bool use_int_hash_for_pods = false;
|
||||||
};
|
};
|
||||||
|
|
||||||
struct MurmurHash3Impl128
|
|
||||||
{
|
|
||||||
static constexpr auto name = "murmurHash3_128";
|
|
||||||
enum { length = 16 };
|
|
||||||
|
|
||||||
static void apply(const char * begin, const size_t size, unsigned char * out_char_data)
|
|
||||||
{
|
|
||||||
MurmurHash3_x64_128(begin, size, 0, out_char_data);
|
|
||||||
}
|
|
||||||
};
|
|
||||||
|
|
||||||
struct ImplCityHash64
|
struct ImplCityHash64
|
||||||
{
|
{
|
||||||
static constexpr auto name = "cityHash64";
|
static constexpr auto name = "cityHash64";
|
||||||
@ -1145,12 +1148,16 @@ using FunctionSipHash128 = FunctionStringHashFixedString<SipHash128Impl>;
|
|||||||
using FunctionCityHash64 = FunctionAnyHash<ImplCityHash64>;
|
using FunctionCityHash64 = FunctionAnyHash<ImplCityHash64>;
|
||||||
using FunctionFarmHash64 = FunctionAnyHash<ImplFarmHash64>;
|
using FunctionFarmHash64 = FunctionAnyHash<ImplFarmHash64>;
|
||||||
using FunctionMetroHash64 = FunctionAnyHash<ImplMetroHash64>;
|
using FunctionMetroHash64 = FunctionAnyHash<ImplMetroHash64>;
|
||||||
|
|
||||||
|
#if !defined(ARCADIA_BUILD)
|
||||||
using FunctionMurmurHash2_32 = FunctionAnyHash<MurmurHash2Impl32>;
|
using FunctionMurmurHash2_32 = FunctionAnyHash<MurmurHash2Impl32>;
|
||||||
using FunctionMurmurHash2_64 = FunctionAnyHash<MurmurHash2Impl64>;
|
using FunctionMurmurHash2_64 = FunctionAnyHash<MurmurHash2Impl64>;
|
||||||
using FunctionGccMurmurHash = FunctionAnyHash<GccMurmurHashImpl>;
|
using FunctionGccMurmurHash = FunctionAnyHash<GccMurmurHashImpl>;
|
||||||
using FunctionMurmurHash3_32 = FunctionAnyHash<MurmurHash3Impl32>;
|
using FunctionMurmurHash3_32 = FunctionAnyHash<MurmurHash3Impl32>;
|
||||||
using FunctionMurmurHash3_64 = FunctionAnyHash<MurmurHash3Impl64>;
|
using FunctionMurmurHash3_64 = FunctionAnyHash<MurmurHash3Impl64>;
|
||||||
using FunctionMurmurHash3_128 = FunctionStringHashFixedString<MurmurHash3Impl128>;
|
using FunctionMurmurHash3_128 = FunctionStringHashFixedString<MurmurHash3Impl128>;
|
||||||
|
#endif
|
||||||
|
|
||||||
using FunctionJavaHash = FunctionAnyHash<JavaHashImpl>;
|
using FunctionJavaHash = FunctionAnyHash<JavaHashImpl>;
|
||||||
using FunctionJavaHashUTF16LE = FunctionAnyHash<JavaHashUTF16LEImpl>;
|
using FunctionJavaHashUTF16LE = FunctionAnyHash<JavaHashUTF16LEImpl>;
|
||||||
using FunctionHiveHash = FunctionAnyHash<HiveHashImpl>;
|
using FunctionHiveHash = FunctionAnyHash<HiveHashImpl>;
|
||||||
|
@ -5,7 +5,6 @@
|
|||||||
#include <Functions/DummyJSONParser.h>
|
#include <Functions/DummyJSONParser.h>
|
||||||
#include <Functions/SimdJSONParser.h>
|
#include <Functions/SimdJSONParser.h>
|
||||||
#include <Functions/RapidJSONParser.h>
|
#include <Functions/RapidJSONParser.h>
|
||||||
#include "config_functions.h"
|
|
||||||
#include <Common/CpuId.h>
|
#include <Common/CpuId.h>
|
||||||
#include <Common/typeid_cast.h>
|
#include <Common/typeid_cast.h>
|
||||||
#include <Common/assert_cast.h>
|
#include <Common/assert_cast.h>
|
||||||
@ -27,6 +26,10 @@
|
|||||||
#include <Interpreters/Context.h>
|
#include <Interpreters/Context.h>
|
||||||
#include <ext/range.h>
|
#include <ext/range.h>
|
||||||
|
|
||||||
|
#if !defined(ARCADIA_BUILD)
|
||||||
|
# include "config_functions.h"
|
||||||
|
#endif
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user