提交 3a7a6d63 编写于 作者: A Alexey Milovidov

Get rid of DataTypeNested [#CLICKHOUSE-2838].

上级 334e349d
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <IO/WriteBufferFromString.h>
#include <string.h>
#include <Common/typeid_cast.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypeNested.h>
#include <DataTypes/DataTypeFactory.h>
#include <Parsers/IAST.h>
#include <Parsers/ASTNameTypePair.h>
namespace DB
......@@ -17,14 +16,6 @@ namespace DB
namespace ErrorCodes
{
extern const int INVALID_NESTED_NAME;
extern const int EMPTY_DATA_PASSED;
extern const int NESTED_TYPE_TOO_DEEP;
}
DataTypeNested::DataTypeNested(const NamesAndTypesListPtr & nested_)
: nested(nested_)
{
}
......@@ -54,45 +45,27 @@ std::string DataTypeNested::extractNestedColumnName(const std::string & nested_n
}
std::string DataTypeNested::getName() const
{
WriteBufferFromOwnString out;
writeCString("Nested(", out);
for (NamesAndTypesList::const_iterator it = nested->begin(); it != nested->end(); ++it)
{
if (it != nested->begin())
writeCString(", ", out);
writeString(it->name, out);
writeChar(' ', out);
writeString(it->type->getName(), out);
}
writeChar(')', out);
return out.str();
}
bool DataTypeNested::equals(const IDataType & rhs) const
{
return typeid(rhs) == typeid(*this) && getName() == rhs.getName();
}
NamesAndTypesListPtr DataTypeNested::expandNestedColumns(const NamesAndTypesList & names_and_types)
{
NamesAndTypesListPtr columns = std::make_shared<NamesAndTypesList>();
for (NamesAndTypesList::const_iterator it = names_and_types.begin(); it != names_and_types.end(); ++it)
{
if (const DataTypeNested * type_nested = typeid_cast<const DataTypeNested *>(&*it->type))
if (const DataTypeArray * type_arr = typeid_cast<const DataTypeArray *>(it->type.get()))
{
const NamesAndTypesList & nested = *type_nested->getNestedTypesList();
for (NamesAndTypesList::const_iterator jt = nested.begin(); jt != nested.end(); ++jt)
if (const DataTypeTuple * type_tuple = typeid_cast<const DataTypeTuple *>(type_arr->getNestedType().get()))
{
String nested_name = DataTypeNested::concatenateNestedName(it->name, jt->name);
columns->push_back(NameAndTypePair(nested_name, std::make_shared<DataTypeArray>(jt->type)));
const DataTypes & elements = type_tuple->getElements();
const Strings & names = type_tuple->getElementNames();
size_t tuple_size = elements.size();
for (size_t i = 0; i < tuple_size; ++i)
{
String nested_name = DataTypeNested::concatenateNestedName(it->name, names[i]);
columns->push_back(NameAndTypePair(nested_name, std::make_shared<DataTypeArray>(elements[i])));
}
}
else
columns->push_back(*it);
}
else
columns->push_back(*it);
......@@ -101,29 +74,13 @@ NamesAndTypesListPtr DataTypeNested::expandNestedColumns(const NamesAndTypesList
}
static DataTypePtr create(const ASTPtr & arguments)
{
if (!arguments || arguments->children.empty())
throw Exception("Nested structure cannot be empty", ErrorCodes::EMPTY_DATA_PASSED);
NamesAndTypesListPtr columns = std::make_shared<NamesAndTypesList>();
for (const auto & child : arguments->children)
{
const ASTNameTypePair & name_and_type_pair = typeid_cast<const ASTNameTypePair &>(*child);
DataTypePtr type = DataTypeFactory::instance().get(name_and_type_pair.type);
if (typeid_cast<const DataTypeNested *>(type.get()))
throw Exception("Nested inside Nested is not allowed", ErrorCodes::NESTED_TYPE_TOO_DEEP);
columns->emplace_back(name_and_type_pair.name, type);
}
return std::make_shared<DataTypeNested>(columns);
}
void registerDataTypeNested(DataTypeFactory & factory)
{
factory.registerDataType("Nested", create);
/// Nested(...) data type is just a sugar for Array(Tuple(...))
factory.registerDataType("Nested", [&factory](const ASTPtr & arguments)
{
return std::make_shared<DataTypeArray>(factory.get("Tuple", arguments));
});
}
}
#pragma once
#include <DataTypes/IDataTypeDummy.h>
#include <Core/NamesAndTypes.h>
namespace DB
{
/** Stores a set of pairs (name, type) for the nested data structure.
* Used only when creating a table. In all other cases, it is not used, since it is expanded into a set of individual columns with types.
*/
class DataTypeNested final : public IDataTypeDummy
namespace DataTypeNested
{
private:
/// Names and types of nested arrays.
NamesAndTypesListPtr nested;
public:
static constexpr bool is_parametric = true;
DataTypeNested(const NamesAndTypesListPtr & nested_);
std::string getName() const override;
const char * getFamilyName() const override { return "Nested"; }
bool equals(const IDataType & rhs) const override;
bool isParametric() const override { return true; }
const NamesAndTypesListPtr & getNestedTypesList() const { return nested; }
static std::string concatenateNestedName(const std::string & nested_table_name, const std::string & nested_field_name);
std::string concatenateNestedName(const std::string & nested_table_name, const std::string & nested_field_name);
/// Returns the prefix of the name to the first '.'. Or the name is unchanged if there is no dot.
static std::string extractNestedTableName(const std::string & nested_name);
std::string extractNestedTableName(const std::string & nested_name);
/// Returns the name suffix after the first dot on the right '.'. Or the name is unchanged if there is no dot.
static std::string extractNestedColumnName(const std::string & nested_name);
std::string extractNestedColumnName(const std::string & nested_name);
/// Creates a new list in which Nested-type columns are replaced by several columns form of `column_name.cell_name`
static NamesAndTypesListPtr expandNestedColumns(const NamesAndTypesList & names_and_types);
NamesAndTypesListPtr expandNestedColumns(const NamesAndTypesList & names_and_types);
};
}
#include <Common/StringUtils.h>
#include <Columns/ColumnTuple.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypeFactory.h>
#include <Parsers/IAST.h>
#include <Parsers/ASTNameTypePair.h>
#include <Common/typeid_cast.h>
#include <IO/WriteHelpers.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteBufferFromString.h>
#include <IO/Operators.h>
#include <ext/map.h>
#include <ext/enumerate.h>
......@@ -16,16 +21,62 @@ namespace DB
namespace ErrorCodes
{
extern const int EMPTY_DATA_PASSED;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int DUPLICATE_COLUMN;
extern const int BAD_ARGUMENTS;
}
DataTypeTuple::DataTypeTuple(const DataTypes & elems_)
: elems(elems_), have_explicit_names(false)
{
/// Automatically assigned names in form of '1', '2', ...
size_t size = elems.size();
names.resize(size);
for (size_t i = 0; i < size; ++i)
names[i] = toString(i + 1);
}
DataTypeTuple::DataTypeTuple(const DataTypes & elems_, const Strings & names_)
: elems(elems_), names(names_), have_explicit_names(true)
{
size_t size = elems.size();
if (names.size() != size)
throw Exception("Wrong number of names passed to constructor of DataTypeTuple", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
std::unordered_set<String> names_set;
for (size_t i = 0; i < size; ++i)
{
if (names[i].empty())
throw Exception("Names of tuple elements cannot be empty", ErrorCodes::BAD_ARGUMENTS);
if (isNumericASCII(names[i][0]))
throw Exception("Explicitly specified names of tuple elements cannot start with digit", ErrorCodes::BAD_ARGUMENTS);
if (!names_set.insert(names[i]).second)
throw Exception("Names of tuple elements must be unique", ErrorCodes::DUPLICATE_COLUMN);
}
}
std::string DataTypeTuple::getName() const
{
std::stringstream s;
size_t size = elems.size();
WriteBufferFromOwnString s;
s << "Tuple(";
for (DataTypes::const_iterator it = elems.begin(); it != elems.end(); ++it)
s << (it == elems.begin() ? "" : ", ") << (*it)->getName();
for (size_t i = 0; i < size; ++i)
{
if (i != 0)
s << ", ";
if (have_explicit_names)
s << backQuoteIfNeed(names[i]) << ' ';
s << elems[i]->getName();
}
s << ")";
return s.str();
......@@ -234,7 +285,7 @@ void DataTypeTuple::enumerateStreams(StreamCallback callback, SubstreamPath path
path.push_back(Substream::TupleElement);
for (const auto i : ext::range(0, ext::size(elems)))
{
path.back().tuple_element = i + 1;
path.back().tuple_element_name = names[i];
elems[i]->enumerateStreams(callback, path);
}
}
......@@ -250,7 +301,7 @@ void DataTypeTuple::serializeBinaryBulkWithMultipleStreams(
path.push_back(Substream::TupleElement);
for (const auto i : ext::range(0, ext::size(elems)))
{
path.back().tuple_element = i + 1;
path.back().tuple_element_name = names[i];
elems[i]->serializeBinaryBulkWithMultipleStreams(
extractElementColumn(column, i), getter, offset, limit, position_independent_encoding, path);
}
......@@ -267,7 +318,7 @@ void DataTypeTuple::deserializeBinaryBulkWithMultipleStreams(
path.push_back(Substream::TupleElement);
for (const auto i : ext::range(0, ext::size(elems)))
{
path.back().tuple_element = i + 1;
path.back().tuple_element_name = names[i];
elems[i]->deserializeBinaryBulkWithMultipleStreams(
extractElementColumn(column, i), getter, limit, avg_value_size_hint, position_independent_encoding, path);
}
......@@ -355,10 +406,26 @@ static DataTypePtr create(const ASTPtr & arguments)
DataTypes nested_types;
nested_types.reserve(arguments->children.size());
Strings names;
names.reserve(arguments->children.size());
for (const ASTPtr & child : arguments->children)
nested_types.emplace_back(DataTypeFactory::instance().get(child));
{
if (const ASTNameTypePair * name_and_type_pair = typeid_cast<const ASTNameTypePair *>(child.get()))
{
nested_types.emplace_back(DataTypeFactory::instance().get(name_and_type_pair->type));
names.emplace_back(name_and_type_pair->name);
}
else
nested_types.emplace_back(DataTypeFactory::instance().get(child));
}
return std::make_shared<DataTypeTuple>(nested_types);
if (names.empty())
return std::make_shared<DataTypeTuple>(nested_types);
else if (names.size() != nested_types.size())
throw Exception("Names are specified not for all elements of Tuple type", ErrorCodes::BAD_ARGUMENTS);
else
return std::make_shared<DataTypeTuple>(nested_types, names);
}
......
......@@ -9,15 +9,24 @@ namespace DB
/** Tuple data type.
* Used as an intermediate result when evaluating expressions.
* Also can be used as a column - the result of the query execution.
* Can not be saved to tables.
*
* Tuple elements can have names.
* If an element is unnamed, it will have automatically assigned name like '1', '2', '3' corresponding to its position.
* Manually assigned names must not begin with digit. Names must be unique.
*
* All tuples with same size and types of elements are equivalent for expressions, regardless to names of elements.
*/
class DataTypeTuple final : public IDataType
{
private:
DataTypes elems;
Strings names;
bool have_explicit_names;
public:
static constexpr bool is_parametric = true;
DataTypeTuple(const DataTypes & elems_) : elems(elems_) {}
DataTypeTuple(const DataTypes & elems);
DataTypeTuple(const DataTypes & elems, const Strings & names);
std::string getName() const override;
const char * getFamilyName() const override { return "Tuple"; }
......@@ -78,6 +87,7 @@ public:
size_t getSizeOfValueInMemory() const override;
const DataTypes & getElements() const { return elems; }
const Strings & getElementNames() const { return names; }
};
}
......
......@@ -85,7 +85,7 @@ String IDataType::getFileNameForStream(const String & column_name, const IDataTy
else if (elem.type == Substream::ArrayElements)
++array_level;
else if (elem.type == Substream::TupleElement)
stream_name += "." + toString(elem.tuple_element);
stream_name += "." + escapeForFileName(elem.tuple_element_name);
}
return stream_name;
}
......
......@@ -82,7 +82,7 @@ public:
Type type;
/// Index of tuple element, starting at 1.
size_t tuple_element = 0;
String tuple_element_name;
Substream(Type type) : type(type) {}
};
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册