Fix notNullIn with NULL argument.

This commit is contained in:
Nikolai Kochetov 2020-05-14 17:21:38 +03:00
parent a808af85b5
commit b556b7c76e
4 changed files with 47 additions and 121 deletions

View File

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

View File

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

View File

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

View File

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