diff --git a/dbms/include/DB/Functions/FunctionsArray.h b/dbms/include/DB/Functions/FunctionsArray.h index f560bd5e67e..0df0d6101b2 100644 --- a/dbms/include/DB/Functions/FunctionsArray.h +++ b/dbms/include/DB/Functions/FunctionsArray.h @@ -212,6 +212,7 @@ struct ArrayIndexNumImpl throw Exception{"Internal error", ErrorCodes::LOGICAL_ERROR}; } + /// Both function arguments are ordinary. template static void vectorCase1( const PaddedPODArray & data, const ColumnArray::Offsets_t & offsets, @@ -241,6 +242,7 @@ struct ArrayIndexNumImpl } } + /// The 2nd function argument is nullable. template static void vectorCase2( const PaddedPODArray & 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 static void vectorCase3( const PaddedPODArray & 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 static void vectorCase4( const PaddedPODArray & data, const ColumnArray::Offsets_t & offsets, @@ -367,6 +368,7 @@ struct ArrayIndexNumImpl } }; +/// Specialization that catches developer errors. template struct ArrayIndexNumImpl { @@ -382,6 +384,8 @@ struct ArrayIndexNumImpl } }; +/// Implementation for arrays of numbers when the 2nd function argument +/// is a NULL value. template struct ArrayIndexNumNullImpl { @@ -423,13 +427,49 @@ struct ArrayIndexNumNullImpl } }; +/// Implementation for arrays of strings when the 2nd function argument +/// is a NULL value. +template +struct ArrayIndexStringNullImpl +{ + static void vector_const( + const ColumnString::Chars_t & data, const ColumnArray::Offsets_t & offsets, const ColumnString::Offsets_t & string_offsets, + PaddedPODArray & result, + const PaddedPODArray * 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 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 & result) + PaddedPODArray & result, + const PaddedPODArray * 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 & result) + PaddedPODArray & result, + const PaddedPODArray * null_map_data, + const PaddedPODArray * 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 * null_map_data = nullptr; + const PaddedPODArray * null_map_item = nullptr; if (arguments.size() > 2) { - const auto & null_map = block.getByPosition(arguments[2]).column; - null_map_data = &static_cast(*null_map).getData(); + const auto & null_map1 = block.getByPosition(arguments[2]).column; + if (null_map1) + null_map_data = &static_cast(*null_map1).getData(); + + const auto & null_map2 = block.getByPosition(arguments[3]).column; + if (null_map2) + null_map_item = &static_cast(*null_map2).getData(); } const auto item_arg = block.getByPosition(arguments[1]).column.get(); - if (const auto item_arg_const = typeid_cast *>(item_arg)) + if (item_arg->isNull()) + { + const auto col_res = std::make_shared(); + block.getByPosition(result).column = col_res; + + ArrayIndexNumNullImpl::vector(col_nested->getData(), col_array->getOffsets(), + col_res->getData(), null_map_data); + } + else if (const auto item_arg_const = typeid_cast *>(item_arg)) { const auto col_res = std::make_shared(); block.getByPosition(result).column = col_res; @@ -599,14 +672,6 @@ private: const auto col_res = std::make_shared(); block.getByPosition(result).column = col_res; - const PaddedPODArray * null_map_item = nullptr; - - if (arguments.size() > 3) - { - const auto & null_map = block.getByPosition(arguments[3]).column; - null_map_item = &static_cast(*null_map).getData(); - } - ArrayIndexNumImpl::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 * null_map_data = nullptr; + const PaddedPODArray * null_map_item = nullptr; + + if (arguments.size() > 2) + { + const auto & col1 = block.getByPosition(arguments[2]).column; + if (col1) + null_map_data = &static_cast(*col1).getData(); + + const auto & col2 = block.getByPosition(arguments[3]).column; + if (col2) + null_map_item = &static_cast(*col2).getData(); + } + const auto item_arg = block.getByPosition(arguments[1]).column.get(); - if (const auto item_arg_const = typeid_cast *>(item_arg)) + if (item_arg->isNull()) + { + const auto col_res = std::make_shared(); + block.getByPosition(result).column = col_res; + + ArrayIndexStringNullImpl::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 *>(item_arg)) { const auto col_res = std::make_shared(); block.getByPosition(result).column = col_res; ArrayIndexStringImpl::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(item_arg)) { @@ -645,7 +735,7 @@ private: ArrayIndexStringImpl::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 * null_map = nullptr; + + if (arguments.size() > 2) + { + const auto & col = block.getByPosition(arguments[3]).column; + if (col) + null_map = &static_cast(*col).getData(); + } + const auto size = item_arg->size(); const auto col_res = std::make_shared(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(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::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(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(*block.getByPosition(arguments[0]).type).getNestedType(); - const auto & nullable_col = static_cast(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(nested_col, col_array->getOffsetsColumn()), - input_type, - "" }, - block.getByPosition(arguments[1]), - + /// 2nd function argument { - null_map, - std::make_shared(), - "" }, + /// 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(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(nested_col, col_array->getOffsetsColumn()); + data.type = static_cast(*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(); + } + 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(*col); + + auto & arg = source_block.unsafeGetByPosition(1); + arg.column = nullable_col.getNestedColumn(); + arg.type = static_cast(*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(); + } + 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(block, arguments, result) diff --git a/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp index 16e4e124b8e..a2c834faa87 100644 --- a/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include @@ -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(&type)) + else if (const DataTypeArray * type_arr = typeid_cast(&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(column).getData(), offset_columns, level + 1); + } + else { ColumnStream & stream = *column_streams[name]; diff --git a/dbms/tests/queries/0_stateless/00362_nullable.reference b/dbms/tests/queries/0_stateless/00362_nullable.reference index 0a5bdc85e85..c6155dcd9c7 100644 --- a/dbms/tests/queries/0_stateless/00362_nullable.reference +++ b/dbms/tests/queries/0_stateless/00362_nullable.reference @@ -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 diff --git a/dbms/tests/queries/0_stateless/00362_nullable.sql b/dbms/tests/queries/0_stateless/00362_nullable.sql index 604163e972f..8e9a48275fa 100644 --- a/dbms/tests/queries/0_stateless/00362_nullable.sql +++ b/dbms/tests/queries/0_stateless/00362_nullable.sql @@ -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;