mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
dbms: Server: Adding nullable type support to functions manipulating arrays. [#METR-19266]
This commit is contained in:
parent
59043c680e
commit
bd73b2d165
@ -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)
|
||||
|
@ -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];
|
||||
|
||||
|
@ -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
|
||||
|
@ -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;
|
||||
|
Loading…
Reference in New Issue
Block a user