dbms: Server: Adding nullable type support to functions manipulating arrays. [#METR-19266]

This commit is contained in:
Alexey Arno 2016-09-15 15:14:25 +03:00
parent 59043c680e
commit bd73b2d165
4 changed files with 650 additions and 123 deletions

View File

@ -212,6 +212,7 @@ struct ArrayIndexNumImpl
throw Exception{"Internal error", ErrorCodes::LOGICAL_ERROR};
}
/// Both function arguments are ordinary.
template <typename ScalarOrVector>
static void vectorCase1(
const PaddedPODArray<T> & data, const ColumnArray::Offsets_t & offsets,
@ -241,6 +242,7 @@ struct ArrayIndexNumImpl
}
}
/// The 2nd function argument is nullable.
template <typename ScalarOrVector>
static void vectorCase2(
const PaddedPODArray<T> & data, const ColumnArray::Offsets_t & offsets,
@ -271,6 +273,7 @@ struct ArrayIndexNumImpl
}
}
/// The 1st function argument is a non-constant array of nullable values.
template <typename ScalarOrVector>
static void vectorCase3(
const PaddedPODArray<T> & data, const ColumnArray::Offsets_t & offsets,
@ -289,14 +292,10 @@ struct ArrayIndexNumImpl
for (size_t j = 0; j < array_size; ++j)
{
bool hit = false;
if (null_map_data[current_offset + j] == 1)
{
}
else if (compare(data[current_offset + j], value, i))
hit = true;
if (hit)
{
if (!IndexConv::apply(j, current))
break;
@ -308,6 +307,8 @@ struct ArrayIndexNumImpl
}
}
/// The 1st function argument is a non-constant array of nullable values.
/// The 2nd function argument is nullable.
template <typename ScalarOrVector>
static void vectorCase4(
const PaddedPODArray<T> & data, const ColumnArray::Offsets_t & offsets,
@ -367,6 +368,7 @@ struct ArrayIndexNumImpl
}
};
/// Specialization that catches developer errors.
template <typename T, typename IndexConv>
struct ArrayIndexNumImpl<T, Null, IndexConv>
{
@ -382,6 +384,8 @@ struct ArrayIndexNumImpl<T, Null, IndexConv>
}
};
/// Implementation for arrays of numbers when the 2nd function argument
/// is a NULL value.
template <typename T, typename IndexConv>
struct ArrayIndexNumNullImpl
{
@ -423,13 +427,49 @@ struct ArrayIndexNumNullImpl
}
};
/// Implementation for arrays of strings when the 2nd function argument
/// is a NULL value.
template <typename IndexConv>
struct ArrayIndexStringNullImpl
{
static void vector_const(
const ColumnString::Chars_t & data, const ColumnArray::Offsets_t & offsets, const ColumnString::Offsets_t & string_offsets,
PaddedPODArray<typename IndexConv::ResultType> & result,
const PaddedPODArray<UInt8> * null_map_data)
{
const auto size = offsets.size();
result.resize(size);
ColumnArray::Offset_t current_offset = 0;
for (size_t i = 0; i < size; ++i)
{
const auto array_size = offsets[i] - current_offset;
typename IndexConv::ResultType current = 0;
for (size_t j = 0; j < array_size; ++j)
{
size_t k = (current_offset == 0 && j == 0) ? 0 : current_offset + j - 1;
if (null_map_data && ((*null_map_data)[k] == 1))
{
if (!IndexConv::apply(j, current))
break;
}
}
result[i] = current;
current_offset = offsets[i];
}
}
};
template <typename IndexConv>
struct ArrayIndexStringImpl
{
static void vector_const(
const ColumnString::Chars_t & data, const ColumnArray::Offsets_t & offsets, const ColumnString::Offsets_t & string_offsets,
const String & value,
PaddedPODArray<typename IndexConv::ResultType> & result)
PaddedPODArray<typename IndexConv::ResultType> & result,
const PaddedPODArray<UInt8> * null_map_data)
{
const auto size = offsets.size();
const auto value_size = value.size();
@ -449,7 +489,11 @@ struct ArrayIndexStringImpl
ColumnArray::Offset_t string_size = string_offsets[current_offset + j] - string_pos;
if (string_size == value_size + 1 && 0 == memcmp(value.data(), &data[string_pos], value_size))
size_t k = (current_offset == 0 && j == 0) ? 0 : current_offset + j - 1;
if (null_map_data && ((*null_map_data)[k] == 1))
{
}
else if (string_size == value_size + 1 && 0 == memcmp(value.data(), &data[string_pos], value_size))
{
if (!IndexConv::apply(j, current))
break;
@ -464,7 +508,9 @@ struct ArrayIndexStringImpl
static void vector_vector(
const ColumnString::Chars_t & data, const ColumnArray::Offsets_t & offsets, const ColumnString::Offsets_t & string_offsets,
const ColumnString::Chars_t & item_values, const ColumnString::Offsets_t & item_offsets,
PaddedPODArray<typename IndexConv::ResultType> & result)
PaddedPODArray<typename IndexConv::ResultType> & result,
const PaddedPODArray<UInt8> * null_map_data,
const PaddedPODArray<UInt8> * null_map_item)
{
const auto size = offsets.size();
result.resize(size);
@ -485,7 +531,18 @@ struct ArrayIndexStringImpl
ColumnArray::Offset_t string_size = string_offsets[current_offset + j] - string_pos;
if (string_size == value_size && 0 == memcmp(&item_values[value_pos], &data[string_pos], value_size))
bool hit = false;
size_t k = (current_offset == 0 && j == 0) ? 0 : current_offset + j - 1;
if (null_map_data && ((*null_map_data)[k] == 1))
{
if (null_map_item && ((*null_map_item)[i] == 1))
hit = true;
}
else if (string_size == value_size && 0 == memcmp(&item_values[value_pos], &data[string_pos], value_size))
hit = true;
if (hit)
{
if (!IndexConv::apply(j, current))
break;
@ -572,17 +629,33 @@ private:
if (!col_nested)
return false;
/// Null maps of the 1st and second function arguments,
/// if it applies.
const PaddedPODArray<UInt8> * null_map_data = nullptr;
const PaddedPODArray<UInt8> * null_map_item = nullptr;
if (arguments.size() > 2)
{
const auto & null_map = block.getByPosition(arguments[2]).column;
null_map_data = &static_cast<const ColumnUInt8 &>(*null_map).getData();
const auto & null_map1 = block.getByPosition(arguments[2]).column;
if (null_map1)
null_map_data = &static_cast<const ColumnUInt8 &>(*null_map1).getData();
const auto & null_map2 = block.getByPosition(arguments[3]).column;
if (null_map2)
null_map_item = &static_cast<const ColumnUInt8 &>(*null_map2).getData();
}
const auto item_arg = block.getByPosition(arguments[1]).column.get();
if (const auto item_arg_const = typeid_cast<const ColumnConst<U> *>(item_arg))
if (item_arg->isNull())
{
const auto col_res = std::make_shared<ResultColumnType>();
block.getByPosition(result).column = col_res;
ArrayIndexNumNullImpl<T, IndexConv>::vector(col_nested->getData(), col_array->getOffsets(),
col_res->getData(), null_map_data);
}
else if (const auto item_arg_const = typeid_cast<const ColumnConst<U> *>(item_arg))
{
const auto col_res = std::make_shared<ResultColumnType>();
block.getByPosition(result).column = col_res;
@ -599,14 +672,6 @@ private:
const auto col_res = std::make_shared<ResultColumnType>();
block.getByPosition(result).column = col_res;
const PaddedPODArray<UInt8> * null_map_item = nullptr;
if (arguments.size() > 3)
{
const auto & null_map = block.getByPosition(arguments[3]).column;
null_map_item = &static_cast<const ColumnUInt8 &>(*null_map).getData();
}
ArrayIndexNumImpl<T, U, IndexConv>::vector(col_nested->getData(), col_array->getOffsets(),
item_arg_vector->getData(), col_res->getData(), null_map_data, null_map_item);
}
@ -628,15 +693,40 @@ private:
if (!col_nested)
return false;
/// Null maps of the 1st and second function arguments,
/// if it applies.
const PaddedPODArray<UInt8> * null_map_data = nullptr;
const PaddedPODArray<UInt8> * null_map_item = nullptr;
if (arguments.size() > 2)
{
const auto & col1 = block.getByPosition(arguments[2]).column;
if (col1)
null_map_data = &static_cast<const ColumnUInt8 &>(*col1).getData();
const auto & col2 = block.getByPosition(arguments[3]).column;
if (col2)
null_map_item = &static_cast<const ColumnUInt8 &>(*col2).getData();
}
const auto item_arg = block.getByPosition(arguments[1]).column.get();
if (const auto item_arg_const = typeid_cast<const ColumnConst<String> *>(item_arg))
if (item_arg->isNull())
{
const auto col_res = std::make_shared<ResultColumnType>();
block.getByPosition(result).column = col_res;
ArrayIndexStringNullImpl<IndexConv>::vector_const(col_nested->getChars(), col_array->getOffsets(),
col_nested->getOffsets(), col_res->getData(), null_map_data);
}
else if (const auto item_arg_const = typeid_cast<const ColumnConst<String> *>(item_arg))
{
const auto col_res = std::make_shared<ResultColumnType>();
block.getByPosition(result).column = col_res;
ArrayIndexStringImpl<IndexConv>::vector_const(col_nested->getChars(), col_array->getOffsets(),
col_nested->getOffsets(), item_arg_const->getData(), col_res->getData());
col_nested->getOffsets(), item_arg_const->getData(), col_res->getData(),
null_map_data);
}
else if (const auto item_arg_vector = typeid_cast<const ColumnString *>(item_arg))
{
@ -645,7 +735,7 @@ private:
ArrayIndexStringImpl<IndexConv>::vector_vector(col_nested->getChars(), col_array->getOffsets(),
col_nested->getOffsets(), item_arg_vector->getChars(), item_arg_vector->getOffsets(),
col_res->getData());
col_res->getData(), null_map_data, null_map_item);
}
else
return false;
@ -683,6 +773,16 @@ private:
}
else
{
/// Null map of the 2nd function argument, if it applies.
const PaddedPODArray<UInt8> * null_map = nullptr;
if (arguments.size() > 2)
{
const auto & col = block.getByPosition(arguments[3]).column;
if (col)
null_map = &static_cast<const ColumnUInt8 &>(*col).getData();
}
const auto size = item_arg->size();
const auto col_res = std::make_shared<ResultColumnType>(size);
block.getByPosition(result).column = col_res;
@ -695,9 +795,23 @@ private:
data[row] = 0;
for (size_t i = 0, size = arr.size(); i < size; ++i)
if (apply_visitor(FieldVisitorAccurateEquals(), arr[i], value))
{
bool hit = false;
if (arr[i].isNull())
{
if (null_map && ((*null_map)[row] == 1))
hit = true;
}
else if (apply_visitor(FieldVisitorAccurateEquals(), arr[i], value))
hit = true;
if (hit)
{
if (!IndexConv::apply(i, data[row]))
break;
}
}
}
}
@ -740,7 +854,7 @@ private:
public:
/// Получить имя функции.
/// Get function name.
String getName() const override
{
return name;
@ -761,7 +875,8 @@ public:
const DataTypeArray * array_type = typeid_cast<const DataTypeArray *>(arguments[0].get());
if (!array_type)
throw Exception("First argument for function " + getName() + " must be array.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
throw Exception("First argument for function " + getName() + " must be an array.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
if (!arguments[1]->isNull())
{
@ -770,8 +885,8 @@ public:
if (!(observed_type0->behavesAsNumber() && observed_type1->behavesAsNumber())
&& observed_type0->getName() != observed_type1->getName())
throw Exception("Type of array elements and second argument for function "
+ getName() + " must be identical. Passed: "
throw Exception("Types of array and 2nd argument of function "
+ getName() + " must be identical up to nullability. Passed: "
+ arguments[0]->getName() + " and " + arguments[1]->getName() + ".",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
@ -779,46 +894,62 @@ public:
return std::make_shared<typename DataTypeFromFieldType<typename IndexConv::ResultType>::Type>();
}
/// Выполнить функцию над блоком.
/// Perform function on the given block.
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
/// Check data nullability.
/// If one or both arguments passed to this function are nullable,
/// we create a new block that contains non-nullable parameters:
/// - if the 1st argument is a non-constant array of nullable values,
/// it is turned into a non-constant array of ordinary values + a null
/// byte map;
/// - if the 2nd argument is a nullable value, it is turned into an
/// ordinary value + a null byte map.
/// Note that since constant arrays have quite a specific structure
/// (they are vectors of Fields, which may represent the NULL value),
/// they do not require any preprocessing
/// Check if the 1st function argument is a non-constant array of nullable
/// values.
bool is_nullable;
const ColumnArray * col_array = nullptr;
col_array = typeid_cast<const ColumnArray *>(block.getByPosition(arguments[0]).column.get());
if (col_array)
is_nullable = col_array->getData().isNullable();
else
is_nullable = false;
/// Check nullability of the 2nd function argument.
bool is_arg_nullable = block.getByPosition(arguments[1]).column->isNullable();
if (!is_nullable && !is_arg_nullable)
{
/// Simple case: no nullable value is passed.
perform(block, arguments, result);
}
else
{
Block source_block;
const auto & input_type = static_cast<const DataTypeNullable &>(*block.getByPosition(arguments[0]).type).getNestedType();
const auto & nullable_col = static_cast<const ColumnNullable &>(col_array->getData());
const auto & nested_col = nullable_col.getNestedColumn();
const auto & null_map = nullable_col.getNullValuesByteMap();
source_block =
/// Template of the block on which we will actually apply the function.
/// Its elements will be filled later.
Block source_block =
{
/// 1st function argument (data)
{
std::make_shared<ColumnArray>(nested_col, col_array->getOffsetsColumn()),
input_type,
""
},
block.getByPosition(arguments[1]),
/// 2nd function argument
{
null_map,
std::make_shared<DataTypeUInt8>(),
""
},
/// 1st argument null map
{
},
/// 2nd argument null map
{
},
/// Function result.
{
nullptr,
block.getByPosition(result).type,
@ -826,15 +957,56 @@ public:
}
};
perform(source_block, {0, 1, 2}, 3);
if (is_nullable)
{
const auto & nullable_col = static_cast<const ColumnNullable &>(col_array->getData());
const auto & nested_col = nullable_col.getNestedColumn();
const ColumnWithTypeAndName & source_col = source_block.unsafeGetByPosition(3);
auto & data = source_block.unsafeGetByPosition(0);
data.column = std::make_shared<ColumnArray>(nested_col, col_array->getOffsetsColumn());
data.type = static_cast<const DataTypeNullable &>(*block.getByPosition(arguments[0]).type).getNestedType();
auto & null_map = source_block.unsafeGetByPosition(2);
null_map.column = nullable_col.getNullValuesByteMap();
null_map.type = std::make_shared<DataTypeUInt8>();
}
else
{
auto & data = source_block.unsafeGetByPosition(0);
data = block.getByPosition(arguments[0]);
}
if (is_arg_nullable)
{
const auto & col = block.getByPosition(arguments[1]).column;
const auto & nullable_col = static_cast<const ColumnNullable &>(*col);
auto & arg = source_block.unsafeGetByPosition(1);
arg.column = nullable_col.getNestedColumn();
arg.type = static_cast<const DataTypeNullable &>(*block.getByPosition(arguments[1]).type).getNestedType();
auto & null_map = source_block.unsafeGetByPosition(3);
null_map.column = nullable_col.getNullValuesByteMap();
null_map.type = std::make_shared<DataTypeUInt8>();
}
else
{
auto & arg = source_block.unsafeGetByPosition(1);
arg = block.getByPosition(arguments[1]);
}
/// Now perform the function.
perform(source_block, {0, 1, 2, 3}, 4);
/// Move the result to its final position.
const ColumnWithTypeAndName & source_col = source_block.unsafeGetByPosition(4);
ColumnWithTypeAndName & dest_col = block.unsafeGetByPosition(result);
dest_col.column = source_col.column;
dest_col.column = std::move(source_col.column);
}
}
private:
/// Perform function on the given block. Internal version.
void perform(Block & block, const ColumnNumbers & arguments, size_t result)
{
if (!(executeNumber<UInt8>(block, arguments, result)

View File

@ -4,6 +4,7 @@
#include <DB/DataTypes/DataTypeNested.h>
#include <DB/DataTypes/DataTypeArray.h>
#include <DB/DataTypes/DataTypeNullable.h>
#include <DB/Columns/ColumnArray.h>
#include <DB/Columns/ColumnNullable.h>
#include <DB/Common/StringUtils.h>
@ -144,12 +145,10 @@ void IMergedBlockOutputStream::writeData(const String & name, const IDataType &
/// Then write data.
writeData(name, nested_type, nested_col, offset_columns, level);
return;
}
/// Для массивов требуется сначала сериализовать размеры, а потом значения.
if (const DataTypeArray * type_arr = typeid_cast<const DataTypeArray *>(&type))
else if (const DataTypeArray * type_arr = typeid_cast<const DataTypeArray *>(&type))
{
/// Для массивов требуется сначала сериализовать размеры, а потом значения.
String size_name = DataTypeNested::extractNestedTableName(name)
+ ARRAY_SIZES_COLUMN_NAME_SUFFIX + toString(level);
@ -189,8 +188,10 @@ void IMergedBlockOutputStream::writeData(const String & name, const IDataType &
prev_mark += limit;
}
}
}
writeData(name, *type_arr->getNestedType(), typeid_cast<const ColumnArray &>(column).getData(), offset_columns, level + 1);
}
else
{
ColumnStream & stream = *column_streams[name];

View File

@ -2,34 +2,34 @@
\N
\N
\N
1 1 [1] [1] a a ['a'] ['a'] a\0\0 a\0\0 ['a\0\0'] ['a\0\0']
1 1 [1] [1] a a ['a'] ['a'] a\0\0 a\0\0 \N ['a\0\0']
1 1 [1] [1] a a ['a'] ['a'] \N a\0\0 ['a\0\0'] ['a\0\0']
1 1 [1] [1] a a \N ['a'] a\0\0 a\0\0 ['a\0\0'] ['a\0\0']
1 1 [1] [1] \N a ['a'] ['a'] a\0\0 a\0\0 ['a\0\0'] ['a\0\0']
1 1 \N [1] a a ['a'] ['a'] a\0\0 a\0\0 ['a\0\0'] ['a\0\0']
\N 1 [1] [1] a a ['a'] ['a'] a\0\0 a\0\0 ['a\0\0'] ['a\0\0']
1 1 [1] [1] a a ['a'] ['a'] a\0\0 a\0\0 ['a\0\0'] ['a\0\0']
1 1 [1] [1] a a ['a'] ['a'] a\0\0 a\0\0 \N ['a\0\0']
1 1 [1] [1] a a ['a'] ['a'] \N a\0\0 ['a\0\0'] ['a\0\0']
1 1 [1] [1] a a \N ['a'] a\0\0 a\0\0 ['a\0\0'] ['a\0\0']
1 1 [1] [1] \N a ['a'] ['a'] a\0\0 a\0\0 ['a\0\0'] ['a\0\0']
1 1 \N [1] a a ['a'] ['a'] a\0\0 a\0\0 ['a\0\0'] ['a\0\0']
\N 1 [1] [1] a a ['a'] ['a'] a\0\0 a\0\0 ['a\0\0'] ['a\0\0']
1 1 [1] [1] a a ['a'] ['a'] a\0\0 a\0\0 ['a\0\0'] ['a\0\0']
1 1 [1] [1] a a ['a'] ['a'] a\0\0 a\0\0 \N ['a\0\0']
1 1 [1] [1] a a ['a'] ['a'] \N a\0\0 ['a\0\0'] ['a\0\0']
1 1 [1] [1] a a \N ['a'] a\0\0 a\0\0 ['a\0\0'] ['a\0\0']
1 1 [1] [1] \N a ['a'] ['a'] a\0\0 a\0\0 ['a\0\0'] ['a\0\0']
1 1 \N [1] a a ['a'] ['a'] a\0\0 a\0\0 ['a\0\0'] ['a\0\0']
\N 1 [1] [1] a a ['a'] ['a'] a\0\0 a\0\0 ['a\0\0'] ['a\0\0']
1 1 [1] [1] a a ['a'] ['a'] a\0\0 a\0\0 ['a\0\0'] ['a\0\0'] 0000-00-00
1 1 [1] [1] a a ['a'] ['a'] a\0\0 a\0\0 \N ['a\0\0'] 0000-00-00
1 1 [1] [1] a a ['a'] ['a'] \N a\0\0 ['a\0\0'] ['a\0\0'] 0000-00-00
1 1 [1] [1] a a \N ['a'] a\0\0 a\0\0 ['a\0\0'] ['a\0\0'] 0000-00-00
1 1 [1] [1] \N a ['a'] ['a'] a\0\0 a\0\0 ['a\0\0'] ['a\0\0'] 0000-00-00
1 1 \N [1] a a ['a'] ['a'] a\0\0 a\0\0 ['a\0\0'] ['a\0\0'] 0000-00-00
\N 1 [1] [1] a a ['a'] ['a'] a\0\0 a\0\0 ['a\0\0'] ['a\0\0'] 0000-00-00
1 1 [1] [1] a a [a] [a] [1] [a]
1 1 [1] [1] a a [a] [a] [1] [\N]
1 1 [1] [1] a a [a] [a] [\N] [a]
1 1 [1] [1] a a \N [a] [1] [a]
1 1 [1] [1] \N a [a] [a] [1] [a]
1 1 \N [1] a a [a] [a] [1] [a]
\N 1 [1] [1] a a [a] [a] [1] [a]
1 1 [1] [1] a a [a] [a] [1] [a]
1 1 [1] [1] a a [a] [a] [1] [\N]
1 1 [1] [1] a a [a] [a] [\N] [a]
1 1 [1] [1] a a \N [a] [1] [a]
1 1 [1] [1] \N a [a] [a] [1] [a]
1 1 \N [1] a a [a] [a] [1] [a]
\N 1 [1] [1] a a [a] [a] [1] [a]
1 1 [1] [1] a a [a] [a] [1] [a]
1 1 [1] [1] a a [a] [a] [1] [\N]
1 1 [1] [1] a a [a] [a] [\N] [a]
1 1 [1] [1] a a \N [a] [1] [a]
1 1 [1] [1] \N a [a] [a] [1] [a]
1 1 \N [1] a a [a] [a] [1] [a]
\N 1 [1] [1] a a [a] [a] [1] [a]
1 1 [1] [1] a a [a] [a] [1] [a] 0000-00-00
1 1 [1] [1] a a [a] [a] [1] [\N] 0000-00-00
1 1 [1] [1] a a [a] [a] [\N] [a] 0000-00-00
1 1 [1] [1] a a \N [a] [1] [a] 0000-00-00
1 1 [1] [1] \N a [a] [a] [1] [a] 0000-00-00
1 1 \N [1] a a [a] [a] [1] [a] 0000-00-00
\N 1 [1] [1] a a [a] [a] [1] [a] 0000-00-00
[2]
2 7
4 3
@ -95,7 +95,7 @@
A
A\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0
[1,2]
['A','B']
[A,B]
3
2
2
@ -105,3 +105,173 @@ A\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0
1
99
99
[\N]
[\N,\N,\N]
[\N,2,3]
[1,\N,3]
[1,2,\N]
[\N,b,c]
[a,\N,c]
[a,b,\N]
1
\N
2
3
a
\N
c
d
1
\N
2
3
2
\N
2
2
\N
3
3
\N
3
\N
7
7
7
\N
\N
\N
4
\N
4
\N
a
\N
a
a
\N
bc
bc
\N
bc
\N
def
def
def
\N
\N
\N
ghij
\N
ghij
\N
2
3
7
4
\N
a
\N
\N
\N
\N
1
1
1
1
0
1
1
1
1
1
1
1
0
1
1
1
0
1
1
0
1
1
1
0
1
1
1
1
0
1
1
0
1
1
0
1
0
0
1
0
1
0
0
0
0
0
0
1
1
1
0
0
1
1
1
1
1
1
0
1
0
0
1
0
0
1
0
1
1
0
0
0
0
1
0
0
0
0
1
1
1
1
1
1
1
0
1
1
1
0
1
1
0
0
0
1
0
0
0
1

View File

@ -14,17 +14,16 @@ col1 Nullable(UInt64), col2 UInt64,
col3 Nullable(Array(UInt64)), col4 Array(UInt64),
col5 Nullable(String), col6 String,
col7 Nullable(Array(String)), col8 Array(String),
col9 Nullable(FixedString(3)), col10 FixedString(3),
col11 Nullable(Array(FixedString(3))), col12 Array(FixedString(3))) Engine = Memory;
col9 Array(Nullable(UInt64)), col10 Array(Nullable(String))) Engine = Memory;
INSERT INTO test.test1(col1,col2,col3,col4,col5,col6,col7,col8,col9,col10,col11,col12) VALUES (1, 1, [1], [1], 'a', 'a', ['a'], ['a'], toFixedString('a', 3), toFixedString('a', 3), [toFixedString('a',3)], [toFixedString('a',3)]);
INSERT INTO test.test1(col1,col2,col3,col4,col5,col6,col7,col8,col9,col10,col11,col12) VALUES (NULL, 1, [1], [1], 'a', 'a', ['a'], ['a'], toFixedString('a', 3), toFixedString('a', 3), [toFixedString('a',3)], [toFixedString('a',3)]);
INSERT INTO test.test1(col1,col2,col3,col4,col5,col6,col7,col8,col9,col10,col11,col12) VALUES (1, 1, NULL, [1], 'a', 'a', ['a'], ['a'], toFixedString('a', 3), toFixedString('a', 3), [toFixedString('a',3)], [toFixedString('a',3)]);
INSERT INTO test.test1(col1,col2,col3,col4,col5,col6,col7,col8,col9,col10,col11,col12) VALUES (1, 1, [1], [1], NULL, 'a', ['a'], ['a'], toFixedString('a', 3), toFixedString('a', 3), [toFixedString('a',3)], [toFixedString('a',3)]);
INSERT INTO test.test1(col1,col2,col3,col4,col5,col6,col7,col8,col9,col10,col11,col12) VALUES (1, 1, [1], [1], 'a', 'a', NULL, ['a'], toFixedString('a', 3), toFixedString('a', 3), [toFixedString('a',3)], [toFixedString('a',3)]);
INSERT INTO test.test1(col1,col2,col3,col4,col5,col6,col7,col8,col9,col10,col11,col12) VALUES (1, 1, [1], [1], 'a', 'a', ['a'], ['a'], NULL, toFixedString('a', 3), [toFixedString('a',3)], [toFixedString('a',3)]);
INSERT INTO test.test1(col1,col2,col3,col4,col5,col6,col7,col8,col9,col10,col11,col12) VALUES (1, 1, [1], [1], 'a', 'a', ['a'], ['a'], toFixedString('a', 3), toFixedString('a', 3), NULL, [toFixedString('a',3)]);
SELECT col1, col2, col3, col4, col5, col6, col7, col8, col9, col10, col11, col12 FROM test.test1 ORDER BY col1,col2,col3,col4,col5,col6,col7,col8,col9,col10,col11,col12 ASC;
INSERT INTO test.test1(col1,col2,col3,col4,col5,col6,col7,col8,col9,col10) VALUES (1, 1, [1], [1], 'a', 'a', ['a'], ['a'], [1], ['a']);
INSERT INTO test.test1(col1,col2,col3,col4,col5,col6,col7,col8,col9,col10) VALUES (NULL, 1, [1], [1], 'a', 'a', ['a'], ['a'], [1], ['a']);
INSERT INTO test.test1(col1,col2,col3,col4,col5,col6,col7,col8,col9,col10) VALUES (1, 1, NULL, [1], 'a', 'a', ['a'], ['a'], [1], ['a']);
INSERT INTO test.test1(col1,col2,col3,col4,col5,col6,col7,col8,col9,col10) VALUES (1, 1, [1], [1], NULL, 'a', ['a'], ['a'], [1], ['a']);
INSERT INTO test.test1(col1,col2,col3,col4,col5,col6,col7,col8,col9,col10) VALUES (1, 1, [1], [1], 'a', 'a', NULL, ['a'], [1], ['a']);
INSERT INTO test.test1(col1,col2,col3,col4,col5,col6,col7,col8,col9,col10) VALUES (1, 1, [1], [1], 'a', 'a', ['a'], ['a'], [NULL], ['a']);
INSERT INTO test.test1(col1,col2,col3,col4,col5,col6,col7,col8,col9,col10) VALUES (1, 1, [1], [1], 'a', 'a', ['a'], ['a'], [1], [NULL]);
SELECT col1, col2, col3, col4, col5, col6, col7, col8, col9, col10 FROM test.test1 ORDER BY col1,col2,col3,col4,col5,col6,col7,col8,col9,col10 ASC;
/* TinyLog engine */
@ -34,17 +33,16 @@ col1 Nullable(UInt64), col2 UInt64,
col3 Nullable(Array(UInt64)), col4 Array(UInt64),
col5 Nullable(String), col6 String,
col7 Nullable(Array(String)), col8 Array(String),
col9 Nullable(FixedString(3)), col10 FixedString(3),
col11 Nullable(Array(FixedString(3))), col12 Array(FixedString(3))) Engine = TinyLog;
col9 Array(Nullable(UInt64)), col10 Array(Nullable(String))) Engine = TinyLog;
INSERT INTO test.test1(col1,col2,col3,col4,col5,col6,col7,col8,col9,col10,col11,col12) VALUES (1, 1, [1], [1], 'a', 'a', ['a'], ['a'], toFixedString('a', 3), toFixedString('a', 3), [toFixedString('a',3)], [toFixedString('a',3)]);
INSERT INTO test.test1(col1,col2,col3,col4,col5,col6,col7,col8,col9,col10,col11,col12) VALUES (NULL, 1, [1], [1], 'a', 'a', ['a'], ['a'], toFixedString('a', 3), toFixedString('a', 3), [toFixedString('a',3)], [toFixedString('a',3)]);
INSERT INTO test.test1(col1,col2,col3,col4,col5,col6,col7,col8,col9,col10,col11,col12) VALUES (1, 1, NULL, [1], 'a', 'a', ['a'], ['a'], toFixedString('a', 3), toFixedString('a', 3), [toFixedString('a',3)], [toFixedString('a',3)]);
INSERT INTO test.test1(col1,col2,col3,col4,col5,col6,col7,col8,col9,col10,col11,col12) VALUES (1, 1, [1], [1], NULL, 'a', ['a'], ['a'], toFixedString('a', 3), toFixedString('a', 3), [toFixedString('a',3)], [toFixedString('a',3)]);
INSERT INTO test.test1(col1,col2,col3,col4,col5,col6,col7,col8,col9,col10,col11,col12) VALUES (1, 1, [1], [1], 'a', 'a', NULL, ['a'], toFixedString('a', 3), toFixedString('a', 3), [toFixedString('a',3)], [toFixedString('a',3)]);
INSERT INTO test.test1(col1,col2,col3,col4,col5,col6,col7,col8,col9,col10,col11,col12) VALUES (1, 1, [1], [1], 'a', 'a', ['a'], ['a'], NULL, toFixedString('a', 3), [toFixedString('a',3)], [toFixedString('a',3)]);
INSERT INTO test.test1(col1,col2,col3,col4,col5,col6,col7,col8,col9,col10,col11,col12) VALUES (1, 1, [1], [1], 'a', 'a', ['a'], ['a'], toFixedString('a', 3), toFixedString('a', 3), NULL, [toFixedString('a',3)]);
SELECT col1, col2, col3, col4, col5, col6, col7, col8, col9, col10, col11, col12 FROM test.test1 ORDER BY col1,col2,col3,col4,col5,col6,col7,col8,col9,col10,col11,col12 ASC;
INSERT INTO test.test1(col1,col2,col3,col4,col5,col6,col7,col8,col9,col10) VALUES (1, 1, [1], [1], 'a', 'a', ['a'], ['a'], [1], ['a']);
INSERT INTO test.test1(col1,col2,col3,col4,col5,col6,col7,col8,col9,col10) VALUES (NULL, 1, [1], [1], 'a', 'a', ['a'], ['a'], [1], ['a']);
INSERT INTO test.test1(col1,col2,col3,col4,col5,col6,col7,col8,col9,col10) VALUES (1, 1, NULL, [1], 'a', 'a', ['a'], ['a'], [1], ['a']);
INSERT INTO test.test1(col1,col2,col3,col4,col5,col6,col7,col8,col9,col10) VALUES (1, 1, [1], [1], NULL, 'a', ['a'], ['a'], [1], ['a']);
INSERT INTO test.test1(col1,col2,col3,col4,col5,col6,col7,col8,col9,col10) VALUES (1, 1, [1], [1], 'a', 'a', NULL, ['a'], [1], ['a']);
INSERT INTO test.test1(col1,col2,col3,col4,col5,col6,col7,col8,col9,col10) VALUES (1, 1, [1], [1], 'a', 'a', ['a'], ['a'], [NULL], ['a']);
INSERT INTO test.test1(col1,col2,col3,col4,col5,col6,col7,col8,col9,col10) VALUES (1, 1, [1], [1], 'a', 'a', ['a'], ['a'], [1], [NULL]);
SELECT col1, col2, col3, col4, col5, col6, col7, col8, col9, col10 FROM test.test1 ORDER BY col1,col2,col3,col4,col5,col6,col7,col8,col9,col10 ASC;
/* Log engine */
@ -54,17 +52,16 @@ col1 Nullable(UInt64), col2 UInt64,
col3 Nullable(Array(UInt64)), col4 Array(UInt64),
col5 Nullable(String), col6 String,
col7 Nullable(Array(String)), col8 Array(String),
col9 Nullable(FixedString(3)), col10 FixedString(3),
col11 Nullable(Array(FixedString(3))), col12 Array(FixedString(3))) Engine = Log;
col9 Array(Nullable(UInt64)), col10 Array(Nullable(String))) Engine = Log;
INSERT INTO test.test1(col1,col2,col3,col4,col5,col6,col7,col8,col9,col10,col11,col12) VALUES (1, 1, [1], [1], 'a', 'a', ['a'], ['a'], toFixedString('a', 3), toFixedString('a', 3), [toFixedString('a',3)], [toFixedString('a',3)]);
INSERT INTO test.test1(col1,col2,col3,col4,col5,col6,col7,col8,col9,col10,col11,col12) VALUES (NULL, 1, [1], [1], 'a', 'a', ['a'], ['a'], toFixedString('a', 3), toFixedString('a', 3), [toFixedString('a',3)], [toFixedString('a',3)]);
INSERT INTO test.test1(col1,col2,col3,col4,col5,col6,col7,col8,col9,col10,col11,col12) VALUES (1, 1, NULL, [1], 'a', 'a', ['a'], ['a'], toFixedString('a', 3), toFixedString('a', 3), [toFixedString('a',3)], [toFixedString('a',3)]);
INSERT INTO test.test1(col1,col2,col3,col4,col5,col6,col7,col8,col9,col10,col11,col12) VALUES (1, 1, [1], [1], NULL, 'a', ['a'], ['a'], toFixedString('a', 3), toFixedString('a', 3), [toFixedString('a',3)], [toFixedString('a',3)]);
INSERT INTO test.test1(col1,col2,col3,col4,col5,col6,col7,col8,col9,col10,col11,col12) VALUES (1, 1, [1], [1], 'a', 'a', NULL, ['a'], toFixedString('a', 3), toFixedString('a', 3), [toFixedString('a',3)], [toFixedString('a',3)]);
INSERT INTO test.test1(col1,col2,col3,col4,col5,col6,col7,col8,col9,col10,col11,col12) VALUES (1, 1, [1], [1], 'a', 'a', ['a'], ['a'], NULL, toFixedString('a', 3), [toFixedString('a',3)], [toFixedString('a',3)]);
INSERT INTO test.test1(col1,col2,col3,col4,col5,col6,col7,col8,col9,col10,col11,col12) VALUES (1, 1, [1], [1], 'a', 'a', ['a'], ['a'], toFixedString('a', 3), toFixedString('a', 3), NULL, [toFixedString('a',3)]);
SELECT col1, col2, col3, col4, col5, col6, col7, col8, col9, col10, col11, col12 FROM test.test1 ORDER BY col1,col2,col3,col4,col5,col6,col7,col8,col9,col10,col11,col12 ASC;
INSERT INTO test.test1(col1,col2,col3,col4,col5,col6,col7,col8,col9,col10) VALUES (1, 1, [1], [1], 'a', 'a', ['a'], ['a'], [1], ['a']);
INSERT INTO test.test1(col1,col2,col3,col4,col5,col6,col7,col8,col9,col10) VALUES (NULL, 1, [1], [1], 'a', 'a', ['a'], ['a'], [1], ['a']);
INSERT INTO test.test1(col1,col2,col3,col4,col5,col6,col7,col8,col9,col10) VALUES (1, 1, NULL, [1], 'a', 'a', ['a'], ['a'], [1], ['a']);
INSERT INTO test.test1(col1,col2,col3,col4,col5,col6,col7,col8,col9,col10) VALUES (1, 1, [1], [1], NULL, 'a', ['a'], ['a'], [1], ['a']);
INSERT INTO test.test1(col1,col2,col3,col4,col5,col6,col7,col8,col9,col10) VALUES (1, 1, [1], [1], 'a', 'a', NULL, ['a'], [1], ['a']);
INSERT INTO test.test1(col1,col2,col3,col4,col5,col6,col7,col8,col9,col10) VALUES (1, 1, [1], [1], 'a', 'a', ['a'], ['a'], [NULL], ['a']);
INSERT INTO test.test1(col1,col2,col3,col4,col5,col6,col7,col8,col9,col10) VALUES (1, 1, [1], [1], 'a', 'a', ['a'], ['a'], [1], [NULL]);
SELECT col1, col2, col3, col4, col5, col6, col7, col8, col9, col10 FROM test.test1 ORDER BY col1,col2,col3,col4,col5,col6,col7,col8,col9,col10 ASC;
/* MergeTree engine */
@ -74,18 +71,17 @@ col1 Nullable(UInt64), col2 UInt64,
col3 Nullable(Array(UInt64)), col4 Array(UInt64),
col5 Nullable(String), col6 String,
col7 Nullable(Array(String)), col8 Array(String),
col9 Nullable(FixedString(3)), col10 FixedString(3),
col11 Nullable(Array(FixedString(3))), col12 Array(FixedString(3)),
col13 Date) Engine = MergeTree(col13, (col2, col13), 8192);
col9 Array(Nullable(UInt64)), col10 Array(Nullable(String)),
col11 Date) Engine = MergeTree(col11, (col2, col11), 8192);
INSERT INTO test.test1(col1,col2,col3,col4,col5,col6,col7,col8,col9,col10,col11,col12,col13) VALUES (1, 1, [1], [1], 'a', 'a', ['a'], ['a'], toFixedString('a', 3), toFixedString('a', 3), [toFixedString('a',3)], [toFixedString('a',3)], '1970-01-01');
INSERT INTO test.test1(col1,col2,col3,col4,col5,col6,col7,col8,col9,col10,col11,col12,col13) VALUES (NULL, 1, [1], [1], 'a', 'a', ['a'], ['a'], toFixedString('a', 3), toFixedString('a', 3), [toFixedString('a',3)], [toFixedString('a',3)], '1970-01-01');
INSERT INTO test.test1(col1,col2,col3,col4,col5,col6,col7,col8,col9,col10,col11,col12,col13) VALUES (1, 1, NULL, [1], 'a', 'a', ['a'], ['a'], toFixedString('a', 3), toFixedString('a', 3), [toFixedString('a',3)], [toFixedString('a',3)], '1970-01-01');
INSERT INTO test.test1(col1,col2,col3,col4,col5,col6,col7,col8,col9,col10,col11,col12,col13) VALUES (1, 1, [1], [1], NULL, 'a', ['a'], ['a'], toFixedString('a', 3), toFixedString('a', 3), [toFixedString('a',3)], [toFixedString('a',3)], '1970-01-01');
INSERT INTO test.test1(col1,col2,col3,col4,col5,col6,col7,col8,col9,col10,col11,col12,col13) VALUES (1, 1, [1], [1], 'a', 'a', NULL, ['a'], toFixedString('a', 3), toFixedString('a', 3), [toFixedString('a',3)], [toFixedString('a',3)], '1970-01-01');
INSERT INTO test.test1(col1,col2,col3,col4,col5,col6,col7,col8,col9,col10,col11,col12,col13) VALUES (1, 1, [1], [1], 'a', 'a', ['a'], ['a'], NULL, toFixedString('a', 3), [toFixedString('a',3)], [toFixedString('a',3)], '1970-01-01');
INSERT INTO test.test1(col1,col2,col3,col4,col5,col6,col7,col8,col9,col10,col11,col12,col13) VALUES (1, 1, [1], [1], 'a', 'a', ['a'], ['a'], toFixedString('a', 3), toFixedString('a', 3), NULL, [toFixedString('a',3)], '1970-01-01');
SELECT col1, col2, col3, col4, col5, col6, col7, col8, col9, col10, col11, col12, col13 FROM test.test1 ORDER BY col1,col2,col3,col4,col5,col6,col7,col8,col9,col10,col11,col12 ASC;
INSERT INTO test.test1(col1,col2,col3,col4,col5,col6,col7,col8,col9,col10,col11) VALUES (1, 1, [1], [1], 'a', 'a', ['a'], ['a'], [1], ['a'], '1970-01-01');
INSERT INTO test.test1(col1,col2,col3,col4,col5,col6,col7,col8,col9,col10,col11) VALUES (NULL, 1, [1], [1], 'a', 'a', ['a'], ['a'], [1], ['a'], '1970-01-01');
INSERT INTO test.test1(col1,col2,col3,col4,col5,col6,col7,col8,col9,col10,col11) VALUES (1, 1, NULL, [1], 'a', 'a', ['a'], ['a'], [1], ['a'], '1970-01-01');
INSERT INTO test.test1(col1,col2,col3,col4,col5,col6,col7,col8,col9,col10,col11) VALUES (1, 1, [1], [1], NULL, 'a', ['a'], ['a'], [1], ['a'], '1970-01-01');
INSERT INTO test.test1(col1,col2,col3,col4,col5,col6,col7,col8,col9,col10,col11) VALUES (1, 1, [1], [1], 'a', 'a', NULL, ['a'], [1], ['a'], '1970-01-01');
INSERT INTO test.test1(col1,col2,col3,col4,col5,col6,col7,col8,col9,col10,col11) VALUES (1, 1, [1], [1], 'a', 'a', ['a'], ['a'], [NULL], ['a'], '1970-01-01');
INSERT INTO test.test1(col1,col2,col3,col4,col5,col6,col7,col8,col9,col10,col11) VALUES (1, 1, [1], [1], 'a', 'a', ['a'], ['a'], [1], [NULL], '1970-01-01');
SELECT col1, col2, col3, col4, col5, col6, col7, col8, col9, col10, col11 FROM test.test1 ORDER BY col1,col2,col3,col4,col5,col6,col7,col8,col9,col10,col11 ASC;
/* Insert with expression */
@ -163,3 +159,191 @@ DROP TABLE IF EXISTS test.test1;
CREATE TABLE test.test1(cond1 Nullable(UInt8), then1 Int8, cond2 UInt8, then2 Nullable(UInt16), then3 Nullable(Float32)) Engine=TinyLog;
INSERT INTO test.test1(cond1,then1,cond2,then2,then3) VALUES(1,1,1,42,99)(0,7,1,99,42)(NULL,6,2,99,NULL);
SELECT multiIf(cond1,then1,cond2,then2,then3) FROM test.test1;
/* Array functions */
SELECT [NULL];
SELECT [NULL,NULL,NULL];
SELECT [NULL,2,3];
SELECT [1,NULL,3];
SELECT [1,2,NULL];
SELECT [NULL,'b','c'];
SELECT ['a',NULL,'c'];
SELECT ['a','b',NULL];
/* arrayElement */
/* constant arrays */
SELECT arrayElement([1,NULL,2,3], 1);
SELECT arrayElement([1,NULL,2,3], 2);
SELECT arrayElement([1,NULL,2,3], 3);
SELECT arrayElement([1,NULL,2,3], 4);
SELECT arrayElement(['a',NULL,'c','d'], 1);
SELECT arrayElement(['a',NULL,'c','d'], 2);
SELECT arrayElement(['a',NULL,'c','d'], 3);
SELECT arrayElement(['a',NULL,'c','d'], 4);
DROP TABLE IF EXISTS test.test1;
CREATE TABLE test.test1(col1 UInt64) Engine=TinyLog;
INSERT INTO test.test1(col1) VALUES(1),(2),(3),(4);
SELECT arrayElement([1,NULL,2,3], col1) FROM test.test1;
/* variable arrays */
DROP TABLE IF EXISTS test.test1;
CREATE TABLE test.test1(col1 Array(Nullable(UInt64))) Engine=TinyLog;
INSERT INTO test.test1(col1) VALUES([2,3,7,NULL]);
INSERT INTO test.test1(col1) VALUES([NULL,3,7,4]);
INSERT INTO test.test1(col1) VALUES([2,NULL,7,NULL]);
INSERT INTO test.test1(col1) VALUES([2,3,NULL,4]);
INSERT INTO test.test1(col1) VALUES([NULL,NULL,NULL,NULL]);
SELECT arrayElement(col1, 1) FROM test.test1;
SELECT arrayElement(col1, 2) FROM test.test1;
SELECT arrayElement(col1, 3) FROM test.test1;
SELECT arrayElement(col1, 4) FROM test.test1;
DROP TABLE IF EXISTS test.test1;
CREATE TABLE test.test1(col1 Array(Nullable(String))) Engine=TinyLog;
INSERT INTO test.test1(col1) VALUES(['a','bc','def',NULL]);
INSERT INTO test.test1(col1) VALUES([NULL,'bc','def','ghij']);
INSERT INTO test.test1(col1) VALUES(['a',NULL,'def',NULL]);
INSERT INTO test.test1(col1) VALUES(['a','bc',NULL,'ghij']);
INSERT INTO test.test1(col1) VALUES([NULL,NULL,NULL,NULL]);
SELECT arrayElement(col1, 1) FROM test.test1;
SELECT arrayElement(col1, 2) FROM test.test1;
SELECT arrayElement(col1, 3) FROM test.test1;
SELECT arrayElement(col1, 4) FROM test.test1;
DROP TABLE IF EXISTS test.test1;
CREATE TABLE test.test1(col1 Array(Nullable(UInt64)), col2 UInt64) Engine=TinyLog;
INSERT INTO test.test1(col1,col2) VALUES([2,3,7,NULL], 1);
INSERT INTO test.test1(col1,col2) VALUES([NULL,3,7,4], 2);
INSERT INTO test.test1(col1,col2) VALUES([2,NULL,7,NULL], 3);
INSERT INTO test.test1(col1,col2) VALUES([2,3,NULL,4],4);
INSERT INTO test.test1(col1,col2) VALUES([NULL,NULL,NULL,NULL],3);
SELECT arrayElement(col1,col2) FROM test.test1;
DROP TABLE IF EXISTS test.test1;
CREATE TABLE test.test1(col1 Array(Nullable(String)), col2 UInt64) Engine=TinyLog;
INSERT INTO test.test1(col1,col2) VALUES(['a','bc','def',NULL], 1);
INSERT INTO test.test1(col1,col2) VALUES([NULL,'bc','def','ghij'], 2);
INSERT INTO test.test1(col1,col2) VALUES(['a',NULL,'def','ghij'], 3);
INSERT INTO test.test1(col1,col2) VALUES(['a','bc',NULL,'ghij'],4);
INSERT INTO test.test1(col1,col2) VALUES([NULL,NULL,NULL,NULL],3);
SELECT arrayElement(col1,col2) FROM test.test1;
/* has */
/* constant arrays */
SELECT has([1,NULL,2,3], 1);
SELECT has([1,NULL,2,3], NULL);
SELECT has([1,NULL,2,3], 2);
SELECT has([1,NULL,2,3], 3);
SELECT has([1,NULL,2,3], 4);
SELECT has(['a',NULL,'def','ghij'], 'a');
SELECT has(['a',NULL,'def','ghij'], NULL);
SELECT has(['a',NULL,'def','ghij'], 'def');
SELECT has(['a',NULL,'def','ghij'], 'ghij');
DROP TABLE IF EXISTS test.test1;
CREATE TABLE test.test1(col1 UInt64) Engine=TinyLog;
INSERT INTO test.test1(col1) VALUES(1),(2),(3),(4);
SELECT has([1,NULL,2,3], col1) FROM test.test1;
DROP TABLE IF EXISTS test.test1;
CREATE TABLE test.test1(col1 Nullable(UInt64)) Engine=TinyLog;
INSERT INTO test.test1(col1) VALUES(1),(2),(3),(4),(NULL);
SELECT has([1,NULL,2,3], col1) FROM test.test1;
DROP TABLE IF EXISTS test.test1;
CREATE TABLE test.test1(col1 String) Engine=TinyLog;
INSERT INTO test.test1(col1) VALUES('a'),('bc'),('def'),('ghij');
SELECT has(['a',NULL,'def','ghij'], col1) FROM test.test1;
DROP TABLE IF EXISTS test.test1;
CREATE TABLE test.test1(col1 Nullable(String)) Engine=TinyLog;
INSERT INTO test.test1(col1) VALUES('a'),('bc'),('def'),('ghij'),(NULL);
SELECT has(['a',NULL,'def','ghij'], col1) FROM test.test1;
/* variable arrays */
DROP TABLE IF EXISTS test.test1;
CREATE TABLE test.test1(col1 Array(Nullable(UInt64))) Engine=TinyLog;
INSERT INTO test.test1(col1) VALUES([2,3,7,NULL]);
INSERT INTO test.test1(col1) VALUES([NULL,3,7,4]);
INSERT INTO test.test1(col1) VALUES([2,NULL,7,NULL]);
INSERT INTO test.test1(col1) VALUES([2,3,NULL,4]);
INSERT INTO test.test1(col1) VALUES([NULL,NULL,NULL,NULL]);
SELECT has(col1, 2) FROM test.test1;
SELECT has(col1, 3) FROM test.test1;
SELECT has(col1, 4) FROM test.test1;
SELECT has(col1, 5) FROM test.test1;
SELECT has(col1, 7) FROM test.test1;
SELECT has(col1, NULL) FROM test.test1;
DROP TABLE IF EXISTS test.test1;
CREATE TABLE test.test1(col1 Array(Nullable(String))) Engine=TinyLog;
INSERT INTO test.test1(col1) VALUES(['a','bc','def',NULL]);
INSERT INTO test.test1(col1) VALUES([NULL,'bc','def','ghij']);
INSERT INTO test.test1(col1) VALUES(['a',NULL,'def',NULL]);
INSERT INTO test.test1(col1) VALUES(['a','bc',NULL,'ghij']);
INSERT INTO test.test1(col1) VALUES([NULL,NULL,NULL,NULL]);
SELECT has(col1, 'a') FROM test.test1;
SELECT has(col1, 'bc') FROM test.test1;
SELECT has(col1, 'def') FROM test.test1;
SELECT has(col1, 'ghij') FROM test.test1;
SELECT has(col1, NULL) FROM test.test1;
DROP TABLE IF EXISTS test.test1;
CREATE TABLE test.test1(col1 Array(Nullable(UInt64)), col2 UInt64) Engine=TinyLog;
INSERT INTO test.test1(col1,col2) VALUES([2,3,7,NULL], 2);
INSERT INTO test.test1(col1,col2) VALUES([NULL,3,7,4], 3);
INSERT INTO test.test1(col1,col2) VALUES([2,NULL,7,NULL], 7);
INSERT INTO test.test1(col1,col2) VALUES([2,3,NULL,4],5);
SELECT has(col1,col2) FROM test.test1;
DROP TABLE IF EXISTS test.test1;
CREATE TABLE test.test1(col1 Array(Nullable(UInt64)), col2 Nullable(UInt64)) Engine=TinyLog;
INSERT INTO test.test1(col1,col2) VALUES([2,3,7,NULL], 2);
INSERT INTO test.test1(col1,col2) VALUES([NULL,3,7,4], 3);
INSERT INTO test.test1(col1,col2) VALUES([2,NULL,7,NULL], 7);
INSERT INTO test.test1(col1,col2) VALUES([2,3,NULL,4],5);
INSERT INTO test.test1(col1,col2) VALUES([NULL,NULL,NULL,NULL],NULL);
SELECT has(col1,col2) FROM test.test1;
DROP TABLE IF EXISTS test.test1;
CREATE TABLE test.test1(col1 Array(Nullable(String)), col2 String) Engine=TinyLog;
INSERT INTO test.test1(col1,col2) VALUES(['a','bc','def',NULL], 'a');
INSERT INTO test.test1(col1,col2) VALUES([NULL,'bc','def','ghij'], 'bc');
INSERT INTO test.test1(col1,col2) VALUES(['a',NULL,'def','ghij'], 'def');
INSERT INTO test.test1(col1,col2) VALUES(['a','bc',NULL,'ghij'], 'ghij');
SELECT has(col1,col2) FROM test.test1;
DROP TABLE IF EXISTS test.test1;
CREATE TABLE test.test1(col1 Array(Nullable(String)), col2 Nullable(String)) Engine=TinyLog;
INSERT INTO test.test1(col1,col2) VALUES(['a','bc','def',NULL], 'a');
INSERT INTO test.test1(col1,col2) VALUES([NULL,'bc','def','ghij'], 'bc');
INSERT INTO test.test1(col1,col2) VALUES(['a',NULL,'def','ghij'], 'def');
INSERT INTO test.test1(col1,col2) VALUES(['a','bc',NULL,'ghij'], 'ghij');
INSERT INTO test.test1(col1,col2) VALUES([NULL,NULL,NULL,NULL], NULL);
SELECT has(col1,col2) FROM test.test1;