Fix incompatible result type for scalar queries with empty result.

This commit is contained in:
Nikolai Kochetov 2021-07-16 16:38:35 +03:00
parent 6ebc11ff4e
commit d996d0bae9
4 changed files with 88 additions and 3 deletions

View File

@ -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);
}

View File

@ -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());

View File

@ -3,3 +3,6 @@
1 1
('2015-01-02','Hello')
('2015-01-02','Hello') ('2015-01-02','Hello') 1 1
\N
(1,2)
[1]

View File

@ -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;