mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
CLICKHOUSE-4523 Fix "Column '0' already exists" in SELECT .. PREWHERE on column with DEFAULT (#5397)
* CLICKHOUSE-4523 Fix "Column '0' already exists" in SELECT .. PREWHERE on column with DEFAULT * fix style * Fix build * Update iostream_debug_helpers.cpp * Update evaluateMissingDefaults.cpp
This commit is contained in:
parent
10ed439b04
commit
6c1cb02172
@ -10,6 +10,7 @@
|
||||
#include <Functions/IFunction.h>
|
||||
#include <IO/WriteBufferFromOStream.h>
|
||||
#include <Interpreters/ExpressionAnalyzer.h>
|
||||
#include <Interpreters/ExpressionActions.h>
|
||||
#include <Parsers/IAST.h>
|
||||
#include <Storages/IStorage.h>
|
||||
#include <Common/COW.h>
|
||||
@ -70,7 +71,7 @@ std::ostream & operator<<(std::ostream & stream, const Block & what)
|
||||
|
||||
std::ostream & operator<<(std::ostream & stream, const ColumnWithTypeAndName & what)
|
||||
{
|
||||
stream << "ColumnWithTypeAndName(name = " << what.name << ", type = " << what.type << ", column = ";
|
||||
stream << "ColumnWithTypeAndName(name = " << what.name << ", type = " << *what.type << ", column = ";
|
||||
return dumpValue(stream, what.column) << ")";
|
||||
}
|
||||
|
||||
@ -109,4 +110,56 @@ std::ostream & operator<<(std::ostream & stream, const IAST & what)
|
||||
return stream;
|
||||
}
|
||||
|
||||
std::ostream & operator<<(std::ostream & stream, const ExpressionAction & what)
|
||||
{
|
||||
stream << "ExpressionAction(" << what.toString() << ")";
|
||||
return stream;
|
||||
}
|
||||
|
||||
std::ostream & operator<<(std::ostream & stream, const ExpressionActions & what)
|
||||
{
|
||||
stream << "ExpressionActions(" << what.dumpActions() << ")";
|
||||
return stream;
|
||||
}
|
||||
|
||||
std::ostream & operator<<(std::ostream & stream, const SyntaxAnalyzerResult & what)
|
||||
{
|
||||
stream << "SyntaxAnalyzerResult{";
|
||||
stream << "storage=" << what.storage << "; ";
|
||||
if (!what.source_columns.empty())
|
||||
{
|
||||
stream << "source_columns=";
|
||||
dumpValue(stream, what.source_columns);
|
||||
stream << "; ";
|
||||
}
|
||||
if (!what.aliases.empty())
|
||||
{
|
||||
stream << "aliases=";
|
||||
dumpValue(stream, what.aliases);
|
||||
stream << "; ";
|
||||
}
|
||||
if (!what.array_join_result_to_source.empty())
|
||||
{
|
||||
stream << "array_join_result_to_source=";
|
||||
dumpValue(stream, what.array_join_result_to_source);
|
||||
stream << "; ";
|
||||
}
|
||||
if (!what.array_join_alias_to_name.empty())
|
||||
{
|
||||
stream << "array_join_alias_to_name=";
|
||||
dumpValue(stream, what.array_join_alias_to_name);
|
||||
stream << "; ";
|
||||
}
|
||||
if (!what.array_join_name_to_alias.empty())
|
||||
{
|
||||
stream << "array_join_name_to_alias=";
|
||||
dumpValue(stream, what.array_join_name_to_alias);
|
||||
stream << "; ";
|
||||
}
|
||||
stream << "rewrite_subqueries=" << what.rewrite_subqueries << "; ";
|
||||
stream << "}";
|
||||
|
||||
return stream;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -41,6 +41,14 @@ std::ostream & operator<<(std::ostream & stream, const IAST & what);
|
||||
|
||||
std::ostream & operator<<(std::ostream & stream, const Connection::Packet & what);
|
||||
|
||||
struct ExpressionAction;
|
||||
std::ostream & operator<<(std::ostream & stream, const ExpressionAction & what);
|
||||
|
||||
class ExpressionActions;
|
||||
std::ostream & operator<<(std::ostream & stream, const ExpressionActions & what);
|
||||
|
||||
struct SyntaxAnalyzerResult;
|
||||
std::ostream & operator<<(std::ostream & stream, const SyntaxAnalyzerResult & what);
|
||||
}
|
||||
|
||||
/// some operator<< should be declared before operator<<(... std::shared_ptr<>)
|
||||
|
@ -1,12 +1,14 @@
|
||||
#include "evaluateMissingDefaults.h"
|
||||
|
||||
#include <Core/Block.h>
|
||||
#include <Storages/ColumnDefault.h>
|
||||
#include <Interpreters/SyntaxAnalyzer.h>
|
||||
#include <Interpreters/ExpressionAnalyzer.h>
|
||||
#include <Interpreters/ExpressionActions.h>
|
||||
#include <Interpreters/evaluateMissingDefaults.h>
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
#include <Parsers/ASTWithAlias.h>
|
||||
#include <utility>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -58,7 +60,29 @@ void evaluateMissingDefaults(Block & block,
|
||||
Block copy_block{block};
|
||||
|
||||
auto syntax_result = SyntaxAnalyzer(context).analyze(default_expr_list, block.getNamesAndTypesList());
|
||||
ExpressionAnalyzer{default_expr_list, syntax_result, context}.getActions(true)->execute(copy_block);
|
||||
auto expression_analyzer = ExpressionAnalyzer{default_expr_list, syntax_result, context};
|
||||
auto required_source_columns = expression_analyzer.getRequiredSourceColumns();
|
||||
auto rows_was = copy_block.rows();
|
||||
|
||||
// Delete all not needed columns in DEFAULT expression.
|
||||
// They can intersect with columns added in PREWHERE
|
||||
// test 00950_default_prewhere
|
||||
// CLICKHOUSE-4523
|
||||
for (const auto & delete_column : copy_block.getNamesAndTypesList())
|
||||
{
|
||||
if (std::find(required_source_columns.begin(), required_source_columns.end(), delete_column.name) == required_source_columns.end())
|
||||
{
|
||||
copy_block.erase(delete_column.name);
|
||||
}
|
||||
}
|
||||
|
||||
if (copy_block.columns() == 0)
|
||||
{
|
||||
// Add column to indicate block size in execute()
|
||||
copy_block.insert({DataTypeUInt8().createColumnConst(rows_was, 0u), std::make_shared<DataTypeUInt8>(), "__dummy"});
|
||||
}
|
||||
|
||||
expression_analyzer.getActions(true)->execute(copy_block);
|
||||
|
||||
/// move evaluated columns to the original block, materializing them at the same time
|
||||
size_t pos = 0;
|
||||
|
@ -0,0 +1,7 @@
|
||||
42
|
||||
42 42 42
|
||||
42 42 43
|
||||
43
|
||||
43
|
||||
43
|
||||
42 42 43
|
21
dbms/tests/queries/0_stateless/00950_default_prewhere.sql
Normal file
21
dbms/tests/queries/0_stateless/00950_default_prewhere.sql
Normal file
@ -0,0 +1,21 @@
|
||||
|
||||
DROP TABLE IF EXISTS test_generic_events_all;
|
||||
|
||||
CREATE TABLE test_generic_events_all (APIKey UInt8, SessionType UInt8) ENGINE = MergeTree() PARTITION BY APIKey ORDER BY tuple();
|
||||
INSERT INTO test_generic_events_all VALUES( 42, 42 );
|
||||
ALTER TABLE test_generic_events_all ADD COLUMN OperatingSystem UInt64 DEFAULT 42;
|
||||
SELECT OperatingSystem FROM test_generic_events_all PREWHERE APIKey = 42 WHERE SessionType = 42;
|
||||
SELECT * FROM test_generic_events_all PREWHERE APIKey = 42 WHERE SessionType = 42;
|
||||
|
||||
DROP TABLE IF EXISTS test_generic_events_all;
|
||||
|
||||
CREATE TABLE test_generic_events_all (APIKey UInt8, SessionType UInt8) ENGINE = MergeTree() PARTITION BY APIKey ORDER BY tuple();
|
||||
INSERT INTO test_generic_events_all VALUES( 42, 42 );
|
||||
ALTER TABLE test_generic_events_all ADD COLUMN OperatingSystem UInt64 DEFAULT SessionType+1;
|
||||
SELECT * FROM test_generic_events_all WHERE APIKey = 42 AND SessionType = 42;
|
||||
SELECT OperatingSystem FROM test_generic_events_all WHERE APIKey = 42;
|
||||
SELECT OperatingSystem FROM test_generic_events_all WHERE APIKey = 42 AND SessionType = 42;
|
||||
SELECT OperatingSystem FROM test_generic_events_all PREWHERE APIKey = 42 WHERE SessionType = 42;
|
||||
SELECT * FROM test_generic_events_all PREWHERE APIKey = 42 WHERE SessionType = 42;
|
||||
|
||||
DROP TABLE IF EXISTS test_generic_events_all;
|
12
dbms/tests/queries/bugs/default_prewhere.sql
Normal file
12
dbms/tests/queries/bugs/default_prewhere.sql
Normal file
@ -0,0 +1,12 @@
|
||||
DROP TABLE IF EXISTS test_generic_events_all;
|
||||
CREATE TABLE test_generic_events_all (APIKey UInt8, SessionType UInt8) ENGINE = MergeTree() PARTITION BY APIKey ORDER BY tuple();
|
||||
INSERT INTO test_generic_events_all VALUES( 42, 42 );
|
||||
ALTER TABLE test_generic_events_all ADD COLUMN OperatingSystem UInt64 DEFAULT APIKey+1;
|
||||
SELECT * FROM test_generic_events_all WHERE APIKey = 42 AND SessionType = 42;
|
||||
-- InterpreterSelectQuery: MergeTreeWhereOptimizer: condition "APIKey = 42" moved to PREWHERE
|
||||
SELECT OperatingSystem FROM test_generic_events_all WHERE APIKey = 42;
|
||||
SELECT OperatingSystem FROM test_generic_events_all WHERE APIKey = 42 AND SessionType = 42;
|
||||
SELECT OperatingSystem FROM test_generic_events_all PREWHERE APIKey = 42 WHERE SessionType = 42;
|
||||
SELECT * FROM test_generic_events_all PREWHERE APIKey = 42 WHERE SessionType = 42;
|
||||
|
||||
DROP TABLE IF EXISTS test_generic_events_all;
|
Loading…
Reference in New Issue
Block a user