Skip to content

Commit 633f6d4

Browse files
authored
Merge pull request ClickHouse#78828 from ClickHouse/fix-bad-cast
Fix bad cast in `StorageDistributed`
2 parents 6b007ab + ac7b785 commit 633f6d4

File tree

5 files changed

+39
-6
lines changed

5 files changed

+39
-6
lines changed

src/Parsers/ASTSelectQuery.cpp

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -448,7 +448,7 @@ void ASTSelectQuery::replaceDatabaseAndTable(const StorageID & table_id)
448448
}
449449

450450

451-
void ASTSelectQuery::addTableFunction(ASTPtr & table_function_ptr)
451+
void ASTSelectQuery::addTableFunction(const ASTPtr & table_function_ptr)
452452
{
453453
ASTTableExpression * table_expression = getFirstTableExpression(*this);
454454

src/Parsers/ASTSelectQuery.h

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -153,7 +153,7 @@ class ASTSelectQuery : public IAST
153153
bool withFill() const;
154154
void replaceDatabaseAndTable(const String & database_name, const String & table_name);
155155
void replaceDatabaseAndTable(const StorageID & table_id);
156-
void addTableFunction(ASTPtr & table_function_ptr);
156+
void addTableFunction(const ASTPtr & table_function_ptr);
157157
void updateTreeHashImpl(SipHash & hash_state, bool ignore_aliases) const override;
158158

159159
void setFinal();

src/Storages/StorageDistributed.cpp

Lines changed: 21 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -37,6 +37,7 @@
3737
#include <Common/threadPoolCallbackRunner.h>
3838
#include <Common/typeid_cast.h>
3939

40+
#include <Parsers/ASTAsterisk.h>
4041
#include <Parsers/ASTExpressionList.h>
4142
#include <Parsers/ASTFunction.h>
4243
#include <Parsers/ASTIdentifier.h>
@@ -970,14 +971,30 @@ std::optional<QueryPipeline> StorageDistributed::distributedWriteBetweenDistribu
970971
const auto & settings = local_context->getSettingsRef();
971972
auto new_query = std::dynamic_pointer_cast<ASTInsertQuery>(query.clone());
972973

973-
/// Unwrap view() function.
974974
if (src_distributed.remote_table_function_ptr)
975975
{
976976
const TableFunctionPtr src_table_function =
977977
TableFunctionFactory::instance().get(src_distributed.remote_table_function_ptr, local_context);
978-
const TableFunctionView * view_function =
979-
assert_cast<const TableFunctionView *>(src_table_function.get());
980-
new_query->select = view_function->getSelectQuery().clone();
978+
if (const TableFunctionView * view_function = typeid_cast<const TableFunctionView *>(src_table_function.get()))
979+
{
980+
new_query->select = view_function->getSelectQuery().clone();
981+
}
982+
else
983+
{
984+
const auto select_with_union_query = std::make_shared<ASTSelectWithUnionQuery>();
985+
select_with_union_query->list_of_selects = std::make_shared<ASTExpressionList>();
986+
987+
const auto select = std::make_shared<ASTSelectQuery>();
988+
989+
auto expression_list = std::make_shared<ASTExpressionList>();
990+
expression_list->children.push_back(std::make_shared<ASTAsterisk>());
991+
select->setExpression(ASTSelectQuery::Expression::SELECT, expression_list->clone());
992+
select->addTableFunction(src_distributed.remote_table_function_ptr);
993+
994+
select_with_union_query->list_of_selects->children.push_back(select->clone());
995+
996+
new_query->select = select_with_union_query;
997+
}
981998
}
982999
else
9831000
{

tests/queries/0_stateless/01099_parallel_distributed_insert_select.reference

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -28,6 +28,11 @@ test_shard_localhost
2828
0
2929
1
3030
2
31+
0
32+
1
33+
2
34+
3
35+
4
3136
test_cluster_two_shards_localhost
3237
0 2
3338
1 2

tests/queries/0_stateless/01099_parallel_distributed_insert_select.sql

Lines changed: 11 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -169,6 +169,17 @@ DROP TABLE local_01099_b;
169169
DROP TABLE distributed_01099_a;
170170
DROP TABLE distributed_01099_b;
171171

172+
--- https://github.com/ClickHouse/ClickHouse/issues/78464
173+
CREATE TABLE local_01099_c (n UInt64) ENGINE = Log;
174+
CREATE TABLE distributed_01099_c AS local_01099_c ENGINE = Distributed('test_shard_localhost', currentDatabase(), local_01099_c, rand());
175+
176+
INSERT INTO TABLE FUNCTION clusterAllReplicas('test_shard_localhost', currentDatabase(), 'distributed_01099_c') (n) SELECT number FROM remote('localhost', numbers(5)) tx;
177+
178+
SELECT * FROM distributed_01099_c;
179+
180+
DROP TABLE local_01099_c;
181+
DROP TABLE distributed_01099_c;
182+
172183
--
173184
-- test_cluster_two_shards_localhost
174185
--

0 commit comments

Comments
 (0)