Skip to content

Commit 8e4517e

Browse files
committed
Merge remote-tracking branch 'origin/master' into vector_search_pre_and_post_filtering
2 parents 3c45924 + 15ad477 commit 8e4517e

File tree

14 files changed

+194
-8
lines changed

14 files changed

+194
-8
lines changed

ci/jobs/scripts/check_style/aspell-ignore/en/aspell-dict.txt

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -167,6 +167,7 @@ CityHash
167167
Clangd
168168
ClickBench
169169
ClickCat
170+
ClickHaskell
170171
ClickHouse
171172
ClickHouse's
172173
ClickHouseClient

docs/en/interfaces/third-party/client-libraries.md

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -83,3 +83,4 @@ ClickHouse Inc does **not** maintain the libraries listed below and hasn't done
8383
- [nim-clickhouse](https://github.com/leonardoce/nim-clickhouse)
8484
### Haskell {#haskell}
8585
- [hdbc-clickhouse](https://github.com/zaneli/hdbc-clickhouse)
86+
- [ClickHaskell](https://clickhaskell.dev/)

src/Databases/DatabaseReplicated.cpp

Lines changed: 6 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -24,6 +24,7 @@
2424
#include <Interpreters/DatabaseCatalog.h>
2525
#include <Interpreters/InterpreterCreateQuery.h>
2626
#include <Interpreters/InterpreterSetQuery.h>
27+
#include <Interpreters/NormalizeSelectWithUnionQueryVisitor.h>
2728
#include <Interpreters/ReplicatedDatabaseQueryStatusSource.h>
2829
#include <Interpreters/evaluateConstantExpression.h>
2930
#include <Interpreters/executeDDLQueryOnCluster.h>
@@ -66,6 +67,7 @@ namespace Setting
6667
extern const SettingsDistributedDDLOutputMode distributed_ddl_output_mode;
6768
extern const SettingsInt64 distributed_ddl_task_timeout;
6869
extern const SettingsBool throw_on_unsupported_query_inside_transaction;
70+
extern const SettingsSetOperationMode union_default_mode;
6971
}
7072

7173
namespace ServerSetting
@@ -1463,8 +1465,11 @@ void DatabaseReplicated::recoverLostReplica(const ZooKeeperPtr & current_zookeep
14631465
}
14641466

14651467
auto query_ast = parseQueryFromMetadataInZooKeeper(table_name, create_query_string);
1466-
14671468
auto create_query_context = make_query_context();
1469+
1470+
NormalizeSelectWithUnionQueryVisitor::Data data{create_query_context->getSettingsRef()[Setting::union_default_mode]};
1471+
NormalizeSelectWithUnionQueryVisitor{data}.visit(query_ast);
1472+
14681473
/// Check larger comment in DatabaseOnDisk::createTableFromAST
14691474
/// TL;DR applySettingsFromQuery will move the settings from engine to query level
14701475
/// making it possible to overcome a backward incompatible change.

src/Interpreters/OptimizeIfWithConstantConditionVisitor.cpp

Lines changed: 11 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -98,13 +98,19 @@ void OptimizeIfWithConstantConditionVisitorData::visit(ASTFunction & function_no
9898
if (tryExtractConstValueFromCondition(condition_expr, condition))
9999
{
100100
ASTPtr replace_ast = condition ? then_expr : else_expr;
101+
bool replacement_supports_alias = dynamic_cast<ASTWithAlias *>(replace_ast.get());
102+
String if_alias = ast->tryGetAlias();
103+
/// We cannot set the resulting alias if the replace ast does not support it (e.g. ASTAsterisk), so it's better to do nothing
104+
if (!if_alias.empty() && !replacement_supports_alias)
105+
return;
106+
101107
ASTPtr child_copy = ast;
102108
String replace_alias = replace_ast->tryGetAlias();
103-
String if_alias = ast->tryGetAlias();
104109

105110
if (replace_alias.empty())
106111
{
107-
replace_ast->setAlias(if_alias);
112+
if (!if_alias.empty())
113+
replace_ast->setAlias(if_alias);
108114
ast = replace_ast;
109115
}
110116
else
@@ -113,12 +119,14 @@ void OptimizeIfWithConstantConditionVisitorData::visit(ASTFunction & function_no
113119
/// But IAST has only method for deep copy of subtree.
114120
/// This can be a reason of performance degradation in case of deep queries.
115121
ASTPtr replace_ast_deep_copy = replace_ast->clone();
116-
replace_ast_deep_copy->setAlias(if_alias);
122+
if (!if_alias.empty())
123+
replace_ast_deep_copy->setAlias(if_alias);
117124
ast = replace_ast_deep_copy;
118125
}
119126

120127
if (!if_alias.empty())
121128
{
129+
ast->setAlias(if_alias);
122130
auto alias_it = aliases.find(if_alias);
123131
if (alias_it != aliases.end() && alias_it->second.get() == child_copy.get())
124132
alias_it->second = ast;

src/Interpreters/TreeRewriter.cpp

Lines changed: 4 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -366,9 +366,12 @@ void renameDuplicatedColumns(const ASTSelectQuery * select_query)
366366
for (auto & expr : elements)
367367
{
368368
auto name = expr->getAliasOrColumnName();
369-
370369
if (!assigned_column_names.insert(name).second)
371370
{
371+
/// We can't rename with aliases if it doesn't support alias (e.g. asterisk)
372+
if (!dynamic_cast<ASTWithAlias *>(expr.get()))
373+
continue;
374+
372375
size_t i = 1;
373376
while (all_column_names.end() != all_column_names.find(name + "_" + toString(i)))
374377
++i;

tests/integration/test_matview_union_replicated/__init__.py

Whitespace-only changes.
Lines changed: 86 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,86 @@
1+
import pytest
2+
from helpers.cluster import ClickHouseCluster
3+
from helpers.test_tools import assert_eq_with_retry
4+
5+
cluster = ClickHouseCluster(__file__)
6+
7+
# Create two nodes - one main node and one replica
8+
node1 = cluster.add_instance("node1", with_zookeeper=True)
9+
node2 = cluster.add_instance("node2", with_zookeeper=True)
10+
11+
@pytest.fixture(scope="module")
12+
def started_cluster():
13+
try:
14+
cluster.start()
15+
yield cluster
16+
finally:
17+
cluster.shutdown()
18+
19+
def test_matview_union_replicated(started_cluster):
20+
21+
# Create replicated database, source and target tables and matview
22+
node1.query("DROP DATABASE IF EXISTS union_test_replicated SYNC")
23+
node1.query("CREATE DATABASE union_test_replicated ENGINE=Replicated('/test/union_replica' , 'shard1', 'replica' || '1');")
24+
25+
node1.query("""
26+
CREATE TABLE union_test_replicated.source_1
27+
(
28+
timestamp DateTime,
29+
value Float64
30+
)
31+
ENGINE = ReplicatedMergeTree
32+
ORDER BY timestamp
33+
""")
34+
35+
node1.query("""
36+
CREATE TABLE union_test_replicated.source_2
37+
(
38+
timestamp DateTime,
39+
value Float64
40+
)
41+
ENGINE = ReplicatedMergeTree
42+
ORDER BY timestamp
43+
""")
44+
45+
node1.query("""
46+
CREATE TABLE union_test_replicated.target
47+
(
48+
timestamp DateTime,
49+
value Float64
50+
)
51+
ENGINE = ReplicatedMergeTree
52+
ORDER BY timestamp
53+
""")
54+
55+
node1.query("""
56+
CREATE MATERIALIZED VIEW union_test_replicated.mv_test TO union_test_replicated.target AS
57+
WITH source_data AS
58+
(
59+
SELECT timestamp, value FROM union_test_replicated.source_1
60+
UNION ALL
61+
SELECT timestamp, value FROM union_test_replicated.source_2
62+
)
63+
SELECT timestamp, value FROM source_data
64+
""")
65+
66+
# Verify INSERT works on Node #1
67+
node1.query("INSERT INTO union_test_replicated.source_1 VALUES (now(), 1)")
68+
69+
# Attach replica on second node
70+
node2.query("DROP DATABASE IF EXISTS union_test_replicated SYNC")
71+
node2.query("CREATE DATABASE union_test_replicated ENGINE=Replicated('/test/union_replica' , 'shard1', 'replica' || '2');")
72+
node2.query("SYSTEM SYNC DATABASE REPLICA union_test_replicated")
73+
74+
# Verify the table structure on replica
75+
assert_eq_with_retry(
76+
node2,
77+
"DESCRIBE TABLE union_test_replicated.source_1",
78+
"timestamp\tDateTime\t\t\t\t\nvalue\tFloat64\t\t\t\t\n"
79+
)
80+
81+
# Run INSERT on replica
82+
node2.query("INSERT INTO union_test_replicated.source_1 VALUES (now(), 1)")
83+
84+
# Clean up
85+
node1.query("DROP DATABASE IF EXISTS union_test_replicated SYNC")
86+
node2.query("DROP DATABASE IF EXISTS union_test_replicated SYNC")

tests/queries/0_stateless/03355_join_to_in_optimization.sql

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -3,8 +3,8 @@ SET join_algorithm = 'hash';
33

44
DROP TABLE IF EXISTS t1;
55
DROP TABLE IF EXISTS t2;
6-
CREATE TABLE t1 (`id` Int32, key String, key2 String) ENGINE = MergeTree ORDER BY id;
7-
CREATE TABLE t2 (`id` Int32, key String, key2 String) ENGINE = MergeTree ORDER BY id;
6+
CREATE TABLE t1 (`id` Int32, key String, key2 String) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity=8192;
7+
CREATE TABLE t2 (`id` Int32, key String, key2 String) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity=8192;
88
INSERT INTO t1 VALUES (1, '111', '111'),(2, '222', '2'),(2, '222', '222'),(3, '333', '333');
99
INSERT INTO t2 VALUES (2, 'AAA', 'AAA'),(2, 'AAA', 'a'),(3, 'BBB', 'BBB'),(4, 'CCC', 'CCC');
1010

@@ -92,4 +92,4 @@ INNER JOIN system.processes AS b
9292
ON (a.query_id = b.query_id) AND (a.query_id = b.query_id)
9393
WHERE current_database = currentDatabase()
9494
FORMAT Null
95-
SETTINGS query_plan_use_new_logical_join_step = true, query_plan_convert_join_to_in = true;
95+
SETTINGS query_plan_use_new_logical_join_step = true, query_plan_convert_join_to_in = true;

tests/queries/0_stateless/03443_alias_with_asterisk.reference

Whitespace-only changes.
Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1 @@
1+
CREATE TABLE t0 (c0 Int ALIAS if(NULL, 1, *)) ENGINE = Memory; -- { serverError UNKNOWN_IDENTIFIER }

0 commit comments

Comments
 (0)