mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 23:52:03 +00:00
Merge pull request #62264 from Algunenano/nested_includes_Ñ
Remove more nested includes
This commit is contained in:
commit
7df11b1852
@ -13,8 +13,6 @@
|
||||
#include <tuple>
|
||||
#include <limits>
|
||||
|
||||
#include <boost/math/special_functions/fpclassify.hpp>
|
||||
|
||||
// NOLINTBEGIN(*)
|
||||
|
||||
/// Use same extended double for all platforms
|
||||
@ -22,6 +20,7 @@
|
||||
#define CONSTEXPR_FROM_DOUBLE constexpr
|
||||
using FromDoubleIntermediateType = long double;
|
||||
#else
|
||||
#include <boost/math/special_functions/fpclassify.hpp>
|
||||
#include <boost/multiprecision/cpp_bin_float.hpp>
|
||||
/// `wide_integer_from_builtin` can't be constexpr with non-literal `cpp_bin_float_double_extended`
|
||||
#define CONSTEXPR_FROM_DOUBLE
|
||||
@ -309,6 +308,13 @@ struct integer<Bits, Signed>::_impl
|
||||
constexpr uint64_t max_int = std::numeric_limits<uint64_t>::max();
|
||||
static_assert(std::is_same_v<T, double> || std::is_same_v<T, FromDoubleIntermediateType>);
|
||||
/// Implementation specific behaviour on overflow (if we don't check here, stack overflow will triggered in bigint_cast).
|
||||
#if (LDBL_MANT_DIG == 64)
|
||||
if (!std::isfinite(t))
|
||||
{
|
||||
self = 0;
|
||||
return;
|
||||
}
|
||||
#else
|
||||
if constexpr (std::is_same_v<T, double>)
|
||||
{
|
||||
if (!std::isfinite(t))
|
||||
@ -325,6 +331,7 @@ struct integer<Bits, Signed>::_impl
|
||||
return;
|
||||
}
|
||||
}
|
||||
#endif
|
||||
|
||||
const T alpha = t / static_cast<T>(max_int);
|
||||
|
||||
|
@ -16,6 +16,8 @@
|
||||
#include <IO/ReadBufferFromString.h>
|
||||
#include <Poco/UUIDGenerator.h>
|
||||
#include <base/insertAtEnd.h>
|
||||
|
||||
#include <boost/range/adaptor/map.hpp>
|
||||
#include <boost/range/algorithm/copy.hpp>
|
||||
|
||||
namespace fs = std::filesystem;
|
||||
|
@ -1,17 +1,18 @@
|
||||
#pragma once
|
||||
|
||||
#include <AggregateFunctions/IAggregateFunction_fwd.h>
|
||||
#include <Columns/ColumnSparse.h>
|
||||
#include <Columns/ColumnTuple.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Core/Block.h>
|
||||
#include <Core/ColumnNumbers.h>
|
||||
#include <Core/Field.h>
|
||||
#include <Core/IResolvedFunction.h>
|
||||
#include <Core/ValuesWithType.h>
|
||||
#include <Interpreters/Context_fwd.h>
|
||||
#include <base/types.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/ThreadPool_fwd.h>
|
||||
#include <Core/IResolvedFunction.h>
|
||||
|
||||
#include "config.h"
|
||||
|
||||
@ -46,13 +47,6 @@ class IWindowFunction;
|
||||
using DataTypePtr = std::shared_ptr<const IDataType>;
|
||||
using DataTypes = std::vector<DataTypePtr>;
|
||||
|
||||
using AggregateDataPtr = char *;
|
||||
using AggregateDataPtrs = std::vector<AggregateDataPtr>;
|
||||
using ConstAggregateDataPtr = const char *;
|
||||
|
||||
class IAggregateFunction;
|
||||
using AggregateFunctionPtr = std::shared_ptr<const IAggregateFunction>;
|
||||
|
||||
struct AggregateFunctionProperties;
|
||||
|
||||
/** Aggregate functions interface.
|
||||
|
14
src/AggregateFunctions/IAggregateFunction_fwd.h
Normal file
14
src/AggregateFunctions/IAggregateFunction_fwd.h
Normal file
@ -0,0 +1,14 @@
|
||||
#pragma once
|
||||
|
||||
#include <memory>
|
||||
#include <vector>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
using AggregateDataPtr = char *;
|
||||
using AggregateDataPtrs = std::vector<AggregateDataPtr>;
|
||||
using ConstAggregateDataPtr = const char *;
|
||||
|
||||
class IAggregateFunction;
|
||||
using AggregateFunctionPtr = std::shared_ptr<const IAggregateFunction>;
|
||||
}
|
@ -1,14 +1,12 @@
|
||||
#include <Analyzer/ArrayJoinNode.h>
|
||||
|
||||
#include <Analyzer/ColumnNode.h>
|
||||
#include <Analyzer/Utils.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <IO/WriteBuffer.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
|
||||
#include <Analyzer/Utils.h>
|
||||
#include <Analyzer/ColumnNode.h>
|
||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||
#include <Common/assert_cast.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -1,14 +1,12 @@
|
||||
#include <Analyzer/ColumnNode.h>
|
||||
|
||||
#include <Common/SipHash.h>
|
||||
|
||||
#include <Analyzer/TableNode.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <IO/WriteBuffer.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Common/SipHash.h>
|
||||
#include <Common/assert_cast.h>
|
||||
|
||||
#include <Analyzer/TableNode.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -1,16 +1,14 @@
|
||||
#include <Analyzer/JoinNode.h>
|
||||
#include <Analyzer/ListNode.h>
|
||||
|
||||
#include <Analyzer/Utils.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <IO/WriteBuffer.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
#include <Parsers/ASTSubquery.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTSubquery.h>
|
||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||
|
||||
#include <Analyzer/Utils.h>
|
||||
#include <Common/assert_cast.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -1,11 +1,9 @@
|
||||
#include <Analyzer/WindowNode.h>
|
||||
|
||||
#include <Common/SipHash.h>
|
||||
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <Parsers/ASTWindowDefinition.h>
|
||||
#include <Common/SipHash.h>
|
||||
#include <Common/assert_cast.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -1,22 +1,25 @@
|
||||
#include <Access/Common/AccessEntityType.h>
|
||||
#include <Backups/BackupCoordinationStage.h>
|
||||
#include <Backups/BackupEntriesCollector.h>
|
||||
#include <Backups/BackupEntryFromMemory.h>
|
||||
#include <Backups/IBackupCoordination.h>
|
||||
#include <Backups/BackupCoordinationStage.h>
|
||||
#include <Backups/BackupUtils.h>
|
||||
#include <Backups/DDLAdjustingForBackupVisitor.h>
|
||||
#include <Backups/IBackupCoordination.h>
|
||||
#include <Databases/IDatabase.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/DatabaseCatalog.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/formatAST.h>
|
||||
#include <Storages/IStorage.h>
|
||||
#include <Access/Common/AccessEntityType.h>
|
||||
#include <base/chrono_io.h>
|
||||
#include <base/insertAtEnd.h>
|
||||
#include <base/scope_guard.h>
|
||||
#include <base/sleep.h>
|
||||
#include <Common/escapeForFileName.h>
|
||||
|
||||
#include <boost/range/adaptor/map.hpp>
|
||||
#include <boost/range/algorithm/copy.hpp>
|
||||
#include <base/scope_guard.h>
|
||||
|
||||
#include <filesystem>
|
||||
|
||||
namespace fs = std::filesystem;
|
||||
|
@ -27,6 +27,8 @@
|
||||
#include <Common/scope_guard_safe.h>
|
||||
#include <Common/ThreadPool.h>
|
||||
|
||||
#include <boost/range/adaptor/map.hpp>
|
||||
|
||||
|
||||
namespace CurrentMetrics
|
||||
{
|
||||
|
@ -24,6 +24,9 @@
|
||||
#include <Common/escapeForFileName.h>
|
||||
#include <base/insertAtEnd.h>
|
||||
#include <boost/algorithm/string/join.hpp>
|
||||
|
||||
#include <boost/range/adaptor/map.hpp>
|
||||
|
||||
#include <filesystem>
|
||||
#include <ranges>
|
||||
|
||||
|
@ -1,7 +1,13 @@
|
||||
#include <Columns/ColumnAggregateFunction.h>
|
||||
|
||||
#include <AggregateFunctions/IAggregateFunction.h>
|
||||
#include <Columns/ColumnsCommon.h>
|
||||
#include <Columns/MaskOperations.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <IO/ReadBufferFromString.h>
|
||||
#include <IO/WriteBufferFromArena.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <Processors/Transforms/ColumnGathererTransform.h>
|
||||
#include <Common/AlignedBuffer.h>
|
||||
#include <Common/Arena.h>
|
||||
#include <Common/FieldVisitorToString.h>
|
||||
@ -11,10 +17,6 @@
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Common/iota.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <IO/WriteBufferFromArena.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <Processors/Transforms/ColumnGathererTransform.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -109,6 +111,11 @@ ConstArenas concatArenas(const ConstArenas & array, ConstArenaPtr arena)
|
||||
|
||||
}
|
||||
|
||||
std::string ColumnAggregateFunction::getName() const
|
||||
{
|
||||
return "AggregateFunction(" + func->getName() + ")";
|
||||
}
|
||||
|
||||
MutableColumnPtr ColumnAggregateFunction::convertToValues(MutableColumnPtr column)
|
||||
{
|
||||
/** If the aggregate function returns an unfinalized/unfinished state,
|
||||
|
@ -1,17 +1,9 @@
|
||||
#pragma once
|
||||
|
||||
#include <AggregateFunctions/IAggregateFunction.h>
|
||||
|
||||
#include <AggregateFunctions/IAggregateFunction_fwd.h>
|
||||
#include <Columns/IColumn.h>
|
||||
#include <Common/PODArray.h>
|
||||
|
||||
#include <Core/Field.h>
|
||||
|
||||
#include <IO/ReadBufferFromString.h>
|
||||
#include <IO/WriteBuffer.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Common/PODArray.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -26,6 +18,12 @@ using ArenaPtr = std::shared_ptr<Arena>;
|
||||
using ConstArenaPtr = std::shared_ptr<const Arena>;
|
||||
using ConstArenas = std::vector<ConstArenaPtr>;
|
||||
|
||||
class Context;
|
||||
using ContextPtr = std::shared_ptr<const Context>;
|
||||
|
||||
struct ColumnWithTypeAndName;
|
||||
using ColumnsWithTypeAndName = std::vector<ColumnWithTypeAndName>;
|
||||
|
||||
|
||||
/** Column of states of aggregate functions.
|
||||
* Presented as an array of pointers to the states of aggregate functions (data).
|
||||
@ -121,7 +119,7 @@ public:
|
||||
/// This method is made static and receive MutableColumnPtr object to explicitly destroy it.
|
||||
static MutableColumnPtr convertToValues(MutableColumnPtr column);
|
||||
|
||||
std::string getName() const override { return "AggregateFunction(" + func->getName() + ")"; }
|
||||
std::string getName() const override;
|
||||
const char * getFamilyName() const override { return "AggregateFunction"; }
|
||||
TypeIndex getDataType() const override { return TypeIndex::AggregateFunction; }
|
||||
|
||||
|
@ -1,12 +1,10 @@
|
||||
#pragma once
|
||||
|
||||
#include <Columns/IColumn.h>
|
||||
#include <Core/Field.h>
|
||||
#include <Core/Names.h>
|
||||
#include <Columns/IColumn.h>
|
||||
#include <Common/PODArray.h>
|
||||
#include <Common/HashTable/HashMap.h>
|
||||
#include <DataTypes/Serializations/JSONDataParser.h>
|
||||
#include <DataTypes/Serializations/SubcolumnsTree.h>
|
||||
#include <Common/PODArray.h>
|
||||
|
||||
#include <DataTypes/IDataType.h>
|
||||
|
||||
|
@ -10,6 +10,8 @@
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <base/JSON.h>
|
||||
|
||||
#include <boost/range/adaptor/map.hpp>
|
||||
|
||||
|
||||
namespace fs = std::filesystem;
|
||||
|
||||
|
@ -1,19 +1,17 @@
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/FieldVisitorToString.h>
|
||||
|
||||
#include <Core/Block.h>
|
||||
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
#include <Common/assert_cast.h>
|
||||
|
||||
#include <AggregateFunctions/IAggregateFunction.h>
|
||||
#include <Columns/ColumnAggregateFunction.h>
|
||||
#include <Columns/ColumnConst.h>
|
||||
#include <Columns/ColumnSparse.h>
|
||||
#include <Core/Block.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <base/sort.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/FieldVisitorToString.h>
|
||||
#include <Common/assert_cast.h>
|
||||
|
||||
#include <iterator>
|
||||
#include <base/sort.h>
|
||||
|
||||
#include <boost/algorithm/string.hpp>
|
||||
|
||||
|
||||
|
@ -22,6 +22,12 @@ namespace ErrorCodes
|
||||
extern const int DECIMAL_OVERFLOW;
|
||||
}
|
||||
|
||||
template <is_decimal T>
|
||||
T DecimalField<T>::getScaleMultiplier() const
|
||||
{
|
||||
return DecimalUtils::scaleMultiplier<T>(scale);
|
||||
}
|
||||
|
||||
inline Field getBinaryValue(UInt8 type, ReadBuffer & buf)
|
||||
{
|
||||
switch (static_cast<Field::Types::Which>(type))
|
||||
@ -627,5 +633,9 @@ std::string_view Field::getTypeName() const
|
||||
return fieldTypeToString(which);
|
||||
}
|
||||
|
||||
|
||||
template class DecimalField<Decimal32>;
|
||||
template class DecimalField<Decimal64>;
|
||||
template class DecimalField<Decimal128>;
|
||||
template class DecimalField<Decimal256>;
|
||||
template class DecimalField<DateTime64>;
|
||||
}
|
||||
|
@ -8,7 +8,6 @@
|
||||
#include <functional>
|
||||
|
||||
#include <Core/CompareHelper.h>
|
||||
#include <Core/DecimalFunctions.h>
|
||||
#include <Core/Defines.h>
|
||||
#include <Core/Types.h>
|
||||
#include <Core/UUID.h>
|
||||
@ -151,7 +150,7 @@ public:
|
||||
|
||||
operator T() const { return dec; } /// NOLINT
|
||||
T getValue() const { return dec; }
|
||||
T getScaleMultiplier() const { return DecimalUtils::scaleMultiplier<T>(scale); }
|
||||
T getScaleMultiplier() const;
|
||||
UInt32 getScale() const { return scale; }
|
||||
|
||||
template <typename U>
|
||||
@ -200,6 +199,12 @@ private:
|
||||
UInt32 scale;
|
||||
};
|
||||
|
||||
extern template class DecimalField<Decimal32>;
|
||||
extern template class DecimalField<Decimal64>;
|
||||
extern template class DecimalField<Decimal128>;
|
||||
extern template class DecimalField<Decimal256>;
|
||||
extern template class DecimalField<DateTime64>;
|
||||
|
||||
template <typename T> constexpr bool is_decimal_field = false;
|
||||
template <> constexpr inline bool is_decimal_field<DecimalField<Decimal32>> = true;
|
||||
template <> constexpr inline bool is_decimal_field<DecimalField<Decimal64>> = true;
|
||||
|
@ -2,6 +2,8 @@
|
||||
#include <magic_enum.hpp>
|
||||
#include <Access/Common/SQLSecurityDefs.h>
|
||||
|
||||
#include <boost/range/adaptor/map.hpp>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -1,13 +1,12 @@
|
||||
#pragma once
|
||||
|
||||
#include <Poco/Timespan.h>
|
||||
#include <Poco/URI.h>
|
||||
#include <base/types.h>
|
||||
#include <Core/Field.h>
|
||||
#include <Core/MultiEnum.h>
|
||||
#include <boost/range/adaptor/map.hpp>
|
||||
#include <chrono>
|
||||
#include <string_view>
|
||||
#include <Core/Field.h>
|
||||
#include <Core/MultiEnum.h>
|
||||
#include <base/types.h>
|
||||
#include <Poco/Timespan.h>
|
||||
#include <Poco/URI.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -15,6 +15,7 @@
|
||||
#include <IO/Operators.h>
|
||||
|
||||
#include <AggregateFunctions/AggregateFunctionFactory.h>
|
||||
#include <AggregateFunctions/IAggregateFunction.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTIdentifier_fwd.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
@ -32,6 +33,11 @@ namespace ErrorCodes
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
String DataTypeAggregateFunction::getFunctionName() const
|
||||
{
|
||||
return function->getName();
|
||||
}
|
||||
|
||||
|
||||
String DataTypeAggregateFunction::doGetName() const
|
||||
{
|
||||
@ -52,6 +58,25 @@ size_t DataTypeAggregateFunction::getVersion() const
|
||||
return function->getDefaultVersion();
|
||||
}
|
||||
|
||||
DataTypePtr DataTypeAggregateFunction::getReturnType() const
|
||||
{
|
||||
return function->getResultType();
|
||||
}
|
||||
|
||||
DataTypePtr DataTypeAggregateFunction::getReturnTypeToPredict() const
|
||||
{
|
||||
return function->getReturnTypeToPredict();
|
||||
}
|
||||
|
||||
bool DataTypeAggregateFunction::isVersioned() const
|
||||
{
|
||||
return function->isVersioned();
|
||||
}
|
||||
|
||||
void DataTypeAggregateFunction::updateVersionFromRevision(size_t revision, bool if_empty) const
|
||||
{
|
||||
setVersion(function->getVersionFromRevision(revision), if_empty);
|
||||
}
|
||||
|
||||
String DataTypeAggregateFunction::getNameImpl(bool with_version) const
|
||||
{
|
||||
|
@ -1,7 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <AggregateFunctions/IAggregateFunction.h>
|
||||
|
||||
#include <AggregateFunctions/IAggregateFunction_fwd.h>
|
||||
#include <Core/Field.h>
|
||||
#include <DataTypes/IDataType.h>
|
||||
|
||||
|
||||
@ -39,7 +39,7 @@ public:
|
||||
{
|
||||
}
|
||||
|
||||
String getFunctionName() const { return function->getName(); }
|
||||
String getFunctionName() const;
|
||||
AggregateFunctionPtr getFunction() const { return function; }
|
||||
|
||||
String doGetName() const override;
|
||||
@ -51,8 +51,8 @@ public:
|
||||
|
||||
bool canBeInsideNullable() const override { return false; }
|
||||
|
||||
DataTypePtr getReturnType() const { return function->getResultType(); }
|
||||
DataTypePtr getReturnTypeToPredict() const { return function->getReturnTypeToPredict(); }
|
||||
DataTypePtr getReturnType() const;
|
||||
DataTypePtr getReturnTypeToPredict() const;
|
||||
DataTypes getArgumentsDataTypes() const { return argument_types; }
|
||||
|
||||
MutableColumnPtr createColumn() const override;
|
||||
@ -69,7 +69,7 @@ public:
|
||||
SerializationPtr doGetDefaultSerialization() const override;
|
||||
bool supportsSparseSerialization() const override { return false; }
|
||||
|
||||
bool isVersioned() const { return function->isVersioned(); }
|
||||
bool isVersioned() const;
|
||||
|
||||
/// Version is not empty only if it was parsed from AST or implicitly cast to 0 or version according
|
||||
/// to server revision.
|
||||
@ -84,10 +84,7 @@ public:
|
||||
version = version_;
|
||||
}
|
||||
|
||||
void updateVersionFromRevision(size_t revision, bool if_empty) const
|
||||
{
|
||||
setVersion(function->getVersionFromRevision(revision), if_empty);
|
||||
}
|
||||
void updateVersionFromRevision(size_t revision, bool if_empty) const;
|
||||
};
|
||||
|
||||
void setVersionToAggregateFunctions(DataTypePtr & type, bool if_empty, std::optional<size_t> revision = std::nullopt);
|
||||
|
@ -7,6 +7,7 @@
|
||||
#include <DataTypes/DataTypeFactory.h>
|
||||
|
||||
#include <AggregateFunctions/AggregateFunctionFactory.h>
|
||||
#include <AggregateFunctions/IAggregateFunction.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
|
@ -1,13 +1,18 @@
|
||||
#pragma once
|
||||
|
||||
#include <AggregateFunctions/IAggregateFunction_fwd.h>
|
||||
#include <Core/Field.h>
|
||||
#include <DataTypes/DataTypeCustom.h>
|
||||
#include <AggregateFunctions/IAggregateFunction.h>
|
||||
|
||||
#include <IO/ReadHelpers.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class IDataType;
|
||||
using DataTypePtr = std::shared_ptr<const IDataType>;
|
||||
using DataTypes = std::vector<DataTypePtr>;
|
||||
|
||||
/** The type SimpleAggregateFunction(fct, type) is meant to be used in an AggregatingMergeTree. It behaves like a standard
|
||||
* data type but when rows are merged, an aggregation function is applied.
|
||||
*
|
||||
|
@ -1,17 +1,15 @@
|
||||
#include <DataTypes/Serializations/SerializationAggregateFunction.h>
|
||||
|
||||
#include <IO/WriteHelpers.h>
|
||||
|
||||
#include <AggregateFunctions/IAggregateFunction.h>
|
||||
#include <Columns/ColumnAggregateFunction.h>
|
||||
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Common/AlignedBuffer.h>
|
||||
#include <Common/Arena.h>
|
||||
|
||||
#include <DataTypes/Serializations/SerializationAggregateFunction.h>
|
||||
#include <Formats/FormatSettings.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <IO/ReadBufferFromString.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Common/AlignedBuffer.h>
|
||||
#include <Common/Arena.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -1,6 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <AggregateFunctions/IAggregateFunction.h>
|
||||
#include <AggregateFunctions/IAggregateFunction_fwd.h>
|
||||
|
||||
#include <DataTypes/Serializations/ISerialization.h>
|
||||
|
||||
|
@ -1,51 +1,54 @@
|
||||
#include <Formats/ProtobufSerializer.h>
|
||||
|
||||
#if USE_PROTOBUF
|
||||
# include <Columns/ColumnAggregateFunction.h>
|
||||
# include <Columns/ColumnArray.h>
|
||||
# include <Columns/ColumnDecimal.h>
|
||||
# include <Columns/ColumnLowCardinality.h>
|
||||
# include <Columns/ColumnMap.h>
|
||||
# include <Columns/ColumnNullable.h>
|
||||
# include <Columns/ColumnFixedString.h>
|
||||
# include <Columns/ColumnString.h>
|
||||
# include <Columns/ColumnTuple.h>
|
||||
# include <Columns/ColumnVector.h>
|
||||
# include <Common/PODArray.h>
|
||||
# include <Common/quoteString.h>
|
||||
# include <Core/DecimalComparison.h>
|
||||
# include <DataTypes/DataTypeAggregateFunction.h>
|
||||
# include <DataTypes/DataTypeArray.h>
|
||||
# include <DataTypes/DataTypesDecimal.h>
|
||||
# include <DataTypes/DataTypeDateTime64.h>
|
||||
# include <DataTypes/DataTypeEnum.h>
|
||||
# include <DataTypes/DataTypeFixedString.h>
|
||||
# include <DataTypes/DataTypeLowCardinality.h>
|
||||
# include <DataTypes/DataTypeMap.h>
|
||||
# include <DataTypes/DataTypeNullable.h>
|
||||
# include <DataTypes/DataTypeTuple.h>
|
||||
# include <DataTypes/DataTypeString.h>
|
||||
# include <DataTypes/Serializations/SerializationDecimal.h>
|
||||
# include <DataTypes/Serializations/SerializationFixedString.h>
|
||||
# include <Formats/ProtobufReader.h>
|
||||
# include <Formats/ProtobufWriter.h>
|
||||
# include <Formats/RowInputMissingColumnsFiller.h>
|
||||
# include <IO/Operators.h>
|
||||
# include <IO/ReadBufferFromString.h>
|
||||
# include <IO/ReadHelpers.h>
|
||||
# include <IO/WriteBufferFromString.h>
|
||||
# include <IO/WriteHelpers.h>
|
||||
# include <base/range.h>
|
||||
# include <base/sort.h>
|
||||
# include <google/protobuf/descriptor.h>
|
||||
# include <google/protobuf/descriptor.pb.h>
|
||||
# include <boost/algorithm/string.hpp>
|
||||
# include <boost/container/flat_map.hpp>
|
||||
# include <boost/container/flat_set.hpp>
|
||||
# include <boost/numeric/conversion/cast.hpp>
|
||||
# include <boost/range/algorithm.hpp>
|
||||
# include <boost/range/algorithm_ext/erase.hpp>
|
||||
# include <Common/logger_useful.h>
|
||||
# include <AggregateFunctions/IAggregateFunction.h>
|
||||
# include <Columns/ColumnAggregateFunction.h>
|
||||
# include <Columns/ColumnArray.h>
|
||||
# include <Columns/ColumnDecimal.h>
|
||||
# include <Columns/ColumnFixedString.h>
|
||||
# include <Columns/ColumnLowCardinality.h>
|
||||
# include <Columns/ColumnMap.h>
|
||||
# include <Columns/ColumnNullable.h>
|
||||
# include <Columns/ColumnString.h>
|
||||
# include <Columns/ColumnTuple.h>
|
||||
# include <Columns/ColumnVector.h>
|
||||
# include <Core/DecimalComparison.h>
|
||||
# include <DataTypes/DataTypeAggregateFunction.h>
|
||||
# include <DataTypes/DataTypeArray.h>
|
||||
# include <DataTypes/DataTypeDateTime64.h>
|
||||
# include <DataTypes/DataTypeEnum.h>
|
||||
# include <DataTypes/DataTypeFixedString.h>
|
||||
# include <DataTypes/DataTypeLowCardinality.h>
|
||||
# include <DataTypes/DataTypeMap.h>
|
||||
# include <DataTypes/DataTypeNullable.h>
|
||||
# include <DataTypes/DataTypeString.h>
|
||||
# include <DataTypes/DataTypeTuple.h>
|
||||
# include <DataTypes/DataTypesDecimal.h>
|
||||
# include <DataTypes/Serializations/SerializationDecimal.h>
|
||||
# include <DataTypes/Serializations/SerializationFixedString.h>
|
||||
# include <Formats/ProtobufReader.h>
|
||||
# include <Formats/ProtobufWriter.h>
|
||||
# include <Formats/RowInputMissingColumnsFiller.h>
|
||||
# include <IO/Operators.h>
|
||||
# include <IO/ReadBufferFromString.h>
|
||||
# include <IO/ReadHelpers.h>
|
||||
# include <IO/WriteBufferFromString.h>
|
||||
# include <IO/WriteHelpers.h>
|
||||
# include <base/range.h>
|
||||
# include <base/sort.h>
|
||||
# include <Common/PODArray.h>
|
||||
# include <Common/logger_useful.h>
|
||||
# include <Common/quoteString.h>
|
||||
|
||||
# include <boost/algorithm/string.hpp>
|
||||
# include <boost/container/flat_map.hpp>
|
||||
# include <boost/container/flat_set.hpp>
|
||||
# include <boost/numeric/conversion/cast.hpp>
|
||||
# include <boost/range/algorithm.hpp>
|
||||
# include <boost/range/algorithm_ext/erase.hpp>
|
||||
# include <google/protobuf/descriptor.h>
|
||||
# include <google/protobuf/descriptor.pb.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -5,54 +5,55 @@
|
||||
// sanitizer/asan_interface.h
|
||||
#include <memory>
|
||||
#include <type_traits>
|
||||
#include <base/wide_integer_to_string.h>
|
||||
|
||||
#include <AggregateFunctions/IAggregateFunction.h>
|
||||
#include <Columns/ColumnAggregateFunction.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnConst.h>
|
||||
#include <Columns/ColumnDecimal.h>
|
||||
#include <Columns/ColumnFixedString.h>
|
||||
#include <Columns/ColumnNullable.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnVector.h>
|
||||
#include <Columns/IColumn.h>
|
||||
#include <Core/ColumnWithTypeAndName.h>
|
||||
#include <Core/ColumnsWithTypeAndName.h>
|
||||
#include <Core/DecimalFunctions.h>
|
||||
#include <DataTypes/DataTypeAggregateFunction.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <DataTypes/DataTypeDateTime64.h>
|
||||
#include <DataTypes/DataTypeFactory.h>
|
||||
#include <DataTypes/DataTypeFixedString.h>
|
||||
#include <DataTypes/DataTypeInterval.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypeIPv4andIPv6.h>
|
||||
#include <DataTypes/DataTypeInterval.h>
|
||||
#include <DataTypes/DataTypeLowCardinality.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <DataTypes/DataTypesDecimal.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/IDataType.h>
|
||||
#include <DataTypes/Native.h>
|
||||
#include <DataTypes/NumberTraits.h>
|
||||
#include <DataTypes/getMostSubtype.h>
|
||||
#include <Functions/DivisionUtils.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/IsOperation.h>
|
||||
#include <Functions/castTypeToEither.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/castColumn.h>
|
||||
#include <base/TypeList.h>
|
||||
#include <base/TypeLists.h>
|
||||
#include <base/map.h>
|
||||
#include <base/types.h>
|
||||
#include <base/wide_integer_to_string.h>
|
||||
#include <Common/Arena.h>
|
||||
#include <Common/FieldVisitorsAccurateComparison.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/Arena.h>
|
||||
#include <Core/ColumnWithTypeAndName.h>
|
||||
#include <base/types.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/IColumn.h>
|
||||
#include <Core/ColumnsWithTypeAndName.h>
|
||||
#include <DataTypes/IDataType.h>
|
||||
#include <DataTypes/getMostSubtype.h>
|
||||
#include <base/TypeLists.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeLowCardinality.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
#if USE_EMBEDDED_COMPILER
|
||||
# include <llvm/IR/IRBuilder.h>
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <type_traits>
|
||||
|
||||
#include <AggregateFunctions/IAggregateFunction.h>
|
||||
#include <Columns/ColumnAggregateFunction.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnConst.h>
|
||||
|
@ -1,5 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <Core/DecimalFunctions.h>
|
||||
#include <DataTypes/DataTypeInterval.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Common/DateLUT.h>
|
||||
|
@ -1,18 +1,19 @@
|
||||
#include <cassert>
|
||||
#include <Columns/ColumnFixedString.h>
|
||||
#include <Columns/ColumnMap.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnTuple.h>
|
||||
#include <Columns/ColumnVector.h>
|
||||
#include <Columns/IColumn.h>
|
||||
#include <Core/ColumnWithTypeAndName.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeMap.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include "Columns/ColumnMap.h"
|
||||
#include "DataTypes/DataTypeMap.h"
|
||||
#include <base/arithmeticOverflow.h>
|
||||
|
||||
#include <cassert>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -1,8 +1,9 @@
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <AggregateFunctions/IAggregateFunction.h>
|
||||
#include <Columns/ColumnAggregateFunction.h>
|
||||
#include <DataTypes/DataTypeAggregateFunction.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Common/AlignedBuffer.h>
|
||||
#include <Common/Arena.h>
|
||||
#include <Common/scope_guard_safe.h>
|
||||
|
@ -20,6 +20,7 @@
|
||||
#include <Poco/Util/Application.h>
|
||||
|
||||
#include <istream>
|
||||
#include <sstream>
|
||||
#include <unordered_map>
|
||||
#include <Common/ProxyConfiguration.h>
|
||||
|
||||
|
@ -1,6 +1,8 @@
|
||||
#include <IO/ReadBufferFromIStream.h>
|
||||
#include <Common/Exception.h>
|
||||
|
||||
#include <istream>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <IO/SeekableReadBuffer.h>
|
||||
|
||||
#include <istream>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -5,6 +5,9 @@
|
||||
#include <IO/ReadBuffer.h>
|
||||
#include <IO/WriteBuffer.h>
|
||||
|
||||
#include <istream>
|
||||
#include <ostream>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <IO/WriteBufferFromOStream.h>
|
||||
#include <Common/logger_useful.h>
|
||||
|
||||
#include <ostream>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -1,7 +1,7 @@
|
||||
#include <AggregateFunctions/IAggregateFunction.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <Interpreters/AggregateDescription.h>
|
||||
#include <Common/FieldVisitorToString.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
#include <Common/JSONBuilder.h>
|
||||
|
||||
|
||||
|
@ -1,13 +1,16 @@
|
||||
#pragma once
|
||||
|
||||
#include <AggregateFunctions/IAggregateFunction.h>
|
||||
#include <AggregateFunctions/IAggregateFunction_fwd.h>
|
||||
#include <Core/ColumnNumbers.h>
|
||||
#include <Core/Field.h>
|
||||
#include <Core/Names.h>
|
||||
#include <Core/Types.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class WriteBuffer;
|
||||
|
||||
namespace JSONBuilder { class JSONMap; }
|
||||
|
||||
struct AggregateDescription
|
||||
|
@ -1,5 +1,5 @@
|
||||
#pragma once
|
||||
#include <AggregateFunctions/IAggregateFunction.h>
|
||||
#include <AggregateFunctions/IAggregateFunction_fwd.h>
|
||||
|
||||
#include <Common/HashTable/FixedHashMap.h>
|
||||
#include <Common/HashTable/StringHashMap.h>
|
||||
|
@ -1,11 +1,12 @@
|
||||
#pragma once
|
||||
#include <boost/noncopyable.hpp>
|
||||
#include <memory.h>
|
||||
#include <AggregateFunctions/IAggregateFunction.h>
|
||||
#include <Common/ColumnsHashing.h>
|
||||
#include <Interpreters/AggregatedData.h>
|
||||
#include <Interpreters/AggregationMethod.h>
|
||||
|
||||
#include <memory>
|
||||
#include <boost/noncopyable.hpp>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
class Arena;
|
||||
|
@ -1,3 +1,4 @@
|
||||
#include <AggregateFunctions/IAggregateFunction.h>
|
||||
#include <Interpreters/AggregationUtils.h>
|
||||
|
||||
namespace DB
|
||||
|
@ -7,38 +7,37 @@
|
||||
# include <unistd.h>
|
||||
#endif
|
||||
|
||||
#include <base/sort.h>
|
||||
#include <DataTypes/DataTypeAggregateFunction.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <DataTypes/DataTypeLowCardinality.h>
|
||||
#include <Columns/ColumnTuple.h>
|
||||
#include <Columns/ColumnSparse.h>
|
||||
#include <Formats/NativeWriter.h>
|
||||
#include <Compression/CompressedWriteBuffer.h>
|
||||
#include <Interpreters/Aggregator.h>
|
||||
#include <AggregateFunctions/Combinators/AggregateFunctionArray.h>
|
||||
#include <AggregateFunctions/Combinators/AggregateFunctionState.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <Interpreters/JIT/compileFunction.h>
|
||||
#include <Interpreters/JIT/CompiledExpressionCache.h>
|
||||
#include <Columns/ColumnSparse.h>
|
||||
#include <Columns/ColumnTuple.h>
|
||||
#include <Compression/CompressedWriteBuffer.h>
|
||||
#include <DataTypes/DataTypeAggregateFunction.h>
|
||||
#include <DataTypes/DataTypeLowCardinality.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <Disks/TemporaryFileOnDisk.h>
|
||||
#include <Formats/NativeWriter.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <Interpreters/AggregationUtils.h>
|
||||
#include <Interpreters/Aggregator.h>
|
||||
#include <Interpreters/JIT/CompiledExpressionCache.h>
|
||||
#include <Interpreters/JIT/compileFunction.h>
|
||||
#include <Interpreters/TemporaryDataOnDisk.h>
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
#include <base/sort.h>
|
||||
#include <Common/CacheBase.h>
|
||||
#include <Common/CurrentMetrics.h>
|
||||
#include <Common/CurrentThread.h>
|
||||
#include <Common/JSONBuilder.h>
|
||||
#include <Common/MemoryTracker.h>
|
||||
#include <Common/Stopwatch.h>
|
||||
#include <Common/setThreadName.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Common/formatReadable.h>
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Common/CacheBase.h>
|
||||
#include <Common/MemoryTracker.h>
|
||||
#include <Common/CurrentThread.h>
|
||||
#include <Common/CurrentMetrics.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Common/JSONBuilder.h>
|
||||
#include <Common/scope_guard_safe.h>
|
||||
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
|
||||
#include <Interpreters/AggregationUtils.h>
|
||||
#include <Common/setThreadName.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
|
||||
|
||||
namespace ProfileEvents
|
||||
|
@ -27,6 +27,8 @@
|
||||
#include <Common/noexcept_scope.h>
|
||||
#include <Common/checkStackSize.h>
|
||||
|
||||
#include <boost/range/adaptor/map.hpp>
|
||||
|
||||
#include "config.h"
|
||||
|
||||
#if USE_MYSQL
|
||||
|
@ -6,6 +6,7 @@
|
||||
#include <Columns/ColumnConst.h>
|
||||
#include <Columns/ColumnFixedString.h>
|
||||
#include <Columns/ColumnNullable.h>
|
||||
#include <Columns/ColumnSparse.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnTuple.h>
|
||||
#include <Columns/ColumnVector.h>
|
||||
|
@ -18,6 +18,7 @@
|
||||
#include <Access/ContextAccess.h>
|
||||
|
||||
#include <AggregateFunctions/AggregateFunctionCount.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
|
||||
#include <Interpreters/ApplyWithAliasVisitor.h>
|
||||
#include <Interpreters/ApplyWithSubqueryVisitor.h>
|
||||
|
@ -2,16 +2,17 @@
|
||||
|
||||
#if USE_EMBEDDED_COMPILER
|
||||
|
||||
#include <llvm/IR/BasicBlock.h>
|
||||
#include <llvm/IR/Function.h>
|
||||
#include <llvm/IR/IRBuilder.h>
|
||||
# include <AggregateFunctions/IAggregateFunction.h>
|
||||
# include <Columns/ColumnNullable.h>
|
||||
# include <DataTypes/DataTypeNullable.h>
|
||||
# include <DataTypes/Native.h>
|
||||
# include <Interpreters/JIT/CHJIT.h>
|
||||
# include <Common/ProfileEvents.h>
|
||||
# include <Common/Stopwatch.h>
|
||||
|
||||
#include <Common/Stopwatch.h>
|
||||
#include <Common/ProfileEvents.h>
|
||||
#include <DataTypes/Native.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <Columns/ColumnNullable.h>
|
||||
#include <Interpreters/JIT/CHJIT.h>
|
||||
# include <llvm/IR/BasicBlock.h>
|
||||
# include <llvm/IR/Function.h>
|
||||
# include <llvm/IR/IRBuilder.h>
|
||||
|
||||
namespace
|
||||
{
|
||||
|
@ -4,10 +4,10 @@
|
||||
|
||||
#if USE_EMBEDDED_COMPILER
|
||||
|
||||
#include <Core/SortDescription.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <AggregateFunctions/IAggregateFunction.h>
|
||||
#include <Interpreters/JIT/CHJIT.h>
|
||||
# include <AggregateFunctions/IAggregateFunction_fwd.h>
|
||||
# include <Core/SortDescription.h>
|
||||
# include <Functions/IFunction.h>
|
||||
# include <Interpreters/JIT/CHJIT.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -1,9 +1,10 @@
|
||||
#include <Interpreters/RewriteFunctionToSubcolumnVisitor.h>
|
||||
#include <DataTypes/NestedUtils.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <DataTypes/NestedUtils.h>
|
||||
#include <Interpreters/RewriteFunctionToSubcolumnVisitor.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Common/assert_cast.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -1,10 +1,10 @@
|
||||
#include <Interpreters/WindowDescription.h>
|
||||
|
||||
#include <AggregateFunctions/IAggregateFunction.h>
|
||||
#include <Core/Field.h>
|
||||
#include <Common/FieldVisitorsAccurateComparison.h>
|
||||
#include <Common/FieldVisitorToString.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <Interpreters/WindowDescription.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Common/FieldVisitorToString.h>
|
||||
#include <Common/FieldVisitorsAccurateComparison.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -1,12 +1,12 @@
|
||||
#pragma once
|
||||
|
||||
#include <AggregateFunctions/IAggregateFunction_fwd.h>
|
||||
#include <Core/Field.h>
|
||||
#include <Parsers/IAST_fwd.h>
|
||||
#include <AggregateFunctions/IAggregateFunction.h>
|
||||
#include <Core/SortDescription.h>
|
||||
#include <DataTypes/IDataType.h>
|
||||
#include <Core/Names.h>
|
||||
#include <Core/SortDescription.h>
|
||||
#include <Core/Types.h>
|
||||
#include <DataTypes/IDataType.h>
|
||||
#include <Parsers/IAST_fwd.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -5,9 +5,10 @@
|
||||
#include <IO/ReadBufferFromMemory.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/StringUtils/StringUtils.h>
|
||||
#include <Common/BinStringDecodeHelper.h>
|
||||
#include <Common/PODArray.h>
|
||||
#include <Common/StringUtils/StringUtils.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include "Parsers/CommonParsers.h"
|
||||
|
||||
#include <Parsers/DumpASTNode.h>
|
||||
|
@ -1,9 +1,10 @@
|
||||
#include <Planner/Planner.h>
|
||||
|
||||
#include <Core/ProtocolDefines.h>
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Common/ProfileEvents.h>
|
||||
#include <Columns/ColumnConst.h>
|
||||
#include <Columns/ColumnSet.h>
|
||||
#include <Core/ProtocolDefines.h>
|
||||
#include <Common/ProfileEvents.h>
|
||||
#include <Common/logger_useful.h>
|
||||
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
|
||||
|
@ -1,9 +1,10 @@
|
||||
#pragma once
|
||||
|
||||
#include <AggregateFunctions/IAggregateFunction.h>
|
||||
#include <Columns/ColumnAggregateFunction.h>
|
||||
#include <Common/AlignedBuffer.h>
|
||||
#include <Processors/Merges/Algorithms/IMergingAlgorithmWithDelayedChunk.h>
|
||||
#include <Processors/Merges/Algorithms/MergedData.h>
|
||||
#include <Common/AlignedBuffer.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -1,16 +1,17 @@
|
||||
#include <base/find_symbols.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <AggregateFunctions/AggregateFunctionFactory.h>
|
||||
#include <AggregateFunctions/IAggregateFunction.h>
|
||||
#include <AggregateFunctions/parseAggregateFunctionParameters.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Processors/Merges/Algorithms/Graphite.h>
|
||||
#include <base/find_symbols.h>
|
||||
#include <base/sort.h>
|
||||
|
||||
#include <string_view>
|
||||
#include <vector>
|
||||
#include <unordered_map>
|
||||
|
||||
#include <fmt/format.h>
|
||||
#include <base/sort.h>
|
||||
|
||||
#include <Poco/Util/AbstractConfiguration.h>
|
||||
|
||||
@ -61,6 +62,23 @@ RuleType ruleType(const String & s)
|
||||
throw Exception(DB::ErrorCodes::BAD_ARGUMENTS, "invalid rule type: {}", s);
|
||||
}
|
||||
|
||||
void Pattern::updateHash(SipHash & hash) const
|
||||
{
|
||||
hash.update(rule_type);
|
||||
hash.update(regexp_str);
|
||||
if (function)
|
||||
{
|
||||
hash.update(function->getName());
|
||||
for (const auto & p : function->getParameters())
|
||||
hash.update(toString(p));
|
||||
}
|
||||
for (const auto & r : retentions)
|
||||
{
|
||||
hash.update(r.age);
|
||||
hash.update(r.precision);
|
||||
}
|
||||
}
|
||||
|
||||
static const Graphite::Pattern undef_pattern =
|
||||
{ /// empty pattern for selectPatternForPath
|
||||
.rule_type = RuleTypeAll,
|
||||
|
@ -1,8 +1,9 @@
|
||||
#pragma once
|
||||
|
||||
#include <Common/SipHash.h>
|
||||
#include <AggregateFunctions/IAggregateFunction_fwd.h>
|
||||
#include <Interpreters/Context_fwd.h>
|
||||
#include <Common/OptimizedRegularExpression.h>
|
||||
#include <AggregateFunctions/IAggregateFunction.h>
|
||||
#include <Common/SipHash.h>
|
||||
|
||||
/** Intended for implementation of "rollup" - aggregation (rounding) of older data
|
||||
* for a table with Graphite data (Graphite is the system for time series monitoring).
|
||||
@ -123,22 +124,7 @@ struct Pattern
|
||||
AggregateFunctionPtr function;
|
||||
Retentions retentions; /// Must be ordered by 'age' descending.
|
||||
enum { TypeUndef, TypeRetention, TypeAggregation, TypeAll } type = TypeAll; /// The type of defined pattern, filled automatically
|
||||
void updateHash(SipHash & hash) const
|
||||
{
|
||||
hash.update(rule_type);
|
||||
hash.update(regexp_str);
|
||||
if (function)
|
||||
{
|
||||
hash.update(function->getName());
|
||||
for (const auto & p : function->getParameters())
|
||||
hash.update(toString(p));
|
||||
}
|
||||
for (const auto & r : retentions)
|
||||
{
|
||||
hash.update(r.age);
|
||||
hash.update(r.precision);
|
||||
}
|
||||
}
|
||||
void updateHash(SipHash & hash) const;
|
||||
};
|
||||
|
||||
bool operator==(const Pattern & a, const Pattern & b);
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <memory>
|
||||
#include <Columns/ColumnFixedString.h>
|
||||
#include <DataTypes/DataTypeFixedString.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <IO/Operators.h>
|
||||
|
@ -1,10 +1,10 @@
|
||||
#include <Processors/QueryPlan/WindowStep.h>
|
||||
|
||||
#include <Processors/Transforms/WindowTransform.h>
|
||||
#include <Processors/Transforms/ExpressionTransform.h>
|
||||
#include <QueryPipeline/QueryPipelineBuilder.h>
|
||||
#include <Interpreters/ExpressionActions.h>
|
||||
#include <AggregateFunctions/IAggregateFunction.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <Interpreters/ExpressionActions.h>
|
||||
#include <Processors/QueryPlan/WindowStep.h>
|
||||
#include <Processors/Transforms/ExpressionTransform.h>
|
||||
#include <Processors/Transforms/WindowTransform.h>
|
||||
#include <QueryPipeline/QueryPipelineBuilder.h>
|
||||
#include <Common/JSONBuilder.h>
|
||||
|
||||
namespace DB
|
||||
|
@ -1,23 +1,23 @@
|
||||
#include <Processors/Transforms/WindowTransform.h>
|
||||
|
||||
#include <limits>
|
||||
|
||||
#include <AggregateFunctions/AggregateFunctionFactory.h>
|
||||
#include <Columns/ColumnAggregateFunction.h>
|
||||
#include <Columns/ColumnConst.h>
|
||||
#include <Columns/ColumnLowCardinality.h>
|
||||
#include <Columns/ColumnNullable.h>
|
||||
#include <DataTypes/DataTypeDateTime64.h>
|
||||
#include <DataTypes/DataTypeInterval.h>
|
||||
#include <DataTypes/DataTypeLowCardinality.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/getLeastSupertype.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Interpreters/ExpressionActions.h>
|
||||
#include <Interpreters/convertFieldToType.h>
|
||||
#include <Processors/Transforms/WindowTransform.h>
|
||||
#include <base/arithmeticOverflow.h>
|
||||
#include <Common/Arena.h>
|
||||
#include <Common/FieldVisitorConvertToNumber.h>
|
||||
#include <Common/FieldVisitorsAccurateComparison.h>
|
||||
#include <Columns/ColumnLowCardinality.h>
|
||||
#include <base/arithmeticOverflow.h>
|
||||
#include <Columns/ColumnConst.h>
|
||||
#include <Columns/ColumnAggregateFunction.h>
|
||||
#include <Columns/ColumnNullable.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/getLeastSupertype.h>
|
||||
#include <DataTypes/DataTypeLowCardinality.h>
|
||||
#include <DataTypes/DataTypeInterval.h>
|
||||
#include <Interpreters/ExpressionActions.h>
|
||||
#include <Interpreters/convertFieldToType.h>
|
||||
#include <DataTypes/DataTypeDateTime64.h>
|
||||
|
||||
#include <limits>
|
||||
|
||||
|
||||
/// See https://fmt.dev/latest/api.html#formatting-user-defined-types
|
||||
|
@ -21,6 +21,8 @@ using ExpressionActionsPtr = std::shared_ptr<ExpressionActions>;
|
||||
|
||||
class Arena;
|
||||
|
||||
class IWindowFunction;
|
||||
|
||||
// Runtime data for computing one window function.
|
||||
struct WindowFunctionWorkspace
|
||||
{
|
||||
|
@ -39,6 +39,8 @@
|
||||
#include <cassert>
|
||||
#include <chrono>
|
||||
|
||||
#include <boost/range/adaptor/map.hpp>
|
||||
|
||||
|
||||
#define DBMS_STORAGE_LOG_DATA_FILE_EXTENSION ".bin"
|
||||
#define DBMS_STORAGE_LOG_MARKS_FILE_NAME "__marks.mrk"
|
||||
|
@ -1,7 +1,8 @@
|
||||
#include <Storages/System/StorageSystemGraphite.h>
|
||||
#include <Storages/MergeTree/MergeTreeData.h>
|
||||
#include <AggregateFunctions/IAggregateFunction.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/DatabaseCatalog.h>
|
||||
#include <Storages/MergeTree/MergeTreeData.h>
|
||||
#include <Storages/System/StorageSystemGraphite.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -1,11 +1,11 @@
|
||||
#include <Storages/System/StorageSystemMySQLBinlogs.h>
|
||||
|
||||
#include <Databases/MySQL/MySQLBinlogClientFactory.h>
|
||||
#include <Processors/Sources/SourceFromSingleChunk.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Databases/MySQL/MySQLBinlogClientFactory.h>
|
||||
#include <Processors/ISource.h>
|
||||
#include <Processors/Sources/SourceFromSingleChunk.h>
|
||||
#include <Storages/System/StorageSystemMySQLBinlogs.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -1,14 +1,15 @@
|
||||
#include "StorageSystemRemoteDataPaths.h"
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Disks/IDisk.h>
|
||||
#include <Interpreters/Cache/FileCache.h>
|
||||
#include <Interpreters/Cache/FileCacheFactory.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Processors/Sources/SourceFromSingleChunk.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Disks/IDisk.h>
|
||||
#include <Processors/Sources/SourceFromSingleChunk.h>
|
||||
|
||||
namespace fs = std::filesystem;
|
||||
|
||||
|
@ -25,6 +25,8 @@
|
||||
#include <QueryPipeline/QueryPipelineBuilder.h>
|
||||
#include <DataTypes/DataTypeUUID.h>
|
||||
|
||||
#include <boost/range/adaptor/map.hpp>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -10,6 +10,8 @@
|
||||
#include <Storages/NamedCollectionsHelpers.h>
|
||||
#include <Common/escapeForFileName.h>
|
||||
|
||||
#include <boost/range/adaptor/map.hpp>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user