Skip to content

Commit

Permalink
Backport to 19.7: CLICKHOUSE-4523 Fix "Column '0' already exists" in …
Browse files Browse the repository at this point in the history
…SELECT .. PREWHERE… (#5467)

* 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

(cherry picked from commit 6c1cb02)

* Update evaluateMissingDefaults.cpp
  • Loading branch information
akuzm committed May 31, 2019
1 parent 58ee7a6 commit 52320d1
Show file tree
Hide file tree
Showing 6 changed files with 128 additions and 3 deletions.
55 changes: 54 additions & 1 deletion dbms/src/Core/iostream_debug_helpers.cpp
Expand Up @@ -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>
Expand Down Expand Up @@ -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) << ")";
}

Expand Down Expand Up @@ -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;
}

}
8 changes: 8 additions & 0 deletions dbms/src/Core/iostream_debug_helpers.h
Expand Up @@ -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<>)
Expand Down
28 changes: 26 additions & 2 deletions dbms/src/Interpreters/evaluateMissingDefaults.cpp
@@ -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
Expand Down Expand Up @@ -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;
Expand Down
@@ -0,0 +1,7 @@
42
42 42 42
42 42 43
43
43
43
42 42 43
21 changes: 21 additions & 0 deletions dbms/tests/queries/0_stateless/00950_default_prewhere.sql
@@ -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 changes: 12 additions & 0 deletions dbms/tests/queries/bugs/default_prewhere.sql
@@ -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;

0 comments on commit 52320d1

Please sign in to comment.