mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-03 13:02:00 +00:00
fix tests
This commit is contained in:
parent
6395a8530f
commit
c8ab6b2de2
@ -44,6 +44,10 @@ The function is not applied to columns that are in primary key.
|
||||
|
||||
If `default_aggregate_function` is not specified for a column and the column's type is not SimpleAggregateFunction or AggregateFunction, then any of values in the column within the same primary key is taken. This value can be NULL.
|
||||
|
||||
:::warning
|
||||
When using this parameter, no check is performed on table creation to verify whether the specified function accepts the data types represented in the columns.
|
||||
If the function is unable to work with the data type, an exception will be thrown only during data insertion or the next merge.
|
||||
:::
|
||||
|
||||
**Query clauses**
|
||||
|
||||
|
@ -41,6 +41,11 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
|
||||
|
||||
Если `default_aggregate_function` не указан для колонки и тип колонки не AggregateFunction и не SimpleAggregateFunction, то в качестве значения будет взято любое (включая NULL) из значений в столбце в рамках того же первичного ключа.
|
||||
|
||||
:::warning
|
||||
При использовании этого параметра при создании таблицы не производится проверка, принимает ли указанная функция типы данных, представленные в столбцах.
|
||||
Если функция не может работать с типом данных, исключение будет выброшено лишь во время вставки данных или при следующем мерже.
|
||||
:::
|
||||
|
||||
**Секции запроса**
|
||||
|
||||
При создании таблицы `AggregatingMergeTree` используются те же [секции](mergetree.md), что и при создании таблицы `MergeTree`.
|
||||
|
@ -11,6 +11,7 @@ namespace DB
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int BAD_ARGUMENTS;
|
||||
}
|
||||
|
||||
AggregatingSortedAlgorithm::ColumnsDefinition::ColumnsDefinition() = default;
|
||||
@ -18,7 +19,7 @@ AggregatingSortedAlgorithm::ColumnsDefinition::ColumnsDefinition(ColumnsDefiniti
|
||||
AggregatingSortedAlgorithm::ColumnsDefinition::~ColumnsDefinition() = default;
|
||||
|
||||
static AggregatingSortedAlgorithm::ColumnsDefinition defineColumns(
|
||||
const Block & header, const SortDescription & description, const String & default_aggregate_function = "")
|
||||
const Block & header, const SortDescription & description, const String & default_aggregate_function = "any")
|
||||
{
|
||||
AggregatingSortedAlgorithm::ColumnsDefinition def = {};
|
||||
size_t num_columns = header.columns();
|
||||
@ -62,22 +63,30 @@ static AggregatingSortedAlgorithm::ColumnsDefinition defineColumns(
|
||||
|
||||
def.columns_to_simple_aggregate.emplace_back(std::move(desc));
|
||||
}
|
||||
else if (dynamic_cast<const DataTypeAggregateFunction *>(column.type->getCustomName()))
|
||||
{
|
||||
// standard aggregate function
|
||||
def.columns_to_aggregate.emplace_back(i);
|
||||
}
|
||||
else
|
||||
else if (!default_aggregate_function.empty())
|
||||
{
|
||||
AggregateFunctionProperties properties;
|
||||
auto type = recursiveRemoveLowCardinality(column.type);
|
||||
const auto & func = AggregateFunctionFactory::instance().get(default_aggregate_function, NullsAction::EMPTY, {column.type}, {}, properties);
|
||||
|
||||
if (!func->getResultType()->equals(*removeLowCardinality(column.type)))
|
||||
{
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Incompatible data types between aggregate function '{}' "
|
||||
"which returns {} and column storage type {}",
|
||||
func->getName(), func->getResultType()->getName(), column.type->getName());
|
||||
}
|
||||
|
||||
AggregatingSortedAlgorithm::SimpleAggregateDescription desc(func, i, type.get() == column.type.get() ? nullptr : type, column.type);
|
||||
if (desc.function->allocatesMemoryInArena())
|
||||
def.allocates_memory_in_arena = true;
|
||||
|
||||
def.columns_to_simple_aggregate.emplace_back(std::move(desc));
|
||||
}
|
||||
else
|
||||
{
|
||||
// standard aggregate function
|
||||
def.columns_to_aggregate.emplace_back(i);
|
||||
}
|
||||
}
|
||||
|
||||
return def;
|
||||
|
@ -456,7 +456,7 @@ static StoragePtr create(const StorageFactory::Arguments & args)
|
||||
add_mandatory_param("sign column");
|
||||
break;
|
||||
case MergeTreeData::MergingParams::Aggregating:
|
||||
add_mandatory_param("default aggregate function");
|
||||
add_optional_param("default aggregate function");
|
||||
break;
|
||||
case MergeTreeData::MergingParams::Graphite:
|
||||
add_mandatory_param("'config_element_for_graphite_schema'");
|
||||
|
Loading…
Reference in New Issue
Block a user