diff --git a/base/common/types.h b/base/common/types.h index 8f125146add..238e4e3956b 100644 --- a/base/common/types.h +++ b/base/common/types.h @@ -11,6 +11,10 @@ using Int16 = int16_t; using Int32 = int32_t; using Int64 = int64_t; +#if __cplusplus <= 201703L +using char8_t = unsigned char; +#endif + using UInt8 = char8_t; using UInt16 = uint16_t; using UInt32 = uint32_t; diff --git a/base/common/ya.make b/base/common/ya.make index a41c8f4c583..31a66893330 100644 --- a/base/common/ya.make +++ b/base/common/ya.make @@ -1,12 +1,47 @@ 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( + contrib/libs/cctz/src + contrib/libs/cxxsupp/libcxx-filesystem + contrib/libs/poco/Net contrib/libs/poco/Util + contrib/restricted/boost + contrib/restricted/cityhash-1.0.2 ) SRCS( argsToConfig.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() diff --git a/base/daemon/BaseDaemon.cpp b/base/daemon/BaseDaemon.cpp index c150dc03014..74df1a41a88 100644 --- a/base/daemon/BaseDaemon.cpp +++ b/base/daemon/BaseDaemon.cpp @@ -50,11 +50,13 @@ #include #include #include -#include -#ifdef __APPLE__ -// ucontext is not available without _XOPEN_SOURCE -#define _XOPEN_SOURCE 700 +#if !defined(ARCADIA_BUILD) +# include +#endif + +#if defined(OS_DARWIN) +# define _XOPEN_SOURCE 700 // ucontext is not available without _XOPEN_SOURCE #endif #include @@ -410,7 +412,7 @@ std::string BaseDaemon::getDefaultCorePath() const void BaseDaemon::closeFDs() { -#if defined(__FreeBSD__) || (defined(__APPLE__) && defined(__MACH__)) +#if defined(OS_FREEBSD) || defined(OS_DARWIN) Poco::File proc_path{"/dev/fd"}; #else Poco::File proc_path{"/proc/self/fd"}; @@ -430,7 +432,7 @@ void BaseDaemon::closeFDs() else { int max_fd = -1; -#ifdef _SC_OPEN_MAX +#if defined(_SC_OPEN_MAX) max_fd = sysconf(_SC_OPEN_MAX); if (max_fd == -1) #endif @@ -448,7 +450,7 @@ namespace /// 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 /// care if it breaks. -#if defined(__linux__) && !defined(NDEBUG) +#if defined(OS_LINUX) && !defined(NDEBUG) void debugIncreaseOOMScore() { const std::string new_score = "555"; diff --git a/base/daemon/ya.make b/base/daemon/ya.make new file mode 100644 index 00000000000..1c72af3ed53 --- /dev/null +++ b/base/daemon/ya.make @@ -0,0 +1,14 @@ +LIBRARY() + +NO_COMPILER_WARNINGS() + +PEERDIR( + clickhouse/src/Common +) + +SRCS( + BaseDaemon.cpp + GraphiteWriter.cpp +) + +END() diff --git a/base/loggers/ya.make b/base/loggers/ya.make new file mode 100644 index 00000000000..b1c84042eee --- /dev/null +++ b/base/loggers/ya.make @@ -0,0 +1,15 @@ +LIBRARY() + +PEERDIR( + clickhouse/src/Common +) + +SRCS( + ExtendedLogChannel.cpp + Loggers.cpp + OwnFormattingChannel.cpp + OwnPatternFormatter.cpp + OwnSplitChannel.cpp +) + +END() diff --git a/base/pcg-random/ya.make b/base/pcg-random/ya.make new file mode 100644 index 00000000000..c6a50887178 --- /dev/null +++ b/base/pcg-random/ya.make @@ -0,0 +1,5 @@ +LIBRARY() + +ADDINCL (GLOBAL clickhouse/base/pcg-random) + +END() diff --git a/base/widechar_width/ya.make b/base/widechar_width/ya.make new file mode 100644 index 00000000000..fa0b4f705db --- /dev/null +++ b/base/widechar_width/ya.make @@ -0,0 +1,9 @@ +LIBRARY() + +ADDINCL(GLOBAL clickhouse/base/widechar_width) + +SRCS( + widechar_width.cpp +) + +END() diff --git a/base/ya.make b/base/ya.make index 25ab5886b2c..004da9af2ae 100644 --- a/base/ya.make +++ b/base/ya.make @@ -1,3 +1,7 @@ RECURSE( common + daemon + loggers + pcg-random + widechar_width ) diff --git a/programs/server/HTTPHandler.cpp b/programs/server/HTTPHandler.cpp index ec890c0a96d..77236210741 100644 --- a/programs/server/HTTPHandler.cpp +++ b/programs/server/HTTPHandler.cpp @@ -15,7 +15,6 @@ #include #include #include -#include #include #include #include @@ -36,6 +35,11 @@ #include #include +#if !defined(ARCADIA_BUILD) +# include +#endif + + namespace DB { diff --git a/programs/server/MySQLHandler.cpp b/programs/server/MySQLHandler.cpp index bfab19061ce..97d9f1919ac 100644 --- a/programs/server/MySQLHandler.cpp +++ b/programs/server/MySQLHandler.cpp @@ -1,10 +1,8 @@ -#include - #include "MySQLHandler.h" + #include #include #include -#include #include #include #include @@ -18,11 +16,15 @@ #include #include +#if !defined(ARCADIA_BUILD) +# include +#endif + #if USE_POCO_NETSSL -#include -#include -#include -#include +# include +# include +# include +# include #endif namespace DB diff --git a/programs/server/MySQLHandler.h b/programs/server/MySQLHandler.h index ca5d045beb0..01634dd68dc 100644 --- a/programs/server/MySQLHandler.h +++ b/programs/server/MySQLHandler.h @@ -1,13 +1,17 @@ #pragma once -#include + #include #include #include #include #include "IServer.h" +#if !defined(ARCADIA_BUILD) +# include +#endif + #if USE_POCO_NETSSL -#include +# include #endif namespace CurrentMetrics diff --git a/programs/server/MySQLHandlerFactory.h b/programs/server/MySQLHandlerFactory.h index b7df9fa60e0..74f0bb35a40 100644 --- a/programs/server/MySQLHandlerFactory.h +++ b/programs/server/MySQLHandlerFactory.h @@ -1,11 +1,15 @@ #pragma once -#include #include #include #include "IServer.h" + +#if !defined(ARCADIA_BUILD) +# include +#endif + #if USE_SSL -#include +# include #endif namespace DB diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 5322514e7c2..3490ff6a445 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -15,7 +15,6 @@ #include #include #include -#include #include #include #include @@ -26,7 +25,6 @@ #include #include #include -#include "config_core.h" #include #include #include @@ -59,19 +57,24 @@ #include "MetricsTransmitter.h" #include #include "TCPHandlerFactory.h" -#include "Common/config_version.h" #include #include #include "MySQLHandlerFactory.h" +#if !defined(ARCADIA_BUILD) +# include +# include "config_core.h" +# include "Common/config_version.h" +#endif + #if defined(OS_LINUX) -#include -#include +# include +# include #endif #if USE_POCO_NETSSL -#include -#include +# include +# include #endif namespace CurrentMetrics @@ -248,7 +251,7 @@ int Server::main(const std::vector & /*args*/) const auto memory_amount = getMemoryAmount(); -#if defined(__linux__) +#if defined(OS_LINUX) std::string executable_path = getExecutablePath(); if (executable_path.empty()) executable_path = "/usr/bin/clickhouse"; /// It is used for information messages. @@ -631,7 +634,7 @@ int Server::main(const std::vector & /*args*/) dns_cache_updater = std::make_unique(*global_context, config().getInt("dns_cache_update_period", 15)); } -#if defined(__linux__) +#if defined(OS_LINUX) if (!TaskStatsInfoGetter::checkPermissions()) { LOG_INFO(log, "It looks like the process has no CAP_NET_ADMIN capability, 'taskstats' performance statistics will be disabled." diff --git a/programs/server/TCPHandler.cpp b/programs/server/TCPHandler.cpp index d82c6e31528..70825cb4f7e 100644 --- a/programs/server/TCPHandler.cpp +++ b/programs/server/TCPHandler.cpp @@ -6,7 +6,6 @@ #include #include #include -#include #include #include #include @@ -33,6 +32,10 @@ #include "TCPHandler.h" +#if !defined(ARCADIA_BUILD) +# include +#endif + namespace DB { diff --git a/programs/server/ya.make b/programs/server/ya.make new file mode 100644 index 00000000000..7f0a3969fd2 --- /dev/null +++ b/programs/server/ya.make @@ -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() diff --git a/programs/ya.make b/programs/ya.make new file mode 100644 index 00000000000..6c773c312b8 --- /dev/null +++ b/programs/ya.make @@ -0,0 +1,3 @@ +RECURSE( + server +) diff --git a/src/Access/AccessFlags.h b/src/Access/AccessFlags.h index c8f57fcd419..cbba295be1a 100644 --- a/src/Access/AccessFlags.h +++ b/src/Access/AccessFlags.h @@ -253,7 +253,7 @@ private: } else { - if (nodes.contains(keyword)) + if (nodes.count(keyword)) throw Exception(keyword + " declared twice", ErrorCodes::LOGICAL_ERROR); node = std::make_unique(keyword, node_type); nodes[node->keyword] = node.get(); @@ -279,7 +279,7 @@ private: { auto parent_node = std::make_unique(parent_keyword); 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; owned_nodes[parent_keyword_as_string_view] = std::move(parent_node); } @@ -299,9 +299,9 @@ private: #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); - if (!owned_nodes.contains("ALL")) + if (!owned_nodes.count("ALL")) throw Exception("'ALL' not declared", ErrorCodes::LOGICAL_ERROR); flags_to_keyword_tree = std::move(owned_nodes["ALL"]); diff --git a/src/Access/ContextAccess.cpp b/src/Access/ContextAccess.cpp index 915593f58f0..ab504e32579 100644 --- a/src/Access/ContextAccess.cpp +++ b/src/Access/ContextAccess.cpp @@ -147,9 +147,9 @@ void ContextAccess::setUser(const UserPtr & user_) const current_roles.reserve(params.current_roles.size()); for (const auto & id : params.current_roles) { - if (user->granted_roles.contains(id)) + if (user->granted_roles.count(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); } } @@ -358,7 +358,7 @@ void ContextAccess::checkAdminOption(const UUID & role_id) const return; 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; std::optional role_name = manager->readName(role_id); diff --git a/src/Access/DiskAccessStorage.cpp b/src/Access/DiskAccessStorage.cpp index 12c65e7df1e..263f4e82d95 100644 --- a/src/Access/DiskAccessStorage.cpp +++ b/src/Access/DiskAccessStorage.cpp @@ -560,7 +560,7 @@ std::vector DiskAccessStorage::findAllImpl(std::type_index type) const bool DiskAccessStorage::existsImpl(const UUID & id) const { 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) { 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); scheduleWriteLists(type); } diff --git a/src/Access/ExtendedRoleSet.cpp b/src/Access/ExtendedRoleSet.cpp index eed475bc3cc..145bd0fe7e5 100644 --- a/src/Access/ExtendedRoleSet.cpp +++ b/src/Access/ExtendedRoleSet.cpp @@ -253,44 +253,44 @@ void ExtendedRoleSet::add(const boost::container::flat_set & ids_) 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 & enabled_roles) const { - if (!all && !ids.contains(user_id)) + if (!all && !ids.count(user_id)) { 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) return false; } - if (except_ids.contains(user_id)) + if (except_ids.count(user_id)) return false; 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; } bool ExtendedRoleSet::match(const UUID & user_id, const boost::container::flat_set & enabled_roles) const { - if (!all && !ids.contains(user_id)) + if (!all && !ids.count(user_id)) { 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) return false; } - if (except_ids.contains(user_id)) + if (except_ids.count(user_id)) return false; 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; } diff --git a/src/Access/QuotaCache.cpp b/src/Access/QuotaCache.cpp index 6db3eb66c5d..96cd6b3e8bb 100644 --- a/src/Access/QuotaCache.cpp +++ b/src/Access/QuotaCache.cpp @@ -250,16 +250,18 @@ void QuotaCache::quotaRemoved(const UUID & quota_id) void QuotaCache::chooseQuotaToConsume() { /// `mutex` is already locked. - std::erase_if( - enabled_quotas, - [&](const std::pair> & pr) + + for (auto i = enabled_quotas.begin(), e = enabled_quotas.end(); i != e;) + { + 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); - return false; // keep in the `enabled_quotas` list. - }); + ++i; + } + } } void QuotaCache::chooseQuotaToConsumeFor(EnabledQuota & enabled) diff --git a/src/Access/RoleCache.cpp b/src/Access/RoleCache.cpp index 63e19a3cb40..0263b793017 100644 --- a/src/Access/RoleCache.cpp +++ b/src/Access/RoleCache.cpp @@ -103,16 +103,17 @@ void RoleCache::collectRolesInfo() { /// `mutex` is already locked. - std::erase_if( - enabled_roles, - [&](const std::pair> & pr) + for (auto i = enabled_roles.begin(), e = enabled_roles.end(); i != e;) + { + 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); - return false; // keep in the `enabled_roles` map. - }); + ++i; + } + } } diff --git a/src/Access/RowPolicyCache.cpp b/src/Access/RowPolicyCache.cpp index 44f2cd160d4..04b4202c211 100644 --- a/src/Access/RowPolicyCache.cpp +++ b/src/Access/RowPolicyCache.cpp @@ -178,16 +178,17 @@ void RowPolicyCache::rowPolicyRemoved(const UUID & policy_id) void RowPolicyCache::mixConditions() { /// `mutex` is already locked. - std::erase_if( - enabled_row_policies, - [&](const std::pair> & pr) + for (auto i = enabled_row_policies.begin(), e = enabled_row_policies.end(); i != e;) + { + 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); - return false; // keep in the `enabled_row_policies` map. - }); + ++i; + } + } } diff --git a/src/Access/SettingsProfilesCache.cpp b/src/Access/SettingsProfilesCache.cpp index 552ed324635..f283715e129 100644 --- a/src/Access/SettingsProfilesCache.cpp +++ b/src/Access/SettingsProfilesCache.cpp @@ -104,16 +104,17 @@ void SettingsProfilesCache::setDefaultProfileName(const String & default_profile void SettingsProfilesCache::mergeSettingsAndConstraints() { /// `mutex` is already locked. - std::erase_if( - enabled_settings, - [&](const std::pair> & pr) + for (auto i = enabled_settings.begin(), e = enabled_settings.end(); i != e;) + { + 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); - 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; element.parent_profile.reset(); - if (already_substituted.contains(parent_profile_id)) + if (already_substituted.count(parent_profile_id)) continue; already_substituted.insert(parent_profile_id); diff --git a/src/Access/ya.make b/src/Access/ya.make new file mode 100644 index 00000000000..fb2e23e0684 --- /dev/null +++ b/src/Access/ya.make @@ -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() diff --git a/src/AggregateFunctions/registerAggregateFunctions.cpp b/src/AggregateFunctions/registerAggregateFunctions.cpp index a4fc41e9c06..a9ab1d4f8ea 100644 --- a/src/AggregateFunctions/registerAggregateFunctions.cpp +++ b/src/AggregateFunctions/registerAggregateFunctions.cpp @@ -32,7 +32,9 @@ void registerAggregateFunctions() registerAggregateFunctionUniqUpTo(factory); registerAggregateFunctionTopK(factory); registerAggregateFunctionsBitwise(factory); +#if !defined(ARCADIA_BUILD) registerAggregateFunctionsBitmap(factory); +#endif registerAggregateFunctionsMaxIntersections(factory); registerAggregateFunctionHistogram(factory); registerAggregateFunctionRetention(factory); diff --git a/src/AggregateFunctions/ya.make b/src/AggregateFunctions/ya.make new file mode 100644 index 00000000000..bfa32b6dd78 --- /dev/null +++ b/src/AggregateFunctions/ya.make @@ -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() diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index f530652caae..cc79e3ec2c8 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -19,16 +19,19 @@ #include #include #include -#include #include #include #include #include #include -#include +#if !defined(ARCADIA_BUILD) +# include +# include +#endif + #if USE_POCO_NETSSL -#include +# include #endif namespace CurrentMetrics diff --git a/src/Client/ya.make b/src/Client/ya.make new file mode 100644 index 00000000000..685e0540ad7 --- /dev/null +++ b/src/Client/ya.make @@ -0,0 +1,15 @@ +LIBRARY() + +PEERDIR( + clickhouse/src/Common + contrib/libs/poco/NetSSL_OpenSSL +) + +SRCS( + Connection.cpp + ConnectionPoolWithFailover.cpp + MultiplexedConnections.cpp + TimeoutSetter.cpp +) + +END() diff --git a/src/Columns/Collator.cpp b/src/Columns/Collator.cpp index c3915125fa1..77735564077 100644 --- a/src/Columns/Collator.cpp +++ b/src/Columns/Collator.cpp @@ -1,17 +1,19 @@ #include -#include "config_core.h" +#if !defined(ARCADIA_BUILD) +# include "config_core.h" +#endif #if USE_ICU - #include - #include - #include - #include +# include +# include +# include +# include #else - #ifdef __clang__ - #pragma clang diagnostic ignored "-Wunused-private-field" - #pragma clang diagnostic ignored "-Wmissing-noreturn" - #endif +# if defined(__clang__) +# pragma clang diagnostic ignored "-Wunused-private-field" +# pragma clang diagnostic ignored "-Wmissing-noreturn" +# endif #endif #include diff --git a/src/Columns/ya.make b/src/Columns/ya.make new file mode 100644 index 00000000000..b36a524e775 --- /dev/null +++ b/src/Columns/ya.make @@ -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() diff --git a/src/Common/ClickHouseRevision.cpp b/src/Common/ClickHouseRevision.cpp index 5513922a655..0b81026adca 100644 --- a/src/Common/ClickHouseRevision.cpp +++ b/src/Common/ClickHouseRevision.cpp @@ -1,5 +1,8 @@ #include -#include + +#if !defined(ARCADIA_BUILD) +# include +#endif namespace ClickHouseRevision { diff --git a/src/Common/Exception.cpp b/src/Common/Exception.cpp index bc3d4a78969..edeb3be069a 100644 --- a/src/Common/Exception.cpp +++ b/src/Common/Exception.cpp @@ -8,11 +8,14 @@ #include #include #include -#include #include #include #include +#if !defined(ARCADIA_BUILD) +# include +#endif + namespace DB { diff --git a/src/Common/OpenSSLHelpers.h b/src/Common/OpenSSLHelpers.h index 8155ffd30ef..e77fc3037c1 100644 --- a/src/Common/OpenSSLHelpers.h +++ b/src/Common/OpenSSLHelpers.h @@ -1,8 +1,11 @@ #pragma once -#include -#if USE_SSL -#include +#if !defined(ARCADIA_BUILD) +# include +#endif + +#if USE_SSL +# include namespace DB diff --git a/src/Common/OptimizedRegularExpression.h b/src/Common/OptimizedRegularExpression.h index 0c31558cecb..fddefe596c4 100644 --- a/src/Common/OptimizedRegularExpression.h +++ b/src/Common/OptimizedRegularExpression.h @@ -5,12 +5,16 @@ #include #include #include -#include #include + +#if !defined(ARCADIA_BUILD) +# include +#endif + #if USE_RE2_ST - #include +# include #else - #define re2_st re2 +# define re2_st re2 #endif diff --git a/src/Common/QueryProfiler.cpp b/src/Common/QueryProfiler.cpp index dd9f36fb3ae..a78c2b2ea6c 100644 --- a/src/Common/QueryProfiler.cpp +++ b/src/Common/QueryProfiler.cpp @@ -6,7 +6,6 @@ #include #include #include -#include #include #include diff --git a/src/Common/QueryProfiler.h b/src/Common/QueryProfiler.h index 17280372237..f00a081972e 100644 --- a/src/Common/QueryProfiler.h +++ b/src/Common/QueryProfiler.h @@ -1,11 +1,14 @@ #pragma once #include -#include -#include #include #include +#if !defined(ARCADIA_BUILD) +# include +# include +#endif + namespace Poco { diff --git a/src/Common/StackTrace.cpp b/src/Common/StackTrace.cpp index 785a97e4d74..48633ef01c1 100644 --- a/src/Common/StackTrace.cpp +++ b/src/Common/StackTrace.cpp @@ -3,7 +3,6 @@ #include #include #include -#include #include #include #include @@ -14,8 +13,12 @@ #include #include +#if !defined(ARCADIA_BUILD) +# include +#endif + #if USE_UNWIND -# include +# include #endif std::string signalToErrorMessage(int sig, const siginfo_t & info, const ucontext_t & context) diff --git a/src/Common/XDBCBridgeHelper.h b/src/Common/XDBCBridgeHelper.h index b9d1f2cdcdf..41aeb421394 100644 --- a/src/Common/XDBCBridgeHelper.h +++ b/src/Common/XDBCBridgeHelper.h @@ -13,10 +13,13 @@ #include #include #include -#include #include #include +#if !defined(ARCADIA_BUILD) +# include +#endif + namespace DB { namespace ErrorCodes diff --git a/src/Common/getNumberOfPhysicalCPUCores.cpp b/src/Common/getNumberOfPhysicalCPUCores.cpp index 32b70b76fbd..1eaa64ae2a4 100644 --- a/src/Common/getNumberOfPhysicalCPUCores.cpp +++ b/src/Common/getNumberOfPhysicalCPUCores.cpp @@ -1,11 +1,16 @@ #include #include -#include +#if !defined(ARCADIA_BUILD) +# include +#else +# include +#endif + #if USE_CPUID -# include +# include #elif USE_CPUINFO -# include +# include #endif diff --git a/src/Common/new_delete.cpp b/src/Common/new_delete.cpp index 0aa5f8aacce..ed0d99679e4 100644 --- a/src/Common/new_delete.cpp +++ b/src/Common/new_delete.cpp @@ -1,4 +1,3 @@ -#include #include #include @@ -14,14 +13,13 @@ /// Replace default new/delete with memory tracking versions. /// @sa https://en.cppreference.com/w/cpp/memory/new/operator_new /// https://en.cppreference.com/w/cpp/memory/new/operator_delete -#if !UNBUNDLED namespace Memory { 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 /// @note je_mallocx() != je_malloc(). It's expected they don't differ much in allocation logic. if (likely(size != 0)) @@ -49,18 +47,18 @@ inline ALWAYS_INLINE void untrackMemory(void * ptr [[maybe_unused]], std::size_t { try { -#if USE_JEMALLOC +#if USE_JEMALLOC && JEMALLOC_VERSION_MAJOR >= 5 /// @note It's also possible to use je_malloc_usable_size() here. if (likely(ptr != nullptr)) CurrentMemoryTracker::free(sallocx(ptr, 0)); #else if (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. else CurrentMemoryTracker::free(malloc_usable_size(ptr)); -# endif +# endif #endif } catch (...) @@ -130,26 +128,3 @@ void operator delete[](void * ptr, std::size_t size) noexcept Memory::untrackMemory(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 diff --git a/src/Common/ya.make b/src/Common/ya.make new file mode 100644 index 00000000000..53155ac471f --- /dev/null +++ b/src/Common/ya.make @@ -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() diff --git a/src/Compression/ya.make b/src/Compression/ya.make new file mode 100644 index 00000000000..55b5287e3f3 --- /dev/null +++ b/src/Compression/ya.make @@ -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() diff --git a/src/Core/Field.h b/src/Core/Field.h index 5e62e4ef3e4..152ae29bd1e 100644 --- a/src/Core/Field.h +++ b/src/Core/Field.h @@ -157,7 +157,9 @@ private: template <> struct NearestFieldTypeImpl { using Type = std::conditional_t, Int64, UInt64>; }; template <> struct NearestFieldTypeImpl { using Type = Int64; }; template <> struct NearestFieldTypeImpl { using Type = UInt64; }; +#if __cplusplus > 201703L template <> struct NearestFieldTypeImpl { using Type = UInt64; }; +#endif template <> struct NearestFieldTypeImpl { using Type = UInt64; }; template <> struct NearestFieldTypeImpl { using Type = UInt64; }; diff --git a/src/Core/MySQLProtocol.h b/src/Core/MySQLProtocol.h index 5255c6f263e..20dd25b6b2d 100644 --- a/src/Core/MySQLProtocol.h +++ b/src/Core/MySQLProtocol.h @@ -22,10 +22,14 @@ #include #include #include -#include "config_core.h" + +#if !defined(ARCADIA_BUILD) +# include "config_core.h" +#endif + #if USE_SSL -#include -#include +# include +# include #endif /// Implementation of MySQL wire protocol. diff --git a/src/Core/NamesAndTypes.cpp b/src/Core/NamesAndTypes.cpp index b45e7d771a9..3a55a4328a7 100644 --- a/src/Core/NamesAndTypes.cpp +++ b/src/Core/NamesAndTypes.cpp @@ -137,8 +137,12 @@ NamesAndTypesList NamesAndTypesList::filter(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. - ::google::dense_hash_map types; + /// NOTE: It's better to make a map in `IStorage` than to create it here every time again. +#if !defined(ARCADIA_BUILD) + google::dense_hash_map types; +#else + google::sparsehash::dense_hash_map types; +#endif types.set_empty_key(StringRef()); for (const NameAndTypePair & column : *this) diff --git a/src/Core/ya.make b/src/Core/ya.make new file mode 100644 index 00000000000..4999fe334bc --- /dev/null +++ b/src/Core/ya.make @@ -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() diff --git a/src/DataStreams/ya.make b/src/DataStreams/ya.make new file mode 100644 index 00000000000..e6f8d8a94d7 --- /dev/null +++ b/src/DataStreams/ya.make @@ -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() diff --git a/src/DataTypes/Native.h b/src/DataTypes/Native.h index 3652e85e2d3..754365dfc87 100644 --- a/src/DataTypes/Native.h +++ b/src/DataTypes/Native.h @@ -1,23 +1,25 @@ #pragma once -#include "config_core.h" +#if !defined(ARCADIA_BUILD) +# include "config_core.h" +#endif + #if USE_EMBEDDED_COMPILER +# include +# include +# include +# include +# include +# include +# include +# include -#include -#include -#include -#include -#include -#include -#include -#include +# pragma GCC diagnostic push +# pragma GCC diagnostic ignored "-Wunused-parameter" -#pragma GCC diagnostic push -#pragma GCC diagnostic ignored "-Wunused-parameter" +# include -#include - -#pragma GCC diagnostic pop +# pragma GCC diagnostic pop namespace DB diff --git a/src/DataTypes/ya.make b/src/DataTypes/ya.make new file mode 100644 index 00000000000..c3a1e452d0d --- /dev/null +++ b/src/DataTypes/ya.make @@ -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() diff --git a/src/Databases/DatabaseFactory.cpp b/src/Databases/DatabaseFactory.cpp index f1cea04dc29..b4a5537a192 100644 --- a/src/Databases/DatabaseFactory.cpp +++ b/src/Databases/DatabaseFactory.cpp @@ -9,15 +9,16 @@ #include #include #include -#include "config_core.h" #include "DatabaseFactory.h" #include +#if !defined(ARCADIA_BUILD) +# include "config_core.h" +#endif + #if USE_MYSQL - -#include -#include - +# include +# include #endif diff --git a/src/Databases/DatabaseMySQL.cpp b/src/Databases/DatabaseMySQL.cpp index 1cbbd4b06d9..7528e3fa719 100644 --- a/src/Databases/DatabaseMySQL.cpp +++ b/src/Databases/DatabaseMySQL.cpp @@ -1,29 +1,30 @@ -#include "config_core.h" +#if !defined(ARCADIA_BUILD) +# include "config_core.h" +#endif #if USE_MYSQL +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include -#include +# include +# include namespace DB diff --git a/src/Databases/ya.make b/src/Databases/ya.make new file mode 100644 index 00000000000..4aff1054023 --- /dev/null +++ b/src/Databases/ya.make @@ -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() diff --git a/src/Dictionaries/CMakeLists.txt b/src/Dictionaries/CMakeLists.txt index c7766fbc628..0a7693edc60 100644 --- a/src/Dictionaries/CMakeLists.txt +++ b/src/Dictionaries/CMakeLists.txt @@ -1,14 +1,7 @@ include(${ClickHouse_SOURCE_DIR}/cmake/dbms_glob_sources.cmake) -include(${ClickHouse_SOURCE_DIR}/cmake/generate_code.cmake) 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/) list(REMOVE_ITEM clickhouse_dictionaries_sources DictionaryFactory.cpp DictionarySourceFactory.cpp DictionaryStructure.cpp getDictionaryConfigurationFromAST.cpp) diff --git a/src/Dictionaries/CacheDictionary_generate1.cpp b/src/Dictionaries/CacheDictionary_generate1.cpp new file mode 100644 index 00000000000..edb4b89d550 --- /dev/null +++ b/src/Dictionaries/CacheDictionary_generate1.cpp @@ -0,0 +1,32 @@ +#include +#include + +namespace DB +{ +#define DEFINE(TYPE) \ + void CacheDictionary::get##TYPE(const std::string & attribute_name, const PaddedPODArray & ids, ResultArrayType & out) \ + const \ + { \ + auto & attribute = getAttribute(attribute_name); \ + checkAttributeType(name, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \ + const auto null_value = std::get(attribute.null_values); \ + getItemsNumberImpl(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 +} diff --git a/src/Dictionaries/CacheDictionary_generate1.cpp.in b/src/Dictionaries/CacheDictionary_generate1.cpp.in deleted file mode 100644 index b870a0ed69b..00000000000 --- a/src/Dictionaries/CacheDictionary_generate1.cpp.in +++ /dev/null @@ -1,22 +0,0 @@ -#include -#include - -namespace DB -{ -namespace ErrorCodes -{ - extern const int TYPE_MISMATCH; -} - -using TYPE = @NAME@; -void CacheDictionary::get@NAME@(const std::string & attribute_name, const PaddedPODArray & ids, ResultArrayType & out) const -{ - auto & attribute = getAttribute(attribute_name); - checkAttributeType(name, attribute_name, attribute.type, AttributeUnderlyingType::ut@NAME@); - - const auto null_value = std::get(attribute.null_values); - - getItemsNumberImpl(attribute, ids, out, [&](const size_t) { return null_value; }); -} - -} diff --git a/src/Dictionaries/CacheDictionary_generate2.cpp b/src/Dictionaries/CacheDictionary_generate2.cpp new file mode 100644 index 00000000000..97fa9e1a365 --- /dev/null +++ b/src/Dictionaries/CacheDictionary_generate2.cpp @@ -0,0 +1,34 @@ +#include +#include + +namespace DB +{ +#define DEFINE(TYPE) \ + void CacheDictionary::get##TYPE( \ + const std::string & attribute_name, \ + const PaddedPODArray & ids, \ + const PaddedPODArray & def, \ + ResultArrayType & out) const \ + { \ + auto & attribute = getAttribute(attribute_name); \ + checkAttributeType(name, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \ + getItemsNumberImpl(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 +} diff --git a/src/Dictionaries/CacheDictionary_generate2.cpp.in b/src/Dictionaries/CacheDictionary_generate2.cpp.in deleted file mode 100644 index 367e150c2cb..00000000000 --- a/src/Dictionaries/CacheDictionary_generate2.cpp.in +++ /dev/null @@ -1,23 +0,0 @@ -#include -#include - -namespace DB -{ -namespace ErrorCodes -{ - extern const int TYPE_MISMATCH; -} - -using TYPE = @NAME@; -void CacheDictionary::get@NAME@(const std::string & attribute_name, - const PaddedPODArray & ids, - const PaddedPODArray & def, - ResultArrayType & out) const -{ - auto & attribute = getAttribute(attribute_name); - checkAttributeType(name, attribute_name, attribute.type, AttributeUnderlyingType::ut@NAME@); - - getItemsNumberImpl(attribute, ids, out, [&](const size_t row) { return def[row]; }); -} - -} diff --git a/src/Dictionaries/CacheDictionary_generate3.cpp b/src/Dictionaries/CacheDictionary_generate3.cpp new file mode 100644 index 00000000000..8a94ca6bc20 --- /dev/null +++ b/src/Dictionaries/CacheDictionary_generate3.cpp @@ -0,0 +1,31 @@ +#include +#include + +namespace DB +{ +#define DEFINE(TYPE) \ + void CacheDictionary::get##TYPE( \ + const std::string & attribute_name, const PaddedPODArray & ids, const TYPE def, ResultArrayType & out) const \ + { \ + auto & attribute = getAttribute(attribute_name); \ + checkAttributeType(name, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \ + getItemsNumberImpl(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 +} diff --git a/src/Dictionaries/CacheDictionary_generate3.cpp.in b/src/Dictionaries/CacheDictionary_generate3.cpp.in deleted file mode 100644 index 8e2c26302e8..00000000000 --- a/src/Dictionaries/CacheDictionary_generate3.cpp.in +++ /dev/null @@ -1,20 +0,0 @@ -#include -#include - -namespace DB -{ -namespace ErrorCodes -{ - extern const int TYPE_MISMATCH; -} - -using TYPE = @NAME@; -void CacheDictionary::get@NAME@(const std::string & attribute_name, const PaddedPODArray & ids, const TYPE def, ResultArrayType & out) const -{ - auto & attribute = getAttribute(attribute_name); - checkAttributeType(name, attribute_name, attribute.type, AttributeUnderlyingType::ut@NAME@); - - getItemsNumberImpl(attribute, ids, out, [&](const size_t) { return def; }); -} - -} diff --git a/src/Dictionaries/ComplexKeyCacheDictionary_generate1.cpp b/src/Dictionaries/ComplexKeyCacheDictionary_generate1.cpp new file mode 100644 index 00000000000..6f9761cd064 --- /dev/null +++ b/src/Dictionaries/ComplexKeyCacheDictionary_generate1.cpp @@ -0,0 +1,32 @@ +#include + +namespace DB +{ +#define DEFINE(TYPE) \ + void ComplexKeyCacheDictionary::get##TYPE( \ + const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types, ResultArrayType & 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(attribute.null_values); \ + getItemsNumberImpl(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 +} diff --git a/src/Dictionaries/ComplexKeyCacheDictionary_generate1.cpp.in b/src/Dictionaries/ComplexKeyCacheDictionary_generate1.cpp.in deleted file mode 100644 index 5c0ed408a55..00000000000 --- a/src/Dictionaries/ComplexKeyCacheDictionary_generate1.cpp.in +++ /dev/null @@ -1,22 +0,0 @@ -#include - -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 & 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(attribute.null_values); - - getItemsNumberImpl(attribute, key_columns, out, [&](const size_t) { return null_value; }); -} -} diff --git a/src/Dictionaries/ComplexKeyCacheDictionary_generate2.cpp b/src/Dictionaries/ComplexKeyCacheDictionary_generate2.cpp new file mode 100644 index 00000000000..297da2c91a0 --- /dev/null +++ b/src/Dictionaries/ComplexKeyCacheDictionary_generate2.cpp @@ -0,0 +1,35 @@ +#include + +namespace DB +{ +#define DEFINE(TYPE) \ + void ComplexKeyCacheDictionary::get##TYPE( \ + const std::string & attribute_name, \ + const Columns & key_columns, \ + const DataTypes & key_types, \ + const PaddedPODArray & def, \ + ResultArrayType & out) const \ + { \ + dict_struct.validateKeyTypes(key_types); \ + auto & attribute = getAttribute(attribute_name); \ + checkAttributeType(name, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \ + getItemsNumberImpl(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 +} diff --git a/src/Dictionaries/ComplexKeyCacheDictionary_generate2.cpp.in b/src/Dictionaries/ComplexKeyCacheDictionary_generate2.cpp.in deleted file mode 100644 index b3233cd05e1..00000000000 --- a/src/Dictionaries/ComplexKeyCacheDictionary_generate2.cpp.in +++ /dev/null @@ -1,25 +0,0 @@ -#include - -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 & def, - ResultArrayType & out) const -{ - dict_struct.validateKeyTypes(key_types); - - auto & attribute = getAttribute(attribute_name); - checkAttributeType(name, attribute_name, attribute.type, AttributeUnderlyingType::ut@NAME@); - - getItemsNumberImpl(attribute, key_columns, out, [&](const size_t row) { return def[row]; }); -} -} diff --git a/src/Dictionaries/ComplexKeyCacheDictionary_generate3.cpp b/src/Dictionaries/ComplexKeyCacheDictionary_generate3.cpp new file mode 100644 index 00000000000..222e9c8ee77 --- /dev/null +++ b/src/Dictionaries/ComplexKeyCacheDictionary_generate3.cpp @@ -0,0 +1,35 @@ +#include + +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 & out) const \ + { \ + dict_struct.validateKeyTypes(key_types); \ + auto & attribute = getAttribute(attribute_name); \ + checkAttributeType(name, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \ + getItemsNumberImpl(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 +} diff --git a/src/Dictionaries/ComplexKeyCacheDictionary_generate3.cpp.in b/src/Dictionaries/ComplexKeyCacheDictionary_generate3.cpp.in deleted file mode 100644 index 02e77c01a4a..00000000000 --- a/src/Dictionaries/ComplexKeyCacheDictionary_generate3.cpp.in +++ /dev/null @@ -1,25 +0,0 @@ -#include - -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 & out) const -{ - dict_struct.validateKeyTypes(key_types); - - auto & attribute = getAttribute(attribute_name); - checkAttributeType(name, attribute_name, attribute.type, AttributeUnderlyingType::ut@NAME@); - - getItemsNumberImpl(attribute, key_columns, out, [&](const size_t) { return def; }); -} -} diff --git a/src/Dictionaries/HashedDictionary.h b/src/Dictionaries/HashedDictionary.h index 3f8eec979bb..a361352a8bd 100644 --- a/src/Dictionaries/HashedDictionary.h +++ b/src/Dictionaries/HashedDictionary.h @@ -156,8 +156,14 @@ private: template using CollectionPtrType = std::unique_ptr>; +#if !defined(ARCADIA_BUILD) template using SparseCollectionType = google::sparse_hash_map>; +#else + template + using SparseCollectionType = google::sparsehash::sparse_hash_map>; +#endif + template using SparseCollectionPtrType = std::unique_ptr>; diff --git a/src/Dictionaries/MongoDBBlockInputStream.cpp b/src/Dictionaries/MongoDBBlockInputStream.cpp index b1c2f67cc86..e0ff7cb6529 100644 --- a/src/Dictionaries/MongoDBBlockInputStream.cpp +++ b/src/Dictionaries/MongoDBBlockInputStream.cpp @@ -1,25 +1,27 @@ -#include "config_core.h" +#if !defined(ARCADIA_BUILD) +# include "config_core.h" +#endif + #if USE_POCO_MONGODB +# include +# include +# include -#include -#include -#include +# include +# include +# include +# include -#include -#include -#include -#include - -#include -#include -#include -#include -#include -#include -#include -#include -#include "DictionaryStructure.h" -#include "MongoDBBlockInputStream.h" +# include +# include +# include +# include +# include +# include +# include +# include +# include "DictionaryStructure.h" +# include "MongoDBBlockInputStream.h" namespace DB diff --git a/src/Dictionaries/MongoDBDictionarySource.h b/src/Dictionaries/MongoDBDictionarySource.h index bf4669248dc..622d61c5b09 100644 --- a/src/Dictionaries/MongoDBDictionarySource.h +++ b/src/Dictionaries/MongoDBDictionarySource.h @@ -1,9 +1,12 @@ #pragma once #include -#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 "IDictionarySource.h" @@ -91,4 +94,5 @@ private: }; } + #endif diff --git a/src/Dictionaries/MySQLDictionarySource.cpp b/src/Dictionaries/MySQLDictionarySource.cpp index 00527064d79..4ba16ca19de 100644 --- a/src/Dictionaries/MySQLDictionarySource.cpp +++ b/src/Dictionaries/MySQLDictionarySource.cpp @@ -1,8 +1,8 @@ #include "MySQLDictionarySource.h" + #include #include "DictionarySourceFactory.h" #include "DictionaryStructure.h" -#include "config_core.h" #include "registerDictionaries.h" namespace DB diff --git a/src/Dictionaries/MySQLDictionarySource.h b/src/Dictionaries/MySQLDictionarySource.h index 95e660d220f..34f784cdfeb 100644 --- a/src/Dictionaries/MySQLDictionarySource.h +++ b/src/Dictionaries/MySQLDictionarySource.h @@ -2,9 +2,11 @@ #include -#include "config_core.h" -#if USE_MYSQL +#if !defined(ARCADIA_BUILD) +# include "config_core.h" +#endif +#if USE_MYSQL # include # include # include "DictionaryStructure.h" diff --git a/src/Dictionaries/RedisBlockInputStream.h b/src/Dictionaries/RedisBlockInputStream.h index 86448095787..6e350e7c4f1 100644 --- a/src/Dictionaries/RedisBlockInputStream.h +++ b/src/Dictionaries/RedisBlockInputStream.h @@ -1,8 +1,11 @@ #pragma once -#include "config_core.h" #include +#if !defined(ARCADIA_BUILD) +# include "config_core.h" +#endif + #if USE_POCO_REDIS # include # include diff --git a/src/Dictionaries/RedisDictionarySource.h b/src/Dictionaries/RedisDictionarySource.h index dc62d7dc625..e42bd2678cb 100644 --- a/src/Dictionaries/RedisDictionarySource.h +++ b/src/Dictionaries/RedisDictionarySource.h @@ -1,10 +1,12 @@ #pragma once -#include "config_core.h" #include -#if USE_POCO_REDIS +#if !defined(ARCADIA_BUILD) +# include "config_core.h" +#endif +#if USE_POCO_REDIS # include "DictionaryStructure.h" # include "IDictionarySource.h" @@ -101,4 +103,5 @@ namespace ErrorCodes }; } + #endif diff --git a/src/Dictionaries/XDBCDictionarySource.cpp b/src/Dictionaries/XDBCDictionarySource.cpp index 8647c8ccac5..3fea0a024ef 100644 --- a/src/Dictionaries/XDBCDictionarySource.cpp +++ b/src/Dictionaries/XDBCDictionarySource.cpp @@ -16,9 +16,12 @@ #include "DictionaryStructure.h" #include "readInvalidateQuery.h" -#include #include "registerDictionaries.h" +#if !defined(ARCADIA_BUILD) +# include +#endif + #if USE_POCO_SQLODBC || USE_POCO_DATAODBC # include #endif diff --git a/src/Dictionaries/registerDictionaries.cpp b/src/Dictionaries/registerDictionaries.cpp index 60218d457b4..cbd2d6010cb 100644 --- a/src/Dictionaries/registerDictionaries.cpp +++ b/src/Dictionaries/registerDictionaries.cpp @@ -25,7 +25,9 @@ void registerDictionaries() registerDictionaryRangeHashed(factory); registerDictionaryComplexKeyHashed(factory); registerDictionaryComplexKeyCache(factory); +#if !defined(ARCADIA_BUILD) registerDictionaryTrie(factory); +#endif registerDictionaryFlat(factory); registerDictionaryHashed(factory); registerDictionaryCache(factory); diff --git a/src/Dictionaries/ya.make b/src/Dictionaries/ya.make new file mode 100644 index 00000000000..3f831c3c9fe --- /dev/null +++ b/src/Dictionaries/ya.make @@ -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() diff --git a/src/Disks/DiskS3.h b/src/Disks/DiskS3.h index 1b61ed1cde3..c89fde4b103 100644 --- a/src/Disks/DiskS3.h +++ b/src/Disks/DiskS3.h @@ -1,6 +1,8 @@ #pragma once -#include +#if !defined(ARCADIA_BUILD) +# include +#endif #if USE_AWS_S3 # include "DiskFactory.h" diff --git a/src/Disks/registerDisks.cpp b/src/Disks/registerDisks.cpp index f28918cdbd0..2da39e62b19 100644 --- a/src/Disks/registerDisks.cpp +++ b/src/Disks/registerDisks.cpp @@ -2,7 +2,9 @@ #include "DiskFactory.h" -#include +#if !defined(ARCADIA_BUILD) +# include +#endif namespace DB { diff --git a/src/Disks/ya.make b/src/Disks/ya.make new file mode 100644 index 00000000000..71eb47c489c --- /dev/null +++ b/src/Disks/ya.make @@ -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() diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 7d741004766..d48b2dde206 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -1,12 +1,12 @@ +#include + #include -#include #include #include #include #include #include #include -#include #include #include #include @@ -16,6 +16,10 @@ #include #include +#if !defined(ARCADIA_BUILD) +# include +#endif + namespace DB { @@ -347,12 +351,14 @@ FormatFactory::FormatFactory() registerOutputFormatProcessorJSONCompactEachRow(*this); registerInputFormatProcessorProtobuf(*this); registerOutputFormatProcessorProtobuf(*this); +#if !defined(ARCADIA_BUILD) registerInputFormatProcessorCapnProto(*this); registerInputFormatProcessorORC(*this); registerInputFormatProcessorParquet(*this); registerOutputFormatProcessorParquet(*this); registerInputFormatProcessorAvro(*this); registerOutputFormatProcessorAvro(*this); +#endif registerInputFormatProcessorTemplate(*this); registerOutputFormatProcessorTemplate(*this); registerInputFormatProcessorRegexp(*this); diff --git a/src/Formats/MySQLBlockInputStream.cpp b/src/Formats/MySQLBlockInputStream.cpp index 4f8291ffebe..17c09cdc14d 100644 --- a/src/Formats/MySQLBlockInputStream.cpp +++ b/src/Formats/MySQLBlockInputStream.cpp @@ -1,15 +1,17 @@ -#include "config_core.h" -#if USE_MYSQL +#if !defined(ARCADIA_BUILD) +# include "config_core.h" +#endif -#include -#include -#include -#include -#include -#include -#include -#include -#include "MySQLBlockInputStream.h" +#if USE_MYSQL +# include +# include +# include +# include +# include +# include +# include +# include +# include "MySQLBlockInputStream.h" namespace DB diff --git a/src/Formats/ProtobufColumnMatcher.h b/src/Formats/ProtobufColumnMatcher.h index ed6c632f8e8..03c5ec40fc6 100644 --- a/src/Formats/ProtobufColumnMatcher.h +++ b/src/Formats/ProtobufColumnMatcher.h @@ -1,15 +1,17 @@ #pragma once -#include "config_formats.h" -#if USE_PROTOBUF +#if !defined(ARCADIA_BUILD) +# include "config_formats.h" +#endif -#include -#include -#include -#include -#include -#include -#include +#if USE_PROTOBUF +# include +# include +# include +# include +# include +# include +# include namespace google { @@ -190,4 +192,5 @@ namespace ProtobufColumnMatcher } } + #endif diff --git a/src/Formats/ProtobufReader.cpp b/src/Formats/ProtobufReader.cpp index 5426e8fac62..249bcf12711 100644 --- a/src/Formats/ProtobufReader.cpp +++ b/src/Formats/ProtobufReader.cpp @@ -1,16 +1,14 @@ -#include "config_formats.h" -#if USE_PROTOBUF - #include "ProtobufReader.h" -#include -#include -#include -#include -#include -#include -#include -#include +#if USE_PROTOBUF +# include +# include +# include +# include +# include +# include +# include +# include namespace DB @@ -272,25 +270,25 @@ UInt64 ProtobufReader::SimpleReader::continueReadingVarint(UInt64 first_byte) UInt64 result = (first_byte & ~static_cast(0x80)); char c; -#define PROTOBUF_READER_READ_VARINT_BYTE(byteNo) \ - do \ - { \ - in.readStrict(c); \ - ++cursor; \ - if constexpr ((byteNo) < 10) \ +# define PROTOBUF_READER_READ_VARINT_BYTE(byteNo) \ + do \ { \ - result |= static_cast(static_cast(c)) << (7 * ((byteNo) - 1)); \ - if (likely(!(c & 0x80))) \ - return result; \ - } \ - else \ - { \ - if (likely(c == 1)) \ - return result; \ - } \ - if constexpr ((byteNo) < 9) \ - result &= ~(static_cast(0x80) << (7 * ((byteNo) - 1))); \ - } while (false) + in.readStrict(c); \ + ++cursor; \ + if constexpr ((byteNo) < 10) \ + { \ + result |= static_cast(static_cast(c)) << (7 * ((byteNo)-1)); \ + if (likely(!(c & 0x80))) \ + return result; \ + } \ + else \ + { \ + if (likely(c == 1)) \ + return result; \ + } \ + if constexpr ((byteNo) < 9) \ + result &= ~(static_cast(0x80) << (7 * ((byteNo)-1))); \ + } while (false) PROTOBUF_READER_READ_VARINT_BYTE(2); 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(10); -#undef PROTOBUF_READER_READ_VARINT_BYTE +# undef PROTOBUF_READER_READ_VARINT_BYTE throwUnknownFormat(); } @@ -311,22 +309,22 @@ void ProtobufReader::SimpleReader::ignoreVarint() { char c; -#define PROTOBUF_READER_IGNORE_VARINT_BYTE(byteNo) \ - do \ - { \ - in.readStrict(c); \ - ++cursor; \ - if constexpr ((byteNo) < 10) \ +# define PROTOBUF_READER_IGNORE_VARINT_BYTE(byteNo) \ + do \ { \ - if (likely(!(c & 0x80))) \ - return; \ - } \ - else \ - { \ - if (likely(c == 1)) \ - return; \ - } \ - } while (false) + in.readStrict(c); \ + ++cursor; \ + if constexpr ((byteNo) < 10) \ + { \ + if (likely(!(c & 0x80))) \ + return; \ + } \ + else \ + { \ + if (likely(c == 1)) \ + return; \ + } \ + } while (false) PROTOBUF_READER_IGNORE_VARINT_BYTE(1); 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(9); PROTOBUF_READER_IGNORE_VARINT_BYTE(10); -#undef PROTOBUF_READER_IGNORE_VARINT_BYTE + +# undef PROTOBUF_READER_IGNORE_VARINT_BYTE throwUnknownFormat(); } @@ -694,16 +693,17 @@ private: std::optional> enum_name_to_value_map; }; -#define PROTOBUF_READER_CREATE_CONVERTER_SPECIALIZATION_FOR_STRINGS(field_type_id) \ - template <> \ - std::unique_ptr ProtobufReader::createConverter( \ - const google::protobuf::FieldDescriptor * field) \ - { \ - return std::make_unique(simple_reader, field); \ - } +# define PROTOBUF_READER_CREATE_CONVERTER_SPECIALIZATION_FOR_STRINGS(field_type_id) \ + template <> \ + std::unique_ptr ProtobufReader::createConverter( \ + const google::protobuf::FieldDescriptor * field) \ + { \ + return std::make_unique(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_BYTES) -#undef PROTOBUF_READER_CREATE_CONVERTER_SPECIALIZATION_FOR_STRINGS + +# undef PROTOBUF_READER_CREATE_CONVERTER_SPECIALIZATION_FOR_STRINGS template @@ -850,13 +850,14 @@ private: std::optional> set_of_enum_values; }; -#define PROTOBUF_READER_CREATE_CONVERTER_SPECIALIZATION_FOR_NUMBERS(field_type_id, field_type) \ - template <> \ - std::unique_ptr ProtobufReader::createConverter( \ - const google::protobuf::FieldDescriptor * field) \ - { \ - return std::make_unique>(simple_reader, field); /* NOLINT */ \ - } +# define PROTOBUF_READER_CREATE_CONVERTER_SPECIALIZATION_FOR_NUMBERS(field_type_id, field_type) \ + template <> \ + std::unique_ptr ProtobufReader::createConverter( \ + const google::protobuf::FieldDescriptor * field) \ + { \ + return std::make_unique>(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_SINT32, Int64); 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_FLOAT, float); 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 @@ -1073,10 +1075,10 @@ void ProtobufReader::setTraitsDataAfterMatchingColumns(Message * message) } switch (field.field_descriptor->type()) { -#define PROTOBUF_READER_CONVERTER_CREATING_CASE(field_type_id) \ - case field_type_id: \ - field.data.converter = createConverter(field.field_descriptor); \ - break +# define PROTOBUF_READER_CONVERTER_CREATING_CASE(field_type_id) \ + case field_type_id: \ + field.data.converter = createConverter(field.field_descriptor); \ + break 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_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_BOOL); PROTOBUF_READER_CONVERTER_CREATING_CASE(google::protobuf::FieldDescriptor::TYPE_ENUM); -#undef PROTOBUF_READER_CONVERTER_CREATING_CASE - default: __builtin_unreachable(); +# undef PROTOBUF_READER_CONVERTER_CREATING_CASE + default: + __builtin_unreachable(); } message->data.field_number_to_field_map.emplace(field.field_number, &field); } @@ -1171,4 +1174,5 @@ bool ProtobufReader::readColumnIndex(size_t & column_index) } } + #endif diff --git a/src/Formats/ProtobufReader.h b/src/Formats/ProtobufReader.h index 393b15eb343..a50c2f202f0 100644 --- a/src/Formats/ProtobufReader.h +++ b/src/Formats/ProtobufReader.h @@ -5,13 +5,15 @@ #include #include -#include "config_formats.h" -#if USE_PROTOBUF +#if !defined(ARCADIA_BUILD) +# include "config_formats.h" +#endif -#include -#include "ProtobufColumnMatcher.h" -#include -#include +#if USE_PROTOBUF +# include +# include +# include +# include "ProtobufColumnMatcher.h" namespace google { diff --git a/src/Formats/ProtobufSchemas.cpp b/src/Formats/ProtobufSchemas.cpp index f4973263bc8..9c6ed76ef27 100644 --- a/src/Formats/ProtobufSchemas.cpp +++ b/src/Formats/ProtobufSchemas.cpp @@ -1,10 +1,12 @@ -#include "config_formats.h" -#if USE_PROTOBUF +#if !defined(ARCADIA_BUILD) +# include "config_formats.h" +#endif -#include -#include -#include -#include +#if USE_PROTOBUF +# include +# include +# include +# include namespace DB diff --git a/src/Formats/ProtobufWriter.cpp b/src/Formats/ProtobufWriter.cpp index 8ee309d41c9..d68999c7ce4 100644 --- a/src/Formats/ProtobufWriter.cpp +++ b/src/Formats/ProtobufWriter.cpp @@ -1,18 +1,16 @@ -#include "config_formats.h" -#if USE_PROTOBUF - #include "ProtobufWriter.h" -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include +#if USE_PROTOBUF +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include namespace DB @@ -496,19 +494,19 @@ private: std::optional> enum_value_to_name_map; }; -#define PROTOBUF_WRITER_CREATE_CONVERTER_SPECIALIZATION_FOR_STRINGS(field_type_id) \ - template <> \ - std::unique_ptr ProtobufWriter::createConverter( \ - const google::protobuf::FieldDescriptor * field) \ - { \ - if (shouldSkipNullValue(field)) \ - return std::make_unique>(simple_writer, field); \ - else \ - return std::make_unique>(simple_writer, field); \ - } +# define PROTOBUF_WRITER_CREATE_CONVERTER_SPECIALIZATION_FOR_STRINGS(field_type_id) \ + template <> \ + std::unique_ptr ProtobufWriter::createConverter( \ + const google::protobuf::FieldDescriptor * field) \ + { \ + if (shouldSkipNullValue(field)) \ + return std::make_unique>(simple_writer, field); \ + else \ + return std::make_unique>(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_BYTES) -#undef PROTOBUF_WRITER_CREATE_CONVERTER_SPECIALIZATION_FOR_STRINGS +# undef PROTOBUF_WRITER_CREATE_CONVERTER_SPECIALIZATION_FOR_STRINGS template @@ -606,18 +604,19 @@ private: } }; -#define PROTOBUF_WRITER_CREATE_CONVERTER_SPECIALIZATION_FOR_NUMBERS(field_type_id, field_type) \ - template <> \ - std::unique_ptr ProtobufWriter::createConverter( \ - const google::protobuf::FieldDescriptor * field) \ - { \ - if (shouldSkipNullValue(field)) \ - return std::make_unique>(simple_writer, field); \ - else if (shouldPackRepeated(field)) \ - return std::make_unique>(simple_writer, field); \ - else \ - return std::make_unique>(simple_writer, field); \ - } +# define PROTOBUF_WRITER_CREATE_CONVERTER_SPECIALIZATION_FOR_NUMBERS(field_type_id, field_type) \ + template <> \ + std::unique_ptr ProtobufWriter::createConverter( \ + const google::protobuf::FieldDescriptor * field) \ + { \ + if (shouldSkipNullValue(field)) \ + return std::make_unique>(simple_writer, field); \ + else if (shouldPackRepeated(field)) \ + return std::make_unique>(simple_writer, field); \ + else \ + return std::make_unique>(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_SINT32, Int32); 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_FLOAT, float); 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 @@ -871,10 +870,10 @@ void ProtobufWriter::setTraitsDataAfterMatchingColumns(Message * message) } switch (field.field_descriptor->type()) { -#define PROTOBUF_WRITER_CONVERTER_CREATING_CASE(field_type_id) \ - case field_type_id: \ - field.data.converter = createConverter(field.field_descriptor); \ - break +# define PROTOBUF_WRITER_CONVERTER_CREATING_CASE(field_type_id) \ + case field_type_id: \ + field.data.converter = createConverter(field.field_descriptor); \ + break 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_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_BOOL); PROTOBUF_WRITER_CONVERTER_CREATING_CASE(google::protobuf::FieldDescriptor::TYPE_ENUM); -#undef PROTOBUF_WRITER_CONVERTER_CREATING_CASE +# undef PROTOBUF_WRITER_CONVERTER_CREATING_CASE default: throw Exception( String("Protobuf type '") + field.field_descriptor->type_name() + "' isn't supported", ErrorCodes::NOT_IMPLEMENTED); @@ -991,4 +990,5 @@ void ProtobufWriter::setNestedMessageNeedsRepeat() } } + #endif diff --git a/src/Formats/ProtobufWriter.h b/src/Formats/ProtobufWriter.h index 3862ed88cdb..1778f2adb21 100644 --- a/src/Formats/ProtobufWriter.h +++ b/src/Formats/ProtobufWriter.h @@ -3,17 +3,17 @@ #include #include #include - -#include "config_formats.h" - #include -#if USE_PROTOBUF +#if !defined(ARCADIA_BUILD) +# include "config_formats.h" +#endif -#include "ProtobufColumnMatcher.h" -#include -#include -#include +#if USE_PROTOBUF +# include +# include +# include +# include "ProtobufColumnMatcher.h" namespace google @@ -233,6 +233,8 @@ private: } #else +# include + namespace DB { diff --git a/src/Formats/ya.make b/src/Formats/ya.make new file mode 100644 index 00000000000..de61820e58d --- /dev/null +++ b/src/Formats/ya.make @@ -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() diff --git a/src/Functions/DivisionUtils.h b/src/Functions/DivisionUtils.h index df3b86f721d..5a0c1b8232a 100644 --- a/src/Functions/DivisionUtils.h +++ b/src/Functions/DivisionUtils.h @@ -2,9 +2,11 @@ #include #include -#include #include +#if !defined(ARCADIA_BUILD) +# include +#endif namespace DB { diff --git a/src/Functions/FunctionBase64Conversion.h b/src/Functions/FunctionBase64Conversion.h index 27a873f69fa..5b100bf68ce 100644 --- a/src/Functions/FunctionBase64Conversion.h +++ b/src/Functions/FunctionBase64Conversion.h @@ -1,13 +1,16 @@ -#include "config_functions.h" +#if !defined(ARCADIA_BUILD) +# include "config_functions.h" +#endif + #if USE_BASE64 -#include -#include -#include -#include -#include -#include -#include -#include +# include +# include +# include +# include +# include +# include +# include +# include namespace DB diff --git a/src/Functions/FunctionBinaryArithmetic.h b/src/Functions/FunctionBinaryArithmetic.h index 2ad52d69f09..30b6da8b696 100644 --- a/src/Functions/FunctionBinaryArithmetic.h +++ b/src/Functions/FunctionBinaryArithmetic.h @@ -27,13 +27,16 @@ #include "FunctionFactory.h" #include #include -#include + +#if !defined(ARCADIA_BUILD) +# include +#endif #if USE_EMBEDDED_COMPILER -#pragma GCC diagnostic push -#pragma GCC diagnostic ignored "-Wunused-parameter" -#include -#pragma GCC diagnostic pop +# pragma GCC diagnostic push +# pragma GCC diagnostic ignored "-Wunused-parameter" +# include +# pragma GCC diagnostic pop #endif diff --git a/src/Functions/FunctionIfBase.h b/src/Functions/FunctionIfBase.h index 48aa1c0b6b2..bdb883ee342 100644 --- a/src/Functions/FunctionIfBase.h +++ b/src/Functions/FunctionIfBase.h @@ -1,9 +1,11 @@ #pragma once -#include #include #include +#if !defined(ARCADIA_BUILD) +# include +#endif namespace DB { diff --git a/src/Functions/FunctionMathBinaryFloat64.h b/src/Functions/FunctionMathBinaryFloat64.h index 2927d4ef228..f696e4728fb 100644 --- a/src/Functions/FunctionMathBinaryFloat64.h +++ b/src/Functions/FunctionMathBinaryFloat64.h @@ -8,7 +8,10 @@ #include #include #include -#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. * Disabled due to license compatibility limitations. diff --git a/src/Functions/FunctionMathUnary.h b/src/Functions/FunctionMathUnary.h index 716b52b40c5..b048b994951 100644 --- a/src/Functions/FunctionMathUnary.h +++ b/src/Functions/FunctionMathUnary.h @@ -7,7 +7,10 @@ #include #include #include -#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. * Disabled due to license compatibility limitations. @@ -20,7 +23,7 @@ * Enabled by default. */ #if USE_FASTOPS -#include +# include #endif diff --git a/src/Functions/FunctionUnaryArithmetic.h b/src/Functions/FunctionUnaryArithmetic.h index 89687b5b23e..4e3ccdc8952 100644 --- a/src/Functions/FunctionUnaryArithmetic.h +++ b/src/Functions/FunctionUnaryArithmetic.h @@ -10,13 +10,16 @@ #include #include #include -#include + +#if !defined(ARCADIA_BUILD) +# include +#endif #if USE_EMBEDDED_COMPILER -#pragma GCC diagnostic push -#pragma GCC diagnostic ignored "-Wunused-parameter" -#include -#pragma GCC diagnostic pop +# pragma GCC diagnostic push +# pragma GCC diagnostic ignored "-Wunused-parameter" +# include +# pragma GCC diagnostic pop #endif diff --git a/src/Functions/FunctionsEmbeddedDictionaries.h b/src/Functions/FunctionsEmbeddedDictionaries.h index 12b478a26b6..44fdca9736d 100644 --- a/src/Functions/FunctionsEmbeddedDictionaries.h +++ b/src/Functions/FunctionsEmbeddedDictionaries.h @@ -15,9 +15,12 @@ #include #include #include -#include #include +#if !defined(ARCADIA_BUILD) +# include +#endif + namespace DB { diff --git a/src/Functions/FunctionsExternalDictionaries.h b/src/Functions/FunctionsExternalDictionaries.h index fc3c2c583a9..61ba555be92 100644 --- a/src/Functions/FunctionsExternalDictionaries.h +++ b/src/Functions/FunctionsExternalDictionaries.h @@ -135,7 +135,9 @@ private: !executeDispatchSimple(block, arguments, result, dict_ptr) && !executeDispatchComplex(block, arguments, result, dict_ptr) && !executeDispatchComplex(block, arguments, result, dict_ptr) && +#if !defined(ARCADIA_BUILD) !executeDispatchComplex(block, arguments, result, dict_ptr) && +#endif !executeDispatchComplex(block, arguments, result, dict_ptr)) throw Exception{"Unsupported dictionary type " + dict_ptr->getTypeName(), ErrorCodes::UNKNOWN_TYPE}; } @@ -306,7 +308,9 @@ private: !executeDispatch(block, arguments, result, dict_ptr) && !executeDispatchComplex(block, arguments, result, dict_ptr) && !executeDispatchComplex(block, arguments, result, dict_ptr) && +#if !defined(ARCADIA_BUILD) !executeDispatchComplex(block, arguments, result, dict_ptr) && +#endif !executeDispatchComplex(block, arguments, result, dict_ptr) && !executeDispatchRange(block, arguments, result, dict_ptr)) throw Exception{"Unsupported dictionary type " + dict_ptr->getTypeName(), ErrorCodes::UNKNOWN_TYPE}; @@ -488,8 +492,10 @@ private: !executeDispatch(block, arguments, result, dict_ptr) && !executeDispatchComplex(block, arguments, result, dict_ptr) && !executeDispatchComplex(block, arguments, result, dict_ptr) && - !executeDispatchComplex(block, arguments, result, dict_ptr) && - !executeDispatchComplex(block, arguments, result, dict_ptr)) +#if !defined(ARCADIA_BUILD) + !executeDispatchComplex(block, arguments, result, dict_ptr) && +#endif + !executeDispatchComplex(block, arguments, result, dict_ptr)) throw Exception{"Unsupported dictionary type " + dict_ptr->getTypeName(), ErrorCodes::UNKNOWN_TYPE}; } @@ -826,7 +832,9 @@ private: !executeDispatch(block, arguments, result, dict_ptr) && !executeDispatchComplex(block, arguments, result, dict_ptr) && !executeDispatchComplex(block, arguments, result, dict_ptr) && +#if !defined(ARCADIA_BUILD) !executeDispatchComplex(block, arguments, result, dict_ptr) && +#endif !executeDispatchComplex(block, arguments, result, dict_ptr) && !executeDispatchRange(block, arguments, result, dict_ptr)) throw Exception{"Unsupported dictionary type " + dict_ptr->getTypeName(), ErrorCodes::UNKNOWN_TYPE}; @@ -1086,8 +1094,10 @@ private: !executeDispatch(block, arguments, result, dict_ptr) && !executeDispatchComplex(block, arguments, result, dict_ptr) && !executeDispatchComplex(block, arguments, result, dict_ptr) && - !executeDispatchComplex(block, arguments, result, dict_ptr) && - !executeDispatchComplex(block, arguments, result, dict_ptr)) +#if !defined(ARCADIA_BUILD) + !executeDispatchComplex(block, arguments, result, dict_ptr) && +#endif + !executeDispatchComplex(block, arguments, result, dict_ptr)) throw Exception{"Unsupported dictionary type " + dict_ptr->getTypeName(), ErrorCodes::UNKNOWN_TYPE}; } diff --git a/src/Functions/FunctionsHashing.cpp b/src/Functions/FunctionsHashing.cpp index 8705e6bfaa3..900455a1015 100644 --- a/src/Functions/FunctionsHashing.cpp +++ b/src/Functions/FunctionsHashing.cpp @@ -26,12 +26,14 @@ void registerFunctionsHashing(FunctionFactory & factory) factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); +#if !defined(ARCADIA_BUILD) factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); +#endif #if USE_XXHASH factory.registerFunction(); diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index a589b220351..f647390e1c8 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -3,22 +3,24 @@ #include #include #include -#include -#include +#if !defined(ARCADIA_BUILD) +# include +# include +# include "config_functions.h" +# include "config_core.h" +#endif #include #include #include -#include "config_functions.h" #if USE_XXHASH -# include +# include #endif -#include "config_core.h" #if USE_SSL -# include -# include +# include +# include #endif #include @@ -219,7 +221,7 @@ struct SipHash128Impl } }; - +#if !defined(ARCADIA_BUILD) /** Why we need MurmurHash2? * MurmurHash2 is an outdated hash function, superseded by MurmurHash3 and subsequently by CityHash, xxHash, HighwayHash. * Usually there is no reason to use MurmurHash. @@ -331,6 +333,18 @@ struct MurmurHash3Impl64 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 /// 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. @@ -411,17 +425,6 @@ struct HiveHashImpl 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 { static constexpr auto name = "cityHash64"; @@ -1145,12 +1148,16 @@ using FunctionSipHash128 = FunctionStringHashFixedString; using FunctionCityHash64 = FunctionAnyHash; using FunctionFarmHash64 = FunctionAnyHash; using FunctionMetroHash64 = FunctionAnyHash; + +#if !defined(ARCADIA_BUILD) using FunctionMurmurHash2_32 = FunctionAnyHash; using FunctionMurmurHash2_64 = FunctionAnyHash; using FunctionGccMurmurHash = FunctionAnyHash; using FunctionMurmurHash3_32 = FunctionAnyHash; using FunctionMurmurHash3_64 = FunctionAnyHash; using FunctionMurmurHash3_128 = FunctionStringHashFixedString; +#endif + using FunctionJavaHash = FunctionAnyHash; using FunctionJavaHashUTF16LE = FunctionAnyHash; using FunctionHiveHash = FunctionAnyHash; diff --git a/src/Functions/FunctionsJSON.h b/src/Functions/FunctionsJSON.h index 468b499b78c..acd7c42732c 100644 --- a/src/Functions/FunctionsJSON.h +++ b/src/Functions/FunctionsJSON.h @@ -5,7 +5,6 @@ #include #include #include -#include "config_functions.h" #include #include #include @@ -27,6 +26,10 @@ #include #include +#if !defined(ARCADIA_BUILD) +# include "config_functions.h" +#endif + namespace DB { diff --git a/src/Functions/FunctionsStringRegex.cpp b/src/Functions/FunctionsStringRegex.cpp index 856852ae197..2f4c2432d54 100644 --- a/src/Functions/FunctionsStringRegex.cpp +++ b/src/Functions/FunctionsStringRegex.cpp @@ -17,16 +17,15 @@ #include #include -#include "config_functions.h" -#if USE_HYPERSCAN -# if __has_include() -# include -# else -# include -# endif +#if !defined(ARCADIA_BUILD) +# include "config_functions.h" +# include +#endif + +#if USE_HYPERSCAN +# include #endif -#include #if USE_RE2_ST # include #else diff --git a/src/Functions/IFunction.cpp b/src/Functions/IFunction.cpp index 7355b5df9ad..911810a272c 100644 --- a/src/Functions/IFunction.cpp +++ b/src/Functions/IFunction.cpp @@ -1,6 +1,5 @@ #include -#include #include #include #include @@ -24,11 +23,15 @@ #include #include +#if !defined(ARCADIA_BUILD) +# include +#endif + #if USE_EMBEDDED_COMPILER -#pragma GCC diagnostic push -#pragma GCC diagnostic ignored "-Wunused-parameter" -#include -#pragma GCC diagnostic pop +# pragma GCC diagnostic push +# pragma GCC diagnostic ignored "-Wunused-parameter" +# include +# pragma GCC diagnostic pop #endif diff --git a/src/Functions/IFunction.h b/src/Functions/IFunction.h index 3887f3f1669..5ef6e5aaf22 100644 --- a/src/Functions/IFunction.h +++ b/src/Functions/IFunction.h @@ -2,12 +2,15 @@ #include -#include "config_core.h" #include #include #include #include +#if !defined(ARCADIA_BUILD) +# include "config_core.h" +#endif + /// This file contains user interface for functions. /// For developer interface (in case you need to implement a new function) see IFunctionImpl.h diff --git a/src/Functions/RapidJSONParser.h b/src/Functions/RapidJSONParser.h index ff4ecd506fd..07b0aea3543 100644 --- a/src/Functions/RapidJSONParser.h +++ b/src/Functions/RapidJSONParser.h @@ -1,13 +1,15 @@ #pragma once -#include "config_functions.h" +#if !defined(ARCADIA_BUILD) +# include "config_functions.h" +#endif + #if USE_RAPIDJSON +# include +# include +# include -#include -#include -#include - -#include +# include namespace DB diff --git a/src/Functions/Regexps.h b/src/Functions/Regexps.h index 5d93e823419..433e416a18f 100644 --- a/src/Functions/Regexps.h +++ b/src/Functions/Regexps.h @@ -14,14 +14,12 @@ #include #include +#if !defined(ARCADIA_BUILD) +# include "config_functions.h" +#endif -#include "config_functions.h" #if USE_HYPERSCAN -# if __has_include() -# include -# else -# include -# endif +# include #endif namespace ProfileEvents diff --git a/src/Functions/SimdJSONParser.h b/src/Functions/SimdJSONParser.h index a12119c5c6d..d5630cf1cf1 100644 --- a/src/Functions/SimdJSONParser.h +++ b/src/Functions/SimdJSONParser.h @@ -1,13 +1,15 @@ #pragma once -#include "config_functions.h" +#if !defined(ARCADIA_BUILD) +# include "config_functions.h" +#endif + #if USE_SIMDJSON +# include +# include +# include -#include -#include -#include - -#include +# include namespace DB @@ -137,4 +139,5 @@ private: }; } + #endif diff --git a/src/Functions/base64Encode.cpp b/src/Functions/base64Encode.cpp index f76fedda71e..8cf54eca768 100644 --- a/src/Functions/base64Encode.cpp +++ b/src/Functions/base64Encode.cpp @@ -1,9 +1,12 @@ #include #include -#include "config_functions.h" + +#if !defined(ARCADIA_BUILD) +# include "config_functions.h" +#endif #if USE_BASE64 -#include +# include namespace DB { diff --git a/src/Functions/convertCharset.cpp b/src/Functions/convertCharset.cpp index 372bf08a0d2..9bd1f1b1cf6 100644 --- a/src/Functions/convertCharset.cpp +++ b/src/Functions/convertCharset.cpp @@ -1,20 +1,22 @@ -#include "config_core.h" +#if !defined(ARCADIA_BUILD) +# include "config_core.h" +#endif + #if USE_ICU +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include -#include -#include +# include +# include +# include namespace DB diff --git a/src/Functions/intDiv.cpp b/src/Functions/intDiv.cpp index 062a374c00f..d21e779045f 100644 --- a/src/Functions/intDiv.cpp +++ b/src/Functions/intDiv.cpp @@ -3,7 +3,6 @@ #if defined(__SSE2__) # define LIBDIVIDE_SSE2 1 -# define LIBDIVIDE_VECTOR_TYPE #endif #include diff --git a/src/Functions/registerFunctions.cpp b/src/Functions/registerFunctions.cpp index 233018c7f16..02013e33d16 100644 --- a/src/Functions/registerFunctions.cpp +++ b/src/Functions/registerFunctions.cpp @@ -46,7 +46,9 @@ void registerFunctions() registerFunctionsArithmetic(factory); registerFunctionsArray(factory); registerFunctionsTuple(factory); +#if !defined(ARCADIA_BUILD) registerFunctionsBitmap(factory); +#endif registerFunctionsCoding(factory); registerFunctionsComparison(factory); registerFunctionsConditional(factory); diff --git a/src/Functions/registerFunctionsConsistentHashing.cpp b/src/Functions/registerFunctionsConsistentHashing.cpp index 95a856b6d3c..ceec6dca5e6 100644 --- a/src/Functions/registerFunctionsConsistentHashing.cpp +++ b/src/Functions/registerFunctionsConsistentHashing.cpp @@ -5,14 +5,18 @@ class FunctionFactory; void registerFunctionYandexConsistentHash(FunctionFactory & factory); void registerFunctionJumpConsistentHash(FunctionFactory & factory); +#if !defined(ARCADIA_BUILD) void registerFunctionSumburConsistentHash(FunctionFactory & factory); +#endif void registerFunctionsConsistentHashing(FunctionFactory & factory) { registerFunctionYandexConsistentHash(factory); registerFunctionJumpConsistentHash(factory); +#if !defined(ARCADIA_BUILD) registerFunctionSumburConsistentHash(factory); +#endif } } diff --git a/src/Functions/registerFunctionsGeo.cpp b/src/Functions/registerFunctionsGeo.cpp index cb3c268a19a..2ed02bbd73b 100644 --- a/src/Functions/registerFunctionsGeo.cpp +++ b/src/Functions/registerFunctionsGeo.cpp @@ -1,4 +1,6 @@ -#include "config_functions.h" +#if !defined(ARCADIA_BUILD) +# include "config_functions.h" +#endif namespace DB { diff --git a/src/Functions/registerFunctionsMiscellaneous.cpp b/src/Functions/registerFunctionsMiscellaneous.cpp index 30cab4cc53a..221e14fcce1 100644 --- a/src/Functions/registerFunctionsMiscellaneous.cpp +++ b/src/Functions/registerFunctionsMiscellaneous.cpp @@ -1,4 +1,6 @@ -#include +#if !defined(ARCADIA_BUILD) +# include +#endif namespace DB { diff --git a/src/Functions/registerFunctionsString.cpp b/src/Functions/registerFunctionsString.cpp index cc94e877bbf..5493b98d3fc 100644 --- a/src/Functions/registerFunctionsString.cpp +++ b/src/Functions/registerFunctionsString.cpp @@ -1,4 +1,6 @@ -#include "config_functions.h" +#if !defined(ARCADIA_BUILD) +# include "config_functions.h" +#endif namespace DB { diff --git a/src/Functions/version.cpp b/src/Functions/version.cpp index 34eabd30336..e11f45f160a 100644 --- a/src/Functions/version.cpp +++ b/src/Functions/version.cpp @@ -1,9 +1,11 @@ #include #include #include -#include #include +#if !defined(ARCADIA_BUILD) +# include +#endif namespace DB { diff --git a/src/Functions/ya.make b/src/Functions/ya.make new file mode 100644 index 00000000000..efcae6bf73b --- /dev/null +++ b/src/Functions/ya.make @@ -0,0 +1,418 @@ +LIBRARY() + +ADDINCL( + library/consistent_hashing + contrib/libs/farmhash + contrib/libs/hyperscan/src + contrib/libs/icu/common + contrib/libs/libdivide + contrib/libs/rapidjson/include + contrib/libs/xxhash +) + +PEERDIR( + clickhouse/src/Common + clickhouse/src/Dictionaries + contrib/libs/farmhash + contrib/libs/fastops/fastops + contrib/libs/hyperscan + contrib/libs/icu + contrib/libs/libdivide + contrib/libs/metrohash + contrib/libs/rapidjson + contrib/libs/xxhash + library/consistent_hashing +) + +SRCS( + array/array.cpp + array/arrayAll.cpp + array/arrayAUC.cpp + array/arrayCompact.cpp + array/arrayConcat.cpp + array/arrayCount.cpp + array/arrayCumSum.cpp + array/arrayCumSumNonNegative.cpp + array/arrayDifference.cpp + array/arrayDistinct.cpp + array/arrayElement.cpp + array/arrayEnumerate.cpp + array/arrayEnumerateDense.cpp + array/arrayEnumerateDenseRanked.cpp + array/arrayEnumerateRanked.cpp + array/arrayEnumerateUniq.cpp + array/arrayEnumerateUniqRanked.cpp + array/arrayExists.cpp + array/arrayFill.cpp + array/arrayFilter.cpp + array/arrayFirst.cpp + array/arrayFirstIndex.cpp + array/arrayFlatten.cpp + array/arrayIntersect.cpp + array/arrayJoin.cpp + array/arrayMap.cpp + array/arrayPopBack.cpp + array/arrayPopFront.cpp + array/arrayPushBack.cpp + array/arrayPushFront.cpp + array/arrayReduce.cpp + array/arrayReduceInRanges.cpp + array/arrayResize.cpp + array/arrayReverse.cpp + array/arraySlice.cpp + array/arraySort.cpp + array/arraySplit.cpp + array/arraySum.cpp + array/arrayUniq.cpp + array/arrayWithConstant.cpp + array/arrayZip.cpp + array/countEqual.cpp + array/emptyArray.cpp + array/emptyArrayToSingle.cpp + array/has.cpp + array/hasAll.cpp + array/hasAny.cpp + array/indexOf.cpp + array/length.cpp + array/range.cpp + array/registerFunctionsArray.cpp + GatherUtils/concat.cpp + GatherUtils/createArraySink.cpp + GatherUtils/createArraySource.cpp + GatherUtils/createValueSource.cpp + GatherUtils/has.cpp + GatherUtils/push.cpp + GatherUtils/resizeConstantSize.cpp + GatherUtils/resizeDynamicSize.cpp + GatherUtils/sliceDynamicOffsetBounded.cpp + GatherUtils/sliceDynamicOffsetUnbounded.cpp + GatherUtils/sliceFromLeftConstantOffsetBounded.cpp + GatherUtils/sliceFromLeftConstantOffsetUnbounded.cpp + GatherUtils/sliceFromRightConstantOffsetBounded.cpp + GatherUtils/sliceFromRightConstantOffsetUnbounded.cpp + URL/basename.cpp + URL/cutFragment.cpp + URL/cutQueryString.cpp + URL/cutQueryStringAndFragment.cpp + URL/cutToFirstSignificantSubdomain.cpp + URL/cutURLParameter.cpp + URL/cutWWW.cpp + URL/decodeURLComponent.cpp + URL/domain.cpp + URL/domainWithoutWWW.cpp + URL/extractURLParameter.cpp + URL/extractURLParameterNames.cpp + URL/extractURLParameters.cpp + URL/firstSignificantSubdomain.cpp + URL/fragment.cpp + URL/path.cpp + URL/pathFull.cpp + URL/protocol.cpp + URL/queryString.cpp + URL/queryStringAndFragment.cpp + URL/registerFunctionsURL.cpp + URL/tldLookup.generated.cpp + URL/topLevelDomain.cpp + URL/URLHierarchy.cpp + URL/URLPathHierarchy.cpp + abs.cpp + acos.cpp + addDays.cpp + addHours.cpp + addMinutes.cpp + addMonths.cpp + addQuarters.cpp + addressToLine.cpp + addressToSymbol.cpp + addSeconds.cpp + addWeeks.cpp + addYears.cpp + appendTrailingCharIfAbsent.cpp + asin.cpp + assumeNotNull.cpp + atan.cpp + bar.cpp + base64Decode.cpp + base64Encode.cpp + bitAnd.cpp + bitBoolMaskAnd.cpp + bitBoolMaskOr.cpp + bitCount.cpp + bitNot.cpp + bitOr.cpp + bitRotateLeft.cpp + bitRotateRight.cpp + bitShiftLeft.cpp + bitShiftRight.cpp + bitSwapLastTwo.cpp + bitTest.cpp + bitTestAll.cpp + bitTestAny.cpp + bitWrapperFunc.cpp + bitXor.cpp + blockNumber.cpp + blockSerializedSize.cpp + blockSize.cpp + caseWithExpression.cpp + cbrt.cpp + coalesce.cpp + concat.cpp + convertCharset.cpp + cos.cpp + CRC.cpp + currentDatabase.cpp + currentQuota.cpp + currentRowPolicies.cpp + currentUser.cpp + dateDiff.cpp + defaultValueOfArgumentType.cpp + demange.cpp + divide.cpp + dumpColumnStructure.cpp + e.cpp + empty.cpp + endsWith.cpp + equals.cpp + erf.cpp + erfc.cpp + evalMLMethod.cpp + exp.cpp + exp2.cpp + exp10.cpp + extractTimeZoneFromFunctionArguments.cpp + filesystem.cpp + finalizeAggregation.cpp + formatDateTime.cpp + formatString.cpp + FunctionFactory.cpp + FunctionFQDN.cpp + FunctionHelpers.cpp + FunctionJoinGet.cpp + FunctionsCoding.cpp + FunctionsConversion.cpp + FunctionsEmbeddedDictionaries.cpp + FunctionsExternalDictionaries.cpp + FunctionsExternalModels.cpp + FunctionsFormatting.cpp + FunctionsHashing.cpp + FunctionsJSON.cpp + FunctionsLogical.cpp + FunctionsRandom.cpp + FunctionsRound.cpp + FunctionsStringArray.cpp + FunctionsStringRegex.cpp + FunctionsStringSimilarity.cpp + gcd.cpp + generateUUIDv4.cpp + GeoHash.cpp + geohashDecode.cpp + geohashEncode.cpp + geohashesInBox.cpp + getMacro.cpp + getScalar.cpp + getSizeOfEnumType.cpp + greatCircleDistance.cpp + greater.cpp + greaterOrEquals.cpp + greatest.cpp + hasColumnInTable.cpp + hasToken.cpp + hasTokenCaseInsensitive.cpp + hostName.cpp + identity.cpp + if.cpp + ifNotFinite.cpp + ifNull.cpp + IFunction.cpp + ignore.cpp + ignoreExceptNull.cpp + in.cpp + intDiv.cpp + intDivOrZero.cpp + intExp2.cpp + intExp10.cpp + isConstant.cpp + isFinite.cpp + isInfinite.cpp + isNaN.cpp + isNotNull.cpp + isNull.cpp + isValidUTF8.cpp + jumpConsistentHash.cpp + lcm.cpp + least.cpp + lengthUTF8.cpp + less.cpp + lessOrEquals.cpp + lgamma.cpp + log.cpp + log2.cpp + log10.cpp + lowCardinalityIndices.cpp + lowCardinalityKeys.cpp + lower.cpp + lowerUTF8.cpp + materialize.cpp + minus.cpp + modulo.cpp + moduloOrZero.cpp + multiIf.cpp + multiply.cpp + multiSearchAllPositions.cpp + multiSearchAllPositionsCaseInsensitive.cpp + multiSearchAllPositionsCaseInsensitiveUTF8.cpp + multiSearchAllPositionsUTF8.cpp + multiSearchAny.cpp + multiSearchAnyCaseInsensitive.cpp + multiSearchAnyCaseInsensitiveUTF8.cpp + multiSearchAnyUTF8.cpp + multiSearchFirstIndex.cpp + multiSearchFirstIndexCaseInsensitive.cpp + multiSearchFirstIndexCaseInsensitiveUTF8.cpp + multiSearchFirstIndexUTF8.cpp + multiSearchFirstPosition.cpp + multiSearchFirstPositionCaseInsensitive.cpp + multiSearchFirstPositionCaseInsensitiveUTF8.cpp + multiSearchFirstPositionUTF8.cpp + negate.cpp + neighbor.cpp + notEmpty.cpp + notEquals.cpp + now.cpp + now64.cpp + nullIf.cpp + pi.cpp + plus.cpp + pointInEllipses.cpp + pointInPolygon.cpp + position.cpp + positionCaseInsensitive.cpp + positionCaseInsensitiveUTF8.cpp + positionUTF8.cpp + pow.cpp + rand.cpp + rand64.cpp + randConstant.cpp + randomPrintableASCII.cpp + regexpQuoteMeta.cpp + registerFunctions.cpp + registerFunctionsArithmetic.cpp + registerFunctionsComparison.cpp + registerFunctionsConditional.cpp + registerFunctionsConsistentHashing.cpp + registerFunctionsDateTime.cpp + registerFunctionsGeo.cpp + registerFunctionsHigherOrder.cpp + registerFunctionsIntrospection.cpp + registerFunctionsMath.cpp + registerFunctionsMiscellaneous.cpp + registerFunctionsNull.cpp + registerFunctionsRandom.cpp + registerFunctionsReinterpret.cpp + registerFunctionsString.cpp + registerFunctionsStringSearch.cpp + registerFunctionsTuple.cpp + registerFunctionsVisitParam.cpp + reinterpretAsFixedString.cpp + reinterpretAsString.cpp + reinterpretStringAs.cpp + repeat.cpp + replicate.cpp + reverse.cpp + reverseUTF8.cpp + roundAge.cpp + roundDuration.cpp + roundToExp2.cpp + rowNumberInAllBlocks.cpp + rowNumberInBlock.cpp + runningAccumulate.cpp + runningDifference.cpp + runningDifferenceStartingWithFirstValue.cpp + sigmoid.cpp + sin.cpp + sleep.cpp + sleepEachRow.cpp + sqrt.cpp + startsWith.cpp + substring.cpp + subtractDays.cpp + subtractHours.cpp + subtractMinutes.cpp + subtractMonths.cpp + subtractQuarters.cpp + subtractSeconds.cpp + subtractWeeks.cpp + subtractYears.cpp + tan.cpp + tanh.cpp + tgamma.cpp + throwIf.cpp + timeSlot.cpp + timeSlots.cpp + timezone.cpp + toColumnTypeName.cpp + toCustomWeek.cpp + today.cpp + toDayOfMonth.cpp + toDayOfWeek.cpp + toDayOfYear.cpp + toHour.cpp + toISOWeek.cpp + toISOYear.cpp + toLowCardinality.cpp + toMinute.cpp + toMonday.cpp + toMonth.cpp + toNullable.cpp + toQuarter.cpp + toRelativeDayNum.cpp + toRelativeHourNum.cpp + toRelativeMinuteNum.cpp + toRelativeMonthNum.cpp + toRelativeQuarterNum.cpp + toRelativeSecondNum.cpp + toRelativeWeekNum.cpp + toRelativeYearNum.cpp + toSecond.cpp + toStartOfDay.cpp + toStartOfFifteenMinutes.cpp + toStartOfFiveMinute.cpp + toStartOfHour.cpp + toStartOfInterval.cpp + toStartOfISOYear.cpp + toStartOfMinute.cpp + toStartOfMonth.cpp + toStartOfQuarter.cpp + toStartOfTenMinutes.cpp + toStartOfYear.cpp + toTime.cpp + toTimeZone.cpp + toTypeName.cpp + toValidUTF8.cpp + toYear.cpp + toYYYYMM.cpp + toYYYYMMDD.cpp + toYYYYMMDDhhmmss.cpp + transform.cpp + trap.cpp + trim.cpp + tryBase64Decode.cpp + tuple.cpp + tupleElement.cpp + upper.cpp + upperUTF8.cpp + uptime.cpp + version.cpp + visibleWidth.cpp + visitParamExtractBool.cpp + visitParamExtractFloat.cpp + visitParamExtractInt.cpp + visitParamExtractRaw.cpp + visitParamExtractString.cpp + visitParamExtractUInt.cpp + visitParamHas.cpp + yandexConsistentHash.cpp + yesterday.cpp +) + +END() diff --git a/src/IO/BrotliReadBuffer.cpp b/src/IO/BrotliReadBuffer.cpp index f68424cd54e..ee888b8af22 100644 --- a/src/IO/BrotliReadBuffer.cpp +++ b/src/IO/BrotliReadBuffer.cpp @@ -1,8 +1,10 @@ -#include -#if USE_BROTLI +#if !defined(ARCADIA_BUILD) +# include +#endif -#include "BrotliReadBuffer.h" -#include +#if USE_BROTLI +# include +# include "BrotliReadBuffer.h" namespace DB { diff --git a/src/IO/BrotliWriteBuffer.cpp b/src/IO/BrotliWriteBuffer.cpp index ac1e2b3c188..e3001409e86 100644 --- a/src/IO/BrotliWriteBuffer.cpp +++ b/src/IO/BrotliWriteBuffer.cpp @@ -1,8 +1,10 @@ -#include -#if USE_BROTLI +#if !defined(ARCADIA_BUILD) +# include +#endif -#include -#include +#if USE_BROTLI +# include +# include namespace DB { diff --git a/src/IO/CompressionMethod.cpp b/src/IO/CompressionMethod.cpp index 20f1ea44301..a0a5e19f4fa 100644 --- a/src/IO/CompressionMethod.cpp +++ b/src/IO/CompressionMethod.cpp @@ -7,7 +7,9 @@ #include #include -#include +#if !defined(ARCADIA_BUILD) +# include +#endif namespace DB diff --git a/src/IO/HTTPCommon.cpp b/src/IO/HTTPCommon.cpp index d8ce3427c91..97a0525a9b8 100644 --- a/src/IO/HTTPCommon.cpp +++ b/src/IO/HTTPCommon.cpp @@ -1,6 +1,5 @@ #include -#include #include #include #include @@ -9,14 +8,18 @@ #include +#if !defined(ARCADIA_BUILD) +# include +#endif + #if USE_POCO_NETSSL -#include -#include -#include -#include -#include -#include -#include +# include +# include +# include +# include +# include +# include +# include #endif #include diff --git a/src/IO/ReadWriteBufferFromHTTP.h b/src/IO/ReadWriteBufferFromHTTP.h index f5e6f91537d..2d4e3f0472e 100644 --- a/src/IO/ReadWriteBufferFromHTTP.h +++ b/src/IO/ReadWriteBufferFromHTTP.h @@ -14,10 +14,13 @@ #include #include #include -#include #include #include +#if !defined(ARCADIA_BUILD) +# include +#endif + #define DEFAULT_HTTP_READ_BUFFER_TIMEOUT 1800 #define DEFAULT_HTTP_READ_BUFFER_CONNECTION_TIMEOUT 1 @@ -302,4 +305,3 @@ public: }; } - diff --git a/src/IO/UseSSL.cpp b/src/IO/UseSSL.cpp index 2aad0ef49e2..6770de2cbad 100644 --- a/src/IO/UseSSL.cpp +++ b/src/IO/UseSSL.cpp @@ -1,9 +1,11 @@ #include "UseSSL.h" -#include +#if !defined(ARCADIA_BUILD) +# include +#endif #if USE_POCO_NETSSL -#include +# include #endif namespace DB diff --git a/src/IO/WriteBufferFromHTTPServerResponse.cpp b/src/IO/WriteBufferFromHTTPServerResponse.cpp index 77198ca93ea..0f30f1352e3 100644 --- a/src/IO/WriteBufferFromHTTPServerResponse.cpp +++ b/src/IO/WriteBufferFromHTTPServerResponse.cpp @@ -7,7 +7,10 @@ #include #include #include -#include + +#if !defined(ARCADIA_BUILD) +# include +#endif namespace DB diff --git a/src/IO/WriteBufferFromHTTPServerResponse.h b/src/IO/WriteBufferFromHTTPServerResponse.h index 1307c007656..ffa36c11c5b 100644 --- a/src/IO/WriteBufferFromHTTPServerResponse.h +++ b/src/IO/WriteBufferFromHTTPServerResponse.h @@ -13,7 +13,10 @@ #include #include #include -#include + +#if !defined(ARCADIA_BUILD) +# include +#endif namespace Poco diff --git a/src/IO/ya.make b/src/IO/ya.make new file mode 100644 index 00000000000..81bf6060a94 --- /dev/null +++ b/src/IO/ya.make @@ -0,0 +1,59 @@ +LIBRARY() + +PEERDIR( + clickhouse/src/Common + contrib/libs/brotli/dec + contrib/libs/brotli/enc + contrib/libs/poco/NetSSL_OpenSSL +) + +SRCS( + AIO.cpp + AIOContextPool.cpp + BrotliReadBuffer.cpp + BrotliWriteBuffer.cpp + CascadeWriteBuffer.cpp + CompressionMethod.cpp + copyData.cpp + createReadBufferFromFileBase.cpp + createWriteBufferFromFileBase.cpp + DoubleConverter.cpp + HashingWriteBuffer.cpp + HexWriteBuffer.cpp + HTTPCommon.cpp + LimitReadBuffer.cpp + MemoryReadWriteBuffer.cpp + MMapReadBufferFromFile.cpp + MMapReadBufferFromFileDescriptor.cpp + NullWriteBuffer.cpp + parseDateTimeBestEffort.cpp + PeekableReadBuffer.cpp + Progress.cpp + ReadBufferAIO.cpp + ReadBufferFromFile.cpp + ReadBufferFromFileBase.cpp + ReadBufferFromFileDescriptor.cpp + ReadBufferFromIStream.cpp + ReadBufferFromMemory.cpp + ReadBufferFromPocoSocket.cpp + readFloatText.cpp + ReadHelpers.cpp + ReadWriteBufferFromHTTP.cpp + UseSSL.cpp + WriteBufferAIO.cpp + WriteBufferFromFile.cpp + WriteBufferFromFileBase.cpp + WriteBufferFromFileDescriptor.cpp + WriteBufferFromFileDescriptorDiscardOnFailure.cpp + WriteBufferFromHTTP.cpp + WriteBufferFromHTTPServerResponse.cpp + WriteBufferFromOStream.cpp + WriteBufferFromPocoSocket.cpp + WriteBufferFromTemporaryFile.cpp + WriteBufferValidUTF8.cpp + WriteHelpers.cpp + ZlibDeflatingWriteBuffer.cpp + ZlibInflatingReadBuffer.cpp +) + +END() diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 07c1d7476ad..03fc04d2152 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -25,11 +25,14 @@ #include #include #include -#include #include #include #include +#if !defined(ARCADIA_BUILD) +# include +#endif + namespace ProfileEvents { diff --git a/src/Interpreters/AsynchronousMetrics.cpp b/src/Interpreters/AsynchronousMetrics.cpp index 35a352b67b6..304db048f4f 100644 --- a/src/Interpreters/AsynchronousMetrics.cpp +++ b/src/Interpreters/AsynchronousMetrics.cpp @@ -5,7 +5,6 @@ #include #include #include -#include "config_core.h" #include #include #include @@ -13,12 +12,12 @@ #include #include -#if __has_include() -#include +#if !defined(ARCADIA_BUILD) +# include "config_core.h" #endif #if USE_JEMALLOC - #include +# include #endif @@ -216,9 +215,9 @@ void AsynchronousMetrics::update() set("NumberOfTables", total_number_of_tables); } -#if USE_JEMALLOC +#if USE_JEMALLOC && JEMALLOC_VERSION_MAJOR >= 4 { - #define FOR_EACH_METRIC(M) \ +# define FOR_EACH_METRIC(M) \ M("allocated", size_t) \ M("active", size_t) \ M("metadata", size_t) \ @@ -228,9 +227,9 @@ void AsynchronousMetrics::update() M("retained", size_t) \ M("background_thread.num_threads", size_t) \ M("background_thread.num_runs", uint64_t) \ - M("background_thread.run_interval", uint64_t) \ + M("background_thread.run_interval", uint64_t) - #define GET_METRIC(NAME, TYPE) \ +# define GET_METRIC(NAME, TYPE) \ do \ { \ TYPE value{}; \ @@ -241,13 +240,12 @@ void AsynchronousMetrics::update() FOR_EACH_METRIC(GET_METRIC) - #undef GET_METRIC - #undef FOR_EACH_METRIC +# undef GET_METRIC +# undef FOR_EACH_METRIC } #endif /// Add more metrics as you wish. } - } diff --git a/src/Interpreters/ClientInfo.cpp b/src/Interpreters/ClientInfo.cpp index ed806e5ad57..909582aa308 100644 --- a/src/Interpreters/ClientInfo.cpp +++ b/src/Interpreters/ClientInfo.cpp @@ -6,9 +6,12 @@ #include #include #include -#include #include +#if !defined(ARCADIA_BUILD) +# include +#endif + namespace DB { diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index f68b08bf9f0..6cf1a066b18 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -13,7 +13,6 @@ #include #include #include -#include "config_core.h" #include #include #include @@ -25,6 +24,10 @@ #include #include +#if !defined(ARCADIA_BUILD) +# include "config_core.h" +#endif + namespace Poco { diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index 435e493ffa9..447c54c769a 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -1,4 +1,3 @@ -#include "config_core.h" #include #include #include @@ -15,6 +14,10 @@ #include #include +#if !defined(ARCADIA_BUILD) +# include "config_core.h" +#endif + namespace ProfileEvents { diff --git a/src/Interpreters/ExpressionActions.h b/src/Interpreters/ExpressionActions.h index 0c3027dfbab..5a29eaaab9e 100644 --- a/src/Interpreters/ExpressionActions.h +++ b/src/Interpreters/ExpressionActions.h @@ -7,12 +7,15 @@ #include #include #include -#include "config_core.h" #include #include #include #include +#if !defined(ARCADIA_BUILD) +# include "config_core.h" +#endif + namespace DB { diff --git a/src/Interpreters/ExpressionJIT.h b/src/Interpreters/ExpressionJIT.h index 995fb35e52c..0a103aab378 100644 --- a/src/Interpreters/ExpressionJIT.h +++ b/src/Interpreters/ExpressionJIT.h @@ -1,13 +1,15 @@ #pragma once -#include "config_core.h" -#if USE_EMBEDDED_COMPILER +#if !defined(ARCADIA_BUILD) +# include "config_core.h" +#endif -#include -#include -#include -#include -#include +#if USE_EMBEDDED_COMPILER +# include +# include +# include +# include +# include namespace DB diff --git a/src/Interpreters/ExternalDictionariesLoader.cpp b/src/Interpreters/ExternalDictionariesLoader.cpp index 02388028c73..889b1c58b55 100644 --- a/src/Interpreters/ExternalDictionariesLoader.cpp +++ b/src/Interpreters/ExternalDictionariesLoader.cpp @@ -1,6 +1,9 @@ #include #include -#include "config_core.h" + +#if !defined(ARCADIA_BUILD) +# include "config_core.h" +#endif #if USE_MYSQL # include @@ -32,9 +35,9 @@ ExternalLoader::LoadablePtr ExternalDictionariesLoader::create( void ExternalDictionariesLoader::resetAll() { - #if USE_MYSQL - mysqlxx::PoolFactory::instance().reset(); - #endif +#if USE_MYSQL + mysqlxx::PoolFactory::instance().reset(); +#endif } } diff --git a/src/Interpreters/InterpreterSetRoleQuery.cpp b/src/Interpreters/InterpreterSetRoleQuery.cpp index 2a6f2317a9c..8f085d66c4c 100644 --- a/src/Interpreters/InterpreterSetRoleQuery.cpp +++ b/src/Interpreters/InterpreterSetRoleQuery.cpp @@ -50,7 +50,7 @@ void InterpreterSetRoleQuery::setRole(const ASTSetRoleQuery & query) { for (const auto & id : roles_from_query.getMatchingIDs()) { - if (!user->granted_roles.contains(id)) + if (!user->granted_roles.count(id)) throw Exception("Role should be granted to set current", ErrorCodes::SET_NON_GRANTED_ROLE); new_current_roles.push_back(id); } @@ -85,7 +85,7 @@ void InterpreterSetRoleQuery::updateUserSetDefaultRoles(User & user, const Exten { for (const auto & id : roles_from_query.getMatchingIDs()) { - if (!user.granted_roles.contains(id)) + if (!user.granted_roles.count(id)) throw Exception("Role should be granted to set default", ErrorCodes::SET_NON_GRANTED_ROLE); } } diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index 36713be1ff7..01a84559e92 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -1,7 +1,6 @@ #include #include #include -#include "config_core.h" #include #include #include @@ -32,6 +31,10 @@ #include #include +#if !defined(ARCADIA_BUILD) +# include "config_core.h" +#endif + namespace DB { diff --git a/src/Interpreters/ya.make b/src/Interpreters/ya.make new file mode 100644 index 00000000000..7eb8a899e9c --- /dev/null +++ b/src/Interpreters/ya.make @@ -0,0 +1,138 @@ +LIBRARY() + +ADDINCL( + contrib/libs/libdivide + contrib/libs/pdqsort +) + +PEERDIR( + clickhouse/src/Core + contrib/libs/libdivide + contrib/libs/pdqsort +) + +NO_COMPILER_WARNINGS() + +SRCS( + ActionLocksManager.cpp + ActionsVisitor.cpp + addMissingDefaults.cpp + addTypeConversionToAST.cpp + Aggregator.cpp + ArrayJoinAction.cpp + AsynchronousMetrics.cpp + BloomFilter.cpp + castColumn.cpp + CatBoostModel.cpp + ClientInfo.cpp + Cluster.cpp + ClusterProxy/executeQuery.cpp + ClusterProxy/SelectStreamFactory.cpp + CollectJoinOnKeysVisitor.cpp + ColumnNamesContext.cpp + Context.cpp + convertFieldToType.cpp + createBlockSelector.cpp + CrossToInnerJoinVisitor.cpp + DatabaseAndTableWithAlias.cpp + DatabaseCatalog.cpp + DDLWorker.cpp + DNSCacheUpdater.cpp + EmbeddedDictionaries.cpp + evaluateConstantExpression.cpp + executeQuery.cpp + ExecuteScalarSubqueriesVisitor.cpp + ExpressionActions.cpp + ExpressionAnalyzer.cpp + ExternalDictionariesLoader.cpp + ExternalLoader.cpp + ExternalLoaderDatabaseConfigRepository.cpp + ExternalLoaderTempConfigRepository.cpp + ExternalLoaderXMLConfigRepository.cpp + ExternalModelsLoader.cpp + ExtractExpressionInfoVisitor.cpp + FillingRow.cpp + getClusterName.cpp + getTableExpressions.cpp + HashJoin.cpp + IdentifierSemantic.cpp + IExternalLoadable.cpp + InJoinSubqueriesPreprocessor.cpp + inplaceBlockConversions.cpp + InternalTextLogsQueue.cpp + InterpreterAlterQuery.cpp + InterpreterCheckQuery.cpp + InterpreterCreateQuery.cpp + InterpreterCreateQuotaQuery.cpp + InterpreterCreateRoleQuery.cpp + InterpreterCreateRowPolicyQuery.cpp + InterpreterCreateSettingsProfileQuery.cpp + InterpreterCreateUserQuery.cpp + InterpreterDescribeQuery.cpp + InterpreterDropAccessEntityQuery.cpp + InterpreterDropQuery.cpp + InterpreterExistsQuery.cpp + InterpreterExplainQuery.cpp + InterpreterFactory.cpp + InterpreterGrantQuery.cpp + InterpreterInsertQuery.cpp + InterpreterKillQueryQuery.cpp + InterpreterOptimizeQuery.cpp + InterpreterRenameQuery.cpp + InterpreterSelectQuery.cpp + InterpreterSelectWithUnionQuery.cpp + InterpreterSetQuery.cpp + InterpreterSetRoleQuery.cpp + InterpreterShowCreateAccessEntityQuery.cpp + InterpreterShowCreateQuery.cpp + InterpreterShowGrantsQuery.cpp + InterpreterShowProcesslistQuery.cpp + InterpreterShowQuotasQuery.cpp + InterpreterShowRowPoliciesQuery.cpp + InterpreterShowTablesQuery.cpp + InterpreterSystemQuery.cpp + InterpreterUseQuery.cpp + InterpreterWatchQuery.cpp + interpretSubquery.cpp + join_common.cpp + JoinedTables.cpp + JoinSwitcher.cpp + JoinToSubqueryTransformVisitor.cpp + loadMetadata.cpp + LogicalExpressionsOptimizer.cpp + MarkTableIdentifiersVisitor.cpp + MergeJoin.cpp + MetricLog.cpp + MutationsInterpreter.cpp + NullableUtils.cpp + OptimizeIfChains.cpp + OptimizeIfWithConstantConditionVisitor.cpp + PartLog.cpp + PredicateExpressionsOptimizer.cpp + PredicateRewriteVisitor.cpp + ProcessList.cpp + ProfileEventsExt.cpp + QueryAliasesVisitor.cpp + QueryLog.cpp + QueryNormalizer.cpp + QueryThreadLog.cpp + RenameColumnVisitor.cpp + ReplaceQueryParameterVisitor.cpp + RequiredSourceColumnsVisitor.cpp + RowRefs.cpp + Set.cpp + SetVariants.cpp + sortBlock.cpp + StorageID.cpp + SubqueryForSet.cpp + SyntaxAnalyzer.cpp + SystemLog.cpp + TableJoin.cpp + TablesStatus.cpp + TextLog.cpp + ThreadStatusExt.cpp + TraceLog.cpp + TranslateQualifiedNamesVisitor.cpp +) + +END() diff --git a/src/Parsers/ASTSystemQuery.h b/src/Parsers/ASTSystemQuery.h index f5b9afde4b3..eac96e50cb1 100644 --- a/src/Parsers/ASTSystemQuery.h +++ b/src/Parsers/ASTSystemQuery.h @@ -1,9 +1,12 @@ #pragma once -#include "config_core.h" #include #include +#if !defined(ARCADIA_BUILD) +# include "config_core.h" +#endif + namespace DB { diff --git a/src/Parsers/ya.make b/src/Parsers/ya.make new file mode 100644 index 00000000000..942124c2c7a --- /dev/null +++ b/src/Parsers/ya.make @@ -0,0 +1,113 @@ +LIBRARY() + +PEERDIR( + clickhouse/src/Common +) + +SRCS( + ASTAlterQuery.cpp + ASTAsterisk.cpp + ASTColumnDeclaration.cpp + ASTColumnsMatcher.cpp + ASTConstraintDeclaration.cpp + ASTCreateQuery.cpp + ASTCreateQuotaQuery.cpp + ASTCreateRoleQuery.cpp + ASTCreateRowPolicyQuery.cpp + ASTCreateSettingsProfileQuery.cpp + ASTCreateUserQuery.cpp + ASTDictionary.cpp + ASTDictionaryAttributeDeclaration.cpp + ASTDropAccessEntityQuery.cpp + ASTDropQuery.cpp + ASTExpressionList.cpp + ASTExtendedRoleSet.cpp + ASTFunction.cpp + ASTFunctionWithKeyValueArguments.cpp + ASTGrantQuery.cpp + ASTIdentifier.cpp + ASTInsertQuery.cpp + ASTKillQueryQuery.cpp + ASTLiteral.cpp + ASTOptimizeQuery.cpp + ASTOrderByElement.cpp + ASTPartition.cpp + ASTQualifiedAsterisk.cpp + ASTQueryParameter.cpp + ASTQueryWithOnCluster.cpp + ASTQueryWithOutput.cpp + ASTQueryWithTableAndOutput.cpp + ASTSampleRatio.cpp + ASTSelectQuery.cpp + ASTSelectWithUnionQuery.cpp + ASTSetRoleQuery.cpp + ASTSettingsProfileElement.cpp + ASTShowCreateAccessEntityQuery.cpp + ASTShowGrantsQuery.cpp + ASTShowQuotasQuery.cpp + ASTShowRowPoliciesQuery.cpp + ASTShowTablesQuery.cpp + ASTSubquery.cpp + ASTSystemQuery.cpp + ASTTablesInSelectQuery.cpp + ASTTTLElement.cpp + ASTWithAlias.cpp + CommonParsers.cpp + ExpressionElementParsers.cpp + ExpressionListParsers.cpp + formatAST.cpp + IAST.cpp + iostream_debug_helpers.cpp + IParserBase.cpp + Lexer.cpp + makeASTForLogicalFunction.cpp + parseDatabaseAndTableName.cpp + parseIdentifierOrStringLiteral.cpp + parseIntervalKind.cpp + parseQuery.cpp + ParserAlterQuery.cpp + ParserCase.cpp + ParserCheckQuery.cpp + ParserCreateQuery.cpp + ParserCreateQuotaQuery.cpp + ParserCreateRoleQuery.cpp + ParserCreateRowPolicyQuery.cpp + ParserCreateSettingsProfileQuery.cpp + ParserCreateUserQuery.cpp + ParserDescribeTableQuery.cpp + ParserDictionary.cpp + ParserDictionaryAttributeDeclaration.cpp + ParserDropAccessEntityQuery.cpp + ParserDropQuery.cpp + ParserExtendedRoleSet.cpp + ParserGrantQuery.cpp + ParserInsertQuery.cpp + ParserKillQueryQuery.cpp + ParserOptimizeQuery.cpp + ParserPartition.cpp + ParserQuery.cpp + ParserQueryWithOutput.cpp + ParserRenameQuery.cpp + ParserSampleRatio.cpp + ParserSelectQuery.cpp + ParserSelectWithUnionQuery.cpp + ParserSetQuery.cpp + ParserSetRoleQuery.cpp + ParserSettingsProfileElement.cpp + ParserShowCreateAccessEntityQuery.cpp + ParserShowGrantsQuery.cpp + ParserShowQuotasQuery.cpp + ParserShowRowPoliciesQuery.cpp + ParserShowTablesQuery.cpp + ParserSystemQuery.cpp + ParserTablePropertiesQuery.cpp + ParserTablesInSelectQuery.cpp + ParserUnionQueryElement.cpp + ParserUseQuery.cpp + ParserWatchQuery.cpp + parseUserName.cpp + queryToString.cpp + TokenIterator.cpp +) + +END() diff --git a/src/Processors/Formats/Impl/ProtobufRowInputFormat.h b/src/Processors/Formats/Impl/ProtobufRowInputFormat.h index 029b2c8329e..8dd36722efc 100644 --- a/src/Processors/Formats/Impl/ProtobufRowInputFormat.h +++ b/src/Processors/Formats/Impl/ProtobufRowInputFormat.h @@ -1,11 +1,13 @@ #pragma once -#include "config_formats.h" -#if USE_PROTOBUF +#if !defined(ARCADIA_BUILD) +# include "config_formats.h" +#endif -#include -#include -#include +#if USE_PROTOBUF +# include +# include +# include namespace DB { diff --git a/src/Processors/Formats/Impl/ProtobufRowOutputFormat.h b/src/Processors/Formats/Impl/ProtobufRowOutputFormat.h index ae8ebbf28f4..f216af01d80 100644 --- a/src/Processors/Formats/Impl/ProtobufRowOutputFormat.h +++ b/src/Processors/Formats/Impl/ProtobufRowOutputFormat.h @@ -1,13 +1,15 @@ #pragma once -#include "config_formats.h" -#if USE_PROTOBUF +#if !defined(ARCADIA_BUILD) +# include "config_formats.h" +#endif -#include -#include -#include -#include -#include +#if USE_PROTOBUF +# include +# include +# include +# include +# include namespace google diff --git a/src/Processors/ya.make b/src/Processors/ya.make new file mode 100644 index 00000000000..9822344aec1 --- /dev/null +++ b/src/Processors/ya.make @@ -0,0 +1,100 @@ +LIBRARY() + +PEERDIR( + clickhouse/src/Common + contrib/libs/msgpack + contrib/libs/protobuf_std +) + +SRCS( + Chunk.cpp + ConcatProcessor.cpp + DelayedPortsProcessor.cpp + Executors/ParallelPipelineExecutor.cpp + Executors/PipelineExecutor.cpp + Executors/SequentialPipelineExecutor.cpp + Executors/TreeExecutorBlockInputStream.cpp + ForkProcessor.cpp + Formats/IInputFormat.cpp + Formats/Impl/BinaryRowInputFormat.cpp + Formats/Impl/BinaryRowOutputFormat.cpp + Formats/Impl/ConstantExpressionTemplate.cpp + Formats/Impl/CSVRowInputFormat.cpp + Formats/Impl/CSVRowOutputFormat.cpp + Formats/Impl/JSONCompactEachRowRowInputFormat.cpp + Formats/Impl/JSONCompactEachRowRowOutputFormat.cpp + Formats/Impl/JSONCompactRowOutputFormat.cpp + Formats/Impl/JSONEachRowRowInputFormat.cpp + Formats/Impl/JSONEachRowRowOutputFormat.cpp + Formats/Impl/JSONEachRowWithProgressRowOutputFormat.cpp + Formats/Impl/JSONRowOutputFormat.cpp + Formats/Impl/MsgPackRowInputFormat.cpp + Formats/Impl/MsgPackRowOutputFormat.cpp + Formats/Impl/MySQLOutputFormat.cpp + Formats/Impl/NativeFormat.cpp + Formats/Impl/NullFormat.cpp + Formats/Impl/ODBCDriver2BlockOutputFormat.cpp + Formats/Impl/ODBCDriverBlockOutputFormat.cpp + Formats/Impl/PrettyBlockOutputFormat.cpp + Formats/Impl/PrettyCompactBlockOutputFormat.cpp + Formats/Impl/PrettySpaceBlockOutputFormat.cpp + Formats/Impl/ProtobufRowInputFormat.cpp + Formats/Impl/ProtobufRowOutputFormat.cpp + Formats/Impl/RegexpRowInputFormat.cpp + Formats/Impl/TabSeparatedRowInputFormat.cpp + Formats/Impl/TabSeparatedRowOutputFormat.cpp + Formats/Impl/TemplateBlockOutputFormat.cpp + Formats/Impl/TemplateRowInputFormat.cpp + Formats/Impl/TSKVRowInputFormat.cpp + Formats/Impl/TSKVRowOutputFormat.cpp + Formats/Impl/ValuesBlockInputFormat.cpp + Formats/Impl/ValuesRowOutputFormat.cpp + Formats/Impl/VerticalRowOutputFormat.cpp + Formats/Impl/XMLRowOutputFormat.cpp + Formats/IOutputFormat.cpp + Formats/IRowInputFormat.cpp + Formats/IRowOutputFormat.cpp + Formats/LazyOutputFormat.cpp + Formats/OutputStreamToOutputFormat.cpp + Formats/RowInputFormatWithDiagnosticInfo.cpp + IAccumulatingTransform.cpp + IInflatingTransform.cpp + IProcessor.cpp + ISimpleTransform.cpp + ISink.cpp + ISource.cpp + LimitTransform.cpp + Pipe.cpp + Port.cpp + QueryPipeline.cpp + ResizeProcessor.cpp + Sources/SinkToOutputStream.cpp + Sources/SourceFromInputStream.cpp + Sources/SourceWithProgress.cpp + Transforms/AddingMissedTransform.cpp + Transforms/AggregatingTransform.cpp + Transforms/ConvertingTransform.cpp + Transforms/CreatingSetsTransform.cpp + Transforms/CubeTransform.cpp + Transforms/DistinctTransform.cpp + Transforms/ExpressionTransform.cpp + Transforms/ExtremesTransform.cpp + Transforms/FillingTransform.cpp + Transforms/FilterTransform.cpp + Transforms/FinishSortingTransform.cpp + Transforms/InflatingExpressionTransform.cpp + Transforms/LimitByTransform.cpp + Transforms/LimitsCheckingTransform.cpp + Transforms/MaterializingTransform.cpp + Transforms/MergeSortingTransform.cpp + Transforms/MergingAggregatedMemoryEfficientTransform.cpp + Transforms/MergingAggregatedTransform.cpp + Transforms/MergingSortedTransform.cpp + Transforms/PartialSortingTransform.cpp + Transforms/ReverseTransform.cpp + Transforms/RollupTransform.cpp + Transforms/SortingTransform.cpp + Transforms/TotalsHavingTransform.cpp +) + +END() diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index 5a792080370..fc69123e696 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -126,8 +126,13 @@ Block IStorage::getSampleBlockForColumns(const Names & column_names) const namespace { - using NamesAndTypesMap = ::google::dense_hash_map; - using UniqueStrings = ::google::dense_hash_set; +#if !defined(ARCADIA_BUILD) + using NamesAndTypesMap = google::dense_hash_map; + using UniqueStrings = google::dense_hash_set; +#else + using NamesAndTypesMap = google::sparsehash::dense_hash_map; + using UniqueStrings = google::sparsehash::dense_hash_set; +#endif String listOfColumns(const NamesAndTypesList & available_columns) { diff --git a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp index e42ac942362..6887b7c0d9d 100644 --- a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp @@ -208,8 +208,7 @@ bool MergeTreeConditionFullText::mayBeTrueOnGranule(MergeTreeIndexGranulePtr idx else if (element.function == RPNElement::FUNCTION_EQUALS || element.function == RPNElement::FUNCTION_NOT_EQUALS) { - rpn_stack.emplace_back( - granule->bloom_filters[element.key_column].contains(*element.bloom_filter), true); + rpn_stack.emplace_back(granule->bloom_filters[element.key_column].contains(*element.bloom_filter), true); if (element.function == RPNElement::FUNCTION_NOT_EQUALS) rpn_stack.back() = !rpn_stack.back(); diff --git a/src/Storages/StorageMySQL.h b/src/Storages/StorageMySQL.h index 3bde1cab4c7..8b98536e4d7 100644 --- a/src/Storages/StorageMySQL.h +++ b/src/Storages/StorageMySQL.h @@ -1,13 +1,16 @@ #pragma once -#include "config_core.h" +#if !defined(ARCADIA_BUILD) +# include "config_core.h" +#endif + #if USE_MYSQL -#include +# include -#include -#include -#include +# include +# include +# include namespace DB diff --git a/src/Storages/System/StorageSystemBuildOptions.cpp b/src/Storages/System/StorageSystemBuildOptions.cpp index 73d1c91f422..4dd4ff90572 100644 --- a/src/Storages/System/StorageSystemBuildOptions.cpp +++ b/src/Storages/System/StorageSystemBuildOptions.cpp @@ -18,11 +18,15 @@ NamesAndTypesList StorageSystemBuildOptions::getNamesAndTypes() void StorageSystemBuildOptions::fillData(MutableColumns & res_columns, const Context &, const SelectQueryInfo &) const { +#if !defined(ARCADIA_BUILD) for (auto it = auto_config_build; *it; it += 2) { res_columns[0]->insert(it[0]); res_columns[1]->insert(it[1]); } +#else + UNUSED(res_columns); +#endif } } diff --git a/src/Storages/registerStorages.cpp b/src/Storages/registerStorages.cpp index f5fab52285d..5ad26b70803 100644 --- a/src/Storages/registerStorages.cpp +++ b/src/Storages/registerStorages.cpp @@ -1,6 +1,3 @@ -#include -#include "config_core.h" - #include #include diff --git a/src/Storages/registerStorages.h b/src/Storages/registerStorages.h index 63a758f5b38..c9874551073 100644 --- a/src/Storages/registerStorages.h +++ b/src/Storages/registerStorages.h @@ -1,6 +1,9 @@ #pragma once -#include -#include "config_core.h" + +#if !defined(ARCADIA_BUILD) +# include +# include "config_core.h" +#endif namespace DB { diff --git a/src/Storages/ya.make b/src/Storages/ya.make new file mode 100644 index 00000000000..6a032d82bdc --- /dev/null +++ b/src/Storages/ya.make @@ -0,0 +1,167 @@ +LIBRARY() + +PEERDIR( + clickhouse/src/Common + contrib/libs/sparsehash +) + +SRCS( + Distributed/DirectoryMonitor.cpp + Distributed/DistributedBlockOutputStream.cpp + LiveView/StorageLiveView.cpp + MergeTree/ActiveDataPartSet.cpp + MergeTree/AllMergeSelector.cpp + MergeTree/BackgroundProcessingPool.cpp + MergeTree/BoolMask.cpp + MergeTree/checkDataPart.cpp + MergeTree/DataPartsExchange.cpp + MergeTree/EphemeralLockInZooKeeper.cpp + MergeTree/IMergedBlockOutputStream.cpp + MergeTree/IMergeTreeDataPart.cpp + MergeTree/IMergeTreeDataPartWriter.cpp + MergeTree/IMergeTreeReader.cpp + MergeTree/KeyCondition.cpp + MergeTree/LevelMergeSelector.cpp + MergeTree/localBackup.cpp + MergeTree/MergedBlockOutputStream.cpp + MergeTree/MergedColumnOnlyOutputStream.cpp + MergeTree/MergeList.cpp + MergeTree/MergeTreeBaseSelectProcessor.cpp + MergeTree/MergeTreeBlockOutputStream.cpp + MergeTree/MergeTreeBlockReadUtils.cpp + MergeTree/MergeTreeData.cpp + MergeTree/MergeTreeDataMergerMutator.cpp + MergeTree/MergeTreeDataPartChecksum.cpp + MergeTree/MergeTreeDataPartCompact.cpp + MergeTree/MergeTreeDataPartTTLInfo.cpp + MergeTree/MergeTreeDataPartType.cpp + MergeTree/MergeTreeDataPartWide.cpp + MergeTree/MergeTreeDataPartWriterCompact.cpp + MergeTree/MergeTreeDataPartWriterWide.cpp + MergeTree/MergeTreeDataSelectExecutor.cpp + MergeTree/MergeTreeDataWriter.cpp + MergeTree/MergeTreeIndexAggregatorBloomFilter.cpp + MergeTree/MergeTreeIndexBloomFilter.cpp + MergeTree/MergeTreeIndexConditionBloomFilter.cpp + MergeTree/MergeTreeIndexFullText.cpp + MergeTree/MergeTreeIndexGranularity.cpp + MergeTree/MergeTreeIndexGranularityInfo.cpp + MergeTree/MergeTreeIndexGranuleBloomFilter.cpp + MergeTree/MergeTreeIndexMinMax.cpp + MergeTree/MergeTreeIndexReader.cpp + MergeTree/MergeTreeIndexSet.cpp + MergeTree/MergeTreeIndices.cpp + MergeTree/MergeTreeMarksLoader.cpp + MergeTree/MergeTreeMutationEntry.cpp + MergeTree/MergeTreePartInfo.cpp + MergeTree/MergeTreePartition.cpp + MergeTree/MergeTreePartsMover.cpp + MergeTree/MergeTreeRangeReader.cpp + MergeTree/MergeTreeReaderCompact.cpp + MergeTree/MergeTreeReaderStream.cpp + MergeTree/MergeTreeReaderWide.cpp + MergeTree/MergeTreeReadPool.cpp + MergeTree/MergeTreeReverseSelectProcessor.cpp + MergeTree/MergeTreeSelectProcessor.cpp + MergeTree/MergeTreeSequentialBlockInputStream.cpp + MergeTree/MergeTreeSettings.cpp + MergeTree/MergeTreeThreadSelectBlockInputProcessor.cpp + MergeTree/MergeTreeWhereOptimizer.cpp + MergeTree/registerStorageMergeTree.cpp + MergeTree/ReplicatedMergeTreeAddress.cpp + MergeTree/ReplicatedMergeTreeAltersSequence.cpp + MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp + MergeTree/ReplicatedMergeTreeCleanupThread.cpp + MergeTree/ReplicatedMergeTreeLogEntry.cpp + MergeTree/ReplicatedMergeTreeMutationEntry.cpp + MergeTree/ReplicatedMergeTreePartCheckThread.cpp + MergeTree/ReplicatedMergeTreePartHeader.cpp + MergeTree/ReplicatedMergeTreeQueue.cpp + MergeTree/ReplicatedMergeTreeRestartingThread.cpp + MergeTree/ReplicatedMergeTreeTableMetadata.cpp + MergeTree/SimpleMergeSelector.cpp + MergeTree/TTLMergeSelector.cpp + System/attachSystemTables.cpp + System/StorageSystemAggregateFunctionCombinators.cpp + System/StorageSystemAsynchronousMetrics.cpp + System/StorageSystemBuildOptions.cpp + System/StorageSystemClusters.cpp + System/StorageSystemCollations.cpp + System/StorageSystemColumns.cpp + System/StorageSystemContributors.cpp + System/StorageSystemContributors.generated.cpp + System/StorageSystemDatabases.cpp + System/StorageSystemDataTypeFamilies.cpp + System/StorageSystemDetachedParts.cpp + System/StorageSystemDictionaries.cpp + System/StorageSystemDisks.cpp + System/StorageSystemEvents.cpp + System/StorageSystemFormats.cpp + System/StorageSystemFunctions.cpp + System/StorageSystemGraphite.cpp + System/StorageSystemMacros.cpp + System/StorageSystemMerges.cpp + System/StorageSystemMergeTreeSettings.cpp + System/StorageSystemMetrics.cpp + System/StorageSystemModels.cpp + System/StorageSystemMutations.cpp + System/StorageSystemNumbers.cpp + System/StorageSystemOne.cpp + System/StorageSystemParts.cpp + System/StorageSystemPartsBase.cpp + System/StorageSystemPartsColumns.cpp + System/StorageSystemProcesses.cpp + System/StorageSystemQuotas.cpp + System/StorageSystemQuotaUsage.cpp + System/StorageSystemReplicas.cpp + System/StorageSystemReplicationQueue.cpp + System/StorageSystemRowPolicies.cpp + System/StorageSystemSettings.cpp + System/StorageSystemStackTrace.cpp + System/StorageSystemStoragePolicies.cpp + System/StorageSystemTableEngines.cpp + System/StorageSystemTableFunctions.cpp + System/StorageSystemTables.cpp + System/StorageSystemZeros.cpp + System/StorageSystemZooKeeper.cpp + AlterCommands.cpp + ColumnDefault.cpp + ColumnsDescription.cpp + ConstraintsDescription.cpp + getStructureOfRemoteTable.cpp + IndicesDescription.cpp + IStorage.cpp + MutationCommands.cpp + PartitionCommands.cpp + ReadInOrderOptimizer.cpp + registerStorages.cpp + StorageBuffer.cpp + StorageDictionary.cpp + StorageDistributed.cpp + StorageFactory.cpp + StorageFile.cpp + StorageGenerateRandom.cpp + StorageInMemoryMetadata.cpp + StorageInput.cpp + StorageJoin.cpp + StorageLog.cpp + StorageLogSettings.cpp + StorageMaterializedView.cpp + StorageMemory.cpp + StorageMerge.cpp + StorageMergeTree.cpp + StorageMySQL.cpp + StorageNull.cpp + StorageReplicatedMergeTree.cpp + StorageSet.cpp + StorageStripeLog.cpp + StorageTinyLog.cpp + StorageURL.cpp + StorageValues.cpp + StorageView.cpp + StorageXDBC.cpp + transformQueryForExternalDatabase.cpp + VirtualColumnUtils.cpp +) + +END() diff --git a/src/TableFunctions/ITableFunctionXDBC.h b/src/TableFunctions/ITableFunctionXDBC.h index 262c237bac2..547fda3f734 100644 --- a/src/TableFunctions/ITableFunctionXDBC.h +++ b/src/TableFunctions/ITableFunctionXDBC.h @@ -4,7 +4,10 @@ #include #include #include -#include + +#if !defined(ARCADIA_BUILD) +# include +#endif namespace DB { diff --git a/src/TableFunctions/TableFunctionMySQL.cpp b/src/TableFunctions/TableFunctionMySQL.cpp index be707c3520d..0afc74f163e 100644 --- a/src/TableFunctions/TableFunctionMySQL.cpp +++ b/src/TableFunctions/TableFunctionMySQL.cpp @@ -1,25 +1,27 @@ -#include "config_core.h" +#if !defined(ARCADIA_BUILD) +# include "config_core.h" +#endif + #if USE_MYSQL +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include "registerTableFunctions.h" -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include "registerTableFunctions.h" - -#include +# include namespace DB diff --git a/src/TableFunctions/registerTableFunctions.h b/src/TableFunctions/registerTableFunctions.h index 3cde28685cc..8c54913676a 100644 --- a/src/TableFunctions/registerTableFunctions.h +++ b/src/TableFunctions/registerTableFunctions.h @@ -1,6 +1,9 @@ #pragma once -#include -#include "config_core.h" + +#if !defined(ARCADIA_BUILD) +# include +# include "config_core.h" +#endif namespace DB { diff --git a/src/TableFunctions/ya.make b/src/TableFunctions/ya.make new file mode 100644 index 00000000000..4d5d53ead88 --- /dev/null +++ b/src/TableFunctions/ya.make @@ -0,0 +1,26 @@ +LIBRARY() + +PEERDIR( + clickhouse/src/Common +) + +SRCS( + ITableFunction.cpp + ITableFunctionFileLike.cpp + ITableFunctionXDBC.cpp + parseColumnsListForTableFunction.cpp + registerTableFunctions.cpp + TableFunctionFactory.cpp + TableFunctionFile.cpp + TableFunctionGenerateRandom.cpp + TableFunctionInput.cpp + TableFunctionMerge.cpp + TableFunctionMySQL.cpp + TableFunctionNumbers.cpp + TableFunctionRemote.cpp + TableFunctionURL.cpp + TableFunctionValues.cpp + TableFunctionZeros.cpp +) + +END() diff --git a/src/ya.make b/src/ya.make new file mode 100644 index 00000000000..95e5914bbe7 --- /dev/null +++ b/src/ya.make @@ -0,0 +1,26 @@ +LIBRARY() + +PEERDIR( + clickhouse/src/Access + clickhouse/src/AggregateFunctions + clickhouse/src/Client + clickhouse/src/Columns + clickhouse/src/Common + clickhouse/src/Compression + clickhouse/src/Core + clickhouse/src/Databases + clickhouse/src/DataStreams + clickhouse/src/DataTypes + clickhouse/src/Formats + clickhouse/src/Dictionaries + clickhouse/src/Disks + clickhouse/src/Functions + clickhouse/src/Interpreters + clickhouse/src/IO + clickhouse/src/Parsers + clickhouse/src/Processors + clickhouse/src/Storages + clickhouse/src/TableFunctions +) + +END() diff --git a/ya.make b/ya.make index 84f1a2163a4..7f7c1bbce2f 100644 --- a/ya.make +++ b/ya.make @@ -7,4 +7,6 @@ OWNER(g:clickhouse) RECURSE( base + programs + src )