mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
Merge remote-tracking branch 'upstream/master' into order-by-efficient
This commit is contained in:
commit
b4991b38bf
@ -176,6 +176,7 @@ QueryProfilerBase<ProfilerImpl>::QueryProfilerBase(const Int32 thread_id, const
|
||||
throw;
|
||||
}
|
||||
#else
|
||||
UNUSED(thread_id, clock_type, period, pause_signal);
|
||||
throw Exception("QueryProfiler cannot work with stock libunwind", ErrorCodes::NOT_IMPLEMENTED);
|
||||
#endif
|
||||
}
|
||||
|
@ -142,4 +142,4 @@
|
||||
|
||||
/// A macro for suppressing warnings about unused variables or function results.
|
||||
/// Useful for structured bindings which have no standard way to declare this.
|
||||
#define UNUSED(X) (void) (X)
|
||||
#define UNUSED(...) (void)(__VA_ARGS__)
|
||||
|
@ -267,4 +267,14 @@ void TranslateQualifiedNamesMatcher::extractJoinUsingColumns(const ASTPtr ast, D
|
||||
}
|
||||
}
|
||||
|
||||
void RestoreQualifiedNamesData::visit(ASTIdentifier & identifier, ASTPtr & ast)
|
||||
{
|
||||
if (IdentifierSemantic::getColumnName(identifier) &&
|
||||
IdentifierSemantic::getMembership(identifier))
|
||||
{
|
||||
ast = identifier.clone();
|
||||
ast->as<ASTIdentifier>()->restoreCompoundName();
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -66,4 +66,15 @@ private:
|
||||
/// It finds columns and translate their names to the normal form. Expand asterisks and qualified asterisks with column names.
|
||||
using TranslateQualifiedNamesVisitor = TranslateQualifiedNamesMatcher::Visitor;
|
||||
|
||||
/// Restore ASTIdentifiers to long form
|
||||
struct RestoreQualifiedNamesData
|
||||
{
|
||||
using TypeToVisit = ASTIdentifier;
|
||||
|
||||
void visit(ASTIdentifier & identifier, ASTPtr & ast);
|
||||
};
|
||||
|
||||
using RestoreQualifiedNamesMatcher = OneTypeMatcher<RestoreQualifiedNamesData>;
|
||||
using RestoreQualifiedNamesVisitor = InDepthNodeVisitor<RestoreQualifiedNamesMatcher, true>;
|
||||
|
||||
}
|
||||
|
@ -43,6 +43,15 @@ void ASTIdentifier::setShortName(const String & new_name)
|
||||
semantic->special = special;
|
||||
}
|
||||
|
||||
void ASTIdentifier::restoreCompoundName()
|
||||
{
|
||||
if (name_parts.empty())
|
||||
return;
|
||||
name = name_parts[0];
|
||||
for (size_t i = 1; i < name_parts.size(); ++i)
|
||||
name += '.' + name_parts[i];
|
||||
}
|
||||
|
||||
void ASTIdentifier::formatImplWithoutAlias(const FormatSettings & settings, FormatState &, FormatStateStacked) const
|
||||
{
|
||||
auto format_element = [&](const String & elem_name)
|
||||
|
@ -38,6 +38,7 @@ public:
|
||||
bool isShort() const { return name_parts.empty() || name == name_parts.back(); }
|
||||
|
||||
void setShortName(const String & new_name);
|
||||
void restoreCompoundName();
|
||||
|
||||
const String & shortName() const
|
||||
{
|
||||
|
@ -34,6 +34,7 @@
|
||||
#include <Interpreters/InterpreterAlterQuery.h>
|
||||
#include <Interpreters/InterpreterDescribeQuery.h>
|
||||
#include <Interpreters/InterpreterSelectQuery.h>
|
||||
#include <Interpreters/TranslateQualifiedNamesVisitor.h>
|
||||
#include <Interpreters/SyntaxAnalyzer.h>
|
||||
#include <Interpreters/createBlockSelector.h>
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
@ -78,10 +79,20 @@ namespace
|
||||
ASTPtr rewriteSelectQuery(const ASTPtr & query, const std::string & database, const std::string & table, ASTPtr table_function_ptr = nullptr)
|
||||
{
|
||||
auto modified_query_ast = query->clone();
|
||||
|
||||
ASTSelectQuery & select_query = modified_query_ast->as<ASTSelectQuery &>();
|
||||
|
||||
/// restore long column names in JOIN ON expressions
|
||||
if (auto tables = select_query.tables())
|
||||
{
|
||||
RestoreQualifiedNamesVisitor::Data data;
|
||||
RestoreQualifiedNamesVisitor(data).visit(tables);
|
||||
}
|
||||
|
||||
if (table_function_ptr)
|
||||
modified_query_ast->as<ASTSelectQuery &>().addTableFunction(table_function_ptr);
|
||||
select_query.addTableFunction(table_function_ptr);
|
||||
else
|
||||
modified_query_ast->as<ASTSelectQuery &>().replaceDatabaseAndTable(database, table);
|
||||
select_query.replaceDatabaseAndTable(database, table);
|
||||
return modified_query_ast;
|
||||
}
|
||||
|
||||
|
@ -1,11 +1,14 @@
|
||||
<test>
|
||||
<name>IPv4 Functions</name>
|
||||
|
||||
<type>once</type>
|
||||
<type>loop</type>
|
||||
<stop_conditions>
|
||||
<all_of>
|
||||
<iterations>10</iterations>
|
||||
<min_time_not_changing_for_ms>10000</min_time_not_changing_for_ms>
|
||||
</all_of>
|
||||
<any_of>
|
||||
<average_speed_not_changing_for_ms>1000</average_speed_not_changing_for_ms>
|
||||
<total_time_ms>2000</total_time_ms>
|
||||
<iterations>50</iterations>
|
||||
<total_time_ms>60000</total_time_ms>
|
||||
</any_of>
|
||||
</stop_conditions>
|
||||
|
||||
@ -49,15 +52,16 @@
|
||||
<fill_query>INSERT INTO mon_{table_suffix} (n) SELECT n FROM mon_{table_suffix}</fill_query>
|
||||
<fill_query>INSERT INTO rnd_{table_suffix} (n) SELECT n FROM rnd_{table_suffix}</fill_query>
|
||||
|
||||
<!-- Using limit to make query finite, allowing it to be run multiple times in a loop, reducing mean error -->
|
||||
<query>INSERT INTO seq_{table_suffix} (n) SELECT number FROM system.numbers LIMIT 100000 SETTINGS max_threads=1</query>
|
||||
<query>INSERT INTO mon_{table_suffix} (n) SELECT number*67+(rand()%67) FROM system.numbers LIMIT 100000 SETTINGS max_threads=1</query>
|
||||
<query>INSERT INTO rnd_{table_suffix} (n) SELECT rand() FROM system.numbers LIMIT 100000 SETTINGS max_threads=1</query>
|
||||
|
||||
<query>INSERT INTO seq_{table_suffix} (n) SELECT number FROM system.numbers SETTINGS max_threads=1</query>
|
||||
<query>INSERT INTO mon_{table_suffix} (n) SELECT number*67+(rand()%67) FROM system.numbers SETTINGS max_threads=1</query>
|
||||
<query>INSERT INTO rnd_{table_suffix} (n) SELECT rand() FROM system.numbers SETTINGS max_threads=1</query>
|
||||
|
||||
|
||||
<query>SELECT count(n) FROM seq_{table_suffix} SETTINGS max_threads=1</query>
|
||||
<query>SELECT count(n) FROM mon_{table_suffix} SETTINGS max_threads=1</query>
|
||||
<query>SELECT count(n) FROM rnd_{table_suffix} SETTINGS max_threads=1</query>
|
||||
<!-- INSERTs above will be run unspecified amount of times, hence size of table is unknown.
|
||||
To make test more reliable, we SELECT fixed number of rows. -->
|
||||
<query>SELECT count(n) FROM seq_{table_suffix} LIMIT 1000000 SETTINGS max_threads=1</query>
|
||||
<query>SELECT count(n) FROM mon_{table_suffix} LIMIT 1000000 SETTINGS max_threads=1</query>
|
||||
<query>SELECT count(n) FROM rnd_{table_suffix} LIMIT 1000000 SETTINGS max_threads=1</query>
|
||||
|
||||
|
||||
<drop_query>DROP TABLE IF EXISTS seq_{table_suffix}</drop_query>
|
||||
|
@ -1,11 +1,14 @@
|
||||
<test>
|
||||
<name>IPv4 Functions</name>
|
||||
|
||||
<type>once</type>
|
||||
<type>loop</type>
|
||||
<stop_conditions>
|
||||
<all_of>
|
||||
<iterations>10</iterations>
|
||||
<min_time_not_changing_for_ms>10000</min_time_not_changing_for_ms>
|
||||
</all_of>
|
||||
<any_of>
|
||||
<average_speed_not_changing_for_ms>1000</average_speed_not_changing_for_ms>
|
||||
<total_time_ms>2000</total_time_ms>
|
||||
<iterations>50</iterations>
|
||||
<total_time_ms>60000</total_time_ms>
|
||||
</any_of>
|
||||
</stop_conditions>
|
||||
|
||||
@ -49,16 +52,16 @@
|
||||
<fill_query>INSERT INTO mon_{table_suffix} (n) SELECT n FROM mon_{table_suffix}</fill_query>
|
||||
<fill_query>INSERT INTO rnd_{table_suffix} (n) SELECT n FROM rnd_{table_suffix}</fill_query>
|
||||
|
||||
<!-- Using limit to make query finite, allowing it to be run multiple times in a loop, reducing mean error -->
|
||||
<query>INSERT INTO seq_{table_suffix} (n) SELECT number/pi() FROM system.numbers LIMIT 100000 SETTINGS max_threads=1</query>
|
||||
<query>INSERT INTO mon_{table_suffix} (n) SELECT number+sin(number) FROM system.numbers LIMIT 100000 SETTINGS max_threads=1</query>
|
||||
<query>INSERT INTO rnd_{table_suffix} (n) SELECT (rand() - 4294967295)/pi() FROM system.numbers LIMIT 100000 SETTINGS max_threads=1</query>
|
||||
|
||||
<query>INSERT INTO seq_{table_suffix} (n) SELECT number/pi() FROM system.numbers SETTINGS max_threads=1</query>
|
||||
<query>INSERT INTO mon_{table_suffix} (n) SELECT number+sin(number) FROM system.numbers SETTINGS max_threads=1</query>
|
||||
<query>INSERT INTO rnd_{table_suffix} (n) SELECT (rand() - 4294967295)/pi() FROM system.numbers SETTINGS max_threads=1</query>
|
||||
|
||||
|
||||
<query>SELECT count(n) FROM seq_{table_suffix} SETTINGS max_threads=1</query>
|
||||
<query>SELECT count(n) FROM mon_{table_suffix} SETTINGS max_threads=1</query>
|
||||
<query>SELECT count(n) FROM rnd_{table_suffix} SETTINGS max_threads=1</query>
|
||||
|
||||
<!-- INSERTs above will be run unspecified amount of times, hence size of table is unknown.
|
||||
To make test more reliable, we SELECT fixed number of rows. -->
|
||||
<query>SELECT count(n) FROM seq_{table_suffix} LIMIT 100000 SETTINGS max_threads=1</query>
|
||||
<query>SELECT count(n) FROM mon_{table_suffix} LIMIT 100000 SETTINGS max_threads=1</query>
|
||||
<query>SELECT count(n) FROM rnd_{table_suffix} LIMIT 100000 SETTINGS max_threads=1</query>
|
||||
|
||||
<drop_query>DROP TABLE IF EXISTS seq_{table_suffix}</drop_query>
|
||||
<drop_query>DROP TABLE IF EXISTS mon_{table_suffix}</drop_query>
|
||||
|
@ -1,5 +1,4 @@
|
||||
<test>
|
||||
<name>count</name>
|
||||
|
||||
<type>loop</type>
|
||||
|
||||
|
@ -1,5 +1,4 @@
|
||||
<test>
|
||||
<name>Moving Sum</name>
|
||||
|
||||
<type>loop</type>
|
||||
|
||||
|
@ -1,5 +1,4 @@
|
||||
<test>
|
||||
<name>merge_tree_many_partitions</name>
|
||||
<type>loop</type>
|
||||
|
||||
<create_query>CREATE TABLE bad_partitions (x UInt64) ENGINE = MergeTree PARTITION BY x ORDER BY x</create_query>
|
||||
|
@ -1,5 +1,4 @@
|
||||
<test>
|
||||
<name>merge_tree_many_partitions_2</name>
|
||||
<type>loop</type>
|
||||
|
||||
<create_query>CREATE TABLE bad_partitions (a UInt64, b UInt64, c UInt64, d UInt64, e UInt64, f UInt64, g UInt64, h UInt64, i UInt64, j UInt64, k UInt64, l UInt64, m UInt64, n UInt64, o UInt64, p UInt64, q UInt64, r UInt64, s UInt64, t UInt64, u UInt64, v UInt64, w UInt64, x UInt64, y UInt64, z UInt64) ENGINE = MergeTree PARTITION BY x ORDER BY x</create_query>
|
||||
|
@ -1,5 +1,4 @@
|
||||
<test>
|
||||
<name>merge_tree_simple_select</name>
|
||||
|
||||
<type>loop</type>
|
||||
|
||||
|
@ -1,5 +1,4 @@
|
||||
<test>
|
||||
<name>number_formatting_formats</name>
|
||||
<type>loop</type>
|
||||
|
||||
<create_query>CREATE TABLE IF NOT EXISTS table_{format} (x UInt64) ENGINE = File(`{format}`)</create_query>
|
||||
|
@ -0,0 +1,8 @@
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
42 42
|
||||
42 42
|
34
dbms/tests/queries/0_stateless/00974_distributed_join_on.sql
Normal file
34
dbms/tests/queries/0_stateless/00974_distributed_join_on.sql
Normal file
@ -0,0 +1,34 @@
|
||||
DROP TABLE IF EXISTS source_table1;
|
||||
DROP TABLE IF EXISTS source_table2;
|
||||
DROP TABLE IF EXISTS distributed_table1;
|
||||
DROP TABLE IF EXISTS distributed_table2;
|
||||
|
||||
CREATE TABLE source_table1 (a Int64, b String) ENGINE = Memory;
|
||||
CREATE TABLE source_table2 (c Int64, d String) ENGINE = Memory;
|
||||
|
||||
INSERT INTO source_table1 VALUES (42, 'qwe');
|
||||
INSERT INTO source_table2 VALUES (42, 'qwe');
|
||||
|
||||
CREATE TABLE distributed_table1 AS source_table1
|
||||
ENGINE = Distributed('test_shard_localhost', currentDatabase(), source_table1);
|
||||
|
||||
CREATE TABLE distributed_table2 AS source_table2
|
||||
ENGINE = Distributed('test_shard_localhost', currentDatabase(), source_table2);
|
||||
|
||||
SET prefer_localhost_replica = 1;
|
||||
SELECT 1 FROM distributed_table1 AS t1 GLOBAL JOIN distributed_table2 AS t2 ON t1.a = t2.c LIMIT 1;
|
||||
SELECT 1 FROM distributed_table1 AS t1 GLOBAL JOIN distributed_table2 AS t2 ON t2.c = t1.a LIMIT 1;
|
||||
SELECT 1 FROM distributed_table1 AS t1 GLOBAL JOIN distributed_table1 AS t2 ON t1.a = t2.a LIMIT 1;
|
||||
|
||||
SET prefer_localhost_replica = 0;
|
||||
SELECT 1 FROM distributed_table1 AS t1 GLOBAL JOIN distributed_table2 AS t2 ON t1.a = t2.c LIMIT 1;
|
||||
SELECT 1 FROM distributed_table1 AS t1 GLOBAL JOIN distributed_table2 AS t2 ON t2.c = t1.a LIMIT 1;
|
||||
SELECT 1 FROM distributed_table1 AS t1 GLOBAL JOIN distributed_table1 AS t2 ON t1.a = t2.a LIMIT 1;
|
||||
|
||||
SELECT t1.a as t1_a, t2.a as t2_a FROM source_table1 AS t1 JOIN source_table1 AS t2 ON t1_a = t2_a LIMIT 1;
|
||||
SELECT t1.a as t1_a, t2.a as t2_a FROM distributed_table1 AS t1 GLOBAL JOIN distributed_table1 AS t2 ON t1_a = t2_a LIMIT 1;
|
||||
|
||||
DROP TABLE source_table1;
|
||||
DROP TABLE source_table2;
|
||||
DROP TABLE distributed_table1;
|
||||
DROP TABLE distributed_table2;
|
Loading…
Reference in New Issue
Block a user