mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 23:21:59 +00:00
Remove cruft
This commit is contained in:
parent
fab8da1c64
commit
8b4a6a2416
@ -177,8 +177,6 @@ DateLUTImpl::DateLUTImpl(const std::string & time_zone_)
|
||||
}
|
||||
|
||||
|
||||
#if !defined(ARCADIA_BUILD) /// Arcadia's variant of CCTZ already has the same implementation.
|
||||
|
||||
/// Prefer to load timezones from blobs linked to the binary.
|
||||
/// The blobs are provided by "tzdata" library.
|
||||
/// This allows to avoid dependency on system tzdata.
|
||||
@ -234,5 +232,3 @@ namespace cctz_extension
|
||||
|
||||
ZoneInfoSourceFactory zone_info_source_factory = custom_factory;
|
||||
}
|
||||
|
||||
#endif
|
||||
|
@ -103,7 +103,6 @@ String LineReader::readLine(const String & first_prompt, const String & second_p
|
||||
continue;
|
||||
}
|
||||
|
||||
#if !defined(ARCADIA_BUILD) /// C++20
|
||||
const char * has_extender = nullptr;
|
||||
for (const auto * extender : extenders)
|
||||
{
|
||||
@ -133,7 +132,6 @@ String LineReader::readLine(const String & first_prompt, const String & second_p
|
||||
if (input.empty())
|
||||
continue;
|
||||
}
|
||||
#endif
|
||||
|
||||
line += (line.empty() ? "" : "\n") + input;
|
||||
|
||||
|
@ -83,10 +83,6 @@
|
||||
# define BOOST_USE_UCONTEXT 1
|
||||
#endif
|
||||
|
||||
#if defined(ARCADIA_BUILD) && defined(BOOST_USE_UCONTEXT)
|
||||
# undef BOOST_USE_UCONTEXT
|
||||
#endif
|
||||
|
||||
/// TODO: Strange enough, there is no way to detect UB sanitizer.
|
||||
|
||||
/// Explicitly allow undefined behaviour for certain functions. Use it as a function attribute.
|
||||
|
@ -49,12 +49,3 @@ namespace
|
||||
#define LOG_WARNING(logger, ...) LOG_IMPL(logger, DB::LogsLevel::warning, Poco::Message::PRIO_WARNING, __VA_ARGS__)
|
||||
#define LOG_ERROR(logger, ...) LOG_IMPL(logger, DB::LogsLevel::error, Poco::Message::PRIO_ERROR, __VA_ARGS__)
|
||||
#define LOG_FATAL(logger, ...) LOG_IMPL(logger, DB::LogsLevel::error, Poco::Message::PRIO_FATAL, __VA_ARGS__)
|
||||
|
||||
|
||||
/// Compatibility for external projects.
|
||||
#if defined(ARCADIA_BUILD)
|
||||
using Poco::Logger;
|
||||
using Poco::Message;
|
||||
using DB::LogsLevel;
|
||||
using DB::CurrentThread;
|
||||
#endif
|
||||
|
@ -3,41 +3,24 @@
|
||||
#pragma GCC diagnostic push
|
||||
#pragma GCC diagnostic ignored "-Wold-style-cast"
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
# include <miniselect/floyd_rivest_select.h> // Y_IGNORE
|
||||
#else
|
||||
# include <algorithm>
|
||||
#endif
|
||||
#include <miniselect/floyd_rivest_select.h>
|
||||
|
||||
template <class RandomIt>
|
||||
void nth_element(RandomIt first, RandomIt nth, RandomIt last)
|
||||
{
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
::miniselect::floyd_rivest_select(first, nth, last);
|
||||
#else
|
||||
::std::nth_element(first, nth, last);
|
||||
#endif
|
||||
}
|
||||
|
||||
template <class RandomIt>
|
||||
void partial_sort(RandomIt first, RandomIt middle, RandomIt last)
|
||||
{
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
::miniselect::floyd_rivest_partial_sort(first, middle, last);
|
||||
#else
|
||||
::std::partial_sort(first, middle, last);
|
||||
#endif
|
||||
}
|
||||
|
||||
template <class RandomIt, class Compare>
|
||||
void partial_sort(RandomIt first, RandomIt middle, RandomIt last, Compare compare)
|
||||
{
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
::miniselect::floyd_rivest_partial_sort(first, middle, last, compare);
|
||||
#else
|
||||
::std::partial_sort(first, middle, last, compare);
|
||||
#endif
|
||||
}
|
||||
|
||||
#pragma GCC diagnostic pop
|
||||
|
||||
}
|
||||
|
@ -63,9 +63,7 @@
|
||||
#include <Common/Elf.h>
|
||||
#include <filesystem>
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
# include <Common/config_version.h>
|
||||
#endif
|
||||
#include <Common/config_version.h>
|
||||
|
||||
#if defined(OS_DARWIN)
|
||||
# pragma GCC diagnostic ignored "-Wunused-macros"
|
||||
|
@ -15,14 +15,12 @@
|
||||
#include <Core/ServerUUID.h>
|
||||
#include <Common/hex.h>
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
# include "Common/config_version.h"
|
||||
# include <Common/config.h>
|
||||
#endif
|
||||
#include "Common/config_version.h"
|
||||
#include <Common/config.h>
|
||||
|
||||
#if USE_SENTRY
|
||||
|
||||
# include <sentry.h> // Y_IGNORE
|
||||
# include <sentry.h>
|
||||
# include <stdio.h>
|
||||
# include <filesystem>
|
||||
|
||||
|
@ -2,7 +2,7 @@
|
||||
#include <errmsg.h>
|
||||
#include <mysql.h>
|
||||
#else
|
||||
#include <mysql/errmsg.h> //Y_IGNORE
|
||||
#include <mysql/errmsg.h>
|
||||
#include <mysql/mysql.h>
|
||||
#endif
|
||||
|
||||
|
@ -1,8 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
# include "config_core.h"
|
||||
#endif
|
||||
#include "config_core.h"
|
||||
|
||||
#include <Access/Credentials.h>
|
||||
#include <base/types.h>
|
||||
|
@ -1,8 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
# include "config_core.h"
|
||||
#endif
|
||||
#include "config_core.h"
|
||||
|
||||
#include <base/types.h>
|
||||
|
||||
|
@ -9,9 +9,7 @@
|
||||
#include <AggregateFunctions/IAggregateFunction.h>
|
||||
#include <Core/DecimalFunctions.h>
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
# include <Common/config.h>
|
||||
#endif
|
||||
#include <Common/config.h>
|
||||
|
||||
#if USE_EMBEDDED_COMPILER
|
||||
# include <llvm/IR/IRBuilder.h>
|
||||
|
@ -9,9 +9,7 @@
|
||||
|
||||
#include <AggregateFunctions/IAggregateFunction.h>
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
# include <Common/config.h>
|
||||
#endif
|
||||
#include <Common/config.h>
|
||||
|
||||
#if USE_EMBEDDED_COMPILER
|
||||
# include <llvm/IR/IRBuilder.h>
|
||||
|
@ -10,9 +10,7 @@
|
||||
#include <AggregateFunctions/IAggregateFunction.h>
|
||||
#include <Common/assert_cast.h>
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
# include <Common/config.h>
|
||||
#endif
|
||||
#include <Common/config.h>
|
||||
|
||||
#if USE_EMBEDDED_COMPILER
|
||||
# include <llvm/IR/IRBuilder.h>
|
||||
|
@ -5,9 +5,7 @@
|
||||
#include <Common/assert_cast.h>
|
||||
#include <AggregateFunctions/IAggregateFunction.h>
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
# include <Common/config.h>
|
||||
#endif
|
||||
#include <Common/config.h>
|
||||
|
||||
#if USE_EMBEDDED_COMPILER
|
||||
# include <llvm/IR/IRBuilder.h>
|
||||
|
@ -14,9 +14,7 @@
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <AggregateFunctions/IAggregateFunction.h>
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
# include <Common/config.h>
|
||||
#endif
|
||||
#include <Common/config.h>
|
||||
|
||||
#if USE_EMBEDDED_COMPILER
|
||||
# include <llvm/IR/IRBuilder.h>
|
||||
|
@ -10,9 +10,7 @@
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
# include <Common/config.h>
|
||||
#endif
|
||||
#include <Common/config.h>
|
||||
|
||||
#if USE_EMBEDDED_COMPILER
|
||||
# include <llvm/IR/IRBuilder.h>
|
||||
|
@ -14,9 +14,7 @@
|
||||
|
||||
#include <AggregateFunctions/IAggregateFunction.h>
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
# include <Common/config.h>
|
||||
#endif
|
||||
#include <Common/config.h>
|
||||
|
||||
#if USE_EMBEDDED_COMPILER
|
||||
# include <llvm/IR/IRBuilder.h>
|
||||
|
@ -395,9 +395,7 @@ private:
|
||||
using Self = AggregateFunctionSumMapFiltered<T, overflow, tuple_argument>;
|
||||
using Base = AggregateFunctionMapBase<T, Self, FieldVisitorSum, overflow, tuple_argument, true>;
|
||||
|
||||
/// ARCADIA_BUILD disallow unordered_set for big ints for some reason
|
||||
static constexpr const bool allow_hash = !is_over_big_int<T>;
|
||||
using ContainerT = std::conditional_t<allow_hash, std::unordered_set<T>, std::set<T>>;
|
||||
using ContainerT = std::unordered_set<T>;
|
||||
|
||||
ContainerT keys_to_keep;
|
||||
|
||||
@ -418,13 +416,10 @@ public:
|
||||
"Aggregate function {} requires an Array as a parameter",
|
||||
getName());
|
||||
|
||||
if constexpr (allow_hash)
|
||||
keys_to_keep.reserve(keys_to_keep_.size());
|
||||
keys_to_keep.reserve(keys_to_keep_.size());
|
||||
|
||||
for (const Field & f : keys_to_keep_)
|
||||
{
|
||||
keys_to_keep.emplace(f.safeGet<T>());
|
||||
}
|
||||
}
|
||||
|
||||
String getName() const override
|
||||
|
@ -9,9 +9,7 @@
|
||||
#include <Common/Exception.h>
|
||||
#include <base/types.h>
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
# include "config_core.h"
|
||||
#endif
|
||||
#include "config_core.h"
|
||||
|
||||
#include <cstddef>
|
||||
#include <memory>
|
||||
|
@ -1,15 +1,13 @@
|
||||
#pragma once
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
# include <Common/config.h>
|
||||
#endif
|
||||
#include <Common/config.h>
|
||||
|
||||
#if USE_DATASKETCHES
|
||||
|
||||
#include <boost/noncopyable.hpp>
|
||||
#include <memory>
|
||||
#include <theta_sketch.hpp> // Y_IGNORE
|
||||
#include <theta_union.hpp> // Y_IGNORE
|
||||
#include <theta_sketch.hpp>
|
||||
#include <theta_union.hpp>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -98,9 +98,7 @@ void registerAggregateFunctions()
|
||||
registerAggregateFunctionUniqUpTo(factory);
|
||||
registerAggregateFunctionTopK(factory);
|
||||
registerAggregateFunctionsBitwise(factory);
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
registerAggregateFunctionsBitmap(factory);
|
||||
#endif
|
||||
registerAggregateFunctionsMaxIntersections(factory);
|
||||
registerAggregateFunctionHistogram(factory);
|
||||
registerAggregateFunctionRetention(factory);
|
||||
|
@ -15,9 +15,7 @@
|
||||
#include <base/range.h>
|
||||
#include <Bridge/IBridgeHelper.h>
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
# include <Common/config.h>
|
||||
#endif
|
||||
#include <Common/config.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -14,9 +14,7 @@
|
||||
#include "Core/Block.h"
|
||||
#include "Core/Protocol.h"
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
# include <Common/config_version.h>
|
||||
#endif
|
||||
#include <Common/config_version.h>
|
||||
#include <Common/UTF8Helpers.h>
|
||||
#include <Common/TerminalSize.h>
|
||||
#include <Common/clearPasswordFromCommandLine.h>
|
||||
|
@ -31,10 +31,8 @@
|
||||
#include <Processors/Executors/PipelineExecutor.h>
|
||||
#include <pcg_random.hpp>
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
# include <Common/config_version.h>
|
||||
# include <Common/config.h>
|
||||
#endif
|
||||
#include <Common/config_version.h>
|
||||
#include <Common/config.h>
|
||||
|
||||
#if USE_SSL
|
||||
# include <Poco/Net/SecureStreamSocket.h>
|
||||
|
@ -4,9 +4,7 @@
|
||||
|
||||
#include <Poco/Net/StreamSocket.h>
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
# include <Common/config.h>
|
||||
#endif
|
||||
#include <Common/config.h>
|
||||
#include <Client/IServerConnection.h>
|
||||
#include <Core/Defines.h>
|
||||
|
||||
|
@ -12,9 +12,7 @@
|
||||
#include <base/setTerminalEcho.h>
|
||||
#include <base/scope_guard.h>
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
#include <readpassphrase/readpassphrase.h> // Y_IGNORE
|
||||
#endif
|
||||
#include <readpassphrase/readpassphrase.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -55,12 +53,10 @@ ConnectionParameters::ConnectionParameters(const Poco::Util::AbstractConfigurati
|
||||
}
|
||||
if (password_prompt)
|
||||
{
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
std::string prompt{"Password for user (" + user + "): "};
|
||||
char buf[1000] = {};
|
||||
if (auto * result = readpassphrase(prompt.c_str(), buf, sizeof(buf), 0))
|
||||
password = result;
|
||||
#endif
|
||||
}
|
||||
|
||||
/// By default compression is disabled if address looks like localhost.
|
||||
|
@ -1,8 +1,6 @@
|
||||
#include <Columns/Collator.h>
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
# include "config_core.h"
|
||||
#endif
|
||||
#include "config_core.h"
|
||||
|
||||
#if USE_ICU
|
||||
# include <unicode/locid.h>
|
||||
|
@ -1,8 +1,5 @@
|
||||
#include <Common/ClickHouseRevision.h>
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
# include <Common/config_version.h>
|
||||
#endif
|
||||
#include <Common/config_version.h>
|
||||
|
||||
namespace ClickHouseRevision
|
||||
{
|
||||
|
@ -55,28 +55,6 @@ public:
|
||||
return locus;
|
||||
}
|
||||
|
||||
/// Used only in arcadia/metrika
|
||||
void readText(ReadBuffer & in)
|
||||
{
|
||||
for (size_t i = 0; i < BITSET_SIZE; ++i)
|
||||
{
|
||||
if (i != 0)
|
||||
assertChar(',', in);
|
||||
readIntText(bitset[i], in);
|
||||
}
|
||||
}
|
||||
|
||||
/// Used only in arcadia/metrika
|
||||
void writeText(WriteBuffer & out) const
|
||||
{
|
||||
for (size_t i = 0; i < BITSET_SIZE; ++i)
|
||||
{
|
||||
if (i != 0)
|
||||
writeCString(",", out);
|
||||
writeIntText(bitset[i], out);
|
||||
}
|
||||
}
|
||||
|
||||
private:
|
||||
/// number of bytes in bitset
|
||||
static constexpr size_t BITSET_SIZE = (static_cast<size_t>(bucket_count) * content_width + 7) / 8;
|
||||
|
@ -1,6 +1,4 @@
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
#include <Common/config.h>
|
||||
#endif
|
||||
#include <Common/config.h>
|
||||
#include "ConfigProcessor.h"
|
||||
#include "YAMLParser.h"
|
||||
|
||||
|
@ -1,8 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
#include <Common/config.h>
|
||||
#endif
|
||||
#include <Common/config.h>
|
||||
|
||||
#include <string>
|
||||
#include <unordered_set>
|
||||
|
@ -1,6 +1,4 @@
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
#include <Common/config.h>
|
||||
#endif
|
||||
#include <Common/config.h>
|
||||
|
||||
#if USE_YAML_CPP
|
||||
#include "YAMLParser.h"
|
||||
@ -19,7 +17,7 @@
|
||||
#include <Poco/DOM/Text.h>
|
||||
#include <Common/Exception.h>
|
||||
|
||||
#include <yaml-cpp/yaml.h> // Y_IGNORE
|
||||
#include <yaml-cpp/yaml.h>
|
||||
|
||||
#include <base/logger_useful.h>
|
||||
|
||||
|
@ -1,8 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
#include <Common/config.h>
|
||||
#endif
|
||||
#include <Common/config.h>
|
||||
|
||||
#include <string>
|
||||
|
||||
|
@ -114,11 +114,7 @@ static DNSResolver::IPAddresses resolveIPAddressImpl(const std::string & host)
|
||||
|
||||
try
|
||||
{
|
||||
#if defined(ARCADIA_BUILD)
|
||||
addresses = Poco::Net::DNS::hostByName(host, &Poco::Net::DNS::DEFAULT_DNS_TIMEOUT, flags).addresses();
|
||||
#else
|
||||
addresses = Poco::Net::DNS::hostByName(host, flags).addresses();
|
||||
#endif
|
||||
}
|
||||
catch (const Poco::Net::DNSException & e)
|
||||
{
|
||||
|
@ -17,9 +17,7 @@
|
||||
#include <Common/ErrorCodes.h>
|
||||
#include <filesystem>
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
# include <Common/config_version.h>
|
||||
#endif
|
||||
#include <Common/config_version.h>
|
||||
|
||||
namespace fs = std::filesystem;
|
||||
|
||||
|
@ -1,6 +1,4 @@
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
# include <Common/config.h>
|
||||
#endif
|
||||
#include <Common/config.h>
|
||||
|
||||
#if USE_SSL
|
||||
#include "OpenSSLHelpers.h"
|
||||
|
@ -1,8 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
# include <Common/config.h>
|
||||
#endif
|
||||
#include <Common/config.h>
|
||||
|
||||
#if USE_SSL
|
||||
# include <base/types.h>
|
||||
|
@ -7,9 +7,7 @@
|
||||
#include <Common/StringSearcher.h>
|
||||
#include <re2/re2.h>
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
# include <Common/config.h>
|
||||
#endif
|
||||
#include <Common/config.h>
|
||||
|
||||
#if USE_RE2_ST
|
||||
# include <re2_st/re2.h>
|
||||
|
@ -4,9 +4,7 @@
|
||||
#include <signal.h>
|
||||
#include <time.h>
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
# include <Common/config.h>
|
||||
#endif
|
||||
#include <Common/config.h>
|
||||
|
||||
|
||||
namespace Poco
|
||||
|
@ -1,24 +1,10 @@
|
||||
#pragma once
|
||||
|
||||
/// SparseHashMap is a wrapper for google::sparse_hash_map.
|
||||
#if defined(ARCADIA_BUILD)
|
||||
#define HASH_FUN_H <unordered_map>
|
||||
template <typename T>
|
||||
struct THash;
|
||||
#endif
|
||||
|
||||
#include <sparsehash/sparse_hash_map>
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
template <class Key, class T, class HashFcn = std::hash<Key>,
|
||||
class EqualKey = std::equal_to<Key>,
|
||||
class Alloc = google::libc_allocator_with_realloc<std::pair<const Key, T>>>
|
||||
using SparseHashMap = google::sparse_hash_map<Key, T, HashFcn, EqualKey, Alloc>;
|
||||
#else
|
||||
template <class Key, class T, class HashFcn = std::hash<Key>,
|
||||
class EqualKey = std::equal_to<Key>,
|
||||
class Alloc = google::sparsehash::libc_allocator_with_realloc<std::pair<const Key, T>>>
|
||||
using SparseHashMap = google::sparsehash::sparse_hash_map<Key, T, HashFcn, EqualKey, Alloc>;
|
||||
|
||||
#undef THash
|
||||
#endif
|
||||
template <class Key, class T, class HashFcn = std::hash<Key>,
|
||||
class EqualKey = std::equal_to<Key>,
|
||||
class Alloc = google::libc_allocator_with_realloc<std::pair<const Key, T>>>
|
||||
using SparseHashMap = google::sparse_hash_map<Key, T, HashFcn, EqualKey, Alloc>;
|
||||
|
@ -13,9 +13,7 @@
|
||||
#include <sstream>
|
||||
#include <unordered_map>
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
# include <Common/config.h>
|
||||
#endif
|
||||
#include <Common/config.h>
|
||||
|
||||
#if USE_UNWIND
|
||||
# include <libunwind.h>
|
||||
@ -201,7 +199,7 @@ void StackTrace::symbolize(
|
||||
const StackTrace::FramePointers & frame_pointers, [[maybe_unused]] size_t offset,
|
||||
size_t size, StackTrace::Frames & frames)
|
||||
{
|
||||
#if defined(__ELF__) && !defined(__FreeBSD__) && !defined(ARCADIA_BUILD)
|
||||
#if defined(__ELF__) && !defined(__FreeBSD__)
|
||||
|
||||
auto symbol_index_ptr = DB::SymbolIndex::instance();
|
||||
const DB::SymbolIndex & symbol_index = *symbol_index_ptr;
|
||||
|
@ -26,7 +26,7 @@ namespace ErrorCodes
|
||||
thread_local ThreadStatus * current_thread = nullptr;
|
||||
thread_local ThreadStatus * main_thread = nullptr;
|
||||
|
||||
#if !defined(SANITIZER) && !defined(ARCADIA_BUILD)
|
||||
#if !defined(SANITIZER)
|
||||
namespace
|
||||
{
|
||||
|
||||
@ -88,7 +88,7 @@ ThreadStatus::ThreadStatus()
|
||||
/// Will set alternative signal stack to provide diagnostics for stack overflow errors.
|
||||
/// If not already installed for current thread.
|
||||
/// Sanitizer makes larger stack usage and also it's incompatible with alternative stack by default (it sets up and relies on its own).
|
||||
#if !defined(SANITIZER) && !defined(ARCADIA_BUILD)
|
||||
#if !defined(SANITIZER)
|
||||
if (!has_alt_stack)
|
||||
{
|
||||
/// Don't repeat tries even if not installed successfully.
|
||||
@ -147,11 +147,9 @@ ThreadStatus::~ThreadStatus()
|
||||
thread_group->threads.erase(this);
|
||||
}
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
/// It may cause segfault if query_context was destroyed, but was not detached
|
||||
auto query_context_ptr = query_context.lock();
|
||||
assert((!query_context_ptr && query_id.empty()) || (query_context_ptr && query_id == query_context_ptr->getCurrentQueryId()));
|
||||
#endif
|
||||
|
||||
if (deleter)
|
||||
deleter();
|
||||
|
@ -9,9 +9,7 @@
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <base/logger_useful.h>
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
# include <Common/config.h>
|
||||
#endif
|
||||
#include <Common/config.h>
|
||||
|
||||
#if USE_SSL
|
||||
# include <Poco/Net/SecureStreamSocket.h>
|
||||
|
@ -1,6 +1,4 @@
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
# include <Common/config.h>
|
||||
#endif
|
||||
#include <Common/config.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <base/types.h>
|
||||
#include <IO/VarInt.h>
|
||||
@ -13,10 +11,10 @@
|
||||
// This depends on BoringSSL-specific API, notably <openssl/aead.h>.
|
||||
#if USE_SSL && USE_INTERNAL_SSL_LIBRARY
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <openssl/digest.h> // Y_IGNORE
|
||||
#include <openssl/digest.h>
|
||||
#include <openssl/err.h>
|
||||
#include <boost/algorithm/hex.hpp>
|
||||
#include <openssl/aead.h> // Y_IGNORE
|
||||
#include <openssl/aead.h>
|
||||
#endif
|
||||
|
||||
// Common part for both parts (with SSL and without)
|
||||
|
@ -1,6 +1,5 @@
|
||||
#pragma once
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
#include <string_view>
|
||||
#include <unordered_map>
|
||||
#include <base/types.h>
|
||||
@ -134,5 +133,3 @@ private:
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
#endif /* NOT Arcadia_build */
|
||||
|
@ -1,6 +1,4 @@
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
# include "config_core.h"
|
||||
#endif
|
||||
#include "config_core.h"
|
||||
|
||||
#include <Compression/CompressionFactory.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
|
@ -1,6 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <libnuraft/nuraft.hxx> // Y_IGNORE
|
||||
#include <libnuraft/nuraft.hxx>
|
||||
#include <city.h>
|
||||
#include <optional>
|
||||
#include <IO/WriteBufferFromFile.h>
|
||||
|
@ -4,7 +4,7 @@
|
||||
#include <map>
|
||||
#include <mutex>
|
||||
#include <Core/Types.h>
|
||||
#include <libnuraft/log_store.hxx> // Y_IGNORE
|
||||
#include <libnuraft/log_store.hxx>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -1,9 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
# include <Common/config.h>
|
||||
# include "config_core.h"
|
||||
#endif
|
||||
#include <Common/config.h>
|
||||
#include "config_core.h"
|
||||
|
||||
#if USE_NURAFT
|
||||
|
||||
|
@ -1,5 +1,5 @@
|
||||
#pragma once
|
||||
#include <libnuraft/log_store.hxx> // Y_IGNORE
|
||||
#include <libnuraft/log_store.hxx>
|
||||
#include <map>
|
||||
#include <mutex>
|
||||
#include <Core/Types.h>
|
||||
|
@ -1,9 +1,7 @@
|
||||
#include <Coordination/KeeperServer.h>
|
||||
#include <Coordination/Defines.h>
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
# include "config_core.h"
|
||||
#endif
|
||||
#include "config_core.h"
|
||||
|
||||
#include <Coordination/LoggerWrapper.h>
|
||||
#include <Coordination/KeeperStateMachine.h>
|
||||
|
@ -1,6 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <libnuraft/nuraft.hxx> // Y_IGNORE
|
||||
#include <libnuraft/nuraft.hxx>
|
||||
#include <Coordination/InMemoryLogStore.h>
|
||||
#include <Coordination/KeeperStateManager.h>
|
||||
#include <Coordination/KeeperStateMachine.h>
|
||||
|
@ -1,5 +1,5 @@
|
||||
#pragma once
|
||||
#include <libnuraft/nuraft.hxx> // Y_IGNORE
|
||||
#include <libnuraft/nuraft.hxx>
|
||||
#include <Coordination/KeeperStorage.h>
|
||||
#include <IO/WriteBuffer.h>
|
||||
#include <IO/ReadBuffer.h>
|
||||
|
@ -2,7 +2,7 @@
|
||||
|
||||
#include <Common/ConcurrentBoundedQueue.h>
|
||||
#include <Coordination/KeeperStorage.h>
|
||||
#include <libnuraft/nuraft.hxx> // Y_IGNORE
|
||||
#include <libnuraft/nuraft.hxx>
|
||||
#include <base/logger_useful.h>
|
||||
#include <Coordination/CoordinationSettings.h>
|
||||
#include <Coordination/KeeperSnapshotManager.h>
|
||||
|
@ -4,7 +4,7 @@
|
||||
#include <string>
|
||||
#include <Coordination/KeeperLogStore.h>
|
||||
#include <Coordination/CoordinationSettings.h>
|
||||
#include <libnuraft/nuraft.hxx> // Y_IGNORE
|
||||
#include <libnuraft/nuraft.hxx>
|
||||
#include <Poco/Util/AbstractConfiguration.h>
|
||||
#include <Coordination/KeeperSnapshotManager.h>
|
||||
|
||||
|
@ -1,6 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <libnuraft/nuraft.hxx> // Y_IGNORE
|
||||
#include <libnuraft/nuraft.hxx>
|
||||
#include <base/logger_useful.h>
|
||||
#include <Core/SettingsEnums.h>
|
||||
|
||||
|
@ -1,7 +1,7 @@
|
||||
#pragma once
|
||||
#include <IO/ReadBufferFromMemory.h>
|
||||
|
||||
#include <libnuraft/nuraft.hxx> // Y_IGNORE
|
||||
#include <libnuraft/nuraft.hxx>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -1,6 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <libnuraft/nuraft.hxx> // Y_IGNORE
|
||||
#include <libnuraft/nuraft.hxx>
|
||||
#include <Core/Types.h>
|
||||
#include <atomic>
|
||||
#include <map>
|
||||
|
@ -1,7 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <IO/WriteBuffer.h>
|
||||
#include <libnuraft/nuraft.hxx> // Y_IGNORE
|
||||
#include <libnuraft/nuraft.hxx>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -1,9 +1,7 @@
|
||||
#include <gtest/gtest.h>
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
# include <Common/config.h>
|
||||
# include "config_core.h"
|
||||
#endif
|
||||
#include <Common/config.h>
|
||||
#include "config_core.h"
|
||||
|
||||
#if USE_NURAFT
|
||||
#include <Poco/ConsoleChannel.h>
|
||||
@ -22,7 +20,7 @@
|
||||
#include <Common/ZooKeeper/ZooKeeperIO.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <base/logger_useful.h>
|
||||
#include <libnuraft/nuraft.hxx> // Y_IGNORE
|
||||
#include <libnuraft/nuraft.hxx>
|
||||
#include <thread>
|
||||
#include <Coordination/KeeperLogStore.h>
|
||||
#include <Coordination/Changelog.h>
|
||||
|
@ -4,9 +4,7 @@
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Core/MySQL/PacketEndpoint.h>
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
# include "config_core.h"
|
||||
#endif
|
||||
#include "config_core.h"
|
||||
|
||||
#if USE_SSL
|
||||
# include <openssl/pem.h>
|
||||
|
@ -1,12 +1,10 @@
|
||||
#pragma once
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
#include "config_core.h"
|
||||
#endif
|
||||
|
||||
#if USE_LIBPQXX
|
||||
|
||||
#include <pqxx/pqxx> // Y_IGNORE
|
||||
#include <pqxx/pqxx>
|
||||
#include <Core/Types.h>
|
||||
#include <boost/noncopyable.hpp>
|
||||
|
||||
|
@ -1,12 +1,10 @@
|
||||
#pragma once
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
#include "config_core.h"
|
||||
#endif
|
||||
|
||||
#if USE_LIBPQXX
|
||||
|
||||
#include <pqxx/pqxx> // Y_IGNORE
|
||||
#include <pqxx/pqxx>
|
||||
#include <Core/Types.h>
|
||||
#include <base/BorrowedObjectPool.h>
|
||||
|
||||
|
@ -1,8 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
#include "config_core.h"
|
||||
#endif
|
||||
|
||||
#if USE_LIBPQXX
|
||||
|
||||
|
@ -1,12 +1,10 @@
|
||||
#pragma once
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
#include "config_core.h"
|
||||
#endif
|
||||
|
||||
#if USE_LIBPQXX
|
||||
|
||||
#include <pqxx/pqxx> // Y_IGNORE
|
||||
#include <pqxx/pqxx>
|
||||
#include <Core/Types.h>
|
||||
#include "Connection.h"
|
||||
#include <Common/Exception.h>
|
||||
|
@ -15,7 +15,7 @@
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/ReadBufferFromString.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include <pqxx/pqxx> // Y_IGNORE
|
||||
#include <pqxx/pqxx>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -1,8 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
#include "config_core.h"
|
||||
#endif
|
||||
|
||||
#if USE_LIBPQXX
|
||||
|
||||
|
@ -1,8 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
# include "config_core.h"
|
||||
#endif
|
||||
#include "config_core.h"
|
||||
|
||||
#if USE_EMBEDDED_COMPILER
|
||||
# include <Common/Exception.h>
|
||||
|
@ -16,9 +16,7 @@
|
||||
#include <Storages/ExternalDataSourceConfiguration.h>
|
||||
#include <filesystem>
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
# include "config_core.h"
|
||||
#endif
|
||||
#include "config_core.h"
|
||||
|
||||
#if USE_MYSQL
|
||||
# include <Core/MySQL/MySQLClient.h>
|
||||
@ -36,7 +34,7 @@
|
||||
#endif
|
||||
|
||||
#if USE_LIBPQXX
|
||||
#include <Databases/PostgreSQL/DatabasePostgreSQL.h> // Y_IGNORE
|
||||
#include <Databases/PostgreSQL/DatabasePostgreSQL.h>
|
||||
#include <Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.h>
|
||||
#include <Storages/PostgreSQL/MaterializedPostgreSQLSettings.h>
|
||||
#include <Storages/StoragePostgreSQL.h>
|
||||
|
@ -1,6 +1,4 @@
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
# include "config_core.h"
|
||||
#endif
|
||||
#include "config_core.h"
|
||||
|
||||
#if USE_MYSQL
|
||||
|
||||
|
@ -1,6 +1,4 @@
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
# include "config_core.h"
|
||||
#endif
|
||||
#include "config_core.h"
|
||||
|
||||
#if USE_MYSQL
|
||||
# include <string>
|
||||
|
@ -1,6 +1,4 @@
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
# include "config_core.h"
|
||||
#endif
|
||||
#include "config_core.h"
|
||||
|
||||
#if USE_MYSQL
|
||||
#include <Core/Block.h>
|
||||
|
@ -1,8 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
# include "config_core.h"
|
||||
#endif
|
||||
#include "config_core.h"
|
||||
|
||||
#if USE_MYSQL
|
||||
|
||||
|
@ -1,6 +1,4 @@
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
# include "config_core.h"
|
||||
#endif
|
||||
#include "config_core.h"
|
||||
|
||||
#if USE_MYSQL
|
||||
|
||||
|
@ -1,8 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
# include "config_core.h"
|
||||
#endif
|
||||
#include "config_core.h"
|
||||
|
||||
#if USE_MYSQL
|
||||
|
||||
|
@ -1,8 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
#include "config_core.h"
|
||||
#endif
|
||||
|
||||
#if USE_LIBPQXX
|
||||
|
||||
|
@ -1,8 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
#include "config_core.h"
|
||||
#endif
|
||||
|
||||
#if USE_LIBPQXX
|
||||
|
||||
|
@ -1,8 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
#include "config_core.h"
|
||||
#endif
|
||||
|
||||
#if USE_LIBPQXX
|
||||
#include <Core/PostgreSQL/ConnectionHolder.h>
|
||||
|
@ -1,15 +1,13 @@
|
||||
#pragma once
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
#include "config_core.h"
|
||||
#endif
|
||||
|
||||
#if USE_SQLITE
|
||||
#include <Core/Names.h>
|
||||
#include <Databases/DatabasesCommon.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
|
||||
#include <sqlite3.h> // Y_IGNORE
|
||||
#include <sqlite3.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -1,13 +1,11 @@
|
||||
#pragma once
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
#include "config_core.h"
|
||||
#endif
|
||||
|
||||
#if USE_SQLITE
|
||||
#include <Core/Types.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <sqlite3.h> // Y_IGNORE
|
||||
#include <sqlite3.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -1,13 +1,11 @@
|
||||
#pragma once
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
#include "config_core.h"
|
||||
#endif
|
||||
|
||||
#if USE_SQLITE
|
||||
|
||||
#include <Storages/StorageSQLite.h>
|
||||
#include <sqlite3.h> // Y_IGNORE
|
||||
#include <sqlite3.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -1,11 +1,9 @@
|
||||
#pragma once
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
#include <Common/config.h>
|
||||
#endif
|
||||
|
||||
#if USE_CASSANDRA
|
||||
#include <cassandra.h> // Y_IGNORE
|
||||
#include <cassandra.h>
|
||||
#include <utility>
|
||||
#include <memory>
|
||||
|
||||
|
@ -1,6 +1,4 @@
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
#include <Common/config.h>
|
||||
#endif
|
||||
|
||||
#if USE_CASSANDRA
|
||||
|
||||
|
@ -2,9 +2,7 @@
|
||||
|
||||
#include <Core/Block.h>
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
# include "config_core.h"
|
||||
#endif
|
||||
#include "config_core.h"
|
||||
|
||||
#if USE_MYSQL
|
||||
# include <base/LocalDateTime.h>
|
||||
|
@ -1,8 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
#include "config_core.h"
|
||||
#endif
|
||||
#include "DictionaryStructure.h"
|
||||
#include "IDictionarySource.h"
|
||||
|
||||
|
@ -14,9 +14,7 @@ void registerDictionarySourceCassandra(DictionarySourceFactory & source_factory)
|
||||
void registerDictionarySourceRedis(DictionarySourceFactory & source_factory);
|
||||
void registerDictionarySourceXDBC(DictionarySourceFactory & source_factory);
|
||||
void registerDictionarySourceJDBC(DictionarySourceFactory & source_factory);
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
void registerDictionarySourcePostgreSQL(DictionarySourceFactory & source_factory);
|
||||
#endif
|
||||
void registerDictionarySourceExecutable(DictionarySourceFactory & source_factory);
|
||||
void registerDictionarySourceExecutablePool(DictionarySourceFactory & source_factory);
|
||||
void registerDictionarySourceHTTP(DictionarySourceFactory & source_factory);
|
||||
@ -46,9 +44,7 @@ void registerDictionaries()
|
||||
registerDictionarySourceCassandra(source_factory);
|
||||
registerDictionarySourceXDBC(source_factory);
|
||||
registerDictionarySourceJDBC(source_factory);
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
registerDictionarySourcePostgreSQL(source_factory);
|
||||
#endif
|
||||
registerDictionarySourceExecutable(source_factory);
|
||||
registerDictionarySourceExecutablePool(source_factory);
|
||||
registerDictionarySourceHTTP(source_factory);
|
||||
|
@ -1,8 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
#include <Common/config.h>
|
||||
#endif
|
||||
|
||||
#if USE_SSL
|
||||
#include <Disks/IDisk.h>
|
||||
|
@ -1,8 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
#include <Common/config.h>
|
||||
#endif
|
||||
|
||||
#include <atomic>
|
||||
#include "Disks/DiskFactory.h"
|
||||
|
@ -1,8 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
#include <Common/config.h>
|
||||
#endif
|
||||
|
||||
#include <IO/ReadBufferFromFile.h>
|
||||
#include <Disks/IDiskRemote.h>
|
||||
|
@ -29,15 +29,15 @@
|
||||
#include <IO/WriteBufferFromS3.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
|
||||
#include <aws/s3/model/CopyObjectRequest.h> // Y_IGNORE
|
||||
#include <aws/s3/model/DeleteObjectsRequest.h> // Y_IGNORE
|
||||
#include <aws/s3/model/GetObjectRequest.h> // Y_IGNORE
|
||||
#include <aws/s3/model/ListObjectsV2Request.h> // Y_IGNORE
|
||||
#include <aws/s3/model/HeadObjectRequest.h> // Y_IGNORE
|
||||
#include <aws/s3/model/CreateMultipartUploadRequest.h> // Y_IGNORE
|
||||
#include <aws/s3/model/CompleteMultipartUploadRequest.h> // Y_IGNORE
|
||||
#include <aws/s3/model/UploadPartCopyRequest.h> // Y_IGNORE
|
||||
#include <aws/s3/model/AbortMultipartUploadRequest.h> // Y_IGNORE
|
||||
#include <aws/s3/model/CopyObjectRequest.h>
|
||||
#include <aws/s3/model/DeleteObjectsRequest.h>
|
||||
#include <aws/s3/model/GetObjectRequest.h>
|
||||
#include <aws/s3/model/ListObjectsV2Request.h>
|
||||
#include <aws/s3/model/HeadObjectRequest.h>
|
||||
#include <aws/s3/model/CreateMultipartUploadRequest.h>
|
||||
#include <aws/s3/model/CompleteMultipartUploadRequest.h>
|
||||
#include <aws/s3/model/UploadPartCopyRequest.h>
|
||||
#include <aws/s3/model/AbortMultipartUploadRequest.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -1,8 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
#include <Common/config.h>
|
||||
#endif
|
||||
|
||||
#if USE_AWS_S3
|
||||
|
||||
|
@ -1,14 +1,12 @@
|
||||
#pragma once
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
#include <Common/config.h>
|
||||
#endif
|
||||
|
||||
#if USE_AWS_S3
|
||||
|
||||
#include <utility>
|
||||
#include <base/types.h>
|
||||
#include <aws/core/client/ClientConfiguration.h> // Y_IGNORE
|
||||
#include <aws/core/client/ClientConfiguration.h>
|
||||
#include <Poco/URI.h>
|
||||
|
||||
namespace DB::S3
|
||||
|
@ -1,8 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
#include <Common/config.h>
|
||||
#endif
|
||||
|
||||
#if USE_AWS_S3
|
||||
|
||||
|
@ -1,8 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
#include <Common/config.h>
|
||||
#endif
|
||||
|
||||
#if USE_AWS_S3
|
||||
|
||||
|
@ -1,6 +1,4 @@
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
#include <Common/config.h>
|
||||
#endif
|
||||
#include <Common/config.h>
|
||||
|
||||
#include <base/logger_useful.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
@ -10,7 +8,7 @@
|
||||
|
||||
#if USE_AWS_S3
|
||||
|
||||
#include <aws/core/client/DefaultRetryStrategy.h> // Y_IGNORE
|
||||
#include <aws/core/client/DefaultRetryStrategy.h>
|
||||
#include <IO/S3Common.h>
|
||||
#include "DiskS3.h"
|
||||
#include "Disks/DiskCacheWrapper.h"
|
||||
|
@ -1,8 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
#include <Common/config.h>
|
||||
#endif
|
||||
|
||||
#include <Disks/IDiskRemote.h>
|
||||
#include <IO/WriteBufferFromFile.h>
|
||||
|
@ -2,9 +2,7 @@
|
||||
|
||||
#include "DiskFactory.h"
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
# include <Common/config.h>
|
||||
#endif
|
||||
#include <Common/config.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -1,8 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
# include "config_formats.h"
|
||||
#endif
|
||||
#include "config_formats.h"
|
||||
|
||||
#if USE_PROTOBUF
|
||||
# include <Common/PODArray.h>
|
||||
|
@ -1,6 +1,4 @@
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
# include "config_formats.h"
|
||||
#endif
|
||||
#include "config_formats.h"
|
||||
|
||||
#if USE_PROTOBUF
|
||||
# include <Formats/FormatSchemaInfo.h>
|
||||
|
@ -1,8 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
# include "config_formats.h"
|
||||
#endif
|
||||
#include "config_formats.h"
|
||||
|
||||
#if USE_PROTOBUF
|
||||
# include <Columns/IColumn.h>
|
||||
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user