ClickHouse/src/Interpreters/addMissingDefaults.cpp

97 lines
3.5 KiB
C++
Raw Normal View History

#include <Interpreters/addMissingDefaults.h>
#include <Common/typeid_cast.h>
#include <DataTypes/NestedUtils.h>
#include <DataTypes/DataTypeArray.h>
#include <Columns/ColumnArray.h>
2020-02-17 15:44:13 +00:00
#include <Interpreters/inplaceBlockConversions.h>
#include <Core/Block.h>
2020-10-02 12:38:50 +00:00
#include <Storages/ColumnsDescription.h>
#include <Interpreters/ExpressionActions.h>
2021-02-05 11:41:44 +00:00
#include <Functions/IFunctionAdaptors.h>
#include <Functions/replicate.h>
#include <Functions/materialize.h>
namespace DB
{
2021-02-05 11:41:44 +00:00
ActionsDAGPtr addMissingDefaults(
const Block & header,
2020-10-02 12:38:50 +00:00
const NamesAndTypesList & required_columns,
const ColumnsDescription & columns,
const Context & context)
{
/// For missing columns of nested structure, you need to create not a column of empty arrays, but a column of arrays of correct lengths.
/// First, remember the offset columns for all arrays in the block.
2021-02-05 11:41:44 +00:00
std::map<String, Names> nested_groups;
2021-02-05 11:41:44 +00:00
for (size_t i = 0, size = header.columns(); i < size; ++i)
{
2021-02-05 11:41:44 +00:00
const auto & elem = header.getByPosition(i);
2021-02-05 11:41:44 +00:00
if (typeid_cast<const ColumnArray *>(&*elem.column))
{
String offsets_name = Nested::extractTableName(elem.name);
2021-02-05 11:41:44 +00:00
auto & group = nested_groups[offsets_name];
if (group.empty())
group.push_back({});
group.push_back(elem.name);
}
}
2021-02-05 11:41:44 +00:00
auto actions = std::make_shared<ActionsDAG>(header.getColumnsWithTypeAndName());
FunctionOverloadResolverPtr func_builder_replicate =
std::make_shared<FunctionOverloadResolverAdaptor>(
std::make_unique<DefaultOverloadResolver>(
std::make_shared<FunctionReplicate>()));
FunctionOverloadResolverPtr func_builder_materialize =
std::make_shared<FunctionOverloadResolverAdaptor>(
std::make_unique<DefaultOverloadResolver>(
std::make_shared<FunctionMaterialize>()));
/// We take given columns from input block and missed columns without default value
/// (default and materialized will be computed later).
for (const auto & column : required_columns)
{
2021-02-05 11:41:44 +00:00
if (header.has(column.name))
continue;
2020-10-02 12:38:50 +00:00
if (columns.hasDefault(column.name))
continue;
String offsets_name = Nested::extractTableName(column.name);
2021-02-05 11:41:44 +00:00
if (nested_groups.count(offsets_name))
{
2021-02-05 11:41:44 +00:00
DataTypePtr nested_type = typeid_cast<const DataTypeArray &>(*column.type).getNestedType();
2021-02-05 11:41:44 +00:00
ColumnPtr nested_column = nested_type->createColumnConstWithDefaultValue(0);
const auto & constant = actions->addColumn({std::move(nested_column), nested_type, column.name}, true);
2021-02-05 11:41:44 +00:00
auto & group = nested_groups[offsets_name];
group[0] = constant.result_name;
2021-02-05 16:35:21 +00:00
actions->addFunction(func_builder_replicate, group, constant.result_name, context);
2021-02-05 11:41:44 +00:00
continue;
}
/** It is necessary to turn a constant column into a full column, since in part of blocks (from other parts),
* it can be full (or the interpreter may decide that it is constant everywhere).
*/
2021-02-05 16:35:21 +00:00
auto new_column = column.type->createColumnConstWithDefaultValue(0);
actions->addColumn({std::move(new_column), column.type, column.name}, true, true);
}
2020-10-02 12:38:50 +00:00
/// Computes explicitly specified values by default and materialized columns.
2021-02-05 15:11:26 +00:00
if (auto dag = evaluateMissingDefaults(header, required_columns, columns, context))
2021-02-05 11:41:44 +00:00
actions = ActionsDAG::merge(std::move(*actions), std::move(*dag));
return actions;
}
}