mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 09:32:06 +00:00
Merge pull request #24178 from kitaisreal/function-move-file
Function move file
This commit is contained in:
commit
0870970e30
@ -106,8 +106,8 @@ endif()
|
||||
list (APPEND clickhouse_common_io_sources ${CONFIG_BUILD})
|
||||
list (APPEND clickhouse_common_io_headers ${CONFIG_VERSION} ${CONFIG_COMMON})
|
||||
|
||||
list (APPEND dbms_sources Functions/IFunction.cpp Functions/IFunctionOld.cpp Functions/FunctionFactory.cpp Functions/FunctionHelpers.cpp Functions/extractTimeZoneFromFunctionArguments.cpp Functions/replicate.cpp Functions/FunctionsLogical.cpp)
|
||||
list (APPEND dbms_headers Functions/IFunctionOld.h Functions/FunctionFactory.h Functions/FunctionHelpers.h Functions/extractTimeZoneFromFunctionArguments.h Functions/replicate.h Functions/FunctionsLogical.h)
|
||||
list (APPEND dbms_sources Functions/IFunction.cpp Functions/FunctionFactory.cpp Functions/FunctionHelpers.cpp Functions/extractTimeZoneFromFunctionArguments.cpp Functions/replicate.cpp Functions/FunctionsLogical.cpp)
|
||||
list (APPEND dbms_headers Functions/IFunction.h Functions/FunctionFactory.h Functions/FunctionHelpers.h Functions/extractTimeZoneFromFunctionArguments.h Functions/replicate.h Functions/FunctionsLogical.h)
|
||||
|
||||
list (APPEND dbms_sources
|
||||
AggregateFunctions/AggregateFunctionFactory.cpp
|
||||
|
@ -5,8 +5,8 @@ add_subdirectory(divide)
|
||||
include("${ClickHouse_SOURCE_DIR}/cmake/dbms_glob_sources.cmake")
|
||||
add_headers_and_sources(clickhouse_functions .)
|
||||
|
||||
list(REMOVE_ITEM clickhouse_functions_sources IFunctionOld.cpp FunctionFactory.cpp FunctionHelpers.cpp)
|
||||
list(REMOVE_ITEM clickhouse_functions_headers IFunctionOld.h FunctionFactory.h FunctionHelpers.h)
|
||||
list(REMOVE_ITEM clickhouse_functions_sources IFunction.cpp FunctionFactory.cpp FunctionHelpers.cpp)
|
||||
list(REMOVE_ITEM clickhouse_functions_headers IFunction.h FunctionFactory.h FunctionHelpers.h)
|
||||
|
||||
add_library(clickhouse_functions ${clickhouse_functions_sources})
|
||||
|
||||
|
@ -6,7 +6,7 @@
|
||||
#include <Core/DecimalFunctions.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/extractTimeZoneFromFunctionArguments.h>
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <common/DateLUTImpl.h>
|
||||
|
||||
|
@ -7,7 +7,7 @@
|
||||
#include <Columns/ColumnVector.h>
|
||||
#include <Columns/ColumnDecimal.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/extractTimeZoneFromFunctionArguments.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <DataTypes/DataTypeDateTime64.h>
|
||||
|
@ -23,7 +23,7 @@
|
||||
#include <Columns/ColumnConst.h>
|
||||
#include <Columns/ColumnAggregateFunction.h>
|
||||
#include "Core/DecimalFunctions.h"
|
||||
#include "IFunctionOld.h"
|
||||
#include "IFunction.h"
|
||||
#include "FunctionHelpers.h"
|
||||
#include "IsOperation.h"
|
||||
#include "DivisionUtils.h"
|
||||
|
@ -2,7 +2,7 @@
|
||||
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Columns/ColumnVector.h>
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <ext/range.h>
|
||||
|
@ -3,7 +3,7 @@
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <DataTypes/DataTypeDateTime64.h>
|
||||
#include <Functions/CustomWeekTransforms.h>
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/TransformDateTime64.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
|
||||
|
@ -7,7 +7,7 @@
|
||||
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/castTypeToEither.h>
|
||||
#include <Functions/extractTimeZoneFromFunctionArguments.h>
|
||||
|
@ -1,7 +1,7 @@
|
||||
#pragma once
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <DataTypes/DataTypeDateTime64.h>
|
||||
#include <Functions/extractTimeZoneFromFunctionArguments.h>
|
||||
#include <Functions/DateTimeTransforms.h>
|
||||
|
@ -1,4 +1,4 @@
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <common/getFQDNOrHostName.h>
|
||||
|
@ -3,7 +3,6 @@
|
||||
#include <Interpreters/Context_fwd.h>
|
||||
#include <Common/IFactoryWithAliases.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunctionAdaptors.h>
|
||||
|
||||
#include <functional>
|
||||
|
@ -1,5 +1,5 @@
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Columns/ColumnTuple.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnFixedString.h>
|
||||
|
@ -1,6 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <DataTypes/Native.h>
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
|
@ -6,7 +6,7 @@
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Columns/ColumnDecimal.h>
|
||||
#include <Columns/ColumnConst.h>
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
|
@ -2,7 +2,7 @@
|
||||
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -5,7 +5,7 @@
|
||||
#include <DataTypes/DataTypesDecimal.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Columns/ColumnDecimal.h>
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
|
@ -1,6 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
|
@ -2,7 +2,7 @@
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/GatherUtils/GatherUtils.h>
|
||||
#include <Functions/GatherUtils/Sources.h>
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/PerformanceAdaptors.h>
|
||||
#include <Functions/TargetSpecific.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
|
@ -1,7 +1,7 @@
|
||||
#pragma once
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Columns/ColumnVector.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
|
@ -4,7 +4,7 @@
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnFixedString.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Interpreters/Context_fwd.h>
|
||||
|
||||
|
||||
|
@ -7,7 +7,7 @@
|
||||
#include <Columns/ColumnVector.h>
|
||||
#include <Columns/ColumnDecimal.h>
|
||||
#include <Columns/ColumnFixedString.h>
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/IsOperation.h>
|
||||
#include <Functions/castTypeToEither.h>
|
||||
|
@ -1,7 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <Functions/extractTimeZoneFromFunctionArguments.h>
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <DataTypes/DataTypeDateTime64.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
|
@ -11,7 +11,7 @@
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/assert_cast.h>
|
||||
|
||||
|
@ -17,7 +17,7 @@
|
||||
#include <DataTypes/DataTypeUUID.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Interpreters/Context_fwd.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Common/IPv6ToBinary.h>
|
||||
|
@ -4,7 +4,7 @@
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Interpreters/Context_fwd.h>
|
||||
|
||||
|
@ -9,7 +9,7 @@
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/EmbeddedDictionaries.h>
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Dictionaries/Embedded/RegionsHierarchy.h>
|
||||
#include <Dictionaries/Embedded/RegionsHierarchies.h>
|
||||
|
@ -27,7 +27,7 @@
|
||||
#include <Interpreters/ExternalDictionariesLoader.h>
|
||||
#include <Interpreters/castColumn.h>
|
||||
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <ext/range.h>
|
||||
|
||||
|
@ -1,6 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Interpreters/Context_fwd.h>
|
||||
|
||||
namespace DB
|
||||
|
@ -39,7 +39,7 @@
|
||||
#include <Columns/ColumnFixedString.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnTuple.h>
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/TargetSpecific.h>
|
||||
#include <Functions/PerformanceAdaptors.h>
|
||||
|
@ -1,6 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Core/AccurateComparison.h>
|
||||
#include <Functions/DummyJSONParser.h>
|
||||
#include <Functions/SimdJSONParser.h>
|
||||
|
@ -4,7 +4,7 @@
|
||||
#include <Core/Defines.h>
|
||||
#include <DataTypes/IDataType.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <type_traits>
|
||||
|
||||
|
@ -9,7 +9,7 @@
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <common/StringRef.h>
|
||||
|
@ -9,7 +9,7 @@
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <common/StringRef.h>
|
||||
|
@ -9,7 +9,7 @@
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <common/StringRef.h>
|
||||
|
@ -2,7 +2,7 @@
|
||||
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Columns/ColumnVector.h>
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/TargetSpecific.h>
|
||||
#include <Functions/PerformanceAdaptors.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
|
@ -9,7 +9,7 @@
|
||||
#include <DataTypes/DataTypeDateTime64.h>
|
||||
#include <Columns/ColumnVector.h>
|
||||
#include <Interpreters/castColumn.h>
|
||||
#include "IFunctionOld.h"
|
||||
#include "IFunction.h"
|
||||
#include <Common/intExp.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Core/Defines.h>
|
||||
|
@ -9,7 +9,7 @@
|
||||
#include <Common/StringUtils/StringUtils.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/Regexps.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
|
@ -9,7 +9,7 @@
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Interpreters/Context_fwd.h>
|
||||
|
||||
namespace DB
|
||||
|
@ -8,7 +8,7 @@
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
|
||||
|
@ -9,7 +9,7 @@
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <common/StringRef.h>
|
||||
|
@ -5,7 +5,7 @@
|
||||
#include <Columns/ColumnVector.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Interpreters/Context_fwd.h>
|
||||
|
||||
namespace DB
|
||||
|
@ -5,7 +5,7 @@
|
||||
#include <DataTypes/DataTypeFixedString.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Common/Volnitsky.h>
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <IO/ReadBufferFromMemory.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
|
@ -16,6 +16,17 @@
|
||||
#include <cstdlib>
|
||||
#include <memory>
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
# include <Common/config.h>
|
||||
#endif
|
||||
|
||||
#if USE_EMBEDDED_COMPILER
|
||||
# pragma GCC diagnostic push
|
||||
# pragma GCC diagnostic ignored "-Wunused-parameter"
|
||||
# include <llvm/IR/IRBuilder.h>
|
||||
# pragma GCC diagnostic pop
|
||||
#endif
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
@ -342,4 +353,77 @@ DataTypePtr IFunctionOverloadResolver::getReturnTypeWithoutLowCardinality(const
|
||||
return getReturnTypeImpl(arguments);
|
||||
}
|
||||
|
||||
|
||||
#if USE_EMBEDDED_COMPILER
|
||||
|
||||
static std::optional<DataTypes> removeNullables(const DataTypes & types)
|
||||
{
|
||||
for (const auto & type : types)
|
||||
{
|
||||
if (!typeid_cast<const DataTypeNullable *>(type.get()))
|
||||
continue;
|
||||
DataTypes filtered;
|
||||
for (const auto & sub_type : types)
|
||||
filtered.emplace_back(removeNullable(sub_type));
|
||||
return filtered;
|
||||
}
|
||||
return {};
|
||||
}
|
||||
|
||||
bool IFunction::isCompilable(const DataTypes & arguments) const
|
||||
{
|
||||
if (useDefaultImplementationForNulls())
|
||||
if (auto denulled = removeNullables(arguments))
|
||||
return isCompilableImpl(*denulled);
|
||||
return isCompilableImpl(arguments);
|
||||
}
|
||||
|
||||
llvm::Value * IFunction::compile(llvm::IRBuilderBase & builder, const DataTypes & arguments, Values values) const
|
||||
{
|
||||
auto denulled_arguments = removeNullables(arguments);
|
||||
if (useDefaultImplementationForNulls() && denulled_arguments)
|
||||
{
|
||||
auto & b = static_cast<llvm::IRBuilder<> &>(builder);
|
||||
|
||||
std::vector<llvm::Value*> unwrapped_values;
|
||||
std::vector<llvm::Value*> is_null_values;
|
||||
|
||||
unwrapped_values.reserve(arguments.size());
|
||||
is_null_values.reserve(arguments.size());
|
||||
|
||||
for (size_t i = 0; i < arguments.size(); ++i)
|
||||
{
|
||||
auto * value = values[i];
|
||||
|
||||
WhichDataType data_type(arguments[i]);
|
||||
if (data_type.isNullable())
|
||||
{
|
||||
unwrapped_values.emplace_back(b.CreateExtractValue(value, {0}));
|
||||
is_null_values.emplace_back(b.CreateExtractValue(value, {1}));
|
||||
}
|
||||
else
|
||||
{
|
||||
unwrapped_values.emplace_back(value);
|
||||
}
|
||||
}
|
||||
|
||||
auto * result = compileImpl(builder, *denulled_arguments, unwrapped_values);
|
||||
|
||||
auto * nullable_structure_type = toNativeType(b, makeNullable(getReturnTypeImpl(*denulled_arguments)));
|
||||
auto * nullable_structure_value = llvm::Constant::getNullValue(nullable_structure_type);
|
||||
|
||||
auto * nullable_structure_with_result_value = b.CreateInsertValue(nullable_structure_value, result, {0});
|
||||
auto * nullable_structure_result_null = b.CreateExtractValue(nullable_structure_with_result_value, {1});
|
||||
|
||||
for (auto * is_null_value : is_null_values)
|
||||
nullable_structure_result_null = b.CreateOr(nullable_structure_result_null, is_null_value);
|
||||
|
||||
return b.CreateInsertValue(nullable_structure_with_result_value, nullable_structure_result_null, {1});
|
||||
}
|
||||
|
||||
return compileImpl(builder, arguments, std::move(values));
|
||||
}
|
||||
|
||||
#endif
|
||||
|
||||
}
|
||||
|
@ -326,4 +326,120 @@ private:
|
||||
|
||||
using FunctionOverloadResolverPtr = std::shared_ptr<IFunctionOverloadResolver>;
|
||||
|
||||
/// Old function interface. Check documentation in IFunction.h.
|
||||
/// If client do not need statefull properties it can implement this interface.
|
||||
class IFunction
|
||||
{
|
||||
public:
|
||||
|
||||
virtual ~IFunction() = default;
|
||||
|
||||
virtual String getName() const = 0;
|
||||
|
||||
virtual ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const = 0;
|
||||
virtual ColumnPtr executeImplDryRun(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const
|
||||
{
|
||||
return executeImpl(arguments, result_type, input_rows_count);
|
||||
}
|
||||
|
||||
/** Default implementation in presence of Nullable arguments or NULL constants as arguments is the following:
|
||||
* if some of arguments are NULL constants then return NULL constant,
|
||||
* if some of arguments are Nullable, then execute function as usual for columns,
|
||||
* where Nullable columns are substituted with nested columns (they have arbitrary values in rows corresponding to NULL value)
|
||||
* and wrap result in Nullable column where NULLs are in all rows where any of arguments are NULL.
|
||||
*/
|
||||
virtual bool useDefaultImplementationForNulls() const { return true; }
|
||||
|
||||
/** If the function have non-zero number of arguments,
|
||||
* and if all arguments are constant, that we could automatically provide default implementation:
|
||||
* arguments are converted to ordinary columns with single value, then function is executed as usual,
|
||||
* and then the result is converted to constant column.
|
||||
*/
|
||||
virtual bool useDefaultImplementationForConstants() const { return false; }
|
||||
|
||||
/** If function arguments has single low cardinality column and all other arguments are constants, call function on nested column.
|
||||
* Otherwise, convert all low cardinality columns to ordinary columns.
|
||||
* Returns ColumnLowCardinality if at least one argument is ColumnLowCardinality.
|
||||
*/
|
||||
virtual bool useDefaultImplementationForLowCardinalityColumns() const { return true; }
|
||||
|
||||
/// If it isn't, will convert all ColumnLowCardinality arguments to full columns.
|
||||
virtual bool canBeExecutedOnLowCardinalityDictionary() const { return true; }
|
||||
|
||||
/** Some arguments could remain constant during this implementation.
|
||||
*/
|
||||
virtual ColumnNumbers getArgumentsThatAreAlwaysConstant() const { return {}; }
|
||||
|
||||
/** True if function can be called on default arguments (include Nullable's) and won't throw.
|
||||
* Counterexample: modulo(0, 0)
|
||||
*/
|
||||
virtual bool canBeExecutedOnDefaultArguments() const { return true; }
|
||||
|
||||
/// Properties from IFunctionBase (see IFunction.h)
|
||||
virtual bool isSuitableForConstantFolding() const { return true; }
|
||||
virtual ColumnPtr getResultIfAlwaysReturnsConstantAndHasArguments(const ColumnsWithTypeAndName & /*arguments*/) const { return nullptr; }
|
||||
virtual bool isInjective(const ColumnsWithTypeAndName & /*sample_columns*/) const { return false; }
|
||||
virtual bool isDeterministic() const { return true; }
|
||||
virtual bool isDeterministicInScopeOfQuery() const { return true; }
|
||||
virtual bool isStateful() const { return false; }
|
||||
virtual bool hasInformationAboutMonotonicity() const { return false; }
|
||||
|
||||
using Monotonicity = IFunctionBase::Monotonicity;
|
||||
virtual Monotonicity getMonotonicityForRange(const IDataType & /*type*/, const Field & /*left*/, const Field & /*right*/) const
|
||||
{
|
||||
throw Exception("Function " + getName() + " has no information about its monotonicity.", ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
/// For non-variadic functions, return number of arguments; otherwise return zero (that should be ignored).
|
||||
virtual size_t getNumberOfArguments() const = 0;
|
||||
|
||||
virtual DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const
|
||||
{
|
||||
throw Exception("getReturnType is not implemented for " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
/// Get the result type by argument type. If the function does not apply to these arguments, throw an exception.
|
||||
virtual DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const
|
||||
{
|
||||
DataTypes data_types(arguments.size());
|
||||
for (size_t i = 0; i < arguments.size(); ++i)
|
||||
data_types[i] = arguments[i].type;
|
||||
|
||||
return getReturnTypeImpl(data_types);
|
||||
}
|
||||
|
||||
virtual bool isVariadic() const { return false; }
|
||||
|
||||
virtual void getLambdaArgumentTypes(DataTypes & /*arguments*/) const
|
||||
{
|
||||
throw Exception("Function " + getName() + " can't have lambda-expressions as arguments", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
|
||||
virtual ColumnNumbers getArgumentsThatDontImplyNullableReturnType(size_t /*number_of_arguments*/) const { return {}; }
|
||||
|
||||
|
||||
#if USE_EMBEDDED_COMPILER
|
||||
|
||||
bool isCompilable(const DataTypes & arguments) const;
|
||||
|
||||
llvm::Value * compile(llvm::IRBuilderBase &, const DataTypes & arguments, Values values) const;
|
||||
|
||||
#endif
|
||||
|
||||
protected:
|
||||
|
||||
#if USE_EMBEDDED_COMPILER
|
||||
|
||||
virtual bool isCompilableImpl(const DataTypes &) const { return false; }
|
||||
|
||||
virtual llvm::Value * compileImpl(llvm::IRBuilderBase &, const DataTypes &, Values) const
|
||||
{
|
||||
throw Exception(getName() + " is not JIT-compilable", ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
#endif
|
||||
};
|
||||
|
||||
using FunctionPtr = std::shared_ptr<IFunction>;
|
||||
|
||||
}
|
||||
|
@ -1,6 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -1,91 +0,0 @@
|
||||
#include "IFunctionOld.h"
|
||||
|
||||
#include <DataTypes/Native.h>
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
# include <Common/config.h>
|
||||
#endif
|
||||
|
||||
#if USE_EMBEDDED_COMPILER
|
||||
# pragma GCC diagnostic push
|
||||
# pragma GCC diagnostic ignored "-Wunused-parameter"
|
||||
# include <llvm/IR/IRBuilder.h>
|
||||
# pragma GCC diagnostic pop
|
||||
#endif
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
#if USE_EMBEDDED_COMPILER
|
||||
|
||||
static std::optional<DataTypes> removeNullables(const DataTypes & types)
|
||||
{
|
||||
for (const auto & type : types)
|
||||
{
|
||||
if (!typeid_cast<const DataTypeNullable *>(type.get()))
|
||||
continue;
|
||||
DataTypes filtered;
|
||||
for (const auto & sub_type : types)
|
||||
filtered.emplace_back(removeNullable(sub_type));
|
||||
return filtered;
|
||||
}
|
||||
return {};
|
||||
}
|
||||
|
||||
bool IFunction::isCompilable(const DataTypes & arguments) const
|
||||
{
|
||||
if (useDefaultImplementationForNulls())
|
||||
if (auto denulled = removeNullables(arguments))
|
||||
return isCompilableImpl(*denulled);
|
||||
return isCompilableImpl(arguments);
|
||||
}
|
||||
|
||||
llvm::Value * IFunction::compile(llvm::IRBuilderBase & builder, const DataTypes & arguments, Values values) const
|
||||
{
|
||||
auto denulled_arguments = removeNullables(arguments);
|
||||
if (useDefaultImplementationForNulls() && denulled_arguments)
|
||||
{
|
||||
auto & b = static_cast<llvm::IRBuilder<> &>(builder);
|
||||
|
||||
std::vector<llvm::Value*> unwrapped_values;
|
||||
std::vector<llvm::Value*> is_null_values;
|
||||
|
||||
unwrapped_values.reserve(arguments.size());
|
||||
is_null_values.reserve(arguments.size());
|
||||
|
||||
for (size_t i = 0; i < arguments.size(); ++i)
|
||||
{
|
||||
auto * value = values[i];
|
||||
|
||||
WhichDataType data_type(arguments[i]);
|
||||
if (data_type.isNullable())
|
||||
{
|
||||
unwrapped_values.emplace_back(b.CreateExtractValue(value, {0}));
|
||||
is_null_values.emplace_back(b.CreateExtractValue(value, {1}));
|
||||
}
|
||||
else
|
||||
{
|
||||
unwrapped_values.emplace_back(value);
|
||||
}
|
||||
}
|
||||
|
||||
auto * result = compileImpl(builder, *denulled_arguments, unwrapped_values);
|
||||
|
||||
auto * nullable_structure_type = toNativeType(b, makeNullable(getReturnTypeImpl(*denulled_arguments)));
|
||||
auto * nullable_structure_value = llvm::Constant::getNullValue(nullable_structure_type);
|
||||
|
||||
auto * nullable_structure_with_result_value = b.CreateInsertValue(nullable_structure_value, result, {0});
|
||||
auto * nullable_structure_result_null = b.CreateExtractValue(nullable_structure_with_result_value, {1});
|
||||
|
||||
for (auto * is_null_value : is_null_values)
|
||||
nullable_structure_result_null = b.CreateOr(nullable_structure_result_null, is_null_value);
|
||||
|
||||
return b.CreateInsertValue(nullable_structure_with_result_value, nullable_structure_result_null, {1});
|
||||
}
|
||||
|
||||
return compileImpl(builder, arguments, std::move(values));
|
||||
}
|
||||
|
||||
#endif
|
||||
|
||||
}
|
@ -1,129 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <Functions/IFunction.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
extern const int NOT_IMPLEMENTED;
|
||||
}
|
||||
|
||||
/// Old function interface. Check documentation in IFunction.h
|
||||
class IFunction
|
||||
{
|
||||
public:
|
||||
|
||||
virtual ~IFunction() = default;
|
||||
|
||||
virtual String getName() const = 0;
|
||||
|
||||
virtual ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const = 0;
|
||||
virtual ColumnPtr executeImplDryRun(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const
|
||||
{
|
||||
return executeImpl(arguments, result_type, input_rows_count);
|
||||
}
|
||||
|
||||
/** Default implementation in presence of Nullable arguments or NULL constants as arguments is the following:
|
||||
* if some of arguments are NULL constants then return NULL constant,
|
||||
* if some of arguments are Nullable, then execute function as usual for columns,
|
||||
* where Nullable columns are substituted with nested columns (they have arbitrary values in rows corresponding to NULL value)
|
||||
* and wrap result in Nullable column where NULLs are in all rows where any of arguments are NULL.
|
||||
*/
|
||||
virtual bool useDefaultImplementationForNulls() const { return true; }
|
||||
|
||||
/** If the function have non-zero number of arguments,
|
||||
* and if all arguments are constant, that we could automatically provide default implementation:
|
||||
* arguments are converted to ordinary columns with single value, then function is executed as usual,
|
||||
* and then the result is converted to constant column.
|
||||
*/
|
||||
virtual bool useDefaultImplementationForConstants() const { return false; }
|
||||
|
||||
/** If function arguments has single low cardinality column and all other arguments are constants, call function on nested column.
|
||||
* Otherwise, convert all low cardinality columns to ordinary columns.
|
||||
* Returns ColumnLowCardinality if at least one argument is ColumnLowCardinality.
|
||||
*/
|
||||
virtual bool useDefaultImplementationForLowCardinalityColumns() const { return true; }
|
||||
|
||||
/// If it isn't, will convert all ColumnLowCardinality arguments to full columns.
|
||||
virtual bool canBeExecutedOnLowCardinalityDictionary() const { return true; }
|
||||
|
||||
/** Some arguments could remain constant during this implementation.
|
||||
*/
|
||||
virtual ColumnNumbers getArgumentsThatAreAlwaysConstant() const { return {}; }
|
||||
|
||||
/** True if function can be called on default arguments (include Nullable's) and won't throw.
|
||||
* Counterexample: modulo(0, 0)
|
||||
*/
|
||||
virtual bool canBeExecutedOnDefaultArguments() const { return true; }
|
||||
|
||||
/// Properties from IFunctionBase (see IFunction.h)
|
||||
virtual bool isSuitableForConstantFolding() const { return true; }
|
||||
virtual ColumnPtr getResultIfAlwaysReturnsConstantAndHasArguments(const ColumnsWithTypeAndName & /*arguments*/) const { return nullptr; }
|
||||
virtual bool isInjective(const ColumnsWithTypeAndName & /*sample_columns*/) const { return false; }
|
||||
virtual bool isDeterministic() const { return true; }
|
||||
virtual bool isDeterministicInScopeOfQuery() const { return true; }
|
||||
virtual bool isStateful() const { return false; }
|
||||
virtual bool hasInformationAboutMonotonicity() const { return false; }
|
||||
|
||||
using Monotonicity = IFunctionBase::Monotonicity;
|
||||
virtual Monotonicity getMonotonicityForRange(const IDataType & /*type*/, const Field & /*left*/, const Field & /*right*/) const
|
||||
{
|
||||
throw Exception("Function " + getName() + " has no information about its monotonicity.", ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
/// For non-variadic functions, return number of arguments; otherwise return zero (that should be ignored).
|
||||
virtual size_t getNumberOfArguments() const = 0;
|
||||
|
||||
virtual DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const
|
||||
{
|
||||
throw Exception("getReturnType is not implemented for " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
/// Get the result type by argument type. If the function does not apply to these arguments, throw an exception.
|
||||
virtual DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const
|
||||
{
|
||||
DataTypes data_types(arguments.size());
|
||||
for (size_t i = 0; i < arguments.size(); ++i)
|
||||
data_types[i] = arguments[i].type;
|
||||
|
||||
return getReturnTypeImpl(data_types);
|
||||
}
|
||||
|
||||
virtual bool isVariadic() const { return false; }
|
||||
|
||||
virtual void getLambdaArgumentTypes(DataTypes & /*arguments*/) const
|
||||
{
|
||||
throw Exception("Function " + getName() + " can't have lambda-expressions as arguments", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
|
||||
virtual ColumnNumbers getArgumentsThatDontImplyNullableReturnType(size_t /*number_of_arguments*/) const { return {}; }
|
||||
|
||||
|
||||
#if USE_EMBEDDED_COMPILER
|
||||
|
||||
bool isCompilable(const DataTypes & arguments) const;
|
||||
|
||||
llvm::Value * compile(llvm::IRBuilderBase &, const DataTypes & arguments, Values values) const;
|
||||
|
||||
#endif
|
||||
|
||||
protected:
|
||||
|
||||
#if USE_EMBEDDED_COMPILER
|
||||
|
||||
virtual bool isCompilableImpl(const DataTypes &) const { return false; }
|
||||
|
||||
virtual llvm::Value * compileImpl(llvm::IRBuilderBase &, const DataTypes &, Values) const
|
||||
{
|
||||
throw Exception(getName() + " is not JIT-compilable", ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
#endif
|
||||
};
|
||||
|
||||
using FunctionPtr = std::shared_ptr<IFunction>;
|
||||
|
||||
}
|
@ -4,7 +4,7 @@
|
||||
#include <DataTypes/NumberTraits.h>
|
||||
#include <Interpreters/castColumn.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <ext/map.h>
|
||||
|
||||
|
@ -1,7 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <Functions/TargetSpecific.h>
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
|
||||
#include <Common/Stopwatch.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
@ -1,5 +1,5 @@
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Common/StringUtils/StringUtils.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
|
@ -7,7 +7,7 @@
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <IO/WriteBufferFromArena.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
|
@ -4,7 +4,7 @@
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Access/AccessFlags.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
@ -3,7 +3,7 @@
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <ext/range.h>
|
||||
|
||||
|
||||
|
@ -8,7 +8,7 @@
|
||||
#include <Columns/ColumnFunction.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Interpreters/Context_fwd.h>
|
||||
|
@ -1,4 +1,4 @@
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/GatherUtils/GatherUtils.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
|
@ -1,4 +1,4 @@
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
|
@ -1,4 +1,4 @@
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
|
@ -1,4 +1,4 @@
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
|
@ -1,5 +1,5 @@
|
||||
#pragma once
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
|
@ -7,7 +7,7 @@
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/getLeastSupertype.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Interpreters/AggregationCommon.h>
|
||||
#include <Common/ColumnsHashing.h>
|
||||
#include <Common/HashTable/ClearableHashMap.h>
|
||||
|
@ -1,4 +1,4 @@
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
|
@ -1,5 +1,5 @@
|
||||
#pragma once
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
|
@ -1,4 +1,4 @@
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
|
@ -1,4 +1,4 @@
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
|
@ -1,5 +1,5 @@
|
||||
#pragma once
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/GatherUtils/GatherUtils.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
|
@ -1,5 +1,5 @@
|
||||
#pragma once
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/GatherUtils/GatherUtils.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/getLeastSupertype.h>
|
||||
|
@ -1,4 +1,4 @@
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
|
@ -1,4 +1,4 @@
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
|
@ -1,4 +1,4 @@
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/GatherUtils/GatherUtils.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
|
@ -1,4 +1,4 @@
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
|
@ -1,4 +1,4 @@
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/GatherUtils/GatherUtils.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
|
@ -1,4 +1,4 @@
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
|
@ -1,4 +1,4 @@
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
|
@ -1,4 +1,4 @@
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
|
@ -1,4 +1,4 @@
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
|
@ -1,5 +1,5 @@
|
||||
#pragma once
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/GatherUtils/GatherUtils.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
|
@ -1,4 +1,4 @@
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
|
@ -1,4 +1,4 @@
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <Core/ColumnNumbers.h>
|
||||
|
@ -1,4 +1,4 @@
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
|
@ -1,5 +1,5 @@
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnVector.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
|
@ -1,4 +1,4 @@
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
|
@ -1,4 +1,4 @@
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <IO/NullWriteBuffer.h>
|
||||
|
@ -1,4 +1,4 @@
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
|
@ -1,6 +1,6 @@
|
||||
#if defined(__ELF__) && !defined(__FreeBSD__)
|
||||
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <Common/SymbolIndex.h>
|
||||
|
@ -2,7 +2,7 @@
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -7,7 +7,7 @@
|
||||
#include <DataTypes/getLeastSupertype.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -7,7 +7,7 @@
|
||||
#include <Functions/GatherUtils/Sinks.h>
|
||||
#include <Functions/GatherUtils/Slices.h>
|
||||
#include <Functions/GatherUtils/Sources.h>
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <ext/map.h>
|
||||
#include <ext/range.h>
|
||||
|
@ -1,6 +1,6 @@
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
|
||||
|
@ -1,4 +1,4 @@
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
|
@ -1,6 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
|
@ -1,4 +1,4 @@
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
|
@ -1,4 +1,4 @@
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
|
@ -5,7 +5,7 @@
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Columns/ColumnDecimal.h>
|
||||
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/extractTimeZoneFromFunctionArguments.h>
|
||||
|
@ -1,4 +1,4 @@
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Core/Field.h>
|
||||
|
||||
|
@ -1,4 +1,4 @@
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Core/Field.h>
|
||||
#include <Columns/ColumnConst.h>
|
||||
|
@ -1,4 +1,4 @@
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <Core/Field.h>
|
||||
|
@ -1,4 +1,4 @@
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <DataTypes/DataTypeLowCardinality.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
|
@ -1,4 +1,4 @@
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <DataTypes/DataTypeAggregateFunction.h>
|
||||
|
@ -5,7 +5,7 @@
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/Regexps.h>
|
||||
|
||||
#include <memory>
|
||||
|
@ -1,7 +1,7 @@
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/IFunctionOld.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <common/find_symbols.h>
|
||||
#include <Common/StringUtils/StringUtils.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