mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 08:32:02 +00:00
Merge pull request #55061 from amosbird/support-empty-tuple-2
Support empty tuple [second implementation]
This commit is contained in:
commit
78506304b7
@ -828,7 +828,7 @@ ColumnPtr ColumnArray::filterTuple(const Filter & filt, ssize_t result_size_hint
|
||||
size_t tuple_size = tuple.tupleSize();
|
||||
|
||||
if (tuple_size == 0)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Empty tuple");
|
||||
return filterGeneric(filt, result_size_hint);
|
||||
|
||||
Columns temporary_arrays(tuple_size);
|
||||
for (size_t i = 0; i < tuple_size; ++i)
|
||||
@ -1265,7 +1265,7 @@ ColumnPtr ColumnArray::replicateTuple(const Offsets & replicate_offsets) const
|
||||
size_t tuple_size = tuple.tupleSize();
|
||||
|
||||
if (tuple_size == 0)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Empty tuple");
|
||||
return replicateGeneric(replicate_offsets);
|
||||
|
||||
Columns temporary_arrays(tuple_size);
|
||||
for (size_t i = 0; i < tuple_size; ++i)
|
||||
|
@ -820,6 +820,9 @@ ColumnPtr recursiveRemoveSparse(const ColumnPtr & column)
|
||||
if (const auto * column_tuple = typeid_cast<const ColumnTuple *>(column.get()))
|
||||
{
|
||||
auto columns = column_tuple->getColumns();
|
||||
if (columns.empty())
|
||||
return column;
|
||||
|
||||
for (auto & element : columns)
|
||||
element = recursiveRemoveSparse(element);
|
||||
|
||||
|
@ -3,14 +3,16 @@
|
||||
#include <Columns/ColumnCompressed.h>
|
||||
#include <Columns/IColumnImpl.h>
|
||||
#include <Core/Field.h>
|
||||
#include <Processors/Transforms/ColumnGathererTransform.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <Common/Arena.h>
|
||||
#include <Common/WeakHash.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Common/iota.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Columns/ColumnsCommon.h>
|
||||
#include <DataTypes/Serializations/SerializationInfoTuple.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <Processors/Transforms/ColumnGathererTransform.h>
|
||||
#include <base/sort.h>
|
||||
|
||||
|
||||
@ -23,6 +25,7 @@ namespace ErrorCodes
|
||||
extern const int NOT_IMPLEMENTED;
|
||||
extern const int CANNOT_INSERT_VALUE_OF_DIFFERENT_SIZE_INTO_TUPLE;
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int SIZES_OF_COLUMNS_DOESNT_MATCH;
|
||||
}
|
||||
|
||||
|
||||
@ -44,6 +47,9 @@ std::string ColumnTuple::getName() const
|
||||
|
||||
ColumnTuple::ColumnTuple(MutableColumns && mutable_columns)
|
||||
{
|
||||
if (mutable_columns.empty())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "This function cannot be used to construct empty tuple. It is a bug");
|
||||
|
||||
columns.reserve(mutable_columns.size());
|
||||
for (auto & column : mutable_columns)
|
||||
{
|
||||
@ -52,15 +58,21 @@ ColumnTuple::ColumnTuple(MutableColumns && mutable_columns)
|
||||
|
||||
columns.push_back(std::move(column));
|
||||
}
|
||||
column_length = columns[0]->size();
|
||||
}
|
||||
|
||||
ColumnTuple::ColumnTuple(size_t len) : column_length(len) {}
|
||||
|
||||
ColumnTuple::Ptr ColumnTuple::create(const Columns & columns)
|
||||
{
|
||||
if (columns.empty())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "This function cannot be used to construct empty tuple. It is a bug");
|
||||
|
||||
for (const auto & column : columns)
|
||||
if (isColumnConst(*column))
|
||||
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "ColumnTuple cannot have ColumnConst as its element");
|
||||
|
||||
auto column_tuple = ColumnTuple::create(MutableColumns());
|
||||
auto column_tuple = ColumnTuple::create(columns[0]->size());
|
||||
column_tuple->columns.assign(columns.begin(), columns.end());
|
||||
|
||||
return column_tuple;
|
||||
@ -68,11 +80,14 @@ ColumnTuple::Ptr ColumnTuple::create(const Columns & columns)
|
||||
|
||||
ColumnTuple::Ptr ColumnTuple::create(const TupleColumns & columns)
|
||||
{
|
||||
if (columns.empty())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "This function cannot be used to construct empty tuple. It is a bug");
|
||||
|
||||
for (const auto & column : columns)
|
||||
if (isColumnConst(*column))
|
||||
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "ColumnTuple cannot have ColumnConst as its element");
|
||||
|
||||
auto column_tuple = ColumnTuple::create(MutableColumns());
|
||||
auto column_tuple = ColumnTuple::create(columns[0]->size());
|
||||
column_tuple->columns = columns;
|
||||
|
||||
return column_tuple;
|
||||
@ -80,6 +95,9 @@ ColumnTuple::Ptr ColumnTuple::create(const TupleColumns & columns)
|
||||
|
||||
MutableColumnPtr ColumnTuple::cloneEmpty() const
|
||||
{
|
||||
if (columns.empty())
|
||||
return ColumnTuple::create(0);
|
||||
|
||||
const size_t tuple_size = columns.size();
|
||||
MutableColumns new_columns(tuple_size);
|
||||
for (size_t i = 0; i < tuple_size; ++i)
|
||||
@ -90,6 +108,9 @@ MutableColumnPtr ColumnTuple::cloneEmpty() const
|
||||
|
||||
MutableColumnPtr ColumnTuple::cloneResized(size_t new_size) const
|
||||
{
|
||||
if (columns.empty())
|
||||
return ColumnTuple::create(new_size);
|
||||
|
||||
const size_t tuple_size = columns.size();
|
||||
MutableColumns new_columns(tuple_size);
|
||||
for (size_t i = 0; i < tuple_size; ++i)
|
||||
@ -98,6 +119,16 @@ MutableColumnPtr ColumnTuple::cloneResized(size_t new_size) const
|
||||
return ColumnTuple::create(std::move(new_columns));
|
||||
}
|
||||
|
||||
size_t ColumnTuple::size() const
|
||||
{
|
||||
if (columns.empty())
|
||||
return column_length;
|
||||
|
||||
/// It's difficult to maintain a consistent `column_length` because there
|
||||
/// are many places that manipulates sub-columns directly.
|
||||
return columns.at(0)->size();
|
||||
}
|
||||
|
||||
Field ColumnTuple::operator[](size_t n) const
|
||||
{
|
||||
Field res;
|
||||
@ -144,6 +175,7 @@ void ColumnTuple::insert(const Field & x)
|
||||
if (tuple.size() != tuple_size)
|
||||
throw Exception(ErrorCodes::CANNOT_INSERT_VALUE_OF_DIFFERENT_SIZE_INTO_TUPLE, "Cannot insert value of different size into tuple");
|
||||
|
||||
++column_length;
|
||||
for (size_t i = 0; i < tuple_size; ++i)
|
||||
columns[i]->insert(tuple[i]);
|
||||
}
|
||||
@ -181,6 +213,7 @@ void ColumnTuple::insertFrom(const IColumn & src_, size_t n)
|
||||
if (src.columns.size() != tuple_size)
|
||||
throw Exception(ErrorCodes::CANNOT_INSERT_VALUE_OF_DIFFERENT_SIZE_INTO_TUPLE, "Cannot insert value of different size into tuple");
|
||||
|
||||
++column_length;
|
||||
for (size_t i = 0; i < tuple_size; ++i)
|
||||
columns[i]->insertFrom(*src.columns[i], n);
|
||||
}
|
||||
@ -199,18 +232,28 @@ void ColumnTuple::insertManyFrom(const IColumn & src, size_t position, size_t le
|
||||
|
||||
void ColumnTuple::insertDefault()
|
||||
{
|
||||
++column_length;
|
||||
for (auto & column : columns)
|
||||
column->insertDefault();
|
||||
}
|
||||
|
||||
void ColumnTuple::popBack(size_t n)
|
||||
{
|
||||
column_length -= n;
|
||||
for (auto & column : columns)
|
||||
column->popBack(n);
|
||||
}
|
||||
|
||||
StringRef ColumnTuple::serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const
|
||||
{
|
||||
if (columns.empty())
|
||||
{
|
||||
/// Has to put one useless byte into Arena, because serialization into zero number of bytes is ambiguous.
|
||||
char * res = arena.allocContinue(1, begin);
|
||||
*res = 0;
|
||||
return { res, 1 };
|
||||
}
|
||||
|
||||
StringRef res(begin, 0);
|
||||
for (const auto & column : columns)
|
||||
{
|
||||
@ -232,6 +275,11 @@ char * ColumnTuple::serializeValueIntoMemory(size_t n, char * memory) const
|
||||
|
||||
const char * ColumnTuple::deserializeAndInsertFromArena(const char * pos)
|
||||
{
|
||||
++column_length;
|
||||
|
||||
if (columns.empty())
|
||||
return pos + 1;
|
||||
|
||||
for (auto & column : columns)
|
||||
pos = column->deserializeAndInsertFromArena(pos);
|
||||
|
||||
@ -272,6 +320,7 @@ void ColumnTuple::updateHashFast(SipHash & hash) const
|
||||
|
||||
void ColumnTuple::insertRangeFrom(const IColumn & src, size_t start, size_t length)
|
||||
{
|
||||
column_length += length;
|
||||
const size_t tuple_size = columns.size();
|
||||
for (size_t i = 0; i < tuple_size; ++i)
|
||||
columns[i]->insertRangeFrom(
|
||||
@ -281,6 +330,12 @@ void ColumnTuple::insertRangeFrom(const IColumn & src, size_t start, size_t leng
|
||||
|
||||
ColumnPtr ColumnTuple::filter(const Filter & filt, ssize_t result_size_hint) const
|
||||
{
|
||||
if (columns.empty())
|
||||
{
|
||||
size_t bytes = countBytesInFilter(filt);
|
||||
return cloneResized(bytes);
|
||||
}
|
||||
|
||||
const size_t tuple_size = columns.size();
|
||||
Columns new_columns(tuple_size);
|
||||
|
||||
@ -292,12 +347,29 @@ ColumnPtr ColumnTuple::filter(const Filter & filt, ssize_t result_size_hint) con
|
||||
|
||||
void ColumnTuple::expand(const Filter & mask, bool inverted)
|
||||
{
|
||||
if (columns.empty())
|
||||
{
|
||||
size_t bytes = countBytesInFilter(mask);
|
||||
if (inverted)
|
||||
bytes = mask.size() - bytes;
|
||||
column_length = bytes;
|
||||
return;
|
||||
}
|
||||
|
||||
for (auto & column : columns)
|
||||
column->expand(mask, inverted);
|
||||
}
|
||||
|
||||
ColumnPtr ColumnTuple::permute(const Permutation & perm, size_t limit) const
|
||||
{
|
||||
if (columns.empty())
|
||||
{
|
||||
if (column_length != perm.size())
|
||||
throw Exception(ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH, "Size of permutation doesn't match size of column");
|
||||
|
||||
return cloneResized(limit ? std::min(column_length, limit) : column_length);
|
||||
}
|
||||
|
||||
const size_t tuple_size = columns.size();
|
||||
Columns new_columns(tuple_size);
|
||||
|
||||
@ -309,6 +381,14 @@ ColumnPtr ColumnTuple::permute(const Permutation & perm, size_t limit) const
|
||||
|
||||
ColumnPtr ColumnTuple::index(const IColumn & indexes, size_t limit) const
|
||||
{
|
||||
if (columns.empty())
|
||||
{
|
||||
if (indexes.size() < limit)
|
||||
throw Exception(ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH, "Size of indexes is less than required");
|
||||
|
||||
return cloneResized(limit ? limit : column_length);
|
||||
}
|
||||
|
||||
const size_t tuple_size = columns.size();
|
||||
Columns new_columns(tuple_size);
|
||||
|
||||
@ -320,6 +400,14 @@ ColumnPtr ColumnTuple::index(const IColumn & indexes, size_t limit) const
|
||||
|
||||
ColumnPtr ColumnTuple::replicate(const Offsets & offsets) const
|
||||
{
|
||||
if (columns.empty())
|
||||
{
|
||||
if (column_length != offsets.size())
|
||||
throw Exception(ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH, "Size of offsets doesn't match size of column");
|
||||
|
||||
return cloneResized(offsets.back());
|
||||
}
|
||||
|
||||
const size_t tuple_size = columns.size();
|
||||
Columns new_columns(tuple_size);
|
||||
|
||||
@ -331,6 +419,22 @@ ColumnPtr ColumnTuple::replicate(const Offsets & offsets) const
|
||||
|
||||
MutableColumns ColumnTuple::scatter(ColumnIndex num_columns, const Selector & selector) const
|
||||
{
|
||||
if (columns.empty())
|
||||
{
|
||||
if (column_length != selector.size())
|
||||
throw Exception(ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH, "Size of selector doesn't match size of column");
|
||||
|
||||
std::vector<size_t> counts(num_columns);
|
||||
for (auto idx : selector)
|
||||
++counts[idx];
|
||||
|
||||
MutableColumns res(num_columns);
|
||||
for (size_t i = 0; i < num_columns; ++i)
|
||||
res[i] = cloneResized(counts[i]);
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
const size_t tuple_size = columns.size();
|
||||
std::vector<MutableColumns> scattered_tuple_elements(tuple_size);
|
||||
|
||||
@ -413,6 +517,9 @@ void ColumnTuple::getPermutationImpl(IColumn::PermutationSortDirection direction
|
||||
res.resize(rows);
|
||||
iota(res.data(), rows, IColumn::Permutation::value_type(0));
|
||||
|
||||
if (columns.empty())
|
||||
return;
|
||||
|
||||
if (limit >= rows)
|
||||
limit = 0;
|
||||
|
||||
@ -603,6 +710,9 @@ void ColumnTuple::takeDynamicStructureFromSourceColumns(const Columns & source_c
|
||||
|
||||
ColumnPtr ColumnTuple::compress() const
|
||||
{
|
||||
if (columns.empty())
|
||||
return Ptr();
|
||||
|
||||
size_t byte_size = 0;
|
||||
Columns compressed;
|
||||
compressed.reserve(columns.size());
|
||||
|
@ -26,6 +26,13 @@ private:
|
||||
explicit ColumnTuple(MutableColumns && columns);
|
||||
ColumnTuple(const ColumnTuple &) = default;
|
||||
|
||||
/// Empty tuple needs a dedicated field to store its size.
|
||||
/// This field used *only* for zero-sized tuples.
|
||||
/// Otherwise `columns[0].size()` should be used to get a size of tuple column
|
||||
size_t column_length;
|
||||
|
||||
/// Dedicated constructor for empty tuples.
|
||||
explicit ColumnTuple(size_t len);
|
||||
public:
|
||||
/** Create immutable column using immutable arguments. This arguments may be shared with other columns.
|
||||
* Use IColumn::mutate in order to make mutable column and mutate shared nested columns.
|
||||
@ -39,6 +46,8 @@ public:
|
||||
requires std::is_rvalue_reference_v<Arg &&>
|
||||
static MutablePtr create(Arg && arg) { return Base::create(std::forward<Arg>(arg)); }
|
||||
|
||||
static MutablePtr create(size_t len_) { return Base::create(len_); }
|
||||
|
||||
std::string getName() const override;
|
||||
const char * getFamilyName() const override { return "Tuple"; }
|
||||
TypeIndex getDataType() const override { return TypeIndex::Tuple; }
|
||||
@ -46,10 +55,7 @@ public:
|
||||
MutableColumnPtr cloneEmpty() const override;
|
||||
MutableColumnPtr cloneResized(size_t size) const override;
|
||||
|
||||
size_t size() const override
|
||||
{
|
||||
return columns.at(0)->size();
|
||||
}
|
||||
size_t size() const override;
|
||||
|
||||
Field operator[](size_t n) const override;
|
||||
void get(size_t n, Field & res) const override;
|
||||
@ -117,6 +123,9 @@ public:
|
||||
bool hasDynamicStructure() const override;
|
||||
void takeDynamicStructureFromSourceColumns(const Columns & source_columns) override;
|
||||
|
||||
/// Empty tuple needs a public method to manage its size.
|
||||
void addSize(size_t delta) { column_length += delta; }
|
||||
|
||||
private:
|
||||
int compareAtImpl(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint, const Collator * collator=nullptr) const;
|
||||
|
||||
|
@ -75,6 +75,9 @@ ColumnPtr recursiveRemoveLowCardinality(const ColumnPtr & column)
|
||||
else if (const auto * column_tuple = typeid_cast<const ColumnTuple *>(column.get()))
|
||||
{
|
||||
auto columns = column_tuple->getColumns();
|
||||
if (columns.empty())
|
||||
return column;
|
||||
|
||||
for (auto & element : columns)
|
||||
element = recursiveRemoveLowCardinality(element);
|
||||
res = ColumnTuple::create(columns);
|
||||
|
@ -29,7 +29,6 @@ namespace ErrorCodes
|
||||
{
|
||||
extern const int BAD_ARGUMENTS;
|
||||
extern const int DUPLICATE_COLUMN;
|
||||
extern const int EMPTY_DATA_PASSED;
|
||||
extern const int NOT_FOUND_COLUMN_IN_BLOCK;
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
extern const int SIZES_OF_COLUMNS_IN_TUPLE_DOESNT_MATCH;
|
||||
@ -181,6 +180,9 @@ static void addElementSafe(const DataTypes & elems, IColumn & column, F && impl)
|
||||
|
||||
MutableColumnPtr DataTypeTuple::createColumn() const
|
||||
{
|
||||
if (elems.empty())
|
||||
return ColumnTuple::create(0);
|
||||
|
||||
size_t size = elems.size();
|
||||
MutableColumns tuple_columns(size);
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
@ -206,6 +208,9 @@ MutableColumnPtr DataTypeTuple::createColumn(const ISerialization & serializatio
|
||||
if (!serialization_tuple)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected serialization to create column of type Tuple");
|
||||
|
||||
if (elems.empty())
|
||||
return IDataType::createColumn(serialization);
|
||||
|
||||
const auto & element_serializations = serialization_tuple->getElementsSerializations();
|
||||
|
||||
size_t size = elems.size();
|
||||
@ -224,6 +229,12 @@ Field DataTypeTuple::getDefault() const
|
||||
|
||||
void DataTypeTuple::insertDefaultInto(IColumn & column) const
|
||||
{
|
||||
if (elems.empty())
|
||||
{
|
||||
column.insertDefault();
|
||||
return;
|
||||
}
|
||||
|
||||
addElementSafe(elems, column, [&]
|
||||
{
|
||||
for (const auto & i : collections::range(0, elems.size()))
|
||||
@ -388,7 +399,7 @@ void DataTypeTuple::forEachChild(const ChildCallback & callback) const
|
||||
static DataTypePtr create(const ASTPtr & arguments)
|
||||
{
|
||||
if (!arguments || arguments->children.empty())
|
||||
throw Exception(ErrorCodes::EMPTY_DATA_PASSED, "Tuple cannot be empty");
|
||||
return std::make_shared<DataTypeTuple>(DataTypes{});
|
||||
|
||||
DataTypes nested_types;
|
||||
nested_types.reserve(arguments->children.size());
|
||||
|
@ -20,7 +20,6 @@ namespace DB
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int EMPTY_DATA_PASSED;
|
||||
extern const int NOT_IMPLEMENTED;
|
||||
}
|
||||
|
||||
@ -146,9 +145,6 @@ DataTypePtr FieldToDataType<on_error>::operator() (const Array & x) const
|
||||
template <LeastSupertypeOnError on_error>
|
||||
DataTypePtr FieldToDataType<on_error>::operator() (const Tuple & tuple) const
|
||||
{
|
||||
if (tuple.empty())
|
||||
throw Exception(ErrorCodes::EMPTY_DATA_PASSED, "Cannot infer type of an empty tuple");
|
||||
|
||||
DataTypes element_types;
|
||||
element_types.reserve(tuple.size());
|
||||
|
||||
|
@ -229,9 +229,10 @@ static std::pair<ColumnPtr, DataTypePtr> recursivlyConvertDynamicColumnToTuple(
|
||||
= recursivlyConvertDynamicColumnToTuple(tuple_columns[i], tuple_types[i]);
|
||||
}
|
||||
|
||||
auto new_column = tuple_size == 0 ? column : ColumnPtr(ColumnTuple::create(new_tuple_columns));
|
||||
return
|
||||
{
|
||||
ColumnTuple::create(new_tuple_columns),
|
||||
new_column,
|
||||
recreateTupleWithElements(*type_tuple, new_tuple_types)
|
||||
};
|
||||
}
|
||||
|
@ -70,13 +70,15 @@ void SerializationInfoTuple::add(const SerializationInfo & other)
|
||||
|
||||
void SerializationInfoTuple::addDefaults(size_t length)
|
||||
{
|
||||
SerializationInfo::addDefaults(length);
|
||||
|
||||
for (const auto & elem : elems)
|
||||
elem->addDefaults(length);
|
||||
}
|
||||
|
||||
void SerializationInfoTuple::replaceData(const SerializationInfo & other)
|
||||
{
|
||||
SerializationInfo::add(other);
|
||||
SerializationInfo::replaceData(other);
|
||||
|
||||
const auto & other_info = assert_cast<const SerializationInfoTuple &>(other);
|
||||
for (const auto & [name, elem] : name_to_elem)
|
||||
@ -94,7 +96,9 @@ MutableSerializationInfoPtr SerializationInfoTuple::clone() const
|
||||
for (const auto & elem : elems)
|
||||
elems_cloned.push_back(elem->clone());
|
||||
|
||||
return std::make_shared<SerializationInfoTuple>(std::move(elems_cloned), names, settings);
|
||||
auto ret = std::make_shared<SerializationInfoTuple>(std::move(elems_cloned), names, settings);
|
||||
ret->data = data;
|
||||
return ret;
|
||||
}
|
||||
|
||||
MutableSerializationInfoPtr SerializationInfoTuple::createWithType(
|
||||
|
@ -91,6 +91,10 @@ static ReturnType addElementSafe(size_t num_elems, IColumn & column, F && impl)
|
||||
restore_elements();
|
||||
return ReturnType(false);
|
||||
}
|
||||
else
|
||||
{
|
||||
assert_cast<ColumnTuple &>(column).addSize(1);
|
||||
}
|
||||
|
||||
// Check that all columns now have the same size.
|
||||
size_t new_size = column.size();
|
||||
@ -564,6 +568,12 @@ void SerializationTuple::enumerateStreams(
|
||||
const StreamCallback & callback,
|
||||
const SubstreamData & data) const
|
||||
{
|
||||
if (elems.empty())
|
||||
{
|
||||
ISerialization::enumerateStreams(settings, callback, data);
|
||||
return;
|
||||
}
|
||||
|
||||
const auto * type_tuple = data.type ? &assert_cast<const DataTypeTuple &>(*data.type) : nullptr;
|
||||
const auto * column_tuple = data.column ? &assert_cast<const ColumnTuple &>(*data.column) : nullptr;
|
||||
const auto * info_tuple = data.serialization_info ? &assert_cast<const SerializationInfoTuple &>(*data.serialization_info) : nullptr;
|
||||
@ -626,6 +636,22 @@ void SerializationTuple::serializeBinaryBulkWithMultipleStreams(
|
||||
SerializeBinaryBulkSettings & settings,
|
||||
SerializeBinaryBulkStatePtr & state) const
|
||||
{
|
||||
if (elems.empty())
|
||||
{
|
||||
if (WriteBuffer * stream = settings.getter(settings.path))
|
||||
{
|
||||
size_t size = column.size();
|
||||
|
||||
if (limit == 0 || offset + limit > size)
|
||||
limit = size - offset;
|
||||
|
||||
for (size_t i = 0; i < limit; ++i)
|
||||
stream->write('0');
|
||||
}
|
||||
|
||||
return;
|
||||
}
|
||||
|
||||
auto * tuple_state = checkAndGetState<SerializeBinaryBulkStateTuple>(state);
|
||||
|
||||
for (size_t i = 0; i < elems.size(); ++i)
|
||||
@ -642,6 +668,24 @@ void SerializationTuple::deserializeBinaryBulkWithMultipleStreams(
|
||||
DeserializeBinaryBulkStatePtr & state,
|
||||
SubstreamsCache * cache) const
|
||||
{
|
||||
if (elems.empty())
|
||||
{
|
||||
auto cached_column = getFromSubstreamsCache(cache, settings.path);
|
||||
if (cached_column)
|
||||
{
|
||||
column = cached_column;
|
||||
}
|
||||
else if (ReadBuffer * stream = settings.getter(settings.path))
|
||||
{
|
||||
auto mutable_column = column->assumeMutable();
|
||||
typeid_cast<ColumnTuple &>(*mutable_column).addSize(stream->tryIgnore(limit));
|
||||
column = std::move(mutable_column);
|
||||
addToSubstreamsCache(cache, settings.path, column);
|
||||
}
|
||||
|
||||
return;
|
||||
}
|
||||
|
||||
auto * tuple_state = checkAndGetState<DeserializeBinaryBulkStateTuple>(state);
|
||||
|
||||
auto mutable_column = column->assumeMutable();
|
||||
@ -650,6 +694,8 @@ void SerializationTuple::deserializeBinaryBulkWithMultipleStreams(
|
||||
settings.avg_value_size_hint = 0;
|
||||
for (size_t i = 0; i < elems.size(); ++i)
|
||||
elems[i]->deserializeBinaryBulkWithMultipleStreams(column_tuple.getColumnPtr(i), limit, settings, tuple_state->states[i], cache);
|
||||
|
||||
typeid_cast<ColumnTuple &>(*mutable_column).addSize(column_tuple.getColumn(0).size());
|
||||
}
|
||||
|
||||
size_t SerializationTuple::getPositionByName(const String & name) const
|
||||
|
@ -294,7 +294,7 @@ Block NativeReader::read()
|
||||
}
|
||||
|
||||
if (res.rows() != rows)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Row count mismatch after desirialization, got: {}, expected: {}", res.rows(), rows);
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Row count mismatch after deserialization, got: {}, expected: {}", res.rows(), rows);
|
||||
|
||||
return res;
|
||||
}
|
||||
|
@ -10,11 +10,6 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
}
|
||||
|
||||
/** tuple(x, y, ...) is a function that allows you to group several columns
|
||||
* tupleElement(tuple, n) is a function that allows you to retrieve a column from tuple.
|
||||
*/
|
||||
@ -45,14 +40,14 @@ public:
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (arguments.empty())
|
||||
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} requires at least one argument.", getName());
|
||||
|
||||
return std::make_shared<DataTypeTuple>(arguments);
|
||||
}
|
||||
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
|
||||
{
|
||||
if (arguments.empty())
|
||||
return ColumnTuple::create(input_rows_count);
|
||||
|
||||
size_t tuple_size = arguments.size();
|
||||
Columns tuple_columns(tuple_size);
|
||||
for (size_t i = 0; i < tuple_size; ++i)
|
||||
|
@ -12,6 +12,7 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
@ -19,6 +20,36 @@ namespace ErrorCodes
|
||||
extern const int ARGUMENT_OUT_OF_BOUND;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
/// Checks that passed data types are tuples and have the same size.
|
||||
/// Returns size of tuples.
|
||||
size_t checkAndGetTuplesSize(const DataTypePtr & lhs_type, const DataTypePtr & rhs_type, const String & function_name = {})
|
||||
{
|
||||
const auto * left_tuple = checkAndGetDataType<DataTypeTuple>(lhs_type.get());
|
||||
const auto * right_tuple = checkAndGetDataType<DataTypeTuple>(rhs_type.get());
|
||||
|
||||
if (!left_tuple)
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Argument 0{} should be tuple, got {}",
|
||||
function_name.empty() ? "" : fmt::format(" of function {}", function_name), lhs_type->getName());
|
||||
|
||||
if (!right_tuple)
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Argument 1{}should be tuple, got {}",
|
||||
function_name.empty() ? "" : fmt::format(" of function {}", function_name), rhs_type->getName());
|
||||
|
||||
const auto & left_types = left_tuple->getElements();
|
||||
const auto & right_types = right_tuple->getElements();
|
||||
|
||||
if (left_types.size() != right_types.size())
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Expected tuples of the same size as arguments{}, got {} and {}",
|
||||
function_name.empty() ? "" : fmt::format(" of function {}", function_name), lhs_type->getName(), rhs_type->getName());
|
||||
return left_types.size();
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
struct PlusName { static constexpr auto name = "plus"; };
|
||||
struct MinusName { static constexpr auto name = "minus"; };
|
||||
struct MultiplyName { static constexpr auto name = "multiply"; };
|
||||
@ -33,8 +64,7 @@ struct L2SquaredLabel { static constexpr auto name = "2Squared"; };
|
||||
struct LinfLabel { static constexpr auto name = "inf"; };
|
||||
struct LpLabel { static constexpr auto name = "p"; };
|
||||
|
||||
/// str starts from the lowercase letter; not constexpr due to the compiler version
|
||||
/*constexpr*/ std::string makeFirstLetterUppercase(const std::string& str)
|
||||
constexpr std::string makeFirstLetterUppercase(const std::string & str)
|
||||
{
|
||||
std::string res(str);
|
||||
res[0] += 'A' - 'a';
|
||||
@ -57,35 +87,13 @@ public:
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
|
||||
{
|
||||
const auto * left_tuple = checkAndGetDataType<DataTypeTuple>(arguments[0].type.get());
|
||||
const auto * right_tuple = checkAndGetDataType<DataTypeTuple>(arguments[1].type.get());
|
||||
size_t tuple_size = checkAndGetTuplesSize(arguments[0].type, arguments[1].type, getName());
|
||||
|
||||
if (!left_tuple)
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Argument 0 of function {} should be tuple, got {}",
|
||||
getName(), arguments[0].type->getName());
|
||||
const auto & left_types = checkAndGetDataType<DataTypeTuple>(arguments[0].type.get())->getElements();
|
||||
const auto & right_types = checkAndGetDataType<DataTypeTuple>(arguments[1].type.get())->getElements();
|
||||
|
||||
if (!right_tuple)
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Argument 1 of function {} should be tuple, got {}",
|
||||
getName(), arguments[1].type->getName());
|
||||
|
||||
const auto & left_types = left_tuple->getElements();
|
||||
const auto & right_types = right_tuple->getElements();
|
||||
|
||||
Columns left_elements;
|
||||
Columns right_elements;
|
||||
if (arguments[0].column)
|
||||
left_elements = getTupleElements(*arguments[0].column);
|
||||
if (arguments[1].column)
|
||||
right_elements = getTupleElements(*arguments[1].column);
|
||||
|
||||
if (left_types.size() != right_types.size())
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Expected tuples of the same size as arguments of function {}. Got {} and {}",
|
||||
getName(), arguments[0].type->getName(), arguments[1].type->getName());
|
||||
|
||||
size_t tuple_size = left_types.size();
|
||||
if (tuple_size == 0)
|
||||
return std::make_shared<DataTypeUInt8>();
|
||||
Columns left_elements = arguments[0].column ? getTupleElements(*arguments[0].column) : Columns();
|
||||
Columns right_elements = arguments[1].column ? getTupleElements(*arguments[1].column) : Columns();
|
||||
|
||||
auto func = FunctionFactory::instance().get(FuncName::name, context);
|
||||
DataTypes types(tuple_size);
|
||||
@ -119,7 +127,7 @@ public:
|
||||
|
||||
size_t tuple_size = left_elements.size();
|
||||
if (tuple_size == 0)
|
||||
return DataTypeUInt8().createColumnConstWithDefaultValue(input_rows_count);
|
||||
return ColumnTuple::create(input_rows_count);
|
||||
|
||||
auto func = FunctionFactory::instance().get(FuncName::name, context);
|
||||
Columns columns(tuple_size);
|
||||
@ -177,9 +185,6 @@ public:
|
||||
cur_elements = getTupleElements(*arguments[0].column);
|
||||
|
||||
size_t tuple_size = cur_types.size();
|
||||
if (tuple_size == 0)
|
||||
return std::make_shared<DataTypeUInt8>();
|
||||
|
||||
auto negate = FunctionFactory::instance().get("negate", context);
|
||||
DataTypes types(tuple_size);
|
||||
for (size_t i = 0; i < tuple_size; ++i)
|
||||
@ -197,7 +202,7 @@ public:
|
||||
}
|
||||
}
|
||||
|
||||
return std::make_shared<DataTypeTuple>(types);
|
||||
return std::make_shared<DataTypeTuple>(std::move(types));
|
||||
}
|
||||
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
|
||||
@ -208,7 +213,7 @@ public:
|
||||
|
||||
size_t tuple_size = cur_elements.size();
|
||||
if (tuple_size == 0)
|
||||
return DataTypeUInt8().createColumnConstWithDefaultValue(input_rows_count);
|
||||
return ColumnTuple::create(input_rows_count);
|
||||
|
||||
auto negate = FunctionFactory::instance().get("negate", context);
|
||||
Columns columns(tuple_size);
|
||||
@ -248,13 +253,9 @@ public:
|
||||
|
||||
const auto & cur_types = cur_tuple->getElements();
|
||||
|
||||
Columns cur_elements;
|
||||
if (arguments[0].column)
|
||||
cur_elements = getTupleElements(*arguments[0].column);
|
||||
Columns cur_elements = arguments[0].column ? getTupleElements(*arguments[0].column) : Columns();
|
||||
|
||||
size_t tuple_size = cur_types.size();
|
||||
if (tuple_size == 0)
|
||||
return std::make_shared<DataTypeUInt8>();
|
||||
|
||||
const auto & p_column = arguments[1];
|
||||
auto func = FunctionFactory::instance().get(FuncName::name, context);
|
||||
@ -285,7 +286,7 @@ public:
|
||||
|
||||
size_t tuple_size = cur_elements.size();
|
||||
if (tuple_size == 0)
|
||||
return DataTypeUInt8().createColumnConstWithDefaultValue(input_rows_count);
|
||||
return ColumnTuple::create(input_rows_count);
|
||||
|
||||
const auto & p_column = arguments[1];
|
||||
auto func = FunctionFactory::instance().get(FuncName::name, context);
|
||||
@ -583,11 +584,14 @@ public:
|
||||
types = {arguments[0]};
|
||||
}
|
||||
|
||||
const auto * interval_last = checkAndGetDataType<DataTypeInterval>(types.back().get());
|
||||
const auto * interval_new = checkAndGetDataType<DataTypeInterval>(arguments[1].get());
|
||||
if (!types.empty())
|
||||
{
|
||||
const auto * interval_last = checkAndGetDataType<DataTypeInterval>(types.back().get());
|
||||
const auto * interval_new = checkAndGetDataType<DataTypeInterval>(arguments[1].get());
|
||||
|
||||
if (!interval_last->equals(*interval_new))
|
||||
types.push_back(arguments[1]);
|
||||
if (!interval_last->equals(*interval_new))
|
||||
types.push_back(arguments[1]);
|
||||
}
|
||||
|
||||
return std::make_shared<DataTypeTuple>(types);
|
||||
}
|
||||
@ -632,14 +636,10 @@ public:
|
||||
size_t tuple_size = cur_elements.size();
|
||||
|
||||
if (tuple_size == 0)
|
||||
{
|
||||
can_be_merged = false;
|
||||
}
|
||||
else
|
||||
{
|
||||
const auto * tuple_last_interval = checkAndGetDataType<DataTypeInterval>(cur_types.back().get());
|
||||
can_be_merged = tuple_last_interval->equals(*second_interval);
|
||||
}
|
||||
return ColumnTuple::create(input_rows_count);
|
||||
|
||||
const auto * tuple_last_interval = checkAndGetDataType<DataTypeInterval>(cur_types.back().get());
|
||||
can_be_merged = tuple_last_interval->equals(*second_interval);
|
||||
|
||||
if (can_be_merged)
|
||||
tuple_columns.resize(tuple_size);
|
||||
@ -726,9 +726,7 @@ public:
|
||||
|
||||
const auto & cur_types = cur_tuple->getElements();
|
||||
|
||||
Columns cur_elements;
|
||||
if (arguments[0].column)
|
||||
cur_elements = getTupleElements(*arguments[0].column);
|
||||
Columns cur_elements = arguments[0].column ? getTupleElements(*arguments[0].column) : Columns();
|
||||
|
||||
size_t tuple_size = cur_types.size();
|
||||
if (tuple_size == 0)
|
||||
@ -1344,6 +1342,11 @@ public:
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
|
||||
{
|
||||
size_t tuple_size = checkAndGetTuplesSize(arguments[0].type, arguments[1].type, getName());
|
||||
if (tuple_size == 0)
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Result of function {} is undefined for empty tuples", getName());
|
||||
|
||||
FunctionDotProduct dot(context);
|
||||
ColumnWithTypeAndName dot_result{dot.getReturnTypeImpl(arguments), {}};
|
||||
|
||||
|
@ -519,9 +519,10 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPartImpl(
|
||||
/// Size of part would not be greater than block.bytes() + epsilon
|
||||
size_t expected_size = block.bytes();
|
||||
|
||||
/// If optimize_on_insert is true, block may become empty after merge.
|
||||
/// There is no need to create empty part.
|
||||
if (expected_size == 0)
|
||||
/// If optimize_on_insert is true, block may become empty after merge. There
|
||||
/// is no need to create empty part. Since expected_size could be zero when
|
||||
/// part only contains empty tuples. As a result, check rows instead.
|
||||
if (block.rows() == 0)
|
||||
return temp_part;
|
||||
|
||||
DB::IMergeTreeDataPart::TTLInfos move_ttl_infos;
|
||||
|
@ -267,6 +267,9 @@ ColumnPtr fillColumnWithRandomData(
|
||||
case TypeIndex::Tuple:
|
||||
{
|
||||
auto elements = typeid_cast<const DataTypeTuple *>(type.get())->getElements();
|
||||
if (elements.empty())
|
||||
return ColumnTuple::create(limit);
|
||||
|
||||
const size_t tuple_size = elements.size();
|
||||
Columns tuple_columns(tuple_size);
|
||||
|
||||
|
@ -82,7 +82,8 @@ SELECT LpNorm((1, 2), toDecimal32(2, 4)); -- { serverError ILLEGAL_TYPE_OF_ARGUM
|
||||
SELECT (1, 2) * toDecimal32(3.1, 8);
|
||||
|
||||
SELECT cosineDistance((1, 2), (2, 3, 4)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
|
||||
SELECT tuple() + tuple(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH }
|
||||
-- TODO: what's the expected value of () + ()? Currently it returns 0.
|
||||
-- SELECT tuple() + tuple(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH }
|
||||
SELECT LpNorm((1, 2, 3)); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH }
|
||||
SELECT max2(1, 2, -1); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH }
|
||||
|
||||
|
@ -10,6 +10,9 @@ SELECT (toIntervalSecond(-1), toIntervalMinute(2), toIntervalMonth(-3), toInterv
|
||||
(1,2)
|
||||
(1,0)
|
||||
---
|
||||
()
|
||||
()
|
||||
---
|
||||
2022-10-12
|
||||
2022-10-10
|
||||
2022-10-12
|
||||
|
@ -12,6 +12,10 @@ SELECT addTupleOfIntervals('2022-10-11'::Date, (INTERVAL 1 DAY, INTERVAL 1 MONTH
|
||||
SELECT subtractTupleOfIntervals('2022-10-11'::Date, (INTERVAL 1 DAY, INTERVAL 1 MONTH));
|
||||
SELECT addInterval((INTERVAL 1 DAY, INTERVAL 1 SECOND), INTERVAL 1 SECOND);
|
||||
SELECT subtractInterval(tuple(INTERVAL 1 DAY, INTERVAL 1 SECOND), INTERVAL 1 SECOND);
|
||||
SELECT '---';
|
||||
|
||||
SELECT addInterval((), INTERVAL 1 MONTH);
|
||||
SELECT subtractInterval(tuple(), INTERVAL 1 SECOND);
|
||||
|
||||
SELECT '---';
|
||||
|
||||
@ -68,4 +72,4 @@ WITH '2022-10-11'::DateTime64 - INTERVAL 1 YEAR - INTERVAL 4 MONTH - INTERVAL 1
|
||||
'2022-10-11'::DateTime64 + (- INTERVAL 1 YEAR - INTERVAL 4 MONTH - INTERVAL 1 SECOND) AS e2,
|
||||
'2022-10-11'::DateTime64 - (INTERVAL 1 YEAR, INTERVAL 4 MONTH, INTERVAL 1 SECOND) AS e3,
|
||||
'2022-10-11'::DateTime64 - INTERVAL '1 YEAR 4 MONTH 1 SECOND' AS e4
|
||||
SELECT e1 == e2 AND e2 == e3 AND e3 == e4, e1;
|
||||
SELECT e1 == e2 AND e2 == e3 AND e3 == e4, e1;
|
||||
|
7
tests/queries/0_stateless/02891_empty_tuple.reference
Normal file
7
tests/queries/0_stateless/02891_empty_tuple.reference
Normal file
@ -0,0 +1,7 @@
|
||||
2
|
||||
() ()
|
||||
() ()
|
||||
()
|
||||
2
|
||||
() [(),()]
|
||||
() []
|
27
tests/queries/0_stateless/02891_empty_tuple.sql
Normal file
27
tests/queries/0_stateless/02891_empty_tuple.sql
Normal file
@ -0,0 +1,27 @@
|
||||
drop table if exists x;
|
||||
|
||||
create table x engine MergeTree order by () as select () as a, () as b;
|
||||
|
||||
insert into x values ((), ());
|
||||
|
||||
select count() from x;
|
||||
|
||||
select * from x order by ();
|
||||
|
||||
select ();
|
||||
|
||||
drop table x;
|
||||
|
||||
drop table if exists x;
|
||||
|
||||
create table x (i Nullable(Tuple())) engine MergeTree order by (); -- { serverError 43 }
|
||||
create table x (i LowCardinality(Tuple())) engine MergeTree order by (); -- { serverError 43 }
|
||||
create table x (i Tuple(), j Array(Tuple())) engine MergeTree order by ();
|
||||
|
||||
insert into x values ((), [(), ()]), ((), []);
|
||||
|
||||
select count() from x;
|
||||
|
||||
select * from x order by () settings max_threads = 1;
|
||||
|
||||
drop table x;
|
Loading…
Reference in New Issue
Block a user