Merge pull request #35682 from CurtizJ/dynamic-columns-6

This commit is contained in:
Vladimir C 2022-04-06 11:48:07 +02:00 committed by GitHub
commit 2ebae2d722
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
8 changed files with 31 additions and 43 deletions

View File

@ -36,7 +36,7 @@ DataTypePtr recursiveRemoveLowCardinality(const DataTypePtr & type)
element = recursiveRemoveLowCardinality(element); element = recursiveRemoveLowCardinality(element);
if (tuple_type->haveExplicitNames()) if (tuple_type->haveExplicitNames())
return std::make_shared<DataTypeTuple>(elements, tuple_type->getElementNames(), tuple_type->serializeNames()); return std::make_shared<DataTypeTuple>(elements, tuple_type->getElementNames());
else else
return std::make_shared<DataTypeTuple>(elements); return std::make_shared<DataTypeTuple>(elements);
} }

View File

@ -64,8 +64,8 @@ static std::optional<Exception> checkTupleNames(const Strings & names)
return {}; return {};
} }
DataTypeTuple::DataTypeTuple(const DataTypes & elems_, const Strings & names_, bool serialize_names_) DataTypeTuple::DataTypeTuple(const DataTypes & elems_, const Strings & names_)
: elems(elems_), names(names_), have_explicit_names(true), serialize_names(serialize_names_) : elems(elems_), names(names_), have_explicit_names(true)
{ {
size_t size = elems.size(); size_t size = elems.size();
if (names.size() != size) if (names.size() != size)
@ -75,11 +75,6 @@ DataTypeTuple::DataTypeTuple(const DataTypes & elems_, const Strings & names_, b
throw std::move(*exception); throw std::move(*exception);
} }
bool DataTypeTuple::canBeCreatedWithNames(const Strings & names)
{
return checkTupleNames(names) == std::nullopt;
}
std::string DataTypeTuple::doGetName() const std::string DataTypeTuple::doGetName() const
{ {
size_t size = elems.size(); size_t size = elems.size();
@ -91,7 +86,7 @@ std::string DataTypeTuple::doGetName() const
if (i != 0) if (i != 0)
s << ", "; s << ", ";
if (have_explicit_names && serialize_names) if (have_explicit_names)
s << backQuoteIfNeed(names[i]) << ' '; s << backQuoteIfNeed(names[i]) << ' ';
s << elems[i]->getName(); s << elems[i]->getName();
@ -206,7 +201,7 @@ bool DataTypeTuple::equals(const IDataType & rhs) const
return false; return false;
for (size_t i = 0; i < size; ++i) for (size_t i = 0; i < size; ++i)
if (!elems[i]->equals(*rhs_tuple.elems[i])) if (!elems[i]->equals(*rhs_tuple.elems[i]) || names[i] != rhs_tuple.names[i])
return false; return false;
return true; return true;
@ -265,31 +260,29 @@ size_t DataTypeTuple::getSizeOfValueInMemory() const
SerializationPtr DataTypeTuple::doGetDefaultSerialization() const SerializationPtr DataTypeTuple::doGetDefaultSerialization() const
{ {
SerializationTuple::ElementSerializations serializations(elems.size()); SerializationTuple::ElementSerializations serializations(elems.size());
bool use_explicit_names = have_explicit_names && serialize_names;
for (size_t i = 0; i < elems.size(); ++i) for (size_t i = 0; i < elems.size(); ++i)
{ {
String elem_name = use_explicit_names ? names[i] : toString(i + 1); String elem_name = have_explicit_names ? names[i] : toString(i + 1);
auto serialization = elems[i]->getDefaultSerialization(); auto serialization = elems[i]->getDefaultSerialization();
serializations[i] = std::make_shared<SerializationNamed>(serialization, elem_name); serializations[i] = std::make_shared<SerializationNamed>(serialization, elem_name);
} }
return std::make_shared<SerializationTuple>(std::move(serializations), use_explicit_names); return std::make_shared<SerializationTuple>(std::move(serializations), have_explicit_names);
} }
SerializationPtr DataTypeTuple::getSerialization(const SerializationInfo & info) const SerializationPtr DataTypeTuple::getSerialization(const SerializationInfo & info) const
{ {
SerializationTuple::ElementSerializations serializations(elems.size()); SerializationTuple::ElementSerializations serializations(elems.size());
const auto & info_tuple = assert_cast<const SerializationInfoTuple &>(info); const auto & info_tuple = assert_cast<const SerializationInfoTuple &>(info);
bool use_explicit_names = have_explicit_names && serialize_names;
for (size_t i = 0; i < elems.size(); ++i) for (size_t i = 0; i < elems.size(); ++i)
{ {
String elem_name = use_explicit_names ? names[i] : toString(i + 1); String elem_name = have_explicit_names ? names[i] : toString(i + 1);
auto serialization = elems[i]->getSerialization(*info_tuple.getElementInfo(i)); auto serialization = elems[i]->getSerialization(*info_tuple.getElementInfo(i));
serializations[i] = std::make_shared<SerializationNamed>(serialization, elem_name); serializations[i] = std::make_shared<SerializationNamed>(serialization, elem_name);
} }
return std::make_shared<SerializationTuple>(std::move(serializations), use_explicit_names); return std::make_shared<SerializationTuple>(std::move(serializations), have_explicit_names);
} }
MutableSerializationInfoPtr DataTypeTuple::createSerializationInfo(const SerializationInfo::Settings & settings) const MutableSerializationInfoPtr DataTypeTuple::createSerializationInfo(const SerializationInfo::Settings & settings) const

View File

@ -22,14 +22,11 @@ private:
DataTypes elems; DataTypes elems;
Strings names; Strings names;
bool have_explicit_names; bool have_explicit_names;
bool serialize_names = true;
public: public:
static constexpr bool is_parametric = true; static constexpr bool is_parametric = true;
explicit DataTypeTuple(const DataTypes & elems); explicit DataTypeTuple(const DataTypes & elems);
DataTypeTuple(const DataTypes & elems, const Strings & names, bool serialize_names_ = true); DataTypeTuple(const DataTypes & elems, const Strings & names);
static bool canBeCreatedWithNames(const Strings & names);
TypeIndex getTypeId() const override { return TypeIndex::Tuple; } TypeIndex getTypeId() const override { return TypeIndex::Tuple; }
std::string doGetName() const override; std::string doGetName() const override;
@ -66,7 +63,6 @@ public:
String getNameByPosition(size_t i) const; String getNameByPosition(size_t i) const;
bool haveExplicitNames() const { return have_explicit_names; } bool haveExplicitNames() const { return have_explicit_names; }
bool serializeNames() const { return serialize_names; }
}; };
} }

View File

@ -2958,8 +2958,7 @@ private:
/// For named tuples allow conversions for tuples with /// For named tuples allow conversions for tuples with
/// different sets of elements. If element exists in @to_type /// different sets of elements. If element exists in @to_type
/// and doesn't exist in @to_type it will be filled by default values. /// and doesn't exist in @to_type it will be filled by default values.
if (from_type->haveExplicitNames() && from_type->serializeNames() if (from_type->haveExplicitNames() && to_type->haveExplicitNames())
&& to_type->haveExplicitNames() && to_type->serializeNames())
{ {
const auto & from_names = from_type->getElementNames(); const auto & from_names = from_type->getElementNames();
std::unordered_map<String, size_t> from_positions; std::unordered_map<String, size_t> from_positions;

View File

@ -54,29 +54,12 @@ public:
bool useDefaultImplementationForNulls() const override { return false; } bool useDefaultImplementationForNulls() const override { return false; }
bool useDefaultImplementationForConstants() const override { return true; } bool useDefaultImplementationForConstants() const override { return true; }
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{ {
if (arguments.empty()) if (arguments.empty())
throw Exception("Function " + getName() + " requires at least one argument.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); throw Exception("Function " + getName() + " requires at least one argument.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
DataTypes types; return std::make_shared<DataTypeTuple>(arguments);
Strings names;
for (const auto & argument : arguments)
{
types.emplace_back(argument.type);
names.emplace_back(argument.name);
}
/// Create named tuple if possible. We don't print tuple element names
/// because they are bad anyway -- aliases are not used, e.g. tuple(1 a)
/// will have element name '1' and not 'a'. If we ever change this, and
/// add the ability to access tuple elements by name, like tuple(1 a).a,
/// we should probably enable printing for better discoverability.
if (DataTypeTuple::canBeCreatedWithNames(names))
return std::make_shared<DataTypeTuple>(types, names, false /*print names*/);
return std::make_shared<DataTypeTuple>(types);
} }
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

View File

@ -0,0 +1 @@
Tuple(foo Int8, k1 Int8, k2 Int8)

View File

@ -0,0 +1,16 @@
-- Tags: no-fasttest
DROP TABLE IF EXISTS t_json;
SET allow_experimental_object_type = 1;
CREATE TABLE t_json(id UInt64, obj JSON) ENGINE = MergeTree ORDER BY id;
INSERT INTO t_json format JSONEachRow {"id": 1, "obj": {"foo": 1, "k1": 2}};
INSERT INTO t_json format JSONEachRow {"id": 2, "obj": {"foo": 1, "k2": 2}};
OPTIMIZE TABLE t_json FINAL;
SELECT any(toTypeName(obj)) from t_json;
DROP TABLE IF EXISTS t_json;

View File

@ -4,7 +4,7 @@ DROP TABLE IF EXISTS test02008;
CREATE TABLE test02008 ( CREATE TABLE test02008 (
col Tuple( col Tuple(
a Tuple(key1 int, key2 int), a Tuple(key1 int, key2 int),
b Tuple(key1 int, key3 int) b Tuple(key1 int, key2 int)
) )
) ENGINE=Memory(); ) ENGINE=Memory();
INSERT INTO test02008 VALUES (tuple(tuple(1, 2), tuple(3, 4))); INSERT INTO test02008 VALUES (tuple(tuple(1, 2), tuple(3, 4)));