mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 00:22:29 +00:00
Merge pull request #23044 from ClickHouse/array-has-any-all-generic
Fix unfinished code in GatherUtils #20272
This commit is contained in:
commit
d3f8a9c23a
@ -82,7 +82,7 @@ inline ALWAYS_INLINE void writeSlice(const GenericArraySlice & slice, GenericArr
|
||||
sink.current_offset += slice.size;
|
||||
}
|
||||
else
|
||||
throw Exception("Function writeSlice expect same column types for GenericArraySlice and GenericArraySink.",
|
||||
throw Exception("Function writeSlice expects same column types for GenericArraySlice and GenericArraySink.",
|
||||
ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
@ -162,7 +162,7 @@ inline ALWAYS_INLINE void writeSlice(const GenericValueSlice & slice, GenericArr
|
||||
++sink.current_offset;
|
||||
}
|
||||
else
|
||||
throw Exception("Function writeSlice expect same column types for GenericValueSlice and GenericArraySink.",
|
||||
throw Exception("Function writeSlice expects same column types for GenericValueSlice and GenericArraySink.",
|
||||
ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
@ -609,7 +609,7 @@ bool sliceHas(const GenericArraySlice & first, const GenericArraySlice & second)
|
||||
{
|
||||
/// Generic arrays should have the same type in order to use column.compareAt(...)
|
||||
if (!first.elements->structureEquals(*second.elements))
|
||||
return false;
|
||||
throw Exception("Function sliceHas expects same column types for slices.", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
auto impl = sliceHasImpl<search_type, GenericArraySlice, GenericArraySlice, sliceEqualElements, insliceEqualElements>;
|
||||
return impl(first, second, nullptr, nullptr);
|
||||
@ -670,7 +670,7 @@ void NO_INLINE arrayAllAny(FirstSource && first, SecondSource && second, ColumnU
|
||||
auto & data = result.getData();
|
||||
for (auto row : ext::range(0, size))
|
||||
{
|
||||
data[row] = static_cast<UInt8>(sliceHas<search_type>(first.getWhole(), second.getWhole()) ? 1 : 0);
|
||||
data[row] = static_cast<UInt8>(sliceHas<search_type>(first.getWhole(), second.getWhole()));
|
||||
first.next();
|
||||
second.next();
|
||||
}
|
||||
|
@ -5,6 +5,7 @@
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/getLeastSupertype.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnFixedString.h>
|
||||
@ -13,9 +14,9 @@
|
||||
#include <Common/FieldVisitorsAccurateComparison.h>
|
||||
#include <Common/memcmpSmall.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include "Columns/ColumnLowCardinality.h"
|
||||
#include "DataTypes/DataTypeLowCardinality.h"
|
||||
#include "Interpreters/castColumn.h"
|
||||
#include <Columns/ColumnLowCardinality.h>
|
||||
#include <DataTypes/DataTypeLowCardinality.h>
|
||||
#include <Interpreters/castColumn.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -373,11 +374,10 @@ public:
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
if (!arguments[1]->onlyNull() && !allowArguments(array_type->getNestedType(), arguments[1]))
|
||||
throw Exception("Types of array and 2nd argument of function \""
|
||||
+ getName() + "\" must be identical up to nullability, cardinality, "
|
||||
"numeric types, or Enum and numeric type. Passed: "
|
||||
+ arguments[0]->getName() + " and " + arguments[1]->getName() + ".",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Types of array and 2nd argument of function `{}` must be identical up to nullability, cardinality, "
|
||||
"numeric types, or Enum and numeric type. Passed: {} and {}.",
|
||||
getName(), arguments[0]->getName(), arguments[1]->getName());
|
||||
|
||||
return std::make_shared<DataTypeNumber<ResultType>>();
|
||||
}
|
||||
@ -494,86 +494,13 @@ private:
|
||||
inline void moveResult() { result_column = std::move(result); }
|
||||
};
|
||||
|
||||
static inline bool allowNested(const DataTypePtr & left, const DataTypePtr & right)
|
||||
{
|
||||
return ((isNativeNumber(left) || isEnum(left)) && isNativeNumber(right)) || left->equals(*right);
|
||||
}
|
||||
|
||||
static inline bool allowArguments(const DataTypePtr & array_inner_type, const DataTypePtr & arg)
|
||||
{
|
||||
if (allowNested(array_inner_type, arg))
|
||||
return true;
|
||||
auto inner_type_decayed = removeNullable(removeLowCardinality(array_inner_type));
|
||||
auto arg_decayed = removeNullable(removeLowCardinality(arg));
|
||||
|
||||
/// Nullable
|
||||
|
||||
const bool array_is_nullable = array_inner_type->isNullable();
|
||||
const bool arg_is_nullable = arg->isNullable();
|
||||
|
||||
const DataTypePtr arg_or_arg_nullable_nested = arg_is_nullable
|
||||
? checkAndGetDataType<DataTypeNullable>(arg.get())->getNestedType()
|
||||
: arg;
|
||||
|
||||
if (array_is_nullable) // comparing Array(Nullable(T)) elem and U
|
||||
{
|
||||
const DataTypePtr array_nullable_nested =
|
||||
checkAndGetDataType<DataTypeNullable>(array_inner_type.get())->getNestedType();
|
||||
|
||||
// We also allow Nullable(T) and LC(U) if the Nullable(T) and U are allowed,
|
||||
// the LC(U) will be converted to U.
|
||||
return allowNested(
|
||||
array_nullable_nested,
|
||||
recursiveRemoveLowCardinality(arg_or_arg_nullable_nested));
|
||||
}
|
||||
else if (arg_is_nullable) // cannot compare Array(T) elem (namely, T) and Nullable(T)
|
||||
return false;
|
||||
|
||||
/// LowCardinality
|
||||
|
||||
const auto * const array_lc_ptr = checkAndGetDataType<DataTypeLowCardinality>(array_inner_type.get());
|
||||
const auto * const arg_lc_ptr = checkAndGetDataType<DataTypeLowCardinality>(arg.get());
|
||||
|
||||
const DataTypePtr array_lc_inner_type = recursiveRemoveLowCardinality(array_inner_type);
|
||||
const DataTypePtr arg_lc_inner_type = recursiveRemoveLowCardinality(arg);
|
||||
|
||||
const bool array_is_lc = nullptr != array_lc_ptr;
|
||||
const bool arg_is_lc = nullptr != arg_lc_ptr;
|
||||
|
||||
const bool array_lc_inner_type_is_nullable = array_is_lc && array_lc_inner_type->isNullable();
|
||||
const bool arg_lc_inner_type_is_nullable = arg_is_lc && arg_lc_inner_type->isNullable();
|
||||
|
||||
if (array_is_lc) // comparing LC(T) and U
|
||||
{
|
||||
const DataTypePtr array_lc_nested_or_lc_nullable_nested = array_lc_inner_type_is_nullable
|
||||
? checkAndGetDataType<DataTypeNullable>(array_lc_inner_type.get())->getNestedType()
|
||||
: array_lc_inner_type;
|
||||
|
||||
if (arg_is_lc) // comparing LC(T) and LC(U)
|
||||
{
|
||||
const DataTypePtr arg_lc_nested_or_lc_nullable_nested = arg_lc_inner_type_is_nullable
|
||||
? checkAndGetDataType<DataTypeNullable>(arg_lc_inner_type.get())->getNestedType()
|
||||
: arg_lc_inner_type;
|
||||
|
||||
return allowNested(
|
||||
array_lc_nested_or_lc_nullable_nested,
|
||||
arg_lc_nested_or_lc_nullable_nested);
|
||||
}
|
||||
else if (arg_is_nullable) // Comparing LC(T) and Nullable(U)
|
||||
{
|
||||
if (!array_lc_inner_type_is_nullable)
|
||||
return false; // Can't compare Array(LC(U)) elem and Nullable(T);
|
||||
|
||||
return allowNested(
|
||||
array_lc_nested_or_lc_nullable_nested,
|
||||
arg_or_arg_nullable_nested);
|
||||
}
|
||||
else // Comparing LC(T) and U (U neither Nullable nor LC)
|
||||
return allowNested(array_lc_nested_or_lc_nullable_nested, arg);
|
||||
}
|
||||
|
||||
if (arg_is_lc) // Allow T and LC(U) if U and T are allowed (the low cardinality column will be converted).
|
||||
return allowNested(array_inner_type, arg_lc_inner_type);
|
||||
|
||||
return false;
|
||||
return ((isNativeNumber(inner_type_decayed) || isEnum(inner_type_decayed)) && isNativeNumber(arg_decayed))
|
||||
|| getLeastSupertype({inner_type_decayed, arg_decayed});
|
||||
}
|
||||
|
||||
#define INTEGRAL_TPL_PACK UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64, Float32, Float64
|
||||
@ -1044,33 +971,38 @@ private:
|
||||
if (!col)
|
||||
return nullptr;
|
||||
|
||||
const IColumn & col_nested = col->getData();
|
||||
DataTypePtr array_elements_type = assert_cast<const DataTypeArray &>(*arguments[0].type).getNestedType();
|
||||
const DataTypePtr & index_type = arguments[1].type;
|
||||
|
||||
DataTypePtr common_type = getLeastSupertype({array_elements_type, index_type});
|
||||
|
||||
ColumnPtr col_nested = castColumn({ col->getDataPtr(), array_elements_type, "" }, common_type);
|
||||
|
||||
const ColumnPtr right_ptr = arguments[1].column->convertToFullColumnIfLowCardinality();
|
||||
const IColumn & item_arg = *right_ptr.get();
|
||||
ColumnPtr item_arg = castColumn({ right_ptr, removeLowCardinality(index_type), "" }, common_type);
|
||||
|
||||
auto col_res = ResultColumnType::create();
|
||||
|
||||
auto [null_map_data, null_map_item] = getNullMaps(arguments);
|
||||
|
||||
if (item_arg.onlyNull())
|
||||
if (item_arg->onlyNull())
|
||||
Impl::Null<ConcreteAction>::process(
|
||||
col->getOffsets(),
|
||||
col_res->getData(),
|
||||
null_map_data);
|
||||
else if (isColumnConst(item_arg))
|
||||
else if (isColumnConst(*item_arg))
|
||||
Impl::Main<ConcreteAction, true>::vector(
|
||||
col_nested,
|
||||
*col_nested,
|
||||
col->getOffsets(),
|
||||
typeid_cast<const ColumnConst &>(item_arg).getDataColumn(),
|
||||
typeid_cast<const ColumnConst &>(*item_arg).getDataColumn(),
|
||||
col_res->getData(), /// TODO This is wrong.
|
||||
null_map_data,
|
||||
nullptr);
|
||||
else
|
||||
Impl::Main<ConcreteAction>::vector(
|
||||
col_nested,
|
||||
*col_nested,
|
||||
col->getOffsets(),
|
||||
item_arg,
|
||||
*item_arg,
|
||||
col_res->getData(),
|
||||
null_map_data,
|
||||
null_map_item);
|
||||
|
@ -13,6 +13,7 @@
|
||||
#include <Interpreters/castColumn.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <ext/range.h>
|
||||
#include <ext/map.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -51,41 +52,13 @@ public:
|
||||
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
|
||||
{
|
||||
size_t rows = input_rows_count;
|
||||
size_t num_args = arguments.size();
|
||||
|
||||
DataTypePtr common_type = nullptr;
|
||||
auto commonType = [&common_type, &arguments]()
|
||||
{
|
||||
if (common_type == nullptr)
|
||||
{
|
||||
DataTypes data_types;
|
||||
data_types.reserve(arguments.size());
|
||||
for (const auto & argument : arguments)
|
||||
data_types.push_back(argument.type);
|
||||
|
||||
common_type = getLeastSupertype(data_types);
|
||||
}
|
||||
|
||||
return common_type;
|
||||
};
|
||||
DataTypePtr common_type = getLeastSupertype(ext::map(arguments, [](auto & arg) { return arg.type; }));
|
||||
|
||||
Columns preprocessed_columns(num_args);
|
||||
|
||||
for (size_t i = 0; i < num_args; ++i)
|
||||
{
|
||||
const auto & argument = arguments[i];
|
||||
ColumnPtr preprocessed_column = argument.column;
|
||||
|
||||
const auto * argument_type = typeid_cast<const DataTypeArray *>(argument.type.get());
|
||||
const auto & nested_type = argument_type->getNestedType();
|
||||
|
||||
/// Converts Array(Nothing) or Array(Nullable(Nothing) to common type. Example: hasAll([Null, 1], [Null]) -> 1
|
||||
if (typeid_cast<const DataTypeNothing *>(removeNullable(nested_type).get()))
|
||||
preprocessed_column = castColumn(argument, commonType());
|
||||
|
||||
preprocessed_columns[i] = std::move(preprocessed_column);
|
||||
}
|
||||
preprocessed_columns[i] = castColumn(arguments[i], common_type);
|
||||
|
||||
std::vector<std::unique_ptr<GatherUtils::IArraySource>> sources;
|
||||
|
||||
@ -100,12 +73,12 @@ public:
|
||||
}
|
||||
|
||||
if (const auto * argument_column_array = typeid_cast<const ColumnArray *>(argument_column.get()))
|
||||
sources.emplace_back(GatherUtils::createArraySource(*argument_column_array, is_const, rows));
|
||||
sources.emplace_back(GatherUtils::createArraySource(*argument_column_array, is_const, input_rows_count));
|
||||
else
|
||||
throw Exception{"Arguments for function " + getName() + " must be arrays.", ErrorCodes::LOGICAL_ERROR};
|
||||
}
|
||||
|
||||
auto result_column = ColumnUInt8::create(rows);
|
||||
auto result_column = ColumnUInt8::create(input_rows_count);
|
||||
auto * result_column_ptr = typeid_cast<ColumnUInt8 *>(result_column.get());
|
||||
GatherUtils::sliceHas(*sources[0], *sources[1], search_type, *result_column_ptr);
|
||||
|
||||
|
@ -34,10 +34,6 @@
|
||||
1
|
||||
0
|
||||
-
|
||||
0
|
||||
0
|
||||
0
|
||||
0
|
||||
-
|
||||
0
|
||||
1
|
||||
|
@ -39,10 +39,10 @@ select hasAny(['a', 'b'], ['a', 'c']);
|
||||
select hasAll(['a', 'b'], ['a', 'c']);
|
||||
select '-';
|
||||
|
||||
select hasAny([1], ['a']);
|
||||
select hasAll([1], ['a']);
|
||||
select hasAll([[1, 2], [3, 4]], ['a', 'c']);
|
||||
select hasAny([[1, 2], [3, 4]], ['a', 'c']);
|
||||
select hasAny([1], ['a']); -- { serverError 386 }
|
||||
select hasAll([1], ['a']); -- { serverError 386 }
|
||||
select hasAll([[1, 2], [3, 4]], ['a', 'c']); -- { serverError 386 }
|
||||
select hasAny([[1, 2], [3, 4]], ['a', 'c']); -- { serverError 386 }
|
||||
select '-';
|
||||
|
||||
select hasAll([[1, 2], [3, 4]], [[1, 2], [3, 5]]);
|
||||
|
@ -20,8 +20,6 @@
|
||||
0
|
||||
1
|
||||
-
|
||||
0
|
||||
0
|
||||
1
|
||||
1
|
||||
0
|
||||
|
@ -25,8 +25,8 @@ select hasSubstr(['a', 'b'], ['a', 'c']);
|
||||
select hasSubstr(['a', 'c', 'b'], ['a', 'c']);
|
||||
select '-';
|
||||
|
||||
select hasSubstr([1], ['a']);
|
||||
select hasSubstr([[1, 2], [3, 4]], ['a', 'c']);
|
||||
select hasSubstr([1], ['a']); -- { serverError 386 }
|
||||
select hasSubstr([[1, 2], [3, 4]], ['a', 'c']); -- { serverError 386 }
|
||||
select hasSubstr([[1, 2], [3, 4], [5, 8]], [[3, 4]]);
|
||||
select hasSubstr([[1, 2], [3, 4], [5, 8]], [[3, 4], [5, 8]]);
|
||||
select hasSubstr([[1, 2], [3, 4], [5, 8]], [[1, 2], [5, 8]]);
|
||||
|
@ -39,9 +39,33 @@ Tuple(Decimal(9, 1), Decimal(18, 1), Decimal(38, 1)) Decimal(9, 1) Decimal(18, 1
|
||||
1 0
|
||||
1 0
|
||||
1 0
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1 0
|
||||
2 0
|
||||
3 0
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
[0.100,0.200,0.300,0.400,0.500,0.600] Array(Decimal(18, 3))
|
||||
[0.100,0.200,0.300,0.700,0.800,0.900] Array(Decimal(38, 3))
|
||||
[0.400,0.500,0.600,0.700,0.800,0.900] Array(Decimal(38, 3))
|
||||
|
@ -58,35 +58,35 @@ SELECT has(a, toDecimal32(0.1, 3)), has(a, toDecimal32(1.0, 3)) FROM decimal;
|
||||
SELECT has(b, toDecimal64(0.4, 3)), has(b, toDecimal64(1.0, 3)) FROM decimal;
|
||||
SELECT has(c, toDecimal128(0.7, 3)), has(c, toDecimal128(1.0, 3)) FROM decimal;
|
||||
|
||||
SELECT has(a, toDecimal32(0.1, 2)) FROM decimal; -- { serverError 43 }
|
||||
SELECT has(a, toDecimal32(0.1, 4)) FROM decimal; -- { serverError 43 }
|
||||
SELECT has(a, toDecimal64(0.1, 3)) FROM decimal; -- { serverError 43 }
|
||||
SELECT has(a, toDecimal128(0.1, 3)) FROM decimal; -- { serverError 43 }
|
||||
SELECT has(b, toDecimal32(0.4, 3)) FROM decimal; -- { serverError 43 }
|
||||
SELECT has(b, toDecimal64(0.4, 2)) FROM decimal; -- { serverError 43 }
|
||||
SELECT has(b, toDecimal64(0.4, 4)) FROM decimal; -- { serverError 43 }
|
||||
SELECT has(b, toDecimal128(0.4, 3)) FROM decimal; -- { serverError 43 }
|
||||
SELECT has(c, toDecimal32(0.7, 3)) FROM decimal; -- { serverError 43 }
|
||||
SELECT has(c, toDecimal64(0.7, 3)) FROM decimal; -- { serverError 43 }
|
||||
SELECT has(c, toDecimal128(0.7, 2)) FROM decimal; -- { serverError 43 }
|
||||
SELECT has(c, toDecimal128(0.7, 4)) FROM decimal; -- { serverError 43 }
|
||||
SELECT has(a, toDecimal32(0.1, 2)) FROM decimal;
|
||||
SELECT has(a, toDecimal32(0.1, 4)) FROM decimal;
|
||||
SELECT has(a, toDecimal64(0.1, 3)) FROM decimal;
|
||||
SELECT has(a, toDecimal128(0.1, 3)) FROM decimal;
|
||||
SELECT has(b, toDecimal32(0.4, 3)) FROM decimal;
|
||||
SELECT has(b, toDecimal64(0.4, 2)) FROM decimal;
|
||||
SELECT has(b, toDecimal64(0.4, 4)) FROM decimal;
|
||||
SELECT has(b, toDecimal128(0.4, 3)) FROM decimal;
|
||||
SELECT has(c, toDecimal32(0.7, 3)) FROM decimal;
|
||||
SELECT has(c, toDecimal64(0.7, 3)) FROM decimal;
|
||||
SELECT has(c, toDecimal128(0.7, 2)) FROM decimal;
|
||||
SELECT has(c, toDecimal128(0.7, 4)) FROM decimal;
|
||||
|
||||
SELECT indexOf(a, toDecimal32(0.1, 3)), indexOf(a, toDecimal32(1.0, 3)) FROM decimal;
|
||||
SELECT indexOf(b, toDecimal64(0.5, 3)), indexOf(b, toDecimal64(1.0, 3)) FROM decimal;
|
||||
SELECT indexOf(c, toDecimal128(0.9, 3)), indexOf(c, toDecimal128(1.0, 3)) FROM decimal;
|
||||
|
||||
SELECT indexOf(a, toDecimal32(0.1, 2)) FROM decimal; -- { serverError 43 }
|
||||
SELECT indexOf(a, toDecimal32(0.1, 4)) FROM decimal; -- { serverError 43 }
|
||||
SELECT indexOf(a, toDecimal64(0.1, 3)) FROM decimal; -- { serverError 43 }
|
||||
SELECT indexOf(a, toDecimal128(0.1, 3)) FROM decimal; -- { serverError 43 }
|
||||
SELECT indexOf(b, toDecimal32(0.4, 3)) FROM decimal; -- { serverError 43 }
|
||||
SELECT indexOf(b, toDecimal64(0.4, 2)) FROM decimal; -- { serverError 43 }
|
||||
SELECT indexOf(b, toDecimal64(0.4, 4)) FROM decimal; -- { serverError 43 }
|
||||
SELECT indexOf(b, toDecimal128(0.4, 3)) FROM decimal; -- { serverError 43 }
|
||||
SELECT indexOf(c, toDecimal32(0.7, 3)) FROM decimal; -- { serverError 43 }
|
||||
SELECT indexOf(c, toDecimal64(0.7, 3)) FROM decimal; -- { serverError 43 }
|
||||
SELECT indexOf(c, toDecimal128(0.7, 2)) FROM decimal; -- { serverError 43 }
|
||||
SELECT indexOf(c, toDecimal128(0.7, 4)) FROM decimal; -- { serverError 43 }
|
||||
SELECT indexOf(a, toDecimal32(0.1, 2)) FROM decimal;
|
||||
SELECT indexOf(a, toDecimal32(0.1, 4)) FROM decimal;
|
||||
SELECT indexOf(a, toDecimal64(0.1, 3)) FROM decimal;
|
||||
SELECT indexOf(a, toDecimal128(0.1, 3)) FROM decimal;
|
||||
SELECT indexOf(b, toDecimal32(0.4, 3)) FROM decimal;
|
||||
SELECT indexOf(b, toDecimal64(0.4, 2)) FROM decimal;
|
||||
SELECT indexOf(b, toDecimal64(0.4, 4)) FROM decimal;
|
||||
SELECT indexOf(b, toDecimal128(0.4, 3)) FROM decimal;
|
||||
SELECT indexOf(c, toDecimal32(0.7, 3)) FROM decimal;
|
||||
SELECT indexOf(c, toDecimal64(0.7, 3)) FROM decimal;
|
||||
SELECT indexOf(c, toDecimal128(0.7, 2)) FROM decimal;
|
||||
SELECT indexOf(c, toDecimal128(0.7, 4)) FROM decimal;
|
||||
|
||||
SELECT arrayConcat(a, b) AS x, toTypeName(x) FROM decimal;
|
||||
SELECT arrayConcat(a, c) AS x, toTypeName(x) FROM decimal;
|
||||
|
@ -1,3 +1,2 @@
|
||||
0
|
||||
1
|
||||
0
|
||||
|
@ -1,3 +1,3 @@
|
||||
SELECT hasAny([['Hello, world']], [[[]]]);
|
||||
SELECT hasAny([['Hello, world']], [[[]]]); -- { serverError 386 }
|
||||
SELECT hasAny([['Hello, world']], [['Hello', 'world'], ['Hello, world']]);
|
||||
SELECT hasAll([['Hello, world']], [['Hello', 'world'], ['Hello, world']]);
|
||||
|
3
tests/queries/0_stateless/01812_has_generic.reference
Normal file
3
tests/queries/0_stateless/01812_has_generic.reference
Normal file
@ -0,0 +1,3 @@
|
||||
1
|
||||
1
|
||||
1
|
3
tests/queries/0_stateless/01812_has_generic.sql
Normal file
3
tests/queries/0_stateless/01812_has_generic.sql
Normal file
@ -0,0 +1,3 @@
|
||||
SELECT has([(1, 2), (3, 4)], (toUInt16(3), 4));
|
||||
SELECT hasAny([(1, 2), (3, 4)], [(toUInt16(3), 4)]);
|
||||
SELECT hasAll([(1, 2), (3, 4)], [(toNullable(1), toUInt64(2)), (toUInt16(3), 4)]);
|
Loading…
Reference in New Issue
Block a user