mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 15:42:02 +00:00
Merge pull request #47984 from ClickHouse/optimize-compilation
Optimize compilation
This commit is contained in:
commit
352a240a20
@ -1,5 +1,6 @@
|
||||
#pragma once
|
||||
#include <base/extended_types.h>
|
||||
#include <base/Decimal_fwd.h>
|
||||
|
||||
#if !defined(NO_SANITIZE_UNDEFINED)
|
||||
#if defined(__clang__)
|
||||
@ -19,23 +20,6 @@ using Decimal64 = Decimal<Int64>;
|
||||
using Decimal128 = Decimal<Int128>;
|
||||
using Decimal256 = Decimal<Int256>;
|
||||
|
||||
template <class T>
|
||||
concept is_decimal =
|
||||
std::is_same_v<T, Decimal32>
|
||||
|| std::is_same_v<T, Decimal64>
|
||||
|| std::is_same_v<T, Decimal128>
|
||||
|| std::is_same_v<T, Decimal256>
|
||||
|| std::is_same_v<T, DateTime64>;
|
||||
|
||||
template <class T>
|
||||
concept is_over_big_int =
|
||||
std::is_same_v<T, Int128>
|
||||
|| std::is_same_v<T, UInt128>
|
||||
|| std::is_same_v<T, Int256>
|
||||
|| std::is_same_v<T, UInt256>
|
||||
|| std::is_same_v<T, Decimal128>
|
||||
|| std::is_same_v<T, Decimal256>;
|
||||
|
||||
template <class T> struct NativeTypeT { using Type = T; };
|
||||
template <is_decimal T> struct NativeTypeT<T> { using Type = typename T::NativeType; };
|
||||
template <class T> using NativeType = typename NativeTypeT<T>::Type;
|
||||
|
46
base/base/Decimal_fwd.h
Normal file
46
base/base/Decimal_fwd.h
Normal file
@ -0,0 +1,46 @@
|
||||
#pragma once
|
||||
|
||||
#include <base/types.h>
|
||||
|
||||
namespace wide
|
||||
{
|
||||
|
||||
template <size_t Bits, typename Signed>
|
||||
class integer;
|
||||
|
||||
}
|
||||
|
||||
using Int128 = wide::integer<128, signed>;
|
||||
using UInt128 = wide::integer<128, unsigned>;
|
||||
using Int256 = wide::integer<256, signed>;
|
||||
using UInt256 = wide::integer<256, unsigned>;
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
template <class> struct Decimal;
|
||||
|
||||
using Decimal32 = Decimal<Int32>;
|
||||
using Decimal64 = Decimal<Int64>;
|
||||
using Decimal128 = Decimal<Int128>;
|
||||
using Decimal256 = Decimal<Int256>;
|
||||
|
||||
class DateTime64;
|
||||
|
||||
template <class T>
|
||||
concept is_decimal =
|
||||
std::is_same_v<T, Decimal32>
|
||||
|| std::is_same_v<T, Decimal64>
|
||||
|| std::is_same_v<T, Decimal128>
|
||||
|| std::is_same_v<T, Decimal256>
|
||||
|| std::is_same_v<T, DateTime64>;
|
||||
|
||||
template <class T>
|
||||
concept is_over_big_int =
|
||||
std::is_same_v<T, Int128>
|
||||
|| std::is_same_v<T, UInt128>
|
||||
|| std::is_same_v<T, Int256>
|
||||
|| std::is_same_v<T, UInt256>
|
||||
|| std::is_same_v<T, Decimal128>
|
||||
|| std::is_same_v<T, Decimal256>;
|
||||
}
|
@ -1,5 +1,6 @@
|
||||
#include <Analyzer/ConstantNode.h>
|
||||
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Common/FieldVisitorToString.h>
|
||||
#include <Common/SipHash.h>
|
||||
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <Core/Field.h>
|
||||
|
||||
#include <Analyzer/IQueryTreeNode.h>
|
||||
#include <Analyzer/ConstantValue.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -11,7 +11,6 @@
|
||||
#include <Parsers/IAST_fwd.h>
|
||||
|
||||
#include <Analyzer/Identifier.h>
|
||||
#include <Analyzer/ConstantValue.h>
|
||||
|
||||
class SipHash;
|
||||
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <Analyzer/IdentifierNode.h>
|
||||
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Common/SipHash.h>
|
||||
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <Analyzer/LambdaNode.h>
|
||||
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Common/SipHash.h>
|
||||
|
||||
#include <IO/WriteBuffer.h>
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <Analyzer/MatcherNode.h>
|
||||
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Common/SipHash.h>
|
||||
|
||||
#include <IO/WriteBuffer.h>
|
||||
|
@ -1,6 +1,5 @@
|
||||
#pragma once
|
||||
|
||||
#include <Parsers/IAST_fwd.h>
|
||||
#include <Interpreters/Context_fwd.h>
|
||||
#include <Analyzer/IQueryTreePass.h>
|
||||
|
||||
|
@ -2,6 +2,7 @@
|
||||
|
||||
#include <fmt/core.h>
|
||||
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Common/SipHash.h>
|
||||
#include <Common/FieldVisitorToString.h>
|
||||
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <Analyzer/SortNode.h>
|
||||
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Common/SipHash.h>
|
||||
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
|
@ -35,6 +35,7 @@
|
||||
#include <Parsers/ASTCreateFunctionQuery.h>
|
||||
#include <Parsers/Access/ASTCreateUserQuery.h>
|
||||
#include <Parsers/ASTDropQuery.h>
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
#include <Parsers/ASTSetQuery.h>
|
||||
#include <Parsers/ASTUseQuery.h>
|
||||
#include <Parsers/ASTSelectWithUnionQuery.h>
|
||||
|
@ -5,7 +5,7 @@
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <base/StringRef.h>
|
||||
#include <Core/Types.h>
|
||||
#include <Core/TypeId.h>
|
||||
|
||||
#include "config.h"
|
||||
|
||||
|
@ -8,6 +8,7 @@
|
||||
#include <Columns/ColumnTuple.h>
|
||||
#include <Common/FieldVisitorToString.h>
|
||||
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Common/randomSeed.h>
|
||||
#include <fmt/core.h>
|
||||
#include <pcg_random.hpp>
|
||||
|
@ -1,9 +1,55 @@
|
||||
#pragma once
|
||||
|
||||
#include <Core/Types.h>
|
||||
#include <Core/Types_fwd.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// @note Except explicitly described you should not assume on TypeIndex numbers and/or their orders in this enum.
|
||||
enum class TypeIndex
|
||||
{
|
||||
Nothing = 0,
|
||||
UInt8,
|
||||
UInt16,
|
||||
UInt32,
|
||||
UInt64,
|
||||
UInt128,
|
||||
UInt256,
|
||||
Int8,
|
||||
Int16,
|
||||
Int32,
|
||||
Int64,
|
||||
Int128,
|
||||
Int256,
|
||||
Float32,
|
||||
Float64,
|
||||
Date,
|
||||
Date32,
|
||||
DateTime,
|
||||
DateTime64,
|
||||
String,
|
||||
FixedString,
|
||||
Enum8,
|
||||
Enum16,
|
||||
Decimal32,
|
||||
Decimal64,
|
||||
Decimal128,
|
||||
Decimal256,
|
||||
UUID,
|
||||
Array,
|
||||
Tuple,
|
||||
Set,
|
||||
Interval,
|
||||
Nullable,
|
||||
Function,
|
||||
AggregateFunction,
|
||||
LowCardinality,
|
||||
Map,
|
||||
Object,
|
||||
IPv4,
|
||||
IPv6,
|
||||
};
|
||||
|
||||
/**
|
||||
* Obtain TypeIndex value from real type if possible.
|
||||
*
|
||||
|
@ -1,5 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <Core/TypeId.h>
|
||||
#include <cstdint>
|
||||
#include <string>
|
||||
#include <vector>
|
||||
@ -44,52 +45,6 @@ struct Null
|
||||
}
|
||||
};
|
||||
|
||||
/// @note Except explicitly described you should not assume on TypeIndex numbers and/or their orders in this enum.
|
||||
enum class TypeIndex
|
||||
{
|
||||
Nothing = 0,
|
||||
UInt8,
|
||||
UInt16,
|
||||
UInt32,
|
||||
UInt64,
|
||||
UInt128,
|
||||
UInt256,
|
||||
Int8,
|
||||
Int16,
|
||||
Int32,
|
||||
Int64,
|
||||
Int128,
|
||||
Int256,
|
||||
Float32,
|
||||
Float64,
|
||||
Date,
|
||||
Date32,
|
||||
DateTime,
|
||||
DateTime64,
|
||||
String,
|
||||
FixedString,
|
||||
Enum8,
|
||||
Enum16,
|
||||
Decimal32,
|
||||
Decimal64,
|
||||
Decimal128,
|
||||
Decimal256,
|
||||
UUID,
|
||||
Array,
|
||||
Tuple,
|
||||
Set,
|
||||
Interval,
|
||||
Nullable,
|
||||
Function,
|
||||
AggregateFunction,
|
||||
LowCardinality,
|
||||
Map,
|
||||
Object,
|
||||
IPv4,
|
||||
IPv6,
|
||||
};
|
||||
|
||||
|
||||
using UInt128 = ::UInt128;
|
||||
using UInt256 = ::UInt256;
|
||||
using Int128 = ::Int128;
|
||||
|
47
src/Core/Types_fwd.h
Normal file
47
src/Core/Types_fwd.h
Normal file
@ -0,0 +1,47 @@
|
||||
#pragma once
|
||||
|
||||
#include <cstdint>
|
||||
#include <string>
|
||||
#include <vector>
|
||||
#include <unordered_set>
|
||||
#include <base/strong_typedef.h>
|
||||
#include <base/defines.h>
|
||||
#include <base/types.h>
|
||||
#include <base/Decimal_fwd.h>
|
||||
|
||||
namespace wide
|
||||
{
|
||||
|
||||
template <size_t Bits, typename Signed>
|
||||
class integer;
|
||||
|
||||
}
|
||||
|
||||
using Int128 = wide::integer<128, signed>;
|
||||
using UInt128 = wide::integer<128, unsigned>;
|
||||
using Int256 = wide::integer<256, signed>;
|
||||
using UInt256 = wide::integer<256, unsigned>;
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
using UUID = StrongTypedef<UInt128, struct UUIDTag>;
|
||||
|
||||
using IPv4 = StrongTypedef<UInt32, struct IPv4Tag>;
|
||||
|
||||
struct IPv6;
|
||||
|
||||
struct Null;
|
||||
|
||||
using UInt128 = ::UInt128;
|
||||
using UInt256 = ::UInt256;
|
||||
using Int128 = ::Int128;
|
||||
using Int256 = ::Int256;
|
||||
|
||||
enum class TypeIndex;
|
||||
|
||||
/// Not a data type in database, defined just for convenience.
|
||||
using Strings = std::vector<String>;
|
||||
using TypeIndexesSet = std::unordered_set<TypeIndex>;
|
||||
|
||||
}
|
@ -2,7 +2,7 @@
|
||||
|
||||
#include <memory>
|
||||
#include <cstddef>
|
||||
#include <Core/Types.h>
|
||||
#include <Core/Types_fwd.h>
|
||||
#include <DataTypes/Serializations/ISerialization.h>
|
||||
|
||||
namespace DB
|
||||
|
@ -1,7 +1,8 @@
|
||||
#pragma once
|
||||
|
||||
#include <DataTypes/DataTypeNumberBase.h>
|
||||
#include <Core/Field.h>
|
||||
#include <Common/DateLUT.h>
|
||||
#include <DataTypes/DataTypeNumberBase.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -1,5 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <base/TypeName.h>
|
||||
#include <Core/TypeId.h>
|
||||
#include <DataTypes/IDataType.h>
|
||||
#include <DataTypes/Serializations/SerializationNumber.h>
|
||||
@ -8,9 +9,6 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
template <typename T>
|
||||
class ColumnVector;
|
||||
|
||||
/** Implements part of the IDataType interface, common to all numbers and for Date and DateTime.
|
||||
*/
|
||||
template <typename T>
|
||||
|
@ -1,7 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <Common/COW.h>
|
||||
#include <Core/Types.h>
|
||||
#include <Core/Types_fwd.h>
|
||||
#include <base/demangle.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Columns/IColumn.h>
|
||||
|
@ -1,10 +1,11 @@
|
||||
#include <DataTypes/Serializations/SerializationEnum.h>
|
||||
|
||||
#include <Columns/ColumnVector.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <Formats/FormatSettings.h>
|
||||
#include <Formats/ProtobufReader.h>
|
||||
#include <Formats/ProtobufWriter.h>
|
||||
#include <Common/assert_cast.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -1,9 +1,10 @@
|
||||
#pragma once
|
||||
|
||||
#include <base/TypeName.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <base/TypeName.h>
|
||||
#include <DataTypes/Serializations/SimpleTextSerialization.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -1,6 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <Core/Types.h>
|
||||
#include <Core/Types_fwd.h>
|
||||
#include <DataTypes/Serializations/ISerialization.h>
|
||||
#include <Poco/JSON/Object.h>
|
||||
|
||||
|
@ -1,11 +1,14 @@
|
||||
#pragma once
|
||||
|
||||
#include <Core/Types.h>
|
||||
#include <DataTypes/Serializations/SimpleTextSerialization.h>
|
||||
#include <Columns/ColumnVector.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
template <typename T>
|
||||
class ColumnVector;
|
||||
|
||||
template <typename T>
|
||||
class SerializationNumber : public SimpleTextSerialization
|
||||
{
|
||||
|
@ -1,13 +1,10 @@
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Core/DecimalFunctions.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
|
||||
#include <Functions/extractTimeZoneFromFunctionArguments.h>
|
||||
|
||||
#include <ctime>
|
||||
|
||||
#include <Core/Field.h>
|
||||
#include <Core/DecimalFunctions.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/extractTimeZoneFromFunctionArguments.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <Functions/extractTimeZoneFromFunctionArguments.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <Columns/ColumnsDateTime.h>
|
||||
#include <Columns/ColumnVector.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -17,6 +17,7 @@
|
||||
#include <Common/LocalDateTime.h>
|
||||
#include <base/StringRef.h>
|
||||
#include <base/arithmeticOverflow.h>
|
||||
#include <base/sort.h>
|
||||
#include <base/unit.h>
|
||||
|
||||
#include <Core/Types.h>
|
||||
@ -38,8 +39,6 @@
|
||||
#include <IO/PeekableReadBuffer.h>
|
||||
#include <IO/VarInt.h>
|
||||
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
|
||||
#include <double-conversion/double-conversion.h>
|
||||
|
||||
static constexpr auto DEFAULT_MAX_STRING_SIZE = 1_GiB;
|
||||
|
@ -17,7 +17,6 @@
|
||||
#include <Interpreters/DatabaseCatalog.h>
|
||||
#include <Interpreters/MergeTreeTransactionHolder.h>
|
||||
#include <IO/IResourceManager.h>
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
#include <Parsers/IAST_fwd.h>
|
||||
#include <Server/HTTP/HTTPContext.h>
|
||||
#include <Storages/ColumnsDescription.h>
|
||||
@ -42,6 +41,8 @@ struct OvercommitTracker;
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class ASTSelectQuery;
|
||||
|
||||
struct ContextSharedPart;
|
||||
class ContextAccess;
|
||||
struct User;
|
||||
|
@ -3,8 +3,6 @@
|
||||
#include <Interpreters/IInterpreter.h>
|
||||
#include <Interpreters/SelectQueryOptions.h>
|
||||
|
||||
#include <Storages/MergeTree/RequestResponse.h>
|
||||
#include <Processors/QueryPlan/QueryPlan.h>
|
||||
#include <Analyzer/QueryTreePassManager.h>
|
||||
#include <Planner/Planner.h>
|
||||
#include <Interpreters/Context_fwd.h>
|
||||
@ -65,9 +63,6 @@ public:
|
||||
|
||||
bool ignoreQuota() const override { return select_query_options.ignore_quota; }
|
||||
|
||||
/// Set merge tree read task callback in context and set collaborate_with_initiator in client info
|
||||
void setMergeTreeReadTaskCallbackAndClientInfo(MergeTreeReadTaskCallback && callback);
|
||||
|
||||
/// Set number_of_current_replica and count_participating_replicas in client_info
|
||||
void setProperClientInfo(size_t replica_number, size_t count_participating_replicas);
|
||||
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/InterpreterSetQuery.h>
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
#include <Parsers/ASTSetQuery.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/ASTExplainQuery.h>
|
||||
|
@ -13,6 +13,8 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class NamesAndTypesList;
|
||||
|
||||
namespace MySQLInterpreter
|
||||
{
|
||||
struct InterpreterDropImpl
|
||||
|
@ -10,6 +10,7 @@
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
|
||||
namespace ProfileEvents
|
||||
{
|
||||
|
@ -7,13 +7,15 @@
|
||||
#include <DataTypes/IDataType.h>
|
||||
#include <Core/Names.h>
|
||||
#include <Core/Types.h>
|
||||
#include <Processors/QueryPlan/FilterStep.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class ASTFunction;
|
||||
|
||||
class ActionsDAG;
|
||||
using ActionsDAGPtr = std::shared_ptr<ActionsDAG>;
|
||||
|
||||
struct WindowFunctionDescription
|
||||
{
|
||||
std::string column_name;
|
||||
|
@ -2,6 +2,7 @@
|
||||
|
||||
#include <Parsers/ASTFunction.h>
|
||||
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <Common/FieldVisitorToString.h>
|
||||
#include <Common/KnownObjectNames.h>
|
||||
|
@ -1,4 +1,6 @@
|
||||
#include <Parsers/ASTQueryWithOutput.h>
|
||||
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Parsers/ASTSetQuery.h>
|
||||
|
||||
namespace DB
|
||||
|
@ -1,3 +1,4 @@
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Parsers/ASTSetQuery.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
|
@ -18,6 +18,7 @@
|
||||
#include <Parsers/ParserCreateQuery.h>
|
||||
#include <Parsers/ParserUnionQueryElement.h>
|
||||
#include <Parsers/parseIntervalKind.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Common/StringUtils/StringUtils.h>
|
||||
|
||||
#include <Parsers/ParserSelectWithUnionQuery.h>
|
||||
|
@ -27,7 +27,8 @@
|
||||
#include <Parsers/Access/ParserShowCreateAccessEntityQuery.h>
|
||||
#include <Parsers/Access/ParserShowGrantsQuery.h>
|
||||
#include <Parsers/Access/ParserShowPrivilegesQuery.h>
|
||||
#include "Common/Exception.h"
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/assert_cast.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -10,6 +10,7 @@
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <Common/UTF8Helpers.h>
|
||||
#include <Common/PODArray.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -12,6 +12,7 @@
|
||||
#include <Processors/QueryPlan/CubeStep.h>
|
||||
#include <Processors/QueryPlan/DistinctStep.h>
|
||||
#include <Processors/QueryPlan/ExpressionStep.h>
|
||||
#include <Processors/QueryPlan/FilterStep.h>
|
||||
#include <Processors/QueryPlan/ITransformingStep.h>
|
||||
#include <Processors/QueryPlan/JoinStep.h>
|
||||
#include <Processors/QueryPlan/Optimizations/Optimizations.h>
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <Processors/QueryPlan/AggregatingStep.h>
|
||||
#include <Processors/QueryPlan/ExpressionStep.h>
|
||||
#include <Processors/QueryPlan/FillingStep.h>
|
||||
#include <Processors/QueryPlan/FilterStep.h>
|
||||
#include <Processors/QueryPlan/ITransformingStep.h>
|
||||
#include <Processors/QueryPlan/JoinStep.h>
|
||||
#include <Processors/QueryPlan/LimitByStep.h>
|
||||
|
@ -11,6 +11,7 @@
|
||||
#include <DataTypes/IDataType.h>
|
||||
#include <DataTypes/DataTypeEnum.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <IO/ReadBufferFromString.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
|
@ -9,6 +9,7 @@
|
||||
#include <Disks/IDisk.h>
|
||||
#include <Common/formatReadable.h>
|
||||
#include <Common/StringUtils/StringUtils.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Interpreters/Set.h>
|
||||
#include <Processors/Sinks/SinkToStorage.h>
|
||||
|
@ -1,6 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/Context_fwd.h>
|
||||
#include <Storages/IStorage.h>
|
||||
#include <Storages/SetSettings.h>
|
||||
|
||||
@ -8,6 +8,9 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class IDisk;
|
||||
using DiskPtr = std::shared_ptr<IDisk>;
|
||||
|
||||
class Set;
|
||||
using SetPtr = std::shared_ptr<Set>;
|
||||
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <Storages/System/StorageSystemBackups.h>
|
||||
#include <Backups/BackupsWorker.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <DataTypes/DataTypeEnum.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Storages/System/StorageSystemDistributionQueue.h>
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <Storages/System/StorageSystemDroppedTables.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypeUUID.h>
|
||||
@ -7,7 +8,7 @@
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/DatabaseCatalog.h>
|
||||
#include "base/types.h"
|
||||
#include <base/types.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
Loading…
Reference in New Issue
Block a user