Move code around to avoid dlsym on Musl

This commit is contained in:
Alexey Milovidov 2021-12-21 16:41:53 +03:00
parent f9d3e45e49
commit 29d28c531f
87 changed files with 175 additions and 153 deletions

View File

@ -9,7 +9,3 @@ add_subdirectory (pcg-random)
add_subdirectory (widechar_width)
add_subdirectory (readpassphrase)
add_subdirectory (bridge)
if (USE_MYSQL)
add_subdirectory (mysqlxx)
endif ()

View File

@ -1,8 +1,6 @@
set (SRCS
argsToConfig.cpp
coverage.cpp
DateLUT.cpp
DateLUTImpl.cpp
demangle.cpp
getFQDNOrHostName.cpp
getMemoryAmount.cpp
@ -18,7 +16,6 @@ set (SRCS
sleep.cpp
terminalColors.cpp
errnoToString.cpp
getResource.cpp
StringRef.cpp
)

View File

@ -1,61 +0,0 @@
add_library (mysqlxx
Connection.cpp
Exception.cpp
Query.cpp
ResultBase.cpp
UseQueryResult.cpp
Row.cpp
Value.cpp
Pool.cpp
PoolFactory.cpp
PoolWithFailover.cpp
)
target_include_directories (mysqlxx PUBLIC ..)
if (NOT USE_INTERNAL_MYSQL_LIBRARY)
set(PLATFORM_LIBRARIES ${CMAKE_DL_LIBS})
if (USE_MYSQL)
target_include_directories (mysqlxx SYSTEM PRIVATE ${MYSQL_INCLUDE_DIR})
endif ()
if (APPLE)
find_library (ICONV_LIBRARY iconv)
set (MYSQLCLIENT_LIBRARIES ${MYSQLCLIENT_LIBRARIES} ${STATIC_MYSQLCLIENT_LIB} ${ICONV_LIBRARY})
elseif (USE_STATIC_LIBRARIES AND STATIC_MYSQLCLIENT_LIB)
set (MYSQLCLIENT_LIBRARIES ${STATIC_MYSQLCLIENT_LIB})
endif ()
endif ()
target_link_libraries (mysqlxx
PUBLIC
common
PRIVATE
${MYSQLCLIENT_LIBRARIES}
${ZLIB_LIBRARIES}
)
if(OPENSSL_LIBRARIES)
target_link_libraries(mysqlxx PRIVATE ${OPENSSL_LIBRARIES})
endif()
target_link_libraries(mysqlxx PRIVATE ${PLATFORM_LIBRARIES})
if (NOT USE_INTERNAL_MYSQL_LIBRARY AND OPENSSL_INCLUDE_DIR)
target_include_directories (mysqlxx SYSTEM PRIVATE ${OPENSSL_INCLUDE_DIR})
endif ()
target_no_warning(mysqlxx reserved-macro-identifier)
if (NOT USE_INTERNAL_MYSQL_LIBRARY AND USE_STATIC_LIBRARIES)
message(WARNING "Statically linking with system mysql/mariadb only works "
"if mysql client libraries are built with same openssl version as "
"we are going to use now. It wouldn't work if GnuTLS is used. "
"Try -D\"USE_INTERNAL_MYSQL_LIBRARY\"=ON or -D\"ENABLE_MYSQL\"=OFF or "
"-D\"USE_STATIC_LIBRARIES\"=OFF")
endif ()
if (ENABLE_TESTS)
add_subdirectory (tests)
endif ()

View File

@ -20,7 +20,7 @@
#include <Common/Config/ConfigProcessor.h>
#include <Common/OpenSSLHelpers.h>
#include <Common/hex.h>
#include <base/getResource.h>
#include <Common/getResource.h>
#include <base/sleep.h>
#include <IO/ReadBufferFromFileDescriptor.h>
#include <IO/WriteBufferFromFileDescriptor.h>

View File

@ -30,7 +30,7 @@
#include <Processors/Executors/PushingPipelineExecutor.h>
#include <Core/Block.h>
#include <base/StringRef.h>
#include <base/DateLUT.h>
#include <Common/DateLUT.h>
#include <base/bit_cast.h>
#include <IO/ReadBufferFromFileDescriptor.h>
#include <IO/WriteBufferFromFileDescriptor.h>

View File

@ -8,8 +8,8 @@
#include <unordered_map>
#include <base/argsToConfig.h>
#include <base/DateLUT.h>
#include <base/LocalDate.h>
#include <Common/DateLUT.h>
#include <Common/LocalDate.h>
#include <base/LineReader.h>
#include <base/scope_guard_safe.h>
#include "Common/Exception.h"

View File

@ -1,8 +1,8 @@
#include "ClientBaseHelpers.h"
#include <base/DateLUT.h>
#include <base/LocalDate.h>
#include <Common/DateLUT.h>
#include <Common/LocalDate.h>
#include <Parsers/Lexer.h>
#include <Common/UTF8Helpers.h>

View File

@ -1,8 +1,9 @@
add_subdirectory(StringUtils)
# after common_io
#add_subdirectory(ZooKeeper)
#add_subdirectory(ConfigProcessor)
if (ENABLE_EXAMPLES)
add_subdirectory(examples)
endif()
if (USE_MYSQL)
add_subdirectory (mysqlxx)
endif ()

View File

@ -18,7 +18,7 @@
#include <Common/ZooKeeper/KeeperException.h>
#include <Common/StringUtils/StringUtils.h>
#include <Common/Exception.h>
#include <base/getResource.h>
#include <Common/getResource.h>
#include <base/errnoToString.h>
#include <IO/WriteBufferFromString.h>
#include <IO/Operators.h>

View File

@ -2,7 +2,7 @@
#include "DateLUTImpl.h"
#include "defines.h"
#include <base/defines.h>
#include <boost/noncopyable.hpp>

View File

@ -3,7 +3,7 @@
#include <cctz/civil_time.h>
#include <cctz/time_zone.h>
#include <cctz/zone_info_source.h>
#include <base/getResource.h>
#include <Common/getResource.h>
#include <Poco/Exception.h>
#include <algorithm>

View File

@ -1,8 +1,8 @@
#pragma once
#include "DayNum.h"
#include "defines.h"
#include "types.h"
#include <base/DayNum.h>
#include <base/defines.h>
#include <base/types.h>
#include <ctime>
#include <cassert>

View File

@ -4,7 +4,7 @@
#include <string>
#include <sstream>
#include <exception>
#include <base/DateLUT.h>
#include <Common/DateLUT.h>
/** Stores a calendar date in broken-down form (year, month, day-in-month).

View File

@ -3,8 +3,8 @@
#include <string>
#include <iomanip>
#include <exception>
#include <base/DateLUT.h>
#include <base/LocalDate.h>
#include <Common/DateLUT.h>
#include <Common/LocalDate.h>
/** Stores calendar date and time in broken-down form.

View File

@ -7,7 +7,7 @@
#include <base/logger_useful.h>
#include <base/errnoToString.h>
#include <Common/ClickHouseRevision.h>
#include <base/LocalDateTime.h>
#include <Common/LocalDateTime.h>
#include <IO/ReadBufferFromFile.h>
#include <IO/LimitReadBuffer.h>

View File

@ -5,6 +5,7 @@
#include <algorithm>
#include <optional>
#include <cassert>
#include <link.h>
@ -85,12 +86,45 @@ namespace
/// https://stackoverflow.com/questions/32088140/multiple-string-tables-in-elf-object
void updateResources(std::string_view name, const void * address, SymbolIndex::Resources & resources)
{
const char * char_address = static_cast<const char *>(address);
if (name.starts_with("_binary_") || name.starts_with("binary_"))
{
std::cerr << name << "\n";
if (name.ends_with("_start"))
{
name = name.substr((name[0] == '_') + strlen("binary_"));
name = name.substr(0, name.size() - strlen("_start"));
resources[name] = std::string_view{char_address, 0};
}
else if (name.ends_with("_end"))
{
name = name.substr((name[0] == '_') + strlen("binary_"));
name = name.substr(0, name.size() - strlen("_end"));
if (auto it = resources.find(name); it != resources.end())
{
auto start = it->second.data();
assert(char_address >= start);
it->second = std::string_view{start, static_cast<size_t>(char_address - start)};
}
}
}
}
/// Based on the code of musl-libc and the answer of Kanalpiroge on
/// https://stackoverflow.com/questions/15779185/list-all-the-functions-symbols-on-the-fly-in-c-code-on-a-linux-architecture
/// It does not extract all the symbols (but only public - exported and used for dynamic linking),
/// but will work if we cannot find or parse ELF files.
void collectSymbolsFromProgramHeaders(dl_phdr_info * info,
std::vector<SymbolIndex::Symbol> & symbols)
void collectSymbolsFromProgramHeaders(
dl_phdr_info * info,
std::vector<SymbolIndex::Symbol> & symbols,
SymbolIndex::Resources & resources)
{
/* Iterate over all headers of the current shared lib
* (first call is for the executable itself)
@ -201,6 +235,8 @@ void collectSymbolsFromProgramHeaders(dl_phdr_info * info,
symbol.address_end = reinterpret_cast<const void *>(info->dlpi_addr + elf_sym[sym_index].st_value + elf_sym[sym_index].st_size);
symbol.name = sym_name;
symbols.push_back(symbol);
updateResources(symbol.name, symbol.address_begin, resources);
}
break;
@ -229,7 +265,8 @@ void collectSymbolsFromELFSymbolTable(
const Elf & elf,
const Elf::Section & symbol_table,
const Elf::Section & string_table,
std::vector<SymbolIndex::Symbol> & symbols)
std::vector<SymbolIndex::Symbol> & symbols,
SymbolIndex::Resources & resources)
{
/// Iterate symbol table.
const ElfSym * symbol_table_entry = reinterpret_cast<const ElfSym *>(symbol_table.begin());
@ -256,6 +293,8 @@ void collectSymbolsFromELFSymbolTable(
symbol.address_end = reinterpret_cast<const void *>(info->dlpi_addr + symbol_table_entry->st_value + symbol_table_entry->st_size);
symbol.name = symbol_name;
symbols.push_back(symbol);
updateResources(symbol.name, symbol.address_begin, resources);
}
}
@ -265,7 +304,8 @@ bool searchAndCollectSymbolsFromELFSymbolTable(
const Elf & elf,
unsigned section_header_type,
const char * string_table_name,
std::vector<SymbolIndex::Symbol> & symbols)
std::vector<SymbolIndex::Symbol> & symbols,
SymbolIndex::Resources & resources)
{
std::optional<Elf::Section> symbol_table;
std::optional<Elf::Section> string_table;
@ -280,17 +320,20 @@ bool searchAndCollectSymbolsFromELFSymbolTable(
return (symbol_table && string_table);
}))
{
std::cerr << "!\n";
return false;
}
collectSymbolsFromELFSymbolTable(info, elf, *symbol_table, *string_table, symbols);
collectSymbolsFromELFSymbolTable(info, elf, *symbol_table, *string_table, symbols, resources);
return true;
}
void collectSymbolsFromELF(dl_phdr_info * info,
void collectSymbolsFromELF(
dl_phdr_info * info,
std::vector<SymbolIndex::Symbol> & symbols,
std::vector<SymbolIndex::Object> & objects,
SymbolIndex::Resources & resources,
String & build_id)
{
/// MSan does not know that the program segments in memory are initialized.
@ -377,10 +420,12 @@ void collectSymbolsFromELF(dl_phdr_info * info,
object.name = object_name;
objects.push_back(std::move(object));
searchAndCollectSymbolsFromELFSymbolTable(info, *objects.back().elf, SHT_SYMTAB, ".strtab", symbols);
searchAndCollectSymbolsFromELFSymbolTable(info, *objects.back().elf, SHT_SYMTAB, ".strtab", symbols, resources);
/// Unneeded because they were parsed from "program headers" of loaded objects.
//searchAndCollectSymbolsFromELFSymbolTable(info, *objects.back().elf, SHT_DYNSYM, ".dynstr", symbols);
/// Unneeded if they were parsed from "program headers" of loaded objects.
#if defined USE_MUSL
searchAndCollectSymbolsFromELFSymbolTable(info, *objects.back().elf, SHT_DYNSYM, ".dynstr", symbols, resources);
#endif
}
@ -392,8 +437,8 @@ int collectSymbols(dl_phdr_info * info, size_t, void * data_ptr)
{
SymbolIndex::Data & data = *reinterpret_cast<SymbolIndex::Data *>(data_ptr);
collectSymbolsFromProgramHeaders(info, data.symbols);
collectSymbolsFromELF(info, data.symbols, data.objects, data.build_id);
collectSymbolsFromProgramHeaders(info, data.symbols, data.resources);
collectSymbolsFromELF(info, data.symbols, data.objects, data.resources, data.build_id);
/* Continue iterations */
return 0;
@ -424,7 +469,7 @@ const T * find(const void * address, const std::vector<T> & vec)
void SymbolIndex::update()
{
dl_iterate_phdr(collectSymbols, &data.symbols);
dl_iterate_phdr(collectSymbols, &data);
std::sort(data.objects.begin(), data.objects.end(), [](const Object & a, const Object & b) { return a.address_begin < b.address_begin; });
std::sort(data.symbols.begin(), data.symbols.end(), [](const Symbol & a, const Symbol & b) { return a.address_begin < b.address_begin; });

View File

@ -4,6 +4,7 @@
#include <vector>
#include <string>
#include <unordered_map>
#include <Common/Elf.h>
#include <boost/noncopyable.hpp>
@ -47,15 +48,27 @@ public:
const std::vector<Symbol> & symbols() const { return data.symbols; }
const std::vector<Object> & objects() const { return data.objects; }
std::string_view getResource(String name) const
{
if (auto it = data.resources.find(name); it != data.resources.end())
return it->second;
return {};
}
/// The BuildID that is generated by compiler.
String getBuildID() const { return data.build_id; }
String getBuildIDHex() const;
using Resources = std::unordered_map<std::string_view /* symbol name */, std::string_view /* blob */>;
struct Data
{
std::vector<Symbol> symbols;
std::vector<Object> objects;
String build_id;
/// Resources (embedded binary data) are located by symbols in form of _binary_name_start and _binary_name_end.
Resources resources;
};
private:
Data data;

View File

@ -1,8 +1,9 @@
#include "getResource.h"
#include "unaligned.h"
#include <dlfcn.h>
#include <string>
#include <boost/algorithm/string/replace.hpp>
#include <Common/SymbolIndex.h>
std::string_view getResource(std::string_view name)
{
@ -13,6 +14,11 @@ std::string_view getResource(std::string_view name)
std::replace(name_replaced.begin(), name_replaced.end(), '.', '_');
boost::replace_all(name_replaced, "+", "_PLUS_");
#if defined USE_MUSL
/// If static linking is used, we cannot use dlsym and have to parse ELF symbol table by ourself.
return DB::SymbolIndex::instance()->getResource(name_replaced);
#else
// In most `dlsym(3)` APIs, one passes the symbol name as it appears via
// something like `nm` or `objdump -t`. For example, a symbol `_foo` would be
// looked up with the string `"_foo"`.
@ -33,8 +39,8 @@ std::string_view getResource(std::string_view name)
std::string symbol_name_start = prefix + name_replaced + "_start";
std::string symbol_name_end = prefix + name_replaced + "_end";
const char* sym_start = reinterpret_cast<const char*>(dlsym(RTLD_DEFAULT, symbol_name_start.c_str()));
const char* sym_end = reinterpret_cast<const char*>(dlsym(RTLD_DEFAULT, symbol_name_end.c_str()));
const char * sym_start = reinterpret_cast<const char *>(dlsym(RTLD_DEFAULT, symbol_name_start.c_str()));
const char * sym_end = reinterpret_cast<const char *>(dlsym(RTLD_DEFAULT, symbol_name_end.c_str()));
if (sym_start && sym_end)
{
@ -42,4 +48,5 @@ std::string_view getResource(std::string_view name)
return { sym_start, resource_size };
}
return {};
#endif
}

View File

@ -0,0 +1,24 @@
add_library (mysqlxx
Connection.cpp
Exception.cpp
Query.cpp
ResultBase.cpp
UseQueryResult.cpp
Row.cpp
Value.cpp
Pool.cpp
PoolFactory.cpp
PoolWithFailover.cpp
)
target_include_directories (mysqlxx PUBLIC .)
target_link_libraries (mysqlxx
clickhouse_common_io
${MYSQLCLIENT_LIBRARIES}
${ZLIB_LIBRARIES}
)
if (ENABLE_TESTS)
add_subdirectory (tests)
endif ()

View File

@ -10,10 +10,10 @@
#include <base/preciseExp10.h>
#include <base/types.h>
#include <base/DateLUT.h>
#include <Common/DateLUT.h>
#include <mysqlxx/Types.h>
#include <base/LocalDateTime.h>
#include <Common/LocalDateTime.h>
namespace mysqlxx

View File

@ -3,8 +3,8 @@
#include <mysqlxx/Connection.h>
#include <mysqlxx/Transaction.h>
#include <mysqlxx/Pool.h>
#include <base/LocalDate.h>
#include <base/LocalDateTime.h>
#include <Common/LocalDate.h>
#include <Common/LocalDateTime.h>
#include <mysqlxx/Null.h>

View File

@ -1,5 +1,5 @@
#include <base/DateLUT.h>
#include <base/DateLUTImpl.h>
#include <Common/DateLUT.h>
#include <Common/DateLUTImpl.h>
#include <gtest/gtest.h>

View File

@ -2,7 +2,7 @@
#include <stdexcept>
#include <gtest/gtest.h>
#include <base/LocalDateTime.h>
#include <Common/LocalDateTime.h>
void fillStackWithGarbage()

View File

@ -5,7 +5,7 @@
#include <IO/MySQLBinlogEventReadBuffer.h>
#include <IO/ReadHelpers.h>
#include <IO/Operators.h>
#include <base/DateLUT.h>
#include <Common/DateLUT.h>
#include <Common/FieldVisitorToString.h>
#include <Core/MySQL/PacketsGeneric.h>
#include <Core/MySQL/PacketsProtocolText.h>

View File

@ -1,7 +1,7 @@
#pragma once
#include <DataTypes/DataTypeNumberBase.h>
#include <base/DateLUT.h>
#include <Common/DateLUT.h>
namespace DB
{

View File

@ -2,7 +2,7 @@
#include <Columns/ColumnVector.h>
#include <Common/assert_cast.h>
#include <base/DateLUT.h>
#include <Common/DateLUT.h>
#include <Formats/FormatSettings.h>
#include <Formats/ProtobufReader.h>
#include <Formats/ProtobufWriter.h>

View File

@ -2,7 +2,7 @@
#include <Columns/ColumnVector.h>
#include <Common/assert_cast.h>
#include <base/DateLUT.h>
#include <Common/DateLUT.h>
#include <Formats/FormatSettings.h>
#include <Formats/ProtobufReader.h>
#include <IO/ReadHelpers.h>

View File

@ -1,6 +1,6 @@
#pragma once
#include <Core/Types.h>
#include <base/DateLUT.h>
#include <Common/DateLUT.h>
class DateLUTImpl;

View File

@ -1,7 +1,7 @@
#include "ExecutableDictionarySource.h"
#include <base/logger_useful.h>
#include <base/LocalDateTime.h>
#include <Common/LocalDateTime.h>
#include <Common/ShellCommand.h>
#include <Processors/Sources/ShellCommandSource.h>

View File

@ -1,7 +1,7 @@
#include "ExecutablePoolDictionarySource.h"
#include <base/logger_useful.h>
#include <base/LocalDateTime.h>
#include <Common/LocalDateTime.h>
#include <Common/ShellCommand.h>
#include <Formats/formatBlock.h>

View File

@ -4,7 +4,7 @@
#include <IO/ReadWriteBufferFromHTTP.h>
#include <Poco/Net/HTTPBasicCredentials.h>
#include <Poco/URI.h>
#include <base/LocalDateTime.h>
#include <Common/LocalDateTime.h>
#include "DictionaryStructure.h"
#include "IDictionarySource.h"
#include <Interpreters/Context.h>

View File

@ -1,7 +1,7 @@
#pragma once
#include <Bridge/LibraryBridgeHelper.h>
#include <base/LocalDateTime.h>
#include <Common/LocalDateTime.h>
#include <Core/UUID.h>
#include "DictionaryStructure.h"
#include <Core/ExternalResultDescription.h>

View File

@ -101,7 +101,7 @@ void registerDictionarySourceMysql(DictionarySourceFactory & factory)
# include <DataTypes/DataTypeString.h>
# include <IO/WriteBufferFromString.h>
# include <IO/WriteHelpers.h>
# include <base/LocalDateTime.h>
# include <Common/LocalDateTime.h>
# include <base/logger_useful.h>
# include "readInvalidateQuery.h"
# include <mysqlxx/Exception.h>

View File

@ -5,7 +5,7 @@
#include "config_core.h"
#if USE_MYSQL
# include <base/LocalDateTime.h>
# include <Common/LocalDateTime.h>
# include <mysqlxx/PoolWithFailover.h>
# include "DictionaryStructure.h"
# include "ExternalQueryBuilder.h"

View File

@ -7,7 +7,7 @@
#if USE_LIBPQXX
#include "ExternalQueryBuilder.h"
#include <Core/Block.h>
#include <base/LocalDateTime.h>
#include <Common/LocalDateTime.h>
#include <base/logger_useful.h>
#include <Core/PostgreSQL/PoolWithFailover.h>

View File

@ -9,7 +9,7 @@
#include <Interpreters/Context.h>
#include <Poco/Net/HTTPRequest.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <base/LocalDateTime.h>
#include <Common/LocalDateTime.h>
#include <base/logger_useful.h>
#include "DictionarySourceFactory.h"
#include "DictionaryStructure.h"

View File

@ -9,7 +9,7 @@
#include <Functions/extractTimeZoneFromFunctionArguments.h>
#include <Functions/IFunction.h>
#include <Common/Exception.h>
#include <base/DateLUTImpl.h>
#include <Common/DateLUTImpl.h>
/// The default mode value to use for the WEEK() function
#define DEFAULT_WEEK_MODE 0

View File

@ -3,8 +3,8 @@
#include <base/types.h>
#include <Core/DecimalFunctions.h>
#include <Common/Exception.h>
#include <base/DateLUTImpl.h>
#include <base/DateLUT.h>
#include <Common/DateLUTImpl.h>
#include <Common/DateLUT.h>
#include <Columns/ColumnVector.h>
#include <Columns/ColumnDecimal.h>
#include <Functions/FunctionHelpers.h>

View File

@ -1,5 +1,5 @@
#pragma once
#include <base/DateLUTImpl.h>
#include <Common/DateLUTImpl.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDate32.h>

View File

@ -1,6 +1,6 @@
#pragma once
#include <base/DateLUT.h>
#include <Common/DateLUT.h>
#include <DataTypes/DataTypeInterval.h>
#include <Functions/IFunction.h>

View File

@ -1,4 +1,4 @@
#include <base/DateLUTImpl.h>
#include <Common/DateLUTImpl.h>
#include <Core/DecimalFunctions.h>
#include <IO/WriteHelpers.h>

View File

@ -4,7 +4,7 @@
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeDateTime64.h>
#include <Columns/ColumnString.h>
#include <base/DateLUT.h>
#include <Common/DateLUT.h>
namespace DB

View File

@ -14,7 +14,7 @@
#include <IO/WriteHelpers.h>
#include <base/DateLUTImpl.h>
#include <Common/DateLUTImpl.h>
#include <base/find_symbols.h>
#include <Core/DecimalFunctions.h>

View File

@ -5,7 +5,7 @@
#include <Core/ServerUUID.h>
#include <Common/SymbolIndex.h>
#include <Common/DNSResolver.h>
#include <base/DateLUT.h>
#include <Common/DateLUT.h>
#if defined(OS_LINUX)
# include <Poco/Environment.h>

View File

@ -3,7 +3,7 @@
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypeDateTime.h>
#include <base/DateLUTImpl.h>
#include <Common/DateLUTImpl.h>
#include <Core/Field.h>

View File

@ -1,4 +1,4 @@
#include <base/DateLUTImpl.h>
#include <Common/DateLUTImpl.h>
#include <Columns/ColumnsNumber.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDate32.h>

View File

@ -1,4 +1,4 @@
#include <base/DateLUT.h>
#include <Common/DateLUT.h>
#include <Core/Field.h>

View File

@ -1,4 +1,4 @@
#include <base/DateLUT.h>
#include <Common/DateLUT.h>
#include <Core/Field.h>

View File

@ -8,9 +8,9 @@
#include <type_traits>
#include <base/DateLUT.h>
#include <base/LocalDate.h>
#include <base/LocalDateTime.h>
#include <Common/DateLUT.h>
#include <Common/LocalDate.h>
#include <Common/LocalDateTime.h>
#include <base/StringRef.h>
#include <base/arithmeticOverflow.h>
#include <base/unit.h>

View File

@ -9,9 +9,9 @@
#include <pcg-random/pcg_random.hpp>
#include <base/DateLUT.h>
#include <base/LocalDate.h>
#include <base/LocalDateTime.h>
#include <Common/DateLUT.h>
#include <Common/LocalDate.h>
#include <Common/LocalDateTime.h>
#include <base/find_symbols.h>
#include <base/StringRef.h>
#include <base/DecomposedFloat.h>

View File

@ -1,4 +1,4 @@
#include <base/DateLUTImpl.h>
#include <Common/DateLUTImpl.h>
#include <Common/StringUtils/StringUtils.h>
#include <IO/ReadBuffer.h>

View File

@ -1,6 +1,6 @@
#include <gtest/gtest.h>
#include <base/DateLUT.h>
#include <Common/DateLUT.h>
#include <IO/WriteHelpers.h>
#include <IO/WriteBufferFromString.h>

View File

@ -12,7 +12,7 @@
#include <DataTypes/DataTypeUUID.h>
#include <DataTypes/DataTypesNumber.h>
#include <Interpreters/ProfileEventsExt.h>
#include <base/DateLUT.h>
#include <Common/DateLUT.h>
#include <base/types.h>
namespace DB

View File

@ -22,7 +22,7 @@
#include <Common/typeid_cast.h>
#include <Common/NaNUtils.h>
#include <base/DateLUT.h>
#include <Common/DateLUT.h>
#include <DataTypes/DataTypeAggregateFunction.h>

View File

@ -14,7 +14,7 @@
#include <DataTypes/DataTypeDate32.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/NestedUtils.h>
#include <base/DateLUTImpl.h>
#include <Common/DateLUTImpl.h>
#include <base/types.h>
#include <Processors/Chunk.h>
#include <Columns/ColumnString.h>

View File

@ -1,7 +1,7 @@
#include <Processors/Merges/Algorithms/GraphiteRollupSortedAlgorithm.h>
#include <AggregateFunctions/IAggregateFunction.h>
#include <base/DateLUTImpl.h>
#include <base/DateLUT.h>
#include <Common/DateLUTImpl.h>
#include <Common/DateLUT.h>
#include <DataTypes/DataTypeDateTime.h>

View File

@ -3,7 +3,7 @@
#include <Storages/TTLDescription.h>
#include <Storages/MergeTree/MergeTreeDataPartTTLInfo.h>
#include <Storages/MergeTree/IMergeTreeDataPart.h>
#include <base/DateLUT.h>
#include <Common/DateLUT.h>
namespace DB
{

View File

@ -6,7 +6,7 @@
#include <Storages/MergeTree/MergeTreeDataPartTTLInfo.h>
#include <Processors/TTL/ITTLAlgorithm.h>
#include <base/DateLUT.h>
#include <Common/DateLUT.h>
namespace DB
{

View File

@ -7,7 +7,7 @@
#include <Processors/TTL/ITTLAlgorithm.h>
#include <Processors/TTL/TTLDeleteAlgorithm.h>
#include <base/DateLUT.h>
#include <Common/DateLUT.h>
namespace DB
{

View File

@ -6,7 +6,7 @@
#include <Poco/Util/LayeredConfiguration.h>
#include <IO/HTTPCommon.h>
#include <base/getResource.h>
#include <Common/getResource.h>
namespace DB

View File

@ -3,7 +3,7 @@
#include <Storages/System/attachSystemTablesImpl.h>
#include <Parsers/ParserCreateQuery.h>
#include <Parsers/parseQuery.h>
#include <base/getResource.h>
#include <Common/getResource.h>
namespace DB
{