Default columns which depends on aliases

This commit is contained in:
alesapin 2020-03-03 17:25:28 +03:00
parent d0bcbf7581
commit d19e21ba19
8 changed files with 99 additions and 3 deletions

View File

@ -74,4 +74,13 @@ const BlockMissingValues::RowsBitMask & BlockMissingValues::getDefaultsBitmask(s
return none;
}
bool BlockMissingValues::hasDefaultBits(size_t column_idx) const
{
auto it = rows_mask_by_column_id.find(column_idx);
if (it == rows_mask_by_column_id.end())
return false;
const auto & col_mask = it->second;
return std::find(col_mask.begin(), col_mask.end(), true) != col_mask.end();
}
}

View File

@ -51,7 +51,10 @@ class BlockMissingValues
public:
using RowsBitMask = std::vector<bool>; /// a bit per row for a column
/// Get mask for column, column_idx is index inside corresponding block
const RowsBitMask & getDefaultsBitmask(size_t column_idx) const;
/// Check that we have to replace default value at least in one of columns
bool hasDefaultBits(size_t column_idx) const;
void setBit(size_t column_idx, size_t row_idx);
bool empty() const { return rows_mask_by_column_id.empty(); }
size_t size() const { return rows_mask_by_column_id.size(); }

View File

@ -56,11 +56,20 @@ Block AddingDefaultsBlockInputStream::readImpl()
if (block_missing_values.empty())
return res;
/// res block alredy has all columns values, with default value for type
/// (not value specified in table). We identify which columns we need to
/// recalculate with help of block_missing_values.
Block evaluate_block{res};
/// remove columns for recalculation
for (const auto & column : column_defaults)
{
if (evaluate_block.has(column.first))
{
size_t column_idx = res.getPositionByName(column.first);
if (block_missing_values.hasDefaultBits(column_idx))
evaluate_block.erase(column.first);
}
}
if (!evaluate_block.columns())
evaluate_block.insert({ColumnConst::create(ColumnUInt8::create(1, 0), res.rows()), std::make_shared<DataTypeUInt8>(), "_dummy"});

View File

@ -11,6 +11,7 @@
#include <Parsers/ASTFunction.h>
#include <utility>
#include <DataTypes/DataTypesNumber.h>
#include <Interpreters/RequiredSourceColumnsVisitor.h>
namespace DB
@ -27,10 +28,20 @@ static ASTPtr requiredExpressions(Block & block, const NamesAndTypesList & requi
const auto it = column_defaults.find(column.name);
/// expressions must be cloned to prevent modification by the ExpressionAnalyzer
if (it != column_defaults.end())
{
auto cast_func = makeASTFunction("CAST", it->second.expression->clone(), std::make_shared<ASTLiteral>(column.type->getName()));
/// expressions must be cloned to prevent modification by the ExpressionAnalyzer
auto column_default_expr = it->second.expression->clone();
RequiredSourceColumnsVisitor::Data columns_context;
RequiredSourceColumnsVisitor(columns_context).visit(column_default_expr);
NameSet required_columns_names = columns_context.requiredColumns();
for (const auto & required_column_name : required_columns_names)
if (auto rit = column_defaults.find(required_column_name);
rit != column_defaults.end() && rit->second.kind == ColumnDefaultKind::Alias)
default_expr_list->children.emplace_back(setAlias(rit->second.expression->clone(), required_column_name));
auto cast_func = makeASTFunction("CAST", column_default_expr, std::make_shared<ASTLiteral>(column.type->getName()));
default_expr_list->children.emplace_back(setAlias(cast_func, it->first));
}
}

View File

@ -0,0 +1,6 @@
1 1
1 1 1
2 2 4
2 2 2 4
3 3 9
3 3 3 9 27

View File

@ -0,0 +1,30 @@
DROP TABLE IF EXISTS table_with_defaults_on_aliases;
CREATE TABLE table_with_defaults_on_aliases (col1 UInt32, col2 ALIAS col1, col3 DEFAULT col2) Engine = MergeTree() ORDER BY tuple();
INSERT INTO table_with_defaults_on_aliases (col1) VALUES (1);
SELECT * FROM table_with_defaults_on_aliases WHERE col1 = 1;
SELECT col1, col2, col3 FROM table_with_defaults_on_aliases WHERE col1 = 1;
ALTER TABLE table_with_defaults_on_aliases ADD COLUMN col4 UInt64 DEFAULT col2 * col3;
INSERT INTO table_with_defaults_on_aliases (col1) VALUES (2);
SELECT * FROM table_with_defaults_on_aliases WHERE col1 = 2;
SELECT col1, col2, col3, col4 FROM table_with_defaults_on_aliases WHERE col1 = 2;
ALTER TABLE table_with_defaults_on_aliases ADD COLUMN col5 UInt64 ALIAS col2 * col4;
INSERT INTO table_with_defaults_on_aliases (col1) VALUES (3);
SELECT * FROM table_with_defaults_on_aliases WHERE col1 = 3;
SELECT col1, col2, col3, col4, col5 FROM table_with_defaults_on_aliases WHERE col1 = 3;
ALTER TABLE table_with_defaults_on_aliases ADD COLUMN col6 UInt64 MATERIALIZED col2 * col4;
DROP TABLE IF EXISTS table_with_defaults_on_aliases;

View File

@ -0,0 +1,2 @@
0 0 test0
ClickHouse is great ClickHouse is fast

View File

@ -0,0 +1,26 @@
DROP TABLE IF EXISTS table_with_complex_default;
CREATE TABLE table_with_complex_default (i Int8, n UInt8 DEFAULT 42, s String DEFAULT concat('test', CAST(n, 'String'))) ENGINE=TinyLog;
INSERT INTO table_with_complex_default FORMAT JSONEachRow {"i":0, "n": 0}
SELECT * FROM table_with_complex_default;
DROP TABLE IF EXISTS table_with_complex_default;
DROP TABLE IF EXISTS test_default_using_alias;
CREATE TABLE test_default_using_alias
(
what String,
a String DEFAULT concat(c, ' is great'),
b String DEFAULT concat(c, ' is fast'),
c String ALIAS concat(what, 'House')
)
ENGINE = TinyLog;
INSERT INTO test_default_using_alias(what) VALUES ('Click');
SELECT a, b FROM test_default_using_alias;
DROP TABLE IF EXISTS test_default_using_alias;