mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
some investigation
This commit is contained in:
parent
9b9360eb2c
commit
4a55186a99
@ -112,7 +112,7 @@ public:
|
||||
|
||||
UInt128 getHash() const override { return hash.getHash(*getRawColumnPtr()); }
|
||||
|
||||
inline std::optional<UInt64> getOrFindIndex(const StringRef& value) const override
|
||||
inline std::optional<UInt64> getOrFindIndex(StringRef value) const override
|
||||
{
|
||||
if (std::optional<UInt64> res = reverse_index.getIndex(value); res)
|
||||
return res;
|
||||
@ -120,8 +120,12 @@ public:
|
||||
auto& nested = *getNestedColumn();
|
||||
|
||||
for (size_t i = 0; i < nested.size(); ++i)
|
||||
if (nested.getDataAt(i) == value)
|
||||
{
|
||||
StringRef index = nested.getDataAt(i);
|
||||
|
||||
if (index == value)
|
||||
return i;
|
||||
}
|
||||
|
||||
return {};
|
||||
}
|
||||
|
@ -86,7 +86,7 @@ public:
|
||||
* region, so it can be easily represented as a @e StringRef. So we pass that ref to this function and get its
|
||||
* index in the dictionary, which can be used to operate with the indices column.
|
||||
*/
|
||||
virtual std::optional<UInt64> getOrFindIndex(const StringRef& value) const = 0;
|
||||
virtual std::optional<UInt64> getOrFindIndex(StringRef value) const = 0;
|
||||
|
||||
void insert(const Field &) override
|
||||
{
|
||||
|
@ -326,15 +326,22 @@ public:
|
||||
|
||||
UInt64 insert(const StringRef & data);
|
||||
|
||||
/// If index is not built, builds it.
|
||||
UInt64 getInsertionPoint(const StringRef & data);
|
||||
/// Returns the found data's index in the dictionary. If index is not built, builds it.
|
||||
UInt64 getInsertionPoint(StringRef data)
|
||||
{
|
||||
if (!index) buildIndex();
|
||||
return getIndexImpl(data);
|
||||
}
|
||||
|
||||
/// Returns the found data's index in the dictionary if the #index is built, otherwise, returns a std::nullopt.
|
||||
std::optional<UInt64> getIndex(const StringRef & data) const;
|
||||
std::optional<UInt64> getIndex(StringRef data) const
|
||||
{
|
||||
if (!index) return {};
|
||||
return getIndexImpl(data);
|
||||
}
|
||||
|
||||
UInt64 lastInsertionPoint() const { return size() + base_index; }
|
||||
|
||||
|
||||
ColumnType * getColumn() const { return column; }
|
||||
size_t size() const;
|
||||
|
||||
@ -387,6 +394,8 @@ private:
|
||||
}
|
||||
|
||||
ColumnUInt64::MutablePtr calcHashes() const;
|
||||
|
||||
UInt64 getIndexImpl(StringRef data) const;
|
||||
};
|
||||
|
||||
|
||||
@ -506,11 +515,8 @@ UInt64 ReverseIndex<IndexType, ColumnType>::insert(const StringRef & data)
|
||||
}
|
||||
|
||||
template <typename IndexType, typename ColumnType>
|
||||
UInt64 ReverseIndex<IndexType, ColumnType>::getInsertionPoint(const StringRef & data)
|
||||
UInt64 ReverseIndex<IndexType, ColumnType>::getIndexImpl(StringRef data) const
|
||||
{
|
||||
if (!index)
|
||||
buildIndex();
|
||||
|
||||
using IteratorType = typename IndexMapType::iterator;
|
||||
IteratorType iterator;
|
||||
|
||||
@ -519,20 +525,4 @@ UInt64 ReverseIndex<IndexType, ColumnType>::getInsertionPoint(const StringRef &
|
||||
|
||||
return iterator == index->end() ? size() + base_index : iterator->getValue();
|
||||
}
|
||||
|
||||
template <typename IndexType, typename ColumnType>
|
||||
std::optional<UInt64> ReverseIndex<IndexType, ColumnType>::getIndex(const StringRef & data) const
|
||||
{
|
||||
if (!index)
|
||||
return {};
|
||||
|
||||
using IteratorType = typename IndexMapType::iterator;
|
||||
IteratorType iterator;
|
||||
|
||||
auto hash = getHash(data);
|
||||
iterator = index->reverseIndexFind(data, hash);
|
||||
|
||||
return iterator == index->end() ? size() + base_index : iterator->getValue();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -16,6 +16,7 @@
|
||||
#include <Common/assert_cast.h>
|
||||
#include "Columns/ColumnLowCardinality.h"
|
||||
#include "DataTypes/DataTypeLowCardinality.h"
|
||||
#include "Interpreters/castColumn.h"
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -24,6 +25,7 @@ namespace DB
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
}
|
||||
|
||||
@ -83,10 +85,10 @@ private:
|
||||
/// compares `lhs against `rhs`, third argument unused
|
||||
static bool compare(const Initial & lhs, const Result & rhs, size_t) { return lhs == rhs; }
|
||||
|
||||
template <class U> static bool compare(size_t lhs, const U & rhs, size_t) { return lhs == rhs; }
|
||||
|
||||
static Initial extract(const PaddedPODArray<Initial> & a, size_t i) { return a[i]; }
|
||||
static size_t extract(const ColumnLowCardinality & a, size_t i) { return a.getIndexAt(i); }
|
||||
|
||||
/// LowCardinality spec. We now the column holds one of the UInt* numbers
|
||||
static UInt64 extract(const IColumn & a, size_t i) { return a.getUInt(i); }
|
||||
|
||||
#pragma GCC diagnostic pop
|
||||
|
||||
@ -658,10 +660,7 @@ struct ArrayIndexGenericNullImpl
|
||||
}
|
||||
};
|
||||
|
||||
/**
|
||||
* Check types extracted from Nullable() and LowCardinality()
|
||||
*/
|
||||
inline bool allowArguments(const DataTypePtr & array_inner_type, const DataTypePtr & arg)
|
||||
inline DataTypePtr extractType(const DataTypePtr& type)
|
||||
{
|
||||
/**
|
||||
* Possible cases for #arg and #array_inner_type:
|
||||
@ -675,17 +674,20 @@ inline bool allowArguments(const DataTypePtr & array_inner_type, const DataTypeP
|
||||
* All other variants are considered wrong (Like N(N(N(T)))).
|
||||
* recursiveRemoveLowCardinality works only if the given type is LC(V).
|
||||
*/
|
||||
DataTypePtr array_extracted =
|
||||
return
|
||||
removeNullable( /// remove outer Nullable, case 3
|
||||
recursiveRemoveLowCardinality( /// remove LC, cases 2 and 4
|
||||
removeNullable( /// remove inner Nullable, case 4
|
||||
array_inner_type)));
|
||||
type)));
|
||||
}
|
||||
|
||||
DataTypePtr arg_extracted =
|
||||
removeNullable(
|
||||
recursiveRemoveLowCardinality(
|
||||
removeNullable(
|
||||
arg)));
|
||||
/**
|
||||
* Check types extracted from Nullable() and LowCardinality()
|
||||
*/
|
||||
inline bool allowArguments(const DataTypePtr & array_inner_type, const DataTypePtr & arg)
|
||||
{
|
||||
const DataTypePtr array_extracted = extractType(array_inner_type);
|
||||
const DataTypePtr arg_extracted = extractType(arg);
|
||||
|
||||
return ((isNativeNumber(array_extracted) || isEnum(array_extracted)) && isNativeNumber(arg_extracted))
|
||||
|| array_extracted->equals(*arg_extracted);
|
||||
@ -801,6 +803,55 @@ private:
|
||||
return true;
|
||||
}
|
||||
|
||||
static StringRef valueHelperCast(char * storage, StringRef arg, TypeIndex left, TypeIndex right)
|
||||
{
|
||||
switch (left)
|
||||
{
|
||||
case TypeIndex::UInt8: return valueHelperCastImpl<UInt8>(storage, arg, right);
|
||||
case TypeIndex::UInt16: return valueHelperCastImpl<UInt16>(storage, arg, right);
|
||||
case TypeIndex::UInt32: return valueHelperCastImpl<UInt32>(storage, arg, right);
|
||||
case TypeIndex::UInt64: return valueHelperCastImpl<UInt32>(storage, arg, right);
|
||||
case TypeIndex::Int8: return valueHelperCastImpl<Int8>(storage, arg, right);
|
||||
case TypeIndex::Int16: return valueHelperCastImpl<Int16>(storage, arg, right);
|
||||
case TypeIndex::Int32: return valueHelperCastImpl<Int32>(storage, arg, right);
|
||||
case TypeIndex::Int64: return valueHelperCastImpl<Int64>(storage, arg, right);
|
||||
case TypeIndex::Float32: return valueHelperCastImpl<Float32>(storage, arg, right);
|
||||
case TypeIndex::Float64: return valueHelperCastImpl<Float64>(storage, arg, right);
|
||||
default:
|
||||
throw Exception("Invalid left argument type", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
}
|
||||
|
||||
template <class T>
|
||||
static StringRef valueHelperCastImpl(char * storage, StringRef arg, TypeIndex right)
|
||||
{
|
||||
const T interpreted = *reinterpret_cast<const T*>(arg.data);
|
||||
|
||||
switch (right)
|
||||
{
|
||||
case TypeIndex::UInt8: return insertStorageElem<UInt8>(storage, interpreted);
|
||||
case TypeIndex::UInt16: return insertStorageElem<UInt16>(storage, interpreted);
|
||||
case TypeIndex::UInt32: return insertStorageElem<UInt32>(storage, interpreted);
|
||||
case TypeIndex::UInt64: return insertStorageElem<UInt32>(storage, interpreted);
|
||||
case TypeIndex::Int8: return insertStorageElem<Int8>(storage, interpreted);
|
||||
case TypeIndex::Int16: return insertStorageElem<Int16>(storage, interpreted);
|
||||
case TypeIndex::Int32: return insertStorageElem<Int32>(storage, interpreted);
|
||||
case TypeIndex::Int64: return insertStorageElem<Int64>(storage, interpreted);
|
||||
case TypeIndex::Float32: return insertStorageElem<Float32>(storage, interpreted);
|
||||
case TypeIndex::Float64: return insertStorageElem<Float64>(storage, interpreted);
|
||||
default:
|
||||
throw Exception("Invalid right argument type", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
}
|
||||
|
||||
template <class T, class U>
|
||||
static StringRef insertStorageElem(char * storage, U elem)
|
||||
{
|
||||
const T target {static_cast<T>(elem)};
|
||||
memcpy(storage, &target, sizeof(T));
|
||||
return {storage, sizeof(T)};
|
||||
}
|
||||
|
||||
/**
|
||||
* 1. Obtain the right-side argument column @e C. If @e C is a non-const column (thus the argument is not constant),
|
||||
* loop through all @e C's values.
|
||||
@ -812,7 +863,7 @@ private:
|
||||
*/
|
||||
bool executeLowCardinality(Block & block, const ColumnNumbers & arguments, size_t result) const
|
||||
{
|
||||
const ColumnArray * col_array = checkAndGetColumn<ColumnArray>(
|
||||
const ColumnArray * const col_array = checkAndGetColumn<ColumnArray>(
|
||||
block.getByPosition(arguments[0]).column.get());
|
||||
|
||||
if (!col_array)
|
||||
@ -824,7 +875,7 @@ private:
|
||||
* 2. LC(Nullable(T)) -- somewhat special as Nullable's getDataAt is slightly slower
|
||||
* (due to nested column invocation).
|
||||
*/
|
||||
const ColumnLowCardinality * col_array_nested_lc =
|
||||
const ColumnLowCardinality * const col_array_nested_lc =
|
||||
checkAndGetColumn<ColumnLowCardinality>(&col_array->getData());
|
||||
|
||||
if (!col_array_nested_lc)
|
||||
@ -837,12 +888,47 @@ private:
|
||||
col_res->getData().resize_fill(col_array->getOffsets().size());
|
||||
|
||||
const auto [null_map_data, null_map_item] = getNullMaps(block, arguments);
|
||||
const IColumn * col_arg = block.getByPosition(arguments[1]).column.get();
|
||||
const IColumn * const col_arg = block.getByPosition(arguments[1]).column.get();
|
||||
|
||||
/**
|
||||
* If the types of #col_arg and #col_array_nested_lc (or its nested column if Nullable) are
|
||||
* non-integral, everything is ok as equal values would give equal StringRef representations.
|
||||
* But this is not true for different integral types:
|
||||
* consider #col_arg = UInt8 and #col_array_nested_lc = UInt16.
|
||||
* The right argument StringRef's size would be 2 (and the left one's -- 1).
|
||||
*
|
||||
* So, for integral types, it's not enough to simply call getDataAt on both arguments.
|
||||
* The left argument (the value whose index is being searched in the indices column) must be casted
|
||||
* to the right argument's side to ensure the StringRefs' equality.
|
||||
*
|
||||
* At first glimpse, we have two issues: signed to unsigned cast (and vice versa) and overflow.
|
||||
* TL;DR -- we don't.
|
||||
*
|
||||
* In fact, we have two sets of values, one for the #col_arg (let's call it Left), and one for the
|
||||
* #col_array_nested_lc (the Right).
|
||||
*
|
||||
* So, if any of the Left values can't be represented by the Right (e.g. -1 for UInt32, which won't
|
||||
* typically be in the dictionary, or max(UInt32) for UInt8), we simply don't care. The function would
|
||||
* return 0 (as it should).
|
||||
*/
|
||||
std::aligned_union_t<sizeof(UInt8), Int64, UInt64, Float64> storage;
|
||||
|
||||
const size_t size = isColumnConst(*col_arg)
|
||||
? 1 /// We have a column with just one value. Arbitrary n is allowed (as the column is const), so take 0.
|
||||
: col_arg->size();
|
||||
|
||||
const bool is_numeric =
|
||||
col_arg->isNumeric()
|
||||
|| col_array_nested_lc->getDictionary().isNumeric()
|
||||
|| col_array_nested_lc->getDictionary().getNestedColumn()->isNumeric(); // Nullable
|
||||
|
||||
const TypeIndex arg_type = col_arg->getDataType();
|
||||
|
||||
const TypeIndex lc_dict_type = col_array_nested_lc->getDictionary().getDataType();
|
||||
const TypeIndex lc_dict_nested_type = lc_dict_type == TypeIndex::Nullable
|
||||
? col_array_nested_lc->getDictionary().getNestedColumn()->getDataType()
|
||||
: lc_dict_type;
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
if (col_arg->onlyNull())
|
||||
@ -857,7 +943,13 @@ private:
|
||||
continue;
|
||||
}
|
||||
|
||||
const StringRef elem = col_arg->getDataAt(i);
|
||||
const StringRef elem = is_numeric && arg_type != lc_dict_nested_type
|
||||
? valueHelperCast(
|
||||
reinterpret_cast<char *>(&storage),
|
||||
col_arg->getDataAt(i),
|
||||
arg_type,
|
||||
lc_dict_nested_type)
|
||||
: col_arg->getDataAt(i);
|
||||
|
||||
if (elem == EMPTY_STRING_REF) /// Possible if the column is Nullable and the data was not present.
|
||||
continue;
|
||||
@ -867,18 +959,12 @@ private:
|
||||
if (!value_index)
|
||||
continue;
|
||||
|
||||
ArrayIndexNumImpl<
|
||||
UInt64, /* Ignored */
|
||||
UInt64, /*Ignored */
|
||||
ConcreteAction,
|
||||
false /* Invoking from LC spec */
|
||||
>::vector(
|
||||
*col_array_nested_lc, /* pass the column itself, the index will be extracted in the impl */
|
||||
col_array->getOffsets(),
|
||||
*value_index, /* target value to search */
|
||||
col_res->getData(),
|
||||
null_map_data,
|
||||
null_map_item);
|
||||
ArrayIndexNumImpl<UInt64, UInt64, ConcreteAction, false /* Invoking from LC spec */ >::vector(
|
||||
col_array_nested_lc->getIndexes(),/* pass the indices column */
|
||||
col_array->getOffsets(),
|
||||
*value_index, /* target value to search */
|
||||
col_res->getData(),
|
||||
null_map_data, null_map_item);
|
||||
}
|
||||
|
||||
block.getByPosition(result).column = std::move(col_res);
|
||||
|
@ -162,8 +162,47 @@ DROP TABLE IF EXISTS bloom_filter_lc_null_types_test;
|
||||
|
||||
DROP TABLE IF EXISTS bloom_filter_array_lc_null_types_test;
|
||||
|
||||
CREATE TABLE bloom_filter_array_lc_null_types_test (order_key Array(LowCardinality(Nullable((UInt64)))), i8 Array(LowCardinality(Nullable((Int8)))), i16 Array(LowCardinality(Nullable((Int16)))), i32 Array(LowCardinality(Nullable((Int32)))), i64 Array(LowCardinality(Nullable((Int64)))), u8 Array(LowCardinality(Nullable((UInt8)))), u16 Array(LowCardinality(Nullable((UInt16)))), u32 Array(LowCardinality(Nullable((UInt32)))), u64 Array(LowCardinality(Nullable((UInt64)))), f32 Array(LowCardinality(Nullable((Float32)))), f64 Array(LowCardinality(Nullable((Float64)))), date Array(LowCardinality(Nullable((Date)))), date_time Array(LowCardinality(Nullable(DateTime('Europe/Moscow')))), str Array(LowCardinality(Nullable((String)))), fixed_string Array(LowCardinality(Nullable(FixedString(5)))), INDEX idx (i8, i16, i32, i64, u8, u16, u32, u64, f32, f64, date, date_time, str, fixed_string) TYPE bloom_filter GRANULARITY 1) ENGINE = MergeTree() ORDER BY order_key SETTINGS index_granularity = 6;
|
||||
INSERT INTO bloom_filter_array_lc_null_types_test SELECT groupArray(number) AS order_key, groupArray(toInt8(number)) AS i8, groupArray(toInt16(number)) AS i16, groupArray(toInt32(number)) AS i32, groupArray(toInt64(number)) AS i64, groupArray(toUInt8(number)) AS u8, groupArray(toUInt16(number)) AS u16, groupArray(toUInt32(number)) AS u32, groupArray(toUInt64(number)) AS u64, groupArray(toFloat32(number)) AS f32, groupArray(toFloat64(number)) AS f64, groupArray(toDate(number, 'Europe/Moscow')) AS date, groupArray(toDateTime(number, 'Europe/Moscow')) AS date_time, groupArray(toString(number)) AS str, groupArray(toFixedString(toString(number), 5)) AS fixed_string FROM (SELECT number FROM system.numbers LIMIT 15);
|
||||
CREATE TABLE bloom_filter_array_lc_null_types_test (
|
||||
order_key Array(LowCardinality(Nullable((UInt64)))),
|
||||
|
||||
i8 Array(LowCardinality(Nullable((Int8)))),
|
||||
i16 Array(LowCardinality(Nullable((Int16)))),
|
||||
i32 Array(LowCardinality(Nullable((Int32)))),
|
||||
i64 Array(LowCardinality(Nullable((Int64)))),
|
||||
u8 Array(LowCardinality(Nullable((UInt8)))),
|
||||
u16 Array(LowCardinality(Nullable((UInt16)))),
|
||||
u32 Array(LowCardinality(Nullable((UInt32)))),
|
||||
u64 Array(LowCardinality(Nullable((UInt64)))),
|
||||
f32 Array(LowCardinality(Nullable((Float32)))),
|
||||
f64 Array(LowCardinality(Nullable((Float64)))),
|
||||
|
||||
date Array(LowCardinality(Nullable((Date)))),
|
||||
date_time Array(LowCardinality(Nullable(DateTime('Europe/Moscow')))),
|
||||
|
||||
str Array(LowCardinality(Nullable((String)))),
|
||||
fixed_string Array(LowCardinality(Nullable(FixedString(5)))),
|
||||
INDEX idx (i8, i16, i32, i64, u8, u16, u32, u64, f32, f64, date, date_time, str, fixed_string)
|
||||
TYPE bloom_filter GRANULARITY 1)
|
||||
ENGINE = MergeTree() ORDER BY order_key SETTINGS index_granularity = 6;
|
||||
|
||||
INSERT INTO bloom_filter_array_lc_null_types_test
|
||||
SELECT groupArray(number) AS order_key,
|
||||
groupArray(toInt8(number)) AS i8,
|
||||
groupArray(toInt16(number)) AS i16,
|
||||
groupArray(toInt32(number)) AS i32,
|
||||
groupArray(toInt64(number)) AS i64,
|
||||
groupArray(toUInt8(number)) AS u8,
|
||||
groupArray(toUInt16(number)) AS u16,
|
||||
groupArray(toUInt32(number)) AS u32,
|
||||
groupArray(toUInt64(number)) AS u64,
|
||||
groupArray(toFloat32(number)) AS f32,
|
||||
groupArray(toFloat64(number)) AS f64,
|
||||
groupArray(toDate(number, 'Europe/Moscow')) AS date,
|
||||
groupArray(toDateTime(number, 'Europe/Moscow')) AS date_time,
|
||||
groupArray(toString(number)) AS str,
|
||||
groupArray(toFixedString(toString(number), 5)) AS fixed_string
|
||||
FROM (SELECT number FROM system.numbers LIMIT 15);
|
||||
|
||||
INSERT INTO bloom_filter_array_lc_null_types_test SELECT groupArray(number) AS order_key, groupArray(toInt8(number)) AS i8, groupArray(toInt16(number)) AS i16, groupArray(toInt32(number)) AS i32, groupArray(toInt64(number)) AS i64, groupArray(toUInt8(number)) AS u8, groupArray(toUInt16(number)) AS u16, groupArray(toUInt32(number)) AS u32, groupArray(toUInt64(number)) AS u64, groupArray(toFloat32(number)) AS f32, groupArray(toFloat64(number)) AS f64, groupArray(toDate(number, 'Europe/Moscow')) AS date, groupArray(toDateTime(number, 'Europe/Moscow')) AS date_time, groupArray(toString(number)) AS str, groupArray(toFixedString(toString(number), 5)) AS fixed_string FROM (SELECT number FROM system.numbers WHERE number >= 5 LIMIT 15);
|
||||
INSERT INTO bloom_filter_array_lc_null_types_test SELECT groupArray(number) AS order_key, groupArray(toInt8(number)) AS i8, groupArray(toInt16(number)) AS i16, groupArray(toInt32(number)) AS i32, groupArray(toInt64(number)) AS i64, groupArray(toUInt8(number)) AS u8, groupArray(toUInt16(number)) AS u16, groupArray(toUInt32(number)) AS u32, groupArray(toUInt64(number)) AS u64, groupArray(toFloat32(number)) AS f32, groupArray(toFloat64(number)) AS f64, groupArray(toDate(number, 'Europe/Moscow')) AS date, groupArray(toDateTime(number, 'Europe/Moscow')) AS date_time, groupArray(toString(number)) AS str, groupArray(toFixedString(toString(number), 5)) AS fixed_string FROM (SELECT number FROM system.numbers WHERE number >= 10 LIMIT 15);
|
||||
INSERT INTO bloom_filter_array_lc_null_types_test SELECT n AS order_key, n AS i8, n AS i16, n AS i32, n AS i64, n AS u8, n AS u16, n AS u32, n AS u64, n AS f32, n AS f64, n AS date, n AS date_time, n AS str, n AS fixed_string FROM (SELECT [NULL] AS n);
|
||||
|
220
tests/queries/0_stateless/01414_low_cardinality_nullable.sql
Normal file
220
tests/queries/0_stateless/01414_low_cardinality_nullable.sql
Normal file
@ -0,0 +1,220 @@
|
||||
DROP TABLE IF EXISTS lc_nullable;
|
||||
|
||||
CREATE TABLE lc_nullable (
|
||||
order_key Array(LowCardinality(Nullable((UInt64)))),
|
||||
|
||||
i8 Array(LowCardinality(Nullable(Int8))),
|
||||
i16 Array(LowCardinality(Nullable(Int16))),
|
||||
i32 Array(LowCardinality(Nullable(Int32))),
|
||||
i64 Array(LowCardinality(Nullable(Int64))),
|
||||
u8 Array(LowCardinality(Nullable(UInt8))),
|
||||
u16 Array(LowCardinality(Nullable(UInt16))),
|
||||
u32 Array(LowCardinality(Nullable(UInt32))),
|
||||
u64 Array(LowCardinality(Nullable(UInt64))),
|
||||
f32 Array(LowCardinality(Nullable(Float32))),
|
||||
f64 Array(LowCardinality(Nullable(Float64))),
|
||||
|
||||
date Array(LowCardinality(Nullable((Date)))),
|
||||
date_time Array(LowCardinality(Nullable(DateTime('Europe/Moscow')))),
|
||||
|
||||
str Array(LowCardinality(Nullable((String)))),
|
||||
fixed_string Array(LowCardinality(Nullable(FixedString(5))))
|
||||
) ENGINE = MergeTree() ORDER BY order_key;
|
||||
|
||||
INSERT INTO lc_nullable SELECT
|
||||
groupArray(number) AS order_key,
|
||||
groupArray(toInt8(number)) AS i8,
|
||||
groupArray(toInt16(number)) AS i16,
|
||||
groupArray(toInt32(number)) AS i32,
|
||||
groupArray(toInt64(number)) AS i64,
|
||||
groupArray(toUInt8(number)) AS u8,
|
||||
groupArray(toUInt16(number)) AS u16,
|
||||
groupArray(toUInt32(number)) AS u32,
|
||||
groupArray(toUInt64(number)) AS u64,
|
||||
groupArray(toFloat32(number)) AS f32,
|
||||
groupArray(toFloat64(number)) AS f64,
|
||||
groupArray(toDate(number, 'Europe/Moscow')) AS date,
|
||||
groupArray(toDateTime(number, 'Europe/Moscow')) AS date_time,
|
||||
groupArray(toString(number)) AS str,
|
||||
groupArray(toFixedString(toString(number), 5)) AS fixed_string
|
||||
FROM (SELECT number FROM system.numbers LIMIT 15);
|
||||
|
||||
INSERT INTO lc_nullable SELECT
|
||||
groupArray(number) AS order_key,
|
||||
groupArray(toInt8(number)) AS i8,
|
||||
groupArray(toInt16(number)) AS i16,
|
||||
groupArray(toInt32(number)) AS i32,
|
||||
groupArray(toInt64(number)) AS i64,
|
||||
groupArray(toUInt8(number)) AS u8,
|
||||
groupArray(toUInt16(number)) AS u16,
|
||||
groupArray(toUInt32(number)) AS u32,
|
||||
groupArray(toUInt64(number)) AS u64,
|
||||
groupArray(toFloat32(number)) AS f32,
|
||||
groupArray(toFloat64(number)) AS f64,
|
||||
groupArray(toDate(number, 'Europe/Moscow')) AS date,
|
||||
groupArray(toDateTime(number, 'Europe/Moscow')) AS date_time,
|
||||
groupArray(toString(number)) AS str,
|
||||
groupArray(toFixedString(toString(number), 5)) AS fixed_string
|
||||
FROM (SELECT -number FROM system.numbers LIMIT 15);
|
||||
|
||||
INSERT INTO lc_nullable SELECT
|
||||
groupArray(number) AS order_key,
|
||||
groupArray(toInt8(number)) AS i8,
|
||||
groupArray(toInt16(number)) AS i16,
|
||||
groupArray(toInt32(number)) AS i32,
|
||||
groupArray(toInt64(number)) AS i64,
|
||||
groupArray(toUInt8(number)) AS u8,
|
||||
groupArray(toUInt16(number)) AS u16,
|
||||
groupArray(toUInt32(number)) AS u32,
|
||||
groupArray(toUInt64(number)) AS u64,
|
||||
groupArray(toFloat32(number)) AS f32,
|
||||
groupArray(toFloat64(number)) AS f64,
|
||||
groupArray(toDate(number, 'Europe/Moscow')) AS date,
|
||||
groupArray(toDateTime(number, 'Europe/Moscow')) AS date_time,
|
||||
groupArray(toString(number)) AS str,
|
||||
groupArray(toFixedString(toString(number), 5)) AS fixed_string
|
||||
FROM (SELECT number FROM system.numbers WHERE number >= 5 LIMIT 15);
|
||||
|
||||
INSERT INTO lc_nullable SELECT
|
||||
groupArray(number) AS order_key,
|
||||
groupArray(toInt8(number)) AS i8,
|
||||
groupArray(toInt16(number)) AS i16,
|
||||
groupArray(toInt32(number)) AS i32,
|
||||
groupArray(toInt64(number)) AS i64,
|
||||
groupArray(toUInt8(number)) AS u8,
|
||||
groupArray(toUInt16(number)) AS u16,
|
||||
groupArray(toUInt32(number)) AS u32,
|
||||
groupArray(toUInt64(number)) AS u64,
|
||||
groupArray(toFloat32(number)) AS f32,
|
||||
groupArray(toFloat64(number)) AS f64,
|
||||
groupArray(toDate(number, 'Europe/Moscow')) AS date,
|
||||
groupArray(toDateTime(number, 'Europe/Moscow')) AS date_time,
|
||||
groupArray(toString(number)) AS str,
|
||||
groupArray(toFixedString(toString(number), 5)) AS fixed_string
|
||||
FROM (SELECT number FROM system.numbers WHERE number >= 10 LIMIT 15);
|
||||
|
||||
INSERT INTO lc_nullable SELECT
|
||||
n AS order_key,
|
||||
n AS i8,
|
||||
n AS i16,
|
||||
n AS i32,
|
||||
n AS i64,
|
||||
n AS u8,
|
||||
n AS u16,
|
||||
n AS u32,
|
||||
n AS u64,
|
||||
n AS f32,
|
||||
n AS f64,
|
||||
n AS date,
|
||||
n AS date_time,
|
||||
n AS str,
|
||||
n AS fixed_string
|
||||
FROM (SELECT [NULL] AS n);
|
||||
|
||||
INSERT INTO lc_nullable SELECT
|
||||
[NULL, n] AS order_key,
|
||||
[NULL, toInt8(n)] AS i8,
|
||||
[NULL, toInt16(n)] AS i16,
|
||||
[NULL, toInt32(n)] AS i32,
|
||||
[NULL, toInt64(n)] AS i64,
|
||||
[NULL, toUInt8(n)] AS u8,
|
||||
[NULL, toUInt16(n)] AS u16,
|
||||
[NULL, toUInt32(n)] AS u32,
|
||||
[NULL, toUInt64(n)] AS u64,
|
||||
[NULL, toFloat32(n)] AS f32,
|
||||
[NULL, toFloat64(n)] AS f64,
|
||||
[NULL, toDate(n, 'Europe/Moscow')] AS date,
|
||||
[NULL, toDateTime(n, 'Europe/Moscow')] AS date_time,
|
||||
[NULL, toString(n)] AS str,
|
||||
[NULL, toFixedString(toString(n), 5)] AS fixed_string
|
||||
FROM (SELECT 100 as n);
|
||||
|
||||
SELECT count() FROM lc_nullable WHERE has(i8, 1);
|
||||
SELECT count() FROM lc_nullable WHERE has(i16, 1);
|
||||
SELECT count() FROM lc_nullable WHERE has(i32, 1);
|
||||
SELECT count() FROM lc_nullable WHERE has(i64, 1);
|
||||
SELECT count() FROM lc_nullable WHERE has(u8, 1);
|
||||
SELECT count() FROM lc_nullable WHERE has(u16, 1);
|
||||
SELECT count() FROM lc_nullable WHERE has(u32, 1);
|
||||
SELECT count() FROM lc_nullable WHERE has(u64, 1);
|
||||
SELECT count() FROM lc_nullable WHERE has(f32, 1);
|
||||
SELECT count() FROM lc_nullable WHERE has(f64, 1);
|
||||
SELECT count() FROM lc_nullable WHERE has(date, toDate('1970-01-02'));
|
||||
SELECT count() FROM lc_nullable WHERE has(date_time, toDateTime('1970-01-01 03:00:01', 'Europe/Moscow'));
|
||||
SELECT count() FROM lc_nullable WHERE has(str, '1');
|
||||
SELECT count() FROM lc_nullable WHERE has(fixed_string, toFixedString('1', 5));
|
||||
|
||||
SELECT count() FROM lc_nullable WHERE has(i8, -1);
|
||||
SELECT count() FROM lc_nullable WHERE has(i16, -1);
|
||||
SELECT count() FROM lc_nullable WHERE has(i32, -1);
|
||||
SELECT count() FROM lc_nullable WHERE has(i64, -1);
|
||||
SELECT count() FROM lc_nullable WHERE has(u8, -1);
|
||||
SELECT count() FROM lc_nullable WHERE has(u16, -1);
|
||||
SELECT count() FROM lc_nullable WHERE has(u32, -1);
|
||||
SELECT count() FROM lc_nullable WHERE has(u64, -1);
|
||||
SELECT count() FROM lc_nullable WHERE has(f32, -1);
|
||||
SELECT count() FROM lc_nullable WHERE has(f64, -1);
|
||||
SELECT count() FROM lc_nullable WHERE has(str, '-1');
|
||||
SELECT count() FROM lc_nullable WHERE has(fixed_string, toFixedString('-1', 5));
|
||||
|
||||
SELECT count() FROM lc_nullable WHERE has(i8, 5);
|
||||
SELECT count() FROM lc_nullable WHERE has(i16, 5);
|
||||
SELECT count() FROM lc_nullable WHERE has(i32, 5);
|
||||
SELECT count() FROM lc_nullable WHERE has(i64, 5);
|
||||
SELECT count() FROM lc_nullable WHERE has(u8, 5);
|
||||
SELECT count() FROM lc_nullable WHERE has(u16, 5);
|
||||
SELECT count() FROM lc_nullable WHERE has(u32, 5);
|
||||
SELECT count() FROM lc_nullable WHERE has(u64, 5);
|
||||
SELECT count() FROM lc_nullable WHERE has(f32, 5);
|
||||
SELECT count() FROM lc_nullable WHERE has(f64, 5);
|
||||
SELECT count() FROM lc_nullable WHERE has(date, toDate('1970-01-06'));
|
||||
SELECT count() FROM lc_nullable WHERE has(date_time, toDateTime('1970-01-01 03:00:05', 'Europe/Moscow'));
|
||||
SELECT count() FROM lc_nullable WHERE has(str, '5');
|
||||
SELECT count() FROM lc_nullable WHERE has(fixed_string, toFixedString('5', 5));
|
||||
|
||||
SELECT count() FROM lc_nullable WHERE has(i8, 10);
|
||||
SELECT count() FROM lc_nullable WHERE has(i16, 10);
|
||||
SELECT count() FROM lc_nullable WHERE has(i32, 10);
|
||||
SELECT count() FROM lc_nullable WHERE has(i64, 10);
|
||||
SELECT count() FROM lc_nullable WHERE has(u8, 10);
|
||||
SELECT count() FROM lc_nullable WHERE has(u16, 10);
|
||||
SELECT count() FROM lc_nullable WHERE has(u32, 10);
|
||||
SELECT count() FROM lc_nullable WHERE has(u64, 10);
|
||||
SELECT count() FROM lc_nullable WHERE has(f32, 10);
|
||||
SELECT count() FROM lc_nullable WHERE has(f64, 10);
|
||||
SELECT count() FROM lc_nullable WHERE has(date, toDate('1970-01-11'));
|
||||
SELECT count() FROM lc_nullable WHERE has(date_time, toDateTime('1970-01-01 03:00:10', 'Europe/Moscow'));
|
||||
SELECT count() FROM lc_nullable WHERE has(str, '10');
|
||||
SELECT count() FROM lc_nullable WHERE has(fixed_string, toFixedString('10', 5));
|
||||
|
||||
SELECT count() FROM lc_nullable WHERE has(i8, NULL);
|
||||
SELECT count() FROM lc_nullable WHERE has(i16, NULL);
|
||||
SELECT count() FROM lc_nullable WHERE has(i32, NULL);
|
||||
SELECT count() FROM lc_nullable WHERE has(i64, NULL);
|
||||
SELECT count() FROM lc_nullable WHERE has(u8, NULL);
|
||||
SELECT count() FROM lc_nullable WHERE has(u16, NULL);
|
||||
SELECT count() FROM lc_nullable WHERE has(u32, NULL);
|
||||
SELECT count() FROM lc_nullable WHERE has(u64, NULL);
|
||||
SELECT count() FROM lc_nullable WHERE has(f32, NULL);
|
||||
SELECT count() FROM lc_nullable WHERE has(f64, NULL);
|
||||
SELECT count() FROM lc_nullable WHERE has(date, NULL);
|
||||
SELECT count() FROM lc_nullable WHERE has(date_time, NULL);
|
||||
SELECT count() FROM lc_nullable WHERE has(str, NULL);
|
||||
SELECT count() FROM lc_nullable WHERE has(fixed_string, NULL);
|
||||
|
||||
SELECT count() FROM lc_nullable WHERE has(i8, 100);
|
||||
SELECT count() FROM lc_nullable WHERE has(i16, 100);
|
||||
SELECT count() FROM lc_nullable WHERE has(i32, 100);
|
||||
SELECT count() FROM lc_nullable WHERE has(i64, 100);
|
||||
SELECT count() FROM lc_nullable WHERE has(u8, 100);
|
||||
SELECT count() FROM lc_nullable WHERE has(u16, 100);
|
||||
SELECT count() FROM lc_nullable WHERE has(u32, 100);
|
||||
SELECT count() FROM lc_nullable WHERE has(u64, 100);
|
||||
SELECT count() FROM lc_nullable WHERE has(f32, 100);
|
||||
SELECT count() FROM lc_nullable WHERE has(f64, 100);
|
||||
SELECT count() FROM lc_nullable WHERE has(date, toDate('1970-04-11'));
|
||||
SELECT count() FROM lc_nullable WHERE has(date_time, toDateTime('1970-01-01 03:01:40', 'Europe/Moscow'));
|
||||
SELECT count() FROM lc_nullable WHERE has(str, '100');
|
||||
SELECT count() FROM lc_nullable WHERE has(fixed_string, toFixedString('100', 5));
|
||||
|
||||
DROP TABLE IF EXISTS lc_nullable;
|
Loading…
Reference in New Issue
Block a user