mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 08:32:02 +00:00
Implement transform_null_in
This commit is contained in:
parent
110eb599c1
commit
53b5dade5e
@ -406,6 +406,7 @@ struct Settings : public SettingsCollection<Settings>
|
||||
M(SettingBool, validate_polygons, true, "Throw exception if polygon is invalid in function pointInPolygon (e.g. self-tangent, self-intersecting). If the setting is false, the function will accept invalid polygons but may silently return wrong result.", 0) \
|
||||
M(SettingUInt64, max_parser_depth, 1000, "Maximum parser depth.", 0) \
|
||||
M(SettingSeconds, temporary_live_view_timeout, DEFAULT_TEMPORARY_LIVE_VIEW_TIMEOUT_SEC, "Timeout after which temporary live view is deleted.", 0) \
|
||||
M(SettingBool, transform_null_in, false, "Enable null verification of the 'IN' operator.", 0) \
|
||||
\
|
||||
/** Obsolete settings that do nothing but left for compatibility reasons. Remove each one after half a year of obsolescence. */ \
|
||||
\
|
||||
|
@ -21,38 +21,62 @@ namespace ErrorCodes
|
||||
* notIn(x, set) - and NOT IN.
|
||||
*/
|
||||
|
||||
template <bool negative, bool global>
|
||||
template <bool negative, bool global, bool null_is_skipped>
|
||||
struct FunctionInName;
|
||||
|
||||
template <>
|
||||
struct FunctionInName<false, false>
|
||||
struct FunctionInName<false, false, true>
|
||||
{
|
||||
static constexpr auto name = "in";
|
||||
};
|
||||
|
||||
template <>
|
||||
struct FunctionInName<false, true>
|
||||
struct FunctionInName<false, true, true>
|
||||
{
|
||||
static constexpr auto name = "globalIn";
|
||||
};
|
||||
|
||||
template <>
|
||||
struct FunctionInName<true, false>
|
||||
struct FunctionInName<true, false, true>
|
||||
{
|
||||
static constexpr auto name = "notIn";
|
||||
};
|
||||
|
||||
template <>
|
||||
struct FunctionInName<true, true>
|
||||
struct FunctionInName<true, true, true>
|
||||
{
|
||||
static constexpr auto name = "globalNotIn";
|
||||
};
|
||||
|
||||
template <bool negative, bool global>
|
||||
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>
|
||||
class FunctionIn : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = FunctionInName<negative, global>::name;
|
||||
static constexpr auto name = FunctionInName<negative, global, null_is_skipped>::name;
|
||||
static FunctionPtr create(const Context &)
|
||||
{
|
||||
return std::make_shared<FunctionIn>();
|
||||
@ -75,6 +99,8 @@ public:
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
bool useDefaultImplementationForNulls() const override { return null_is_skipped; }
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
|
||||
{
|
||||
/// NOTE: after updating this code, check that FunctionIgnoreExceptNull returns the same type of column.
|
||||
@ -123,10 +149,14 @@ public:
|
||||
|
||||
void registerFunctionsIn(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionIn<false, false>>();
|
||||
factory.registerFunction<FunctionIn<false, true>>();
|
||||
factory.registerFunction<FunctionIn<true, false>>();
|
||||
factory.registerFunction<FunctionIn<true, true>>();
|
||||
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>>();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -64,7 +64,7 @@ static NamesAndTypesList::iterator findColumn(const String & name, NamesAndTypes
|
||||
}
|
||||
|
||||
template<typename Collection>
|
||||
static Block createBlockFromCollection(const Collection & collection, const DataTypes & types)
|
||||
static Block createBlockFromCollection(const Collection & collection, const DataTypes & types, const Context & context)
|
||||
{
|
||||
size_t columns_num = types.size();
|
||||
MutableColumns columns(columns_num);
|
||||
@ -77,7 +77,7 @@ static Block createBlockFromCollection(const Collection & collection, const Data
|
||||
if (columns_num == 1)
|
||||
{
|
||||
auto field = convertFieldToType(value, *types[0]);
|
||||
if (!field.isNull())
|
||||
if (!field.isNull() || context.getSettingsRef().transform_null_in)
|
||||
columns[0]->insert(std::move(field));
|
||||
}
|
||||
else
|
||||
@ -100,7 +100,7 @@ static Block createBlockFromCollection(const Collection & collection, const Data
|
||||
for (; i < tuple_size; ++i)
|
||||
{
|
||||
tuple_values[i] = convertFieldToType(tuple[i], *types[i]);
|
||||
if (tuple_values[i].isNull())
|
||||
if (tuple_values[i].isNull() && !context.getSettingsRef().transform_null_in)
|
||||
break;
|
||||
}
|
||||
|
||||
@ -170,23 +170,23 @@ SetPtr makeExplicitSet(
|
||||
if (left_type_depth == right_type_depth)
|
||||
{
|
||||
Array array{right_arg_value};
|
||||
block = createBlockFromCollection(array, set_element_types);
|
||||
block = createBlockFromCollection(array, set_element_types, context);
|
||||
}
|
||||
/// 1 in (1, 2); (1, 2) in ((1, 2), (3, 4)); etc.
|
||||
else if (left_type_depth + 1 == right_type_depth)
|
||||
{
|
||||
auto type_index = right_arg_type->getTypeId();
|
||||
if (type_index == TypeIndex::Tuple)
|
||||
block = createBlockFromCollection(DB::get<const Tuple &>(right_arg_value), set_element_types);
|
||||
block = createBlockFromCollection(DB::get<const Tuple &>(right_arg_value), set_element_types, context);
|
||||
else if (type_index == TypeIndex::Array)
|
||||
block = createBlockFromCollection(DB::get<const Array &>(right_arg_value), set_element_types);
|
||||
block = createBlockFromCollection(DB::get<const Array &>(right_arg_value), set_element_types, context);
|
||||
else
|
||||
throw_unsupported_type(right_arg_type);
|
||||
}
|
||||
else
|
||||
throw_unsupported_type(right_arg_type);
|
||||
|
||||
SetPtr set = std::make_shared<Set>(size_limits, create_ordered_set);
|
||||
SetPtr set = std::make_shared<Set>(size_limits, create_ordered_set, context);
|
||||
|
||||
set->setHeader(block);
|
||||
set->insertFromBlock(block);
|
||||
@ -654,7 +654,7 @@ SetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool no_su
|
||||
return subquery_for_set.set;
|
||||
}
|
||||
|
||||
SetPtr set = std::make_shared<Set>(data.set_size_limit, false);
|
||||
SetPtr set = std::make_shared<Set>(data.set_size_limit, false, data.context);
|
||||
|
||||
/** The following happens for GLOBAL INs:
|
||||
* - in the addExternalStorage function, the IN (SELECT ...) subquery is replaced with IN _data1,
|
||||
|
@ -291,7 +291,7 @@ void SelectQueryExpressionAnalyzer::tryMakeSetForIndexFromSubquery(const ASTPtr
|
||||
auto interpreter_subquery = interpretSubquery(subquery_or_table_name, context, {}, query_options);
|
||||
BlockIO res = interpreter_subquery->execute();
|
||||
|
||||
SetPtr set = std::make_shared<Set>(settings.size_limits_for_set, true);
|
||||
SetPtr set = std::make_shared<Set>(settings.size_limits_for_set, true, context);
|
||||
set->setHeader(res.in->getHeader());
|
||||
|
||||
res.in->readPrefix();
|
||||
|
@ -5,7 +5,7 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
ColumnPtr extractNestedColumnsAndNullMap(ColumnRawPtrs & key_columns, ConstNullMapPtr & null_map)
|
||||
ColumnPtr extractNestedColumnsAndNullMap(ColumnRawPtrs & key_columns, ConstNullMapPtr & null_map, bool exact_null)
|
||||
{
|
||||
ColumnPtr null_map_holder;
|
||||
|
||||
@ -38,7 +38,12 @@ ColumnPtr extractNestedColumnsAndNullMap(ColumnRawPtrs & key_columns, ConstNullM
|
||||
PaddedPODArray<UInt8> & mutable_null_map = assert_cast<ColumnUInt8 &>(*mutable_null_map_holder).getData();
|
||||
const PaddedPODArray<UInt8> & other_null_map = column_nullable->getNullMapData();
|
||||
for (size_t i = 0, size = mutable_null_map.size(); i < size; ++i)
|
||||
mutable_null_map[i] |= other_null_map[i];
|
||||
{
|
||||
if (exact_null)
|
||||
mutable_null_map[i] &= other_null_map[i];
|
||||
else
|
||||
mutable_null_map[i] |= other_null_map[i];
|
||||
}
|
||||
|
||||
null_map_holder = std::move(mutable_null_map_holder);
|
||||
}
|
||||
|
@ -8,6 +8,6 @@ namespace DB
|
||||
* In 'null_map' return a map of positions where at least one column was NULL.
|
||||
* @returns ownership column of null_map.
|
||||
*/
|
||||
ColumnPtr extractNestedColumnsAndNullMap(ColumnRawPtrs & key_columns, ConstNullMapPtr & null_map);
|
||||
ColumnPtr extractNestedColumnsAndNullMap(ColumnRawPtrs & key_columns, ConstNullMapPtr & null_map, bool exact_null = false);
|
||||
|
||||
}
|
||||
|
@ -87,6 +87,8 @@ void NO_INLINE Set::insertFromBlockImplCase(
|
||||
{
|
||||
if ((*null_map)[i])
|
||||
{
|
||||
has_null = true;
|
||||
|
||||
if constexpr (build_filter)
|
||||
{
|
||||
(*out_filter)[i] = false;
|
||||
@ -138,7 +140,7 @@ void Set::setHeader(const Block & header)
|
||||
|
||||
/// We will insert to the Set only keys, where all components are not NULL.
|
||||
ConstNullMapPtr null_map{};
|
||||
ColumnPtr null_map_holder = extractNestedColumnsAndNullMap(key_columns, null_map);
|
||||
ColumnPtr null_map_holder = extractNestedColumnsAndNullMap(key_columns, null_map, context.getSettingsRef().transform_null_in);
|
||||
|
||||
if (fill_set_elements)
|
||||
{
|
||||
@ -228,7 +230,7 @@ static Field extractValueFromNode(const ASTPtr & node, const IDataType & type, c
|
||||
throw Exception("Incorrect element of set. Must be literal or constant expression.", ErrorCodes::INCORRECT_ELEMENT_OF_SET);
|
||||
}
|
||||
|
||||
void Set::createFromAST(const DataTypes & types, ASTPtr node, const Context & context)
|
||||
void Set::createFromAST(const DataTypes & types, ASTPtr node)
|
||||
{
|
||||
/// Will form a block with values from the set.
|
||||
|
||||
@ -249,7 +251,7 @@ void Set::createFromAST(const DataTypes & types, ASTPtr node, const Context & co
|
||||
{
|
||||
Field value = extractValueFromNode(elem, *types[0], context);
|
||||
|
||||
if (!value.isNull())
|
||||
if (!value.isNull() || context.getSettingsRef().transform_null_in)
|
||||
columns[0]->insert(value);
|
||||
}
|
||||
else if (const auto * func = elem->as<ASTFunction>())
|
||||
@ -284,7 +286,7 @@ void Set::createFromAST(const DataTypes & types, ASTPtr node, const Context & co
|
||||
: extractValueFromNode(func->arguments->children[i], *types[i], context);
|
||||
|
||||
/// If at least one of the elements of the tuple has an impossible (outside the range of the type) value, then the entire tuple too.
|
||||
if (value.isNull())
|
||||
if (value.isNull() && !context.getSettings().transform_null_in)
|
||||
break;
|
||||
|
||||
tuple_values[i] = value;
|
||||
@ -348,7 +350,7 @@ ColumnPtr Set::execute(const Block & block, bool negative) const
|
||||
|
||||
/// We will check existence in Set only for keys, where all components are not NULL.
|
||||
ConstNullMapPtr null_map{};
|
||||
ColumnPtr null_map_holder = extractNestedColumnsAndNullMap(key_columns, null_map);
|
||||
ColumnPtr null_map_holder = extractNestedColumnsAndNullMap(key_columns, null_map, context.getSettingsRef().transform_null_in);
|
||||
|
||||
executeOrdinary(key_columns, vec_res, negative, null_map);
|
||||
|
||||
@ -390,7 +392,12 @@ void NO_INLINE Set::executeImplCase(
|
||||
for (size_t i = 0; i < rows; ++i)
|
||||
{
|
||||
if (has_null_map && (*null_map)[i])
|
||||
vec_res[i] = negative;
|
||||
{
|
||||
if (has_null)
|
||||
vec_res[i] = !negative;
|
||||
else
|
||||
vec_res[i] = negative;
|
||||
}
|
||||
else
|
||||
{
|
||||
auto find_result = state.findKey(method.data, i, pool);
|
||||
|
@ -30,9 +30,9 @@ public:
|
||||
/// (that is useful only for checking that some value is in the set and may not store the original values),
|
||||
/// store all set elements in explicit form.
|
||||
/// This is needed for subsequent use for index.
|
||||
Set(const SizeLimits & limits_, bool fill_set_elements_)
|
||||
Set(const SizeLimits & limits_, bool fill_set_elements_, const Context & context_)
|
||||
: log(&Logger::get("Set")),
|
||||
limits(limits_), fill_set_elements(fill_set_elements_)
|
||||
limits(limits_), fill_set_elements(fill_set_elements_), context(context_)
|
||||
{
|
||||
}
|
||||
|
||||
@ -45,7 +45,7 @@ public:
|
||||
* 'types' - types of what are on the left hand side of IN.
|
||||
* 'node' - list of values: 1, 2, 3 or list of tuples: (1, 2), (3, 4), (5, 6).
|
||||
*/
|
||||
void createFromAST(const DataTypes & types, ASTPtr node, const Context & context);
|
||||
void createFromAST(const DataTypes & types, ASTPtr node);
|
||||
|
||||
/** Create a Set from stream.
|
||||
* Call setHeader, then call insertFromBlock for each block.
|
||||
@ -113,6 +113,10 @@ private:
|
||||
/// Do we need to additionally store all elements of the set in explicit form for subsequent use for index.
|
||||
bool fill_set_elements;
|
||||
|
||||
const Context & context;
|
||||
|
||||
bool has_null = false;
|
||||
|
||||
/// Check if set contains all the data.
|
||||
bool is_created = false;
|
||||
|
||||
|
@ -60,25 +60,40 @@ namespace
|
||||
|
||||
using LogAST = DebugASTLog<false>; /// set to true to enable logs
|
||||
|
||||
/// Select implementation of countDistinct based on settings.
|
||||
/// Select implementation of a function based on settings.
|
||||
/// Important that it is done as query rewrite. It means rewritten query
|
||||
/// will be sent to remote servers during distributed query execution,
|
||||
/// and on all remote servers, function implementation will be same.
|
||||
template <char const * func_name>
|
||||
struct CustomizeFunctionsData
|
||||
{
|
||||
using TypeToVisit = ASTFunction;
|
||||
|
||||
const String & count_distinct;
|
||||
const String & customized_func_name;
|
||||
|
||||
void visit(ASTFunction & func, ASTPtr &)
|
||||
{
|
||||
if (Poco::toLower(func.name) == "countdistinct")
|
||||
func.name = count_distinct;
|
||||
if (Poco::toLower(func.name) == func_name)
|
||||
{
|
||||
func.name = customized_func_name;
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
using CustomizeFunctionsMatcher = OneTypeMatcher<CustomizeFunctionsData>;
|
||||
using CustomizeFunctionsVisitor = InDepthNodeVisitor<CustomizeFunctionsMatcher, true>;
|
||||
char countdistinct[] = "countdistinct";
|
||||
using CustomizeFunctionsVisitor = InDepthNodeVisitor<OneTypeMatcher<CustomizeFunctionsData<countdistinct>>, true>;
|
||||
|
||||
char in[] = "in";
|
||||
using CustomizeInVisitor = InDepthNodeVisitor<OneTypeMatcher<CustomizeFunctionsData<in>>, true>;
|
||||
|
||||
char notIn[] = "notin";
|
||||
using CustomizeNotInVisitor = InDepthNodeVisitor<OneTypeMatcher<CustomizeFunctionsData<notIn>>, true>;
|
||||
|
||||
char globalIn[] = "globalin";
|
||||
using CustomizeGlobalInVisitor = InDepthNodeVisitor<OneTypeMatcher<CustomizeFunctionsData<globalIn>>, true>;
|
||||
|
||||
char globalNotIn[] = "globalnotin";
|
||||
using CustomizeGlobalNotInVisitor = InDepthNodeVisitor<OneTypeMatcher<CustomizeFunctionsData<globalNotIn>>, true>;
|
||||
|
||||
|
||||
/// Translate qualified names such as db.table.column, table.column, table_alias.column to names' normal form.
|
||||
@ -889,6 +904,21 @@ void SyntaxAnalyzer::normalize(ASTPtr & query, Aliases & aliases, const Settings
|
||||
CustomizeFunctionsVisitor::Data data{settings.count_distinct_implementation};
|
||||
CustomizeFunctionsVisitor(data).visit(query);
|
||||
|
||||
if (settings.transform_null_in)
|
||||
{
|
||||
CustomizeInVisitor::Data data_null_in{"nullIn"};
|
||||
CustomizeInVisitor(data_null_in).visit(query);
|
||||
|
||||
CustomizeNotInVisitor::Data data_not_null_in{"notNullIn"};
|
||||
CustomizeNotInVisitor(data_not_null_in).visit(query);
|
||||
|
||||
CustomizeGlobalInVisitor::Data data_global_null_in{"globalNullIn"};
|
||||
CustomizeGlobalInVisitor(data_global_null_in).visit(query);
|
||||
|
||||
CustomizeGlobalNotInVisitor::Data data_global_not_null_in{"globalNotNullIn"};
|
||||
CustomizeGlobalNotInVisitor(data_global_not_null_in).visit(query);
|
||||
}
|
||||
|
||||
/// Creates a dictionary `aliases`: alias -> ASTPtr
|
||||
QueryAliasesVisitor(aliases).visit(query);
|
||||
|
||||
|
@ -5,12 +5,12 @@ namespace DB
|
||||
|
||||
inline bool functionIsInOperator(const std::string & name)
|
||||
{
|
||||
return name == "in" || name == "notIn";
|
||||
return name == "in" || name == "notIn" || name == "nullIn" || name == "notNullIn";
|
||||
}
|
||||
|
||||
inline bool functionIsInOrGlobalInOperator(const std::string & name)
|
||||
{
|
||||
return functionIsInOperator(name) || name == "globalIn" || name == "globalNotIn";
|
||||
return functionIsInOperator(name) || name == "globalIn" || name == "globalNotIn" || name == "globalNullIn" || name == "globalNotNullIn";
|
||||
}
|
||||
|
||||
inline bool functionIsLikeOperator(const std::string & name)
|
||||
|
@ -112,7 +112,7 @@ StorageSet::StorageSet(
|
||||
const ConstraintsDescription & constraints_,
|
||||
const Context & context_)
|
||||
: StorageSetOrJoinBase{relative_path_, table_id_, columns_, constraints_, context_},
|
||||
set(std::make_shared<Set>(SizeLimits(), false))
|
||||
set(std::make_shared<Set>(SizeLimits(), false, context_))
|
||||
{
|
||||
Block header = getSampleBlock();
|
||||
header = header.sortColumns();
|
||||
@ -127,7 +127,7 @@ void StorageSet::finishInsert() { set->finishInsert(); }
|
||||
size_t StorageSet::getSize() const { return set->getTotalRowCount(); }
|
||||
|
||||
|
||||
void StorageSet::truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &)
|
||||
void StorageSet::truncate(const ASTPtr &, const Context & context, TableStructureWriteLockHolder &)
|
||||
{
|
||||
Poco::File(path).remove(true);
|
||||
Poco::File(path).createDirectories();
|
||||
@ -137,7 +137,7 @@ void StorageSet::truncate(const ASTPtr &, const Context &, TableStructureWriteLo
|
||||
header = header.sortColumns();
|
||||
|
||||
increment = 0;
|
||||
set = std::make_shared<Set>(SizeLimits(), false);
|
||||
set = std::make_shared<Set>(SizeLimits(), false, context);
|
||||
set->setHeader(header);
|
||||
}
|
||||
|
||||
|
54
tests/queries/0_stateless/01231_operator_null_in.reference
Normal file
54
tests/queries/0_stateless/01231_operator_null_in.reference
Normal file
@ -0,0 +1,54 @@
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
93
tests/queries/0_stateless/01231_operator_null_in.sql
Normal file
93
tests/queries/0_stateless/01231_operator_null_in.sql
Normal file
@ -0,0 +1,93 @@
|
||||
DROP TABLE IF EXISTS null_in;
|
||||
CREATE TABLE null_in (dt DateTime, idx int, i Nullable(int), s Nullable(String)) ENGINE = MergeTree() PARTITION BY dt ORDER BY idx;
|
||||
|
||||
INSERT INTO null_in VALUES (1, 1, 1, '1') (2, 2, NULL, NULL) (3, 3, 3, '3') (4, 4, NULL, NULL) (5, 5, 5, '5');
|
||||
|
||||
SELECT count() == 2 FROM null_in WHERE i in (1, 3, NULL);
|
||||
SELECT count() == 2 FROM null_in WHERE i in range(4);
|
||||
SELECT count() == 2 FROM null_in WHERE s in ('1', '3', NULL);
|
||||
SELECT count() == 2 FROM null_in WHERE i global in (1, 3, NULL);
|
||||
SELECT count() == 2 FROM null_in WHERE i global in range(4);
|
||||
SELECT count() == 2 FROM null_in WHERE s global in ('1', '3', NULL);
|
||||
|
||||
SELECT count() == 1 FROM null_in WHERE i not in (1, 3, NULL);
|
||||
SELECT count() == 1 FROM null_in WHERE i not in range(4);
|
||||
SELECT count() == 1 FROM null_in WHERE s not in ('1', '3', NULL);
|
||||
SELECT count() == 1 FROM null_in WHERE i global not in (1, 3, NULL);
|
||||
SELECT count() == 1 FROM null_in WHERE i global not in range(4);
|
||||
SELECT count() == 1 FROM null_in WHERE s global not in ('1', '3', NULL);
|
||||
|
||||
SET transform_null_in = 1;
|
||||
|
||||
SELECT count() == 4 FROM null_in WHERE i in (1, 3, NULL);
|
||||
SELECT count() == 2 FROM null_in WHERE i in range(4);
|
||||
SELECT count() == 4 FROM null_in WHERE s in ('1', '3', NULL);
|
||||
SELECT count() == 4 FROM null_in WHERE i global in (1, 3, NULL);
|
||||
SELECT count() == 2 FROM null_in WHERE i global in range(4);
|
||||
SELECT count() == 4 FROM null_in WHERE s global in ('1', '3', NULL);
|
||||
|
||||
SELECT count() == 1 FROM null_in WHERE i not in (1, 3, NULL);
|
||||
SELECT count() == 3 FROM null_in WHERE i not in range(4);
|
||||
SELECT count() == 1 FROM null_in WHERE s not in ('1', '3', NULL);
|
||||
SELECT count() == 1 FROM null_in WHERE i global not in (1, 3, NULL);
|
||||
SELECT count() == 3 FROM null_in WHERE i global not in range(4);
|
||||
SELECT count() == 1 FROM null_in WHERE s global not in ('1', '3', NULL);
|
||||
|
||||
SELECT count() == 3 FROM null_in WHERE i not in (1, 3);
|
||||
SELECT count() == 3 FROM null_in WHERE i not in range(4);
|
||||
SELECT count() == 3 FROM null_in WHERE s not in ('1', '3');
|
||||
SELECT count() == 3 FROM null_in WHERE i global not in (1, 3);
|
||||
SELECT count() == 3 FROM null_in WHERE i global not in range(4);
|
||||
SELECT count() == 3 FROM null_in WHERE s global not in ('1', '3');
|
||||
|
||||
DROP TABLE IF EXISTS null_in;
|
||||
|
||||
DROP TABLE IF EXISTS null_in_subquery;
|
||||
CREATE TABLE null_in_subquery (dt DateTime, idx int, i Nullable(UInt64)) ENGINE = MergeTree() PARTITION BY dt ORDER BY idx;
|
||||
INSERT INTO null_in_subquery SELECT number % 3, number, number FROM system.numbers LIMIT 99999;
|
||||
|
||||
SELECT count() == 33333 FROM null_in_subquery WHERE i in (SELECT i FROM null_in_subquery WHERE dt = 0);
|
||||
SELECT count() == 66666 FROM null_in_subquery WHERE i not in (SELECT i FROM null_in_subquery WHERE dt = 1);
|
||||
SELECT count() == 33333 FROM null_in_subquery WHERE i global in (SELECT i FROM null_in_subquery WHERE dt = 2);
|
||||
SELECT count() == 66666 FROM null_in_subquery WHERE i global not in (SELECT i FROM null_in_subquery WHERE dt = 0);
|
||||
|
||||
-- For index column
|
||||
SELECT count() == 33333 FROM null_in_subquery WHERE idx in (SELECT idx FROM null_in_subquery WHERE dt = 0);
|
||||
SELECT count() == 66666 FROM null_in_subquery WHERE idx not in (SELECT idx FROM null_in_subquery WHERE dt = 1);
|
||||
SELECT count() == 33333 FROM null_in_subquery WHERE idx global in (SELECT idx FROM null_in_subquery WHERE dt = 2);
|
||||
SELECT count() == 66666 FROM null_in_subquery WHERE idx global not in (SELECT idx FROM null_in_subquery WHERE dt = 0);
|
||||
|
||||
INSERT INTO null_in_subquery VALUES (0, 123456780, NULL);
|
||||
INSERT INTO null_in_subquery VALUES (1, 123456781, NULL);
|
||||
|
||||
SELECT count() == 33335 FROM null_in_subquery WHERE i in (SELECT i FROM null_in_subquery WHERE dt = 0);
|
||||
SELECT count() == 66666 FROM null_in_subquery WHERE i not in (SELECT i FROM null_in_subquery WHERE dt = 1);
|
||||
SELECT count() == 33333 FROM null_in_subquery WHERE i in (SELECT i FROM null_in_subquery WHERE dt = 2);
|
||||
SELECT count() == 66668 FROM null_in_subquery WHERE i not in (SELECT i FROM null_in_subquery WHERE dt = 2);
|
||||
SELECT count() == 33335 FROM null_in_subquery WHERE i global in (SELECT i FROM null_in_subquery WHERE dt = 0);
|
||||
SELECT count() == 66666 FROM null_in_subquery WHERE i global not in (SELECT i FROM null_in_subquery WHERE dt = 1);
|
||||
SELECT count() == 33333 FROM null_in_subquery WHERE i global in (SELECT i FROM null_in_subquery WHERE dt = 2);
|
||||
SELECT count() == 66668 FROM null_in_subquery WHERE i global not in (SELECT i FROM null_in_subquery WHERE dt = 2);
|
||||
|
||||
DROP TABLE IF EXISTS null_in_subquery;
|
||||
|
||||
|
||||
DROP TABLE IF EXISTS null_in_tuple;
|
||||
CREATE TABLE null_in_tuple (dt DateTime, idx int, t Tuple(Nullable(UInt64), Nullable(String))) ENGINE = MergeTree() PARTITION BY dt ORDER BY idx;
|
||||
INSERT INTO null_in_tuple VALUES (1, 1, (1, '1')) (2, 2, (2, NULL)) (3, 3, (NULL, '3')) (4, 4, (NULL, NULL))
|
||||
|
||||
SET transform_null_in = 0;
|
||||
|
||||
SELECT arraySort(x -> (x.1, x.2), groupArray(t)) == [(1, '1')] FROM null_in_tuple WHERE t in ((1, '1'), (NULL, NULL));
|
||||
SELECT arraySort(x -> (x.1, x.2), groupArray(t)) == [(2, NULL), (NULL, '3'), (NULL, NULL)] FROM null_in_tuple WHERE t not in ((1, '1'), (NULL, NULL));
|
||||
SELECT arraySort(x -> (x.1, x.2), groupArray(t)) == [(1, '1')] FROM null_in_tuple WHERE t global in ((1, '1'), (NULL, NULL));
|
||||
SELECT arraySort(x -> (x.1, x.2), groupArray(t)) == [(2, NULL), (NULL, '3'), (NULL, NULL)] FROM null_in_tuple WHERE t global not in ((1, '1'), (NULL, NULL));
|
||||
|
||||
SET transform_null_in = 1;
|
||||
|
||||
SELECT arraySort(x -> (x.1, x.2), groupArray(t)) == [(1, '1'), (NULL, NULL)] FROM null_in_tuple WHERE t in ((1, '1'), (NULL, NULL));
|
||||
SELECT arraySort(x -> (x.1, x.2), groupArray(t)) == [(2, NULL), (NULL, '3')] FROM null_in_tuple WHERE t not in ((1, '1'), (NULL, NULL));
|
||||
SELECT arraySort(x -> (x.1, x.2), groupArray(t)) == [(1, '1'), (NULL, NULL)] FROM null_in_tuple WHERE t global in ((1, '1'), (NULL, NULL));
|
||||
SELECT arraySort(x -> (x.1, x.2), groupArray(t)) == [(2, NULL), (NULL, '3')] FROM null_in_tuple WHERE t global not in ((1, '1'), (NULL, NULL));
|
||||
|
||||
DROP TABLE IF EXISTS null_in_subquery;
|
Loading…
Reference in New Issue
Block a user