mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-11 17:02:25 +00:00
Merge pull request #35682 from CurtizJ/dynamic-columns-6
This commit is contained in:
commit
2ebae2d722
@ -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);
|
||||||
}
|
}
|
||||||
|
@ -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
|
||||||
|
@ -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; }
|
|
||||||
};
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -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;
|
||||||
|
@ -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
|
||||||
|
1
tests/queries/0_stateless/01825_type_json_9.reference
Normal file
1
tests/queries/0_stateless/01825_type_json_9.reference
Normal file
@ -0,0 +1 @@
|
|||||||
|
Tuple(foo Int8, k1 Int8, k2 Int8)
|
16
tests/queries/0_stateless/01825_type_json_9.sql
Normal file
16
tests/queries/0_stateless/01825_type_json_9.sql
Normal 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;
|
@ -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)));
|
||||||
|
Loading…
Reference in New Issue
Block a user