mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 08:02:02 +00:00
Cleanup code
This commit is contained in:
parent
4d5868d05d
commit
15b8c48ca9
@ -74,7 +74,7 @@ namespace
|
||||
/// groupBitmap needs to know about the data type that was used to create bitmaps.
|
||||
/// We need to look inside the type of its argument to obtain it.
|
||||
const DataTypeAggregateFunction & datatype_aggfunc = dynamic_cast<const DataTypeAggregateFunction &>(*argument_type_ptr);
|
||||
ConstAggregateFunctionPtr aggfunc = datatype_aggfunc.getFunction();
|
||||
AggregateFunctionPtr aggfunc = datatype_aggfunc.getFunction();
|
||||
|
||||
if (aggfunc->getName() != AggregateFunctionGroupBitmapData<UInt8>::name())
|
||||
throw Exception(
|
||||
|
@ -49,8 +49,7 @@ using AggregateDataPtr = char *;
|
||||
using ConstAggregateDataPtr = const char *;
|
||||
|
||||
class IAggregateFunction;
|
||||
using AggregateFunctionPtr = std::shared_ptr<IAggregateFunction>;
|
||||
using ConstAggregateFunctionPtr = std::shared_ptr<const IAggregateFunction>;
|
||||
using AggregateFunctionPtr = std::shared_ptr<const IAggregateFunction>;
|
||||
|
||||
struct AggregateFunctionProperties;
|
||||
|
||||
|
@ -40,12 +40,12 @@ ColumnsWithTypeAndName FunctionNode::getArgumentTypes() const
|
||||
|
||||
FunctionBasePtr FunctionNode::getFunction() const
|
||||
{
|
||||
return std::dynamic_pointer_cast<IFunctionBase>(function);
|
||||
return std::dynamic_pointer_cast<const IFunctionBase>(function);
|
||||
}
|
||||
|
||||
AggregateFunctionPtr FunctionNode::getAggregateFunction() const
|
||||
{
|
||||
return std::dynamic_pointer_cast<IAggregateFunction>(function);
|
||||
return std::dynamic_pointer_cast<const IAggregateFunction>(function);
|
||||
}
|
||||
|
||||
bool FunctionNode::isAggregateFunction() const
|
||||
|
@ -15,10 +15,10 @@ class IFunctionOverloadResolver;
|
||||
using FunctionOverloadResolverPtr = std::shared_ptr<IFunctionOverloadResolver>;
|
||||
|
||||
class IFunctionBase;
|
||||
using FunctionBasePtr = std::shared_ptr<IFunctionBase>;
|
||||
using FunctionBasePtr = std::shared_ptr<const IFunctionBase>;
|
||||
|
||||
class IAggregateFunction;
|
||||
using AggregateFunctionPtr = std::shared_ptr<IAggregateFunction>;
|
||||
using AggregateFunctionPtr = std::shared_ptr<const IAggregateFunction>;
|
||||
|
||||
/** Function node represents function in query tree.
|
||||
* Function syntax: function_name(parameter_1, ...)(argument_1, ...).
|
||||
|
@ -28,7 +28,7 @@ namespace ErrorCodes
|
||||
}
|
||||
|
||||
|
||||
static String getTypeString(const ConstAggregateFunctionPtr & func, std::optional<size_t> version = std::nullopt)
|
||||
static String getTypeString(const AggregateFunctionPtr & func, std::optional<size_t> version = std::nullopt)
|
||||
{
|
||||
WriteBufferFromOwnString stream;
|
||||
|
||||
@ -62,18 +62,18 @@ static String getTypeString(const ConstAggregateFunctionPtr & func, std::optiona
|
||||
}
|
||||
|
||||
|
||||
ColumnAggregateFunction::ColumnAggregateFunction(const ConstAggregateFunctionPtr & func_, std::optional<size_t> version_)
|
||||
ColumnAggregateFunction::ColumnAggregateFunction(const AggregateFunctionPtr & func_, std::optional<size_t> version_)
|
||||
: func(func_), type_string(getTypeString(func, version_)), version(version_)
|
||||
{
|
||||
}
|
||||
|
||||
ColumnAggregateFunction::ColumnAggregateFunction(const ConstAggregateFunctionPtr & func_, const ConstArenas & arenas_)
|
||||
ColumnAggregateFunction::ColumnAggregateFunction(const AggregateFunctionPtr & func_, const ConstArenas & arenas_)
|
||||
: foreign_arenas(arenas_), func(func_), type_string(getTypeString(func))
|
||||
{
|
||||
|
||||
}
|
||||
|
||||
void ColumnAggregateFunction::set(const ConstAggregateFunctionPtr & func_, size_t version_)
|
||||
void ColumnAggregateFunction::set(const AggregateFunctionPtr & func_, size_t version_)
|
||||
{
|
||||
func = func_;
|
||||
version = version_;
|
||||
|
@ -70,7 +70,7 @@ private:
|
||||
ArenaPtr my_arena;
|
||||
|
||||
/// Used for destroying states and for finalization of values.
|
||||
ConstAggregateFunctionPtr func;
|
||||
AggregateFunctionPtr func;
|
||||
|
||||
/// Source column. Used (holds source from destruction),
|
||||
/// if this column has been constructed from another and uses all or part of its values.
|
||||
@ -92,9 +92,9 @@ private:
|
||||
/// Create a new column that has another column as a source.
|
||||
MutablePtr createView() const;
|
||||
|
||||
explicit ColumnAggregateFunction(const ConstAggregateFunctionPtr & func_, std::optional<size_t> version_ = std::nullopt);
|
||||
explicit ColumnAggregateFunction(const AggregateFunctionPtr & func_, std::optional<size_t> version_ = std::nullopt);
|
||||
|
||||
ColumnAggregateFunction(const ConstAggregateFunctionPtr & func_, const ConstArenas & arenas_);
|
||||
ColumnAggregateFunction(const AggregateFunctionPtr & func_, const ConstArenas & arenas_);
|
||||
|
||||
ColumnAggregateFunction(const ColumnAggregateFunction & src_);
|
||||
|
||||
@ -103,10 +103,10 @@ private:
|
||||
public:
|
||||
~ColumnAggregateFunction() override;
|
||||
|
||||
void set(const ConstAggregateFunctionPtr & func_, size_t version_);
|
||||
void set(const AggregateFunctionPtr & func_, size_t version_);
|
||||
|
||||
ConstAggregateFunctionPtr getAggregateFunction() { return func; }
|
||||
ConstAggregateFunctionPtr getAggregateFunction() const { return func; }
|
||||
AggregateFunctionPtr getAggregateFunction() { return func; }
|
||||
AggregateFunctionPtr getAggregateFunction() const { return func; }
|
||||
|
||||
/// If we have another column as a source (owner of data), copy all data to ourself and reset source.
|
||||
/// This is needed before inserting new elements, because we must own these elements (to destroy them in destructor),
|
||||
|
@ -13,7 +13,7 @@ namespace ErrorCodes
|
||||
}
|
||||
|
||||
class IFunctionBase;
|
||||
using FunctionBasePtr = std::shared_ptr<IFunctionBase>;
|
||||
using FunctionBasePtr = std::shared_ptr<const IFunctionBase>;
|
||||
|
||||
/** A column containing a lambda expression.
|
||||
* Behaves like a constant-column. Contains an expression, but not input or output data.
|
||||
|
@ -24,6 +24,6 @@ public:
|
||||
virtual ~IResolvedFunction() = default;
|
||||
};
|
||||
|
||||
using IResolvedFunctionPtr = std::shared_ptr<IResolvedFunction>;
|
||||
using IResolvedFunctionPtr = std::shared_ptr<const IResolvedFunction>;
|
||||
|
||||
}
|
||||
|
@ -19,7 +19,7 @@ namespace DB
|
||||
class DataTypeAggregateFunction final : public IDataType
|
||||
{
|
||||
private:
|
||||
ConstAggregateFunctionPtr function;
|
||||
AggregateFunctionPtr function;
|
||||
DataTypes argument_types;
|
||||
Array parameters;
|
||||
mutable std::optional<size_t> version;
|
||||
@ -30,7 +30,7 @@ private:
|
||||
public:
|
||||
static constexpr bool is_parametric = true;
|
||||
|
||||
DataTypeAggregateFunction(ConstAggregateFunctionPtr function_, const DataTypes & argument_types_,
|
||||
DataTypeAggregateFunction(AggregateFunctionPtr function_, const DataTypes & argument_types_,
|
||||
const Array & parameters_, std::optional<size_t> version_ = std::nullopt)
|
||||
: function(std::move(function_))
|
||||
, argument_types(argument_types_)
|
||||
@ -40,7 +40,7 @@ public:
|
||||
}
|
||||
|
||||
String getFunctionName() const { return function->getName(); }
|
||||
ConstAggregateFunctionPtr getFunction() const { return function; }
|
||||
AggregateFunctionPtr getFunction() const { return function; }
|
||||
|
||||
String doGetName() const override;
|
||||
String getNameWithoutVersion() const;
|
||||
|
@ -108,14 +108,14 @@ void SerializationAggregateFunction::deserializeBinaryBulk(IColumn & column, Rea
|
||||
}
|
||||
}
|
||||
|
||||
static String serializeToString(const ConstAggregateFunctionPtr & function, const IColumn & column, size_t row_num, size_t version)
|
||||
static String serializeToString(const AggregateFunctionPtr & function, const IColumn & column, size_t row_num, size_t version)
|
||||
{
|
||||
WriteBufferFromOwnString buffer;
|
||||
function->serialize(assert_cast<const ColumnAggregateFunction &>(column).getData()[row_num], buffer, version);
|
||||
return buffer.str();
|
||||
}
|
||||
|
||||
static void deserializeFromString(const ConstAggregateFunctionPtr & function, IColumn & column, const String & s, size_t version)
|
||||
static void deserializeFromString(const AggregateFunctionPtr & function, IColumn & column, const String & s, size_t version)
|
||||
{
|
||||
ColumnAggregateFunction & column_concrete = assert_cast<ColumnAggregateFunction &>(column);
|
||||
|
||||
|
@ -11,14 +11,14 @@ namespace DB
|
||||
class SerializationAggregateFunction final : public ISerialization
|
||||
{
|
||||
private:
|
||||
ConstAggregateFunctionPtr function;
|
||||
AggregateFunctionPtr function;
|
||||
String type_name;
|
||||
size_t version;
|
||||
|
||||
public:
|
||||
static constexpr bool is_parametric = true;
|
||||
|
||||
SerializationAggregateFunction(const ConstAggregateFunctionPtr & function_, String type_name_, size_t version_)
|
||||
SerializationAggregateFunction(const AggregateFunctionPtr & function_, String type_name_, size_t version_)
|
||||
: function(function_), type_name(std::move(type_name_)), version(version_) {}
|
||||
|
||||
/// NOTE These two functions for serializing single values are incompatible with the functions below.
|
||||
|
@ -1736,7 +1736,7 @@ namespace
|
||||
}
|
||||
|
||||
const std::shared_ptr<const DataTypeAggregateFunction> aggregate_function_data_type;
|
||||
ConstAggregateFunctionPtr aggregate_function;
|
||||
AggregateFunctionPtr aggregate_function;
|
||||
String text_buffer;
|
||||
};
|
||||
|
||||
|
@ -895,7 +895,7 @@ class FunctionBinaryArithmetic : public IFunction
|
||||
const ColumnAggregateFunction & column = typeid_cast<const ColumnAggregateFunction &>(
|
||||
agg_state_is_const ? assert_cast<const ColumnConst &>(agg_state_column).getDataColumn() : agg_state_column);
|
||||
|
||||
ConstAggregateFunctionPtr function = column.getAggregateFunction();
|
||||
AggregateFunctionPtr function = column.getAggregateFunction();
|
||||
|
||||
size_t size = agg_state_is_const ? 1 : input_rows_count;
|
||||
|
||||
@ -960,7 +960,7 @@ class FunctionBinaryArithmetic : public IFunction
|
||||
const ColumnAggregateFunction & rhs = typeid_cast<const ColumnAggregateFunction &>(
|
||||
rhs_is_const ? assert_cast<const ColumnConst &>(rhs_column).getDataColumn() : rhs_column);
|
||||
|
||||
ConstAggregateFunctionPtr function = lhs.getAggregateFunction();
|
||||
AggregateFunctionPtr function = lhs.getAggregateFunction();
|
||||
|
||||
size_t size = (lhs_is_const && rhs_is_const) ? 1 : input_rows_count;
|
||||
|
||||
|
@ -285,7 +285,7 @@ public:
|
||||
|
||||
};
|
||||
|
||||
using FunctionBasePtr = std::shared_ptr<IFunctionBase>;
|
||||
using FunctionBasePtr = std::shared_ptr<const IFunctionBase>;
|
||||
|
||||
|
||||
/** Creates IFunctionBase from argument types list (chooses one function overload).
|
||||
|
@ -91,7 +91,7 @@ public:
|
||||
if (arguments.size() == 2)
|
||||
column_with_groups = arguments[1].column;
|
||||
|
||||
ConstAggregateFunctionPtr aggregate_function_ptr = column_with_states->getAggregateFunction();
|
||||
AggregateFunctionPtr aggregate_function_ptr = column_with_states->getAggregateFunction();
|
||||
const IAggregateFunction & agg_func = *aggregate_function_ptr;
|
||||
|
||||
AlignedBuffer place(agg_func.sizeOfData(), agg_func.alignOfData());
|
||||
|
@ -17,7 +17,7 @@ class IExecutableFunction;
|
||||
using ExecutableFunctionPtr = std::shared_ptr<IExecutableFunction>;
|
||||
|
||||
class IFunctionBase;
|
||||
using FunctionBasePtr = std::shared_ptr<IFunctionBase>;
|
||||
using FunctionBasePtr = std::shared_ptr<const IFunctionBase>;
|
||||
|
||||
class IFunctionOverloadResolver;
|
||||
using FunctionOverloadResolverPtr = std::shared_ptr<IFunctionOverloadResolver>;
|
||||
|
@ -263,7 +263,7 @@ public:
|
||||
return result;
|
||||
}
|
||||
|
||||
static void applyFunction(IFunctionBase & function, Field & value)
|
||||
static void applyFunction(const IFunctionBase & function, Field & value)
|
||||
{
|
||||
const auto & type = function.getArgumentTypes().at(0);
|
||||
ColumnsWithTypeAndName args{{type->createColumnConst(1, value), type, "x" }};
|
||||
|
@ -18,7 +18,7 @@ struct Range;
|
||||
|
||||
class Context;
|
||||
class IFunctionBase;
|
||||
using FunctionBasePtr = std::shared_ptr<IFunctionBase>;
|
||||
using FunctionBasePtr = std::shared_ptr<const IFunctionBase>;
|
||||
|
||||
class Chunk;
|
||||
|
||||
|
@ -945,7 +945,7 @@ bool KeyCondition::transformConstantWithValidFunctions(
|
||||
DataTypePtr & out_key_column_type,
|
||||
Field & out_value,
|
||||
DataTypePtr & out_type,
|
||||
std::function<bool(IFunctionBase &, const IDataType &)> always_monotonic) const
|
||||
std::function<bool(const IFunctionBase &, const IDataType &)> always_monotonic) const
|
||||
{
|
||||
const auto & sample_block = key_expr->getSampleBlock();
|
||||
|
||||
@ -1076,7 +1076,7 @@ bool KeyCondition::canConstantBeWrappedByMonotonicFunctions(
|
||||
out_key_column_type,
|
||||
out_value,
|
||||
out_type,
|
||||
[](IFunctionBase & func, const IDataType & type)
|
||||
[](const IFunctionBase & func, const IDataType & type)
|
||||
{
|
||||
if (!func.hasInformationAboutMonotonicity())
|
||||
return false;
|
||||
@ -1131,7 +1131,7 @@ bool KeyCondition::canConstantBeWrappedByFunctions(
|
||||
out_key_column_type,
|
||||
out_value,
|
||||
out_type,
|
||||
[](IFunctionBase & func, const IDataType &)
|
||||
[](const IFunctionBase & func, const IDataType &)
|
||||
{
|
||||
return func.isDeterministic();
|
||||
});
|
||||
|
@ -19,7 +19,7 @@ namespace DB
|
||||
class ASTFunction;
|
||||
class Context;
|
||||
class IFunction;
|
||||
using FunctionBasePtr = std::shared_ptr<IFunctionBase>;
|
||||
using FunctionBasePtr = std::shared_ptr<const IFunctionBase>;
|
||||
class ExpressionActions;
|
||||
using ExpressionActionsPtr = std::shared_ptr<ExpressionActions>;
|
||||
struct ActionDAGNodes;
|
||||
@ -427,7 +427,7 @@ private:
|
||||
DataTypePtr & out_key_column_type,
|
||||
Field & out_value,
|
||||
DataTypePtr & out_type,
|
||||
std::function<bool(IFunctionBase &, const IDataType &)> always_monotonic) const;
|
||||
std::function<bool(const IFunctionBase &, const IDataType &)> always_monotonic) const;
|
||||
|
||||
bool canConstantBeWrappedByMonotonicFunctions(
|
||||
const RPNBuilderTreeNode & node,
|
||||
|
@ -61,7 +61,7 @@ void checkTTLExpression(const ExpressionActionsPtr & ttl_expression, const Strin
|
||||
{
|
||||
if (action.node->type == ActionsDAG::ActionType::FUNCTION)
|
||||
{
|
||||
IFunctionBase & func = *action.node->function_base;
|
||||
const IFunctionBase & func = *action.node->function_base;
|
||||
if (!func.isDeterministic())
|
||||
throw Exception(
|
||||
"TTL expression cannot contain non-deterministic functions, "
|
||||
|
Loading…
Reference in New Issue
Block a user