mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 08:02:02 +00:00
Fix incompatible result type for scalar queries with empty result.
This commit is contained in:
parent
6ebc11ff4e
commit
d996d0bae9
@ -181,7 +181,10 @@ ColumnPtr IExecutableFunction::defaultImplementationForNulls(
|
||||
{
|
||||
// Default implementation for nulls returns null result for null arguments,
|
||||
// so the result type must be nullable.
|
||||
assert(result_type->isNullable());
|
||||
if (!result_type->isNullable())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR,
|
||||
"Function {} with Null argument and default implementation for Nulls "
|
||||
"expect to return Nullable result, got {}", result_type->getName());
|
||||
|
||||
return result_type->createColumnConstWithDefaultValue(input_rows_count);
|
||||
}
|
||||
|
@ -1,9 +1,11 @@
|
||||
#include <Interpreters/ExecuteScalarSubqueriesVisitor.h>
|
||||
|
||||
#include <Columns/ColumnTuple.h>
|
||||
#include <Columns/ColumnNullable.h>
|
||||
#include <DataStreams/IBlockInputStream.h>
|
||||
#include <DataStreams/materializeBlock.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
|
||||
@ -119,8 +121,26 @@ void ExecuteScalarSubqueriesMatcher::visit(const ASTSubquery & subquery, ASTPtr
|
||||
|
||||
if (block.rows() == 0)
|
||||
{
|
||||
auto types = interpreter.getSampleBlock().getDataTypes();
|
||||
if (types.size() != 1)
|
||||
types = {std::make_shared<DataTypeTuple>(types)};
|
||||
|
||||
auto & type = types[0];
|
||||
if (!type->isNullable())
|
||||
{
|
||||
if (!type->canBeInsideNullable())
|
||||
throw Exception(ErrorCodes::INCORRECT_RESULT_OF_SCALAR_SUBQUERY,
|
||||
"Scalar subquery returned empty result of type {} which cannot be Nullable",
|
||||
type->getName());
|
||||
|
||||
type = makeNullable(type);
|
||||
}
|
||||
|
||||
/// Interpret subquery with empty result as Null literal
|
||||
auto ast_new = std::make_unique<ASTLiteral>(Null());
|
||||
auto ast_new = makeASTFunction(
|
||||
"CAST",
|
||||
std::make_unique<ASTLiteral>(Null()),
|
||||
std::make_unique<ASTLiteral>(type->getName()));
|
||||
ast_new->setAlias(ast->tryGetAlias());
|
||||
ast = std::move(ast_new);
|
||||
return;
|
||||
@ -140,10 +160,20 @@ void ExecuteScalarSubqueriesMatcher::visit(const ASTSubquery & subquery, ASTPtr
|
||||
size_t columns = block.columns();
|
||||
|
||||
if (columns == 1)
|
||||
{
|
||||
auto & column = block.getByPosition(0);
|
||||
/// Here we wrap type to nullable if we can.
|
||||
/// It is needed cause if subquery return no rows, it's result will be Null.
|
||||
/// In case of many columns, do not check it cause tuple can't be nullable.
|
||||
if (!column.type->isNullable() && column.type->canBeInsideNullable())
|
||||
{
|
||||
column.type = makeNullable(column.type);
|
||||
column.column = makeNullable(column.column);
|
||||
}
|
||||
scalar = block;
|
||||
}
|
||||
else
|
||||
{
|
||||
|
||||
ColumnWithTypeAndName ctn;
|
||||
ctn.type = std::make_shared<DataTypeTuple>(block.getDataTypes());
|
||||
ctn.column = ColumnTuple::create(block.getColumns());
|
||||
|
@ -3,3 +3,6 @@
|
||||
1 1
|
||||
('2015-01-02','Hello')
|
||||
('2015-01-02','Hello') ('2015-01-02','Hello') 1 1
|
||||
\N
|
||||
(1,2)
|
||||
[1]
|
||||
|
@ -8,3 +8,52 @@ SELECT (SELECT toDate('2015-01-02'), 'Hello') AS x, x, identity((SELECT 1)), ide
|
||||
-- SELECT (SELECT uniqState(''));
|
||||
|
||||
SELECT ( SELECT throwIf(1 + dummy) ); -- { serverError 395 }
|
||||
|
||||
-- Scalar subquery with 0 rows must return Null
|
||||
SELECT (SELECT 1 WHERE 0);
|
||||
-- But tuple and array can't be inside nullable
|
||||
SELECT (SELECT 1, 2 WHERE 0); -- { serverError 125 }
|
||||
SELECT (SELECT [1] WHERE 0); -- { serverError 125 }
|
||||
-- Works for not-empty casle
|
||||
SELECT (SELECT 1, 2);
|
||||
SELECT (SELECT [1]);
|
||||
-- Several rows
|
||||
SELECT (SELECT number FROM numbers(2)); -- { serverError 125 }
|
||||
|
||||
-- Bug reproduction form #25411
|
||||
WITH a AS (select (select 1 WHERE 0) as b)
|
||||
select 1
|
||||
from system.one
|
||||
cross join a
|
||||
where a.b = 0;
|
||||
|
||||
-- Reported query
|
||||
drop table if exists t_q1ht4gq_5;
|
||||
create table t_q1ht4gq_5 (c_zeij INTEGER NOT NULL, c_fehk75l TEXT, c_jz TEXT, c_wynzuek TEXT, c_nkt INTEGER NOT NULL, c_g TEXT, c_mc2 TEXT, primary key(c_nkt)) engine = MergeTree();
|
||||
WITH
|
||||
cte_0 AS (select
|
||||
subq_0.c6 as c2,
|
||||
case when 0<>0 then ((select c_zeij from t_q1ht4gq_5 order by c_zeij limit 1 offset 1)
|
||||
+ subq_0.c4) else ((select c_zeij from t_q1ht4gq_5 order by c_zeij limit 1 offset 1)
|
||||
+ subq_0.c4) end as c4
|
||||
from
|
||||
(select
|
||||
ref_0.c_nkt as c4,
|
||||
ref_0.c_nkt as c6
|
||||
from
|
||||
t_q1ht4gq_5 as ref_0
|
||||
) as subq_0
|
||||
)
|
||||
select
|
||||
ref_12.c_zeij as c3
|
||||
from
|
||||
t_q1ht4gq_5 as ref_12
|
||||
where (ref_12.c_jz not in (
|
||||
select
|
||||
ref_14.c_mc2 as c0
|
||||
from
|
||||
t_q1ht4gq_5 as ref_14
|
||||
cross join cte_0 as ref_15
|
||||
where ref_15.c4 > ref_15.c2));
|
||||
|
||||
drop table if exists t_q1ht4gq_5;
|
||||
|
Loading…
Reference in New Issue
Block a user