mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Fix notNullIn with NULL argument.
This commit is contained in:
parent
a808af85b5
commit
b556b7c76e
@ -1,56 +0,0 @@
|
||||
#include <Functions/IFunctionImpl.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** ignoreExceptNull(...) is a function that takes any arguments, and always returns 0 except Null.
|
||||
*/
|
||||
class FunctionIgnoreExceptNull : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "ignoreExceptNull";
|
||||
static FunctionPtr create(const Context &)
|
||||
{
|
||||
return std::make_shared<FunctionIgnoreExceptNull>();
|
||||
}
|
||||
|
||||
bool isVariadic() const override
|
||||
{
|
||||
return true;
|
||||
}
|
||||
size_t getNumberOfArguments() const override
|
||||
{
|
||||
return 0;
|
||||
}
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override
|
||||
{
|
||||
return std::make_shared<DataTypeUInt8>();
|
||||
}
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override
|
||||
{
|
||||
/// This function is mainly used in query analysis instead of "in" functions
|
||||
/// in the case when only header is needed and set for in is not calculated.
|
||||
/// Because of that function must return the same column type as "in" function, which is ColumnUInt8.
|
||||
auto res = ColumnUInt8::create(input_rows_count, 0);
|
||||
block.getByPosition(result).column = std::move(res);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
void registerFunctionIgnoreExceptNull(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionIgnoreExceptNull>();
|
||||
}
|
||||
|
||||
}
|
@ -21,62 +21,34 @@ namespace ErrorCodes
|
||||
* notIn(x, set) - and NOT IN.
|
||||
*/
|
||||
|
||||
template <bool negative, bool global, bool null_is_skipped>
|
||||
template <bool negative, bool global, bool null_is_skipped, bool ignore_set>
|
||||
struct FunctionInName;
|
||||
|
||||
template <>
|
||||
struct FunctionInName<false, false, true>
|
||||
{
|
||||
static constexpr auto name = "in";
|
||||
};
|
||||
template <> struct FunctionInName<false, false, true, false> { static constexpr auto name = "in"; };
|
||||
template <> struct FunctionInName<false, true, true, false> { static constexpr auto name = "globalIn"; };
|
||||
template <> struct FunctionInName<true, false, true, false> { static constexpr auto name = "notIn"; };
|
||||
template <> struct FunctionInName<true, true, true, false> { static constexpr auto name = "globalNotIn"; };
|
||||
template <> struct FunctionInName<false, false, false, false> { static constexpr auto name = "nullIn"; };
|
||||
template <> struct FunctionInName<false, true, false, false> { static constexpr auto name = "globalNullIn"; };
|
||||
template <> struct FunctionInName<true, false, false, false> { static constexpr auto name = "notNullIn"; };
|
||||
template <> struct FunctionInName<true, true, false, false> { static constexpr auto name = "globalNotNullIn"; };
|
||||
template <> struct FunctionInName<false, false, true, true> { static constexpr auto name = "inIgnoreSet"; };
|
||||
template <> struct FunctionInName<false, true, true, true> { static constexpr auto name = "globalInIgnoreSet"; };
|
||||
template <> struct FunctionInName<true, false, true, true> { static constexpr auto name = "notInIgnoreSet"; };
|
||||
template <> struct FunctionInName<true, true, true, true> { static constexpr auto name = "globalNotInIgnoreSet"; };
|
||||
template <> struct FunctionInName<false, false, false, true> { static constexpr auto name = "nullInIgnoreSet"; };
|
||||
template <> struct FunctionInName<false, true, false, true> { static constexpr auto name = "globalNullInIgnoreSet"; };
|
||||
template <> struct FunctionInName<true, false, false, true> { static constexpr auto name = "notNullInIgnoreSet"; };
|
||||
template <> struct FunctionInName<true, true, false, true> { static constexpr auto name = "globalNotNullInIgnoreSet"; };
|
||||
|
||||
template <>
|
||||
struct FunctionInName<false, true, true>
|
||||
{
|
||||
static constexpr auto name = "globalIn";
|
||||
};
|
||||
|
||||
template <>
|
||||
struct FunctionInName<true, false, true>
|
||||
{
|
||||
static constexpr auto name = "notIn";
|
||||
};
|
||||
|
||||
template <>
|
||||
struct FunctionInName<true, true, true>
|
||||
{
|
||||
static constexpr auto name = "globalNotIn";
|
||||
};
|
||||
|
||||
template <>
|
||||
struct FunctionInName<false, false, false>
|
||||
{
|
||||
static constexpr auto name = "nullIn";
|
||||
};
|
||||
|
||||
template <>
|
||||
struct FunctionInName<false, true, false>
|
||||
{
|
||||
static constexpr auto name = "globalNullIn";
|
||||
};
|
||||
|
||||
template <>
|
||||
struct FunctionInName<true, false, false>
|
||||
{
|
||||
static constexpr auto name = "notNullIn";
|
||||
};
|
||||
|
||||
template <>
|
||||
struct FunctionInName<true, true, false>
|
||||
{
|
||||
static constexpr auto name = "globalNotNullIn";
|
||||
};
|
||||
|
||||
template <bool negative, bool global, bool null_is_skipped>
|
||||
template <bool negative, bool global, bool null_is_skipped, bool ignore_set>
|
||||
class FunctionIn : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = FunctionInName<negative, global, null_is_skipped>::name;
|
||||
/// ignore_set flag means that we don't use set from the second argument, just return zero column.
|
||||
/// It is needed to perform type analysis without creation of set.
|
||||
static constexpr auto name = FunctionInName<negative, global, null_is_skipped, ignore_set>::name;
|
||||
|
||||
static FunctionPtr create(const Context &)
|
||||
{
|
||||
return std::make_shared<FunctionIn>();
|
||||
@ -101,9 +73,13 @@ public:
|
||||
|
||||
bool useDefaultImplementationForNulls() const override { return null_is_skipped; }
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, [[maybe_unused]] size_t input_rows_count) override
|
||||
{
|
||||
/// NOTE: after updating this code, check that FunctionIgnoreExceptNull returns the same type of column.
|
||||
if constexpr (ignore_set)
|
||||
{
|
||||
block.getByPosition(result).column = DataTypeUInt8().createColumnConst(input_rows_count, 0u);
|
||||
return;
|
||||
}
|
||||
|
||||
/// Second argument must be ColumnSet.
|
||||
ColumnPtr column_set_ptr = block.getByPosition(arguments[1]).column;
|
||||
@ -146,17 +122,23 @@ public:
|
||||
}
|
||||
};
|
||||
|
||||
template<bool ignore_set>
|
||||
static void registerFunctionsInImpl(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionIn<false, false, true, ignore_set>>();
|
||||
factory.registerFunction<FunctionIn<false, true, true, ignore_set>>();
|
||||
factory.registerFunction<FunctionIn<true, false, true, ignore_set>>();
|
||||
factory.registerFunction<FunctionIn<true, true, true, ignore_set>>();
|
||||
factory.registerFunction<FunctionIn<false, false, false, ignore_set>>();
|
||||
factory.registerFunction<FunctionIn<false, true, false, ignore_set>>();
|
||||
factory.registerFunction<FunctionIn<true, false, false, ignore_set>>();
|
||||
factory.registerFunction<FunctionIn<true, true, false, ignore_set>>();
|
||||
}
|
||||
|
||||
void registerFunctionsIn(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionIn<false, false, true>>();
|
||||
factory.registerFunction<FunctionIn<false, true, true>>();
|
||||
factory.registerFunction<FunctionIn<true, false, true>>();
|
||||
factory.registerFunction<FunctionIn<true, true, true>>();
|
||||
factory.registerFunction<FunctionIn<false, false, false>>();
|
||||
factory.registerFunction<FunctionIn<false, true, false>>();
|
||||
factory.registerFunction<FunctionIn<true, false, false>>();
|
||||
factory.registerFunction<FunctionIn<true, true, false>>();
|
||||
registerFunctionsInImpl<false>(factory);
|
||||
registerFunctionsInImpl<true>(factory);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -29,7 +29,6 @@ void registerFunctionSleep(FunctionFactory &);
|
||||
void registerFunctionSleepEachRow(FunctionFactory &);
|
||||
void registerFunctionMaterialize(FunctionFactory &);
|
||||
void registerFunctionIgnore(FunctionFactory &);
|
||||
void registerFunctionIgnoreExceptNull(FunctionFactory &);
|
||||
void registerFunctionIdentity(FunctionFactory &);
|
||||
void registerFunctionArrayJoin(FunctionFactory &);
|
||||
void registerFunctionReplicate(FunctionFactory &);
|
||||
@ -88,7 +87,6 @@ void registerFunctionsMiscellaneous(FunctionFactory & factory)
|
||||
registerFunctionSleepEachRow(factory);
|
||||
registerFunctionMaterialize(factory);
|
||||
registerFunctionIgnore(factory);
|
||||
registerFunctionIgnoreExceptNull(factory);
|
||||
registerFunctionIdentity(factory);
|
||||
registerFunctionArrayJoin(factory);
|
||||
registerFunctionReplicate(factory);
|
||||
|
@ -381,11 +381,13 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data &
|
||||
if (!data.only_consts)
|
||||
{
|
||||
/// We are in the part of the tree that we are not going to compute. You just need to define types.
|
||||
/// Do not subquery and create sets. We treat "IN" as "ignoreExceptNull" function.
|
||||
/// Do not subquery and create sets. We replace "in*" function to "in*IgnoreSet".
|
||||
|
||||
auto argument_name = node.arguments->children.at(0)->getColumnName();
|
||||
|
||||
data.addAction(ExpressionAction::applyFunction(
|
||||
FunctionFactory::instance().get("ignoreExceptNull", data.context),
|
||||
{ node.arguments->children.at(0)->getColumnName() },
|
||||
FunctionFactory::instance().get(node.name + "IgnoreSet", data.context),
|
||||
{ argument_name, argument_name },
|
||||
column_name.get(ast)));
|
||||
}
|
||||
return;
|
||||
|
Loading…
Reference in New Issue
Block a user