2017-12-23 04:29:34 +00:00
|
|
|
#include <string.h>
|
|
|
|
|
2017-07-13 16:49:09 +00:00
|
|
|
#include <Common/typeid_cast.h>
|
2019-08-21 02:28:04 +00:00
|
|
|
#include <Common/assert_cast.h>
|
2018-01-15 19:07:47 +00:00
|
|
|
#include <Common/StringUtils/StringUtils.h>
|
2013-07-11 17:35:56 +00:00
|
|
|
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <DataTypes/DataTypeArray.h>
|
2017-12-23 04:29:34 +00:00
|
|
|
#include <DataTypes/DataTypeTuple.h>
|
2017-12-25 18:58:39 +00:00
|
|
|
#include <DataTypes/NestedUtils.h>
|
2020-11-10 17:32:00 +00:00
|
|
|
#include <DataTypes/DataTypeNested.h>
|
2018-01-02 09:11:13 +00:00
|
|
|
|
|
|
|
#include <Columns/ColumnArray.h>
|
|
|
|
#include <Columns/ColumnTuple.h>
|
|
|
|
#include <Columns/ColumnConst.h>
|
2017-07-16 04:08:20 +00:00
|
|
|
|
|
|
|
#include <Parsers/IAST.h>
|
2013-08-14 16:52:40 +00:00
|
|
|
|
2013-07-11 17:35:56 +00:00
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
2018-09-12 21:08:55 +00:00
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
|
|
|
extern const int ILLEGAL_COLUMN;
|
|
|
|
extern const int SIZES_OF_ARRAYS_DOESNT_MATCH;
|
|
|
|
}
|
|
|
|
|
2017-12-25 18:58:39 +00:00
|
|
|
namespace Nested
|
|
|
|
{
|
2013-07-11 17:35:56 +00:00
|
|
|
|
2017-12-25 18:58:39 +00:00
|
|
|
std::string concatenateName(const std::string & nested_table_name, const std::string & nested_field_name)
|
2013-07-16 14:55:01 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
return nested_table_name + "." + nested_field_name;
|
2013-07-16 14:55:01 +00:00
|
|
|
}
|
|
|
|
|
2018-01-02 05:54:34 +00:00
|
|
|
|
2021-07-03 00:13:21 +00:00
|
|
|
/** Name can be treated as compound if it contains dot (.) in the middle.
|
2021-06-03 13:11:36 +00:00
|
|
|
*/
|
|
|
|
std::pair<std::string, std::string> splitName(const std::string & name, bool reverse)
|
|
|
|
{
|
2021-07-07 14:05:11 +00:00
|
|
|
auto idx = (reverse ? name.find_last_of('.') : name.find_first_of('.'));
|
2021-07-03 00:13:21 +00:00
|
|
|
if (idx == std::string::npos || idx == 0 || idx + 1 == name.size())
|
2018-01-02 05:54:34 +00:00
|
|
|
return {name, {}};
|
|
|
|
|
2021-07-03 00:13:21 +00:00
|
|
|
return {name.substr(0, idx), name.substr(idx + 1)};
|
2013-07-16 14:55:01 +00:00
|
|
|
}
|
|
|
|
|
2021-12-14 15:58:17 +00:00
|
|
|
std::pair<std::string_view, std::string_view> splitName(const std::string_view & name, bool reverse)
|
2021-12-09 11:35:22 +00:00
|
|
|
{
|
2021-12-14 15:58:17 +00:00
|
|
|
auto idx = (reverse ? name.find_last_of('.') : name.find_first_of('.'));
|
2021-12-09 11:35:22 +00:00
|
|
|
if (idx == std::string::npos || idx == 0 || idx + 1 == name.size())
|
|
|
|
return {name, {}};
|
|
|
|
|
|
|
|
return {name.substr(0, idx), name.substr(idx + 1)};
|
|
|
|
}
|
|
|
|
|
2013-07-16 14:55:01 +00:00
|
|
|
|
2018-01-02 05:54:34 +00:00
|
|
|
std::string extractTableName(const std::string & nested_name)
|
2013-07-30 17:25:46 +00:00
|
|
|
{
|
2020-08-08 00:47:03 +00:00
|
|
|
auto split = splitName(nested_name);
|
|
|
|
return split.first;
|
2013-07-30 17:25:46 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2018-01-02 09:11:13 +00:00
|
|
|
Block flatten(const Block & block)
|
|
|
|
{
|
|
|
|
Block res;
|
|
|
|
|
|
|
|
for (const auto & elem : block)
|
|
|
|
{
|
2020-11-10 17:32:00 +00:00
|
|
|
const DataTypeArray * type_arr = typeid_cast<const DataTypeArray *>(elem.type.get());
|
2020-12-07 19:02:26 +00:00
|
|
|
if (type_arr)
|
2018-01-02 09:11:13 +00:00
|
|
|
{
|
2020-04-20 23:44:51 +00:00
|
|
|
const DataTypeTuple * type_tuple = typeid_cast<const DataTypeTuple *>(type_arr->getNestedType().get());
|
|
|
|
if (type_tuple && type_tuple->haveExplicitNames())
|
2018-01-02 09:11:13 +00:00
|
|
|
{
|
|
|
|
const DataTypes & element_types = type_tuple->getElements();
|
|
|
|
const Strings & names = type_tuple->getElementNames();
|
|
|
|
size_t tuple_size = element_types.size();
|
|
|
|
|
2019-06-27 19:28:52 +00:00
|
|
|
bool is_const = isColumnConst(*elem.column);
|
2018-01-02 09:11:13 +00:00
|
|
|
const ColumnArray * column_array;
|
|
|
|
if (is_const)
|
2019-08-21 02:28:04 +00:00
|
|
|
column_array = typeid_cast<const ColumnArray *>(&assert_cast<const ColumnConst &>(*elem.column).getDataColumn());
|
2018-01-02 09:11:13 +00:00
|
|
|
else
|
|
|
|
column_array = typeid_cast<const ColumnArray *>(elem.column.get());
|
|
|
|
|
|
|
|
const ColumnPtr & column_offsets = column_array->getOffsetsPtr();
|
|
|
|
|
|
|
|
const ColumnTuple & column_tuple = typeid_cast<const ColumnTuple &>(column_array->getData());
|
2019-03-25 01:43:54 +00:00
|
|
|
const auto & element_columns = column_tuple.getColumns();
|
2018-01-02 09:11:13 +00:00
|
|
|
|
|
|
|
for (size_t i = 0; i < tuple_size; ++i)
|
|
|
|
{
|
|
|
|
String nested_name = concatenateName(elem.name, names[i]);
|
|
|
|
ColumnPtr column_array_of_element = ColumnArray::create(element_columns[i], column_offsets);
|
|
|
|
|
|
|
|
res.insert(ColumnWithTypeAndName(
|
|
|
|
is_const
|
|
|
|
? ColumnConst::create(std::move(column_array_of_element), block.rows())
|
|
|
|
: std::move(column_array_of_element),
|
|
|
|
std::make_shared<DataTypeArray>(element_types[i]),
|
|
|
|
nested_name));
|
|
|
|
}
|
|
|
|
}
|
|
|
|
else
|
|
|
|
res.insert(elem);
|
|
|
|
}
|
|
|
|
else
|
|
|
|
res.insert(elem);
|
|
|
|
}
|
|
|
|
|
|
|
|
return res;
|
|
|
|
}
|
|
|
|
|
2020-12-07 19:02:26 +00:00
|
|
|
namespace
|
2017-12-25 21:10:46 +00:00
|
|
|
{
|
2018-01-02 05:54:34 +00:00
|
|
|
|
2020-12-07 19:02:26 +00:00
|
|
|
using NameToDataType = std::map<String, DataTypePtr>;
|
|
|
|
|
|
|
|
NameToDataType getSubcolumnsOfNested(const NamesAndTypesList & names_and_types)
|
|
|
|
{
|
|
|
|
std::unordered_map<String, NamesAndTypesList> nested;
|
2018-01-02 05:54:34 +00:00
|
|
|
for (const auto & name_type : names_and_types)
|
|
|
|
{
|
2020-11-10 17:32:00 +00:00
|
|
|
const DataTypeArray * type_arr = typeid_cast<const DataTypeArray *>(name_type.type.get());
|
|
|
|
|
|
|
|
/// Ignore true Nested type, but try to unite flatten arrays to Nested type.
|
|
|
|
if (!isNested(name_type.type) && type_arr)
|
2018-01-02 05:54:34 +00:00
|
|
|
{
|
2020-08-08 00:47:03 +00:00
|
|
|
auto split = splitName(name_type.name);
|
|
|
|
if (!split.second.empty())
|
|
|
|
nested[split.first].emplace_back(split.second, type_arr->getNestedType());
|
2018-01-02 05:54:34 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-12-07 19:02:26 +00:00
|
|
|
std::map<String, DataTypePtr> nested_types;
|
|
|
|
|
2020-11-10 17:32:00 +00:00
|
|
|
for (const auto & [name, elems] : nested)
|
|
|
|
nested_types.emplace(name, createNested(elems.getTypes(), elems.getNames()));
|
|
|
|
|
2020-12-07 19:02:26 +00:00
|
|
|
return nested_types;
|
|
|
|
}
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
|
|
NamesAndTypesList collect(const NamesAndTypesList & names_and_types)
|
|
|
|
{
|
|
|
|
NamesAndTypesList res;
|
|
|
|
auto nested_types = getSubcolumnsOfNested(names_and_types);
|
|
|
|
|
|
|
|
for (const auto & name_type : names_and_types)
|
2021-09-02 17:22:01 +00:00
|
|
|
if (!isArray(name_type.type) || !nested_types.count(splitName(name_type.name).first))
|
2020-12-07 19:02:26 +00:00
|
|
|
res.push_back(name_type);
|
|
|
|
|
|
|
|
for (const auto & name_type : nested_types)
|
|
|
|
res.emplace_back(name_type.first, name_type.second);
|
|
|
|
|
|
|
|
return res;
|
|
|
|
}
|
|
|
|
|
|
|
|
NamesAndTypesList convertToSubcolumns(const NamesAndTypesList & names_and_types)
|
|
|
|
{
|
|
|
|
auto nested_types = getSubcolumnsOfNested(names_and_types);
|
|
|
|
auto res = names_and_types;
|
|
|
|
|
2020-11-10 17:32:00 +00:00
|
|
|
for (auto & name_type : res)
|
|
|
|
{
|
2021-09-02 17:22:01 +00:00
|
|
|
if (!isArray(name_type.type))
|
|
|
|
continue;
|
|
|
|
|
2020-11-10 17:32:00 +00:00
|
|
|
auto split = splitName(name_type.name);
|
|
|
|
if (name_type.isSubcolumn() || split.second.empty())
|
|
|
|
continue;
|
|
|
|
|
|
|
|
auto it = nested_types.find(split.first);
|
|
|
|
if (it != nested_types.end())
|
|
|
|
name_type = NameAndTypePair{split.first, split.second, it->second, it->second->getSubcolumnType(split.second)};
|
|
|
|
}
|
2018-01-02 05:54:34 +00:00
|
|
|
|
|
|
|
return res;
|
2013-08-14 16:52:40 +00:00
|
|
|
}
|
2015-10-04 03:17:36 +00:00
|
|
|
|
2018-09-12 21:08:55 +00:00
|
|
|
|
|
|
|
void validateArraySizes(const Block & block)
|
|
|
|
{
|
|
|
|
/// Nested prefix -> position of first column in block.
|
|
|
|
std::map<std::string, size_t> nested;
|
|
|
|
|
|
|
|
for (size_t i = 0, size = block.columns(); i < size; ++i)
|
|
|
|
{
|
|
|
|
const auto & elem = block.getByPosition(i);
|
|
|
|
|
2018-09-12 22:44:42 +00:00
|
|
|
if (isArray(elem.type))
|
2018-09-12 21:08:55 +00:00
|
|
|
{
|
|
|
|
if (!typeid_cast<const ColumnArray *>(elem.column.get()))
|
|
|
|
throw Exception("Column with Array type is not represented by ColumnArray column: " + elem.column->dumpStructure(), ErrorCodes::ILLEGAL_COLUMN);
|
|
|
|
|
2020-08-08 00:47:03 +00:00
|
|
|
auto split = splitName(elem.name);
|
2018-09-12 21:08:55 +00:00
|
|
|
|
|
|
|
/// Is it really a column of Nested data structure.
|
2020-08-08 00:47:03 +00:00
|
|
|
if (!split.second.empty())
|
2018-09-12 21:08:55 +00:00
|
|
|
{
|
2020-08-08 00:47:03 +00:00
|
|
|
auto [it, inserted] = nested.emplace(split.first, i);
|
2018-09-12 21:08:55 +00:00
|
|
|
|
|
|
|
/// It's not the first column of Nested data structure.
|
|
|
|
if (!inserted)
|
|
|
|
{
|
2019-08-21 02:28:04 +00:00
|
|
|
const ColumnArray & first_array_column = assert_cast<const ColumnArray &>(*block.getByPosition(it->second).column);
|
|
|
|
const ColumnArray & another_array_column = assert_cast<const ColumnArray &>(*elem.column);
|
2018-09-12 21:08:55 +00:00
|
|
|
|
|
|
|
if (!first_array_column.hasEqualOffsets(another_array_column))
|
|
|
|
throw Exception("Elements '" + block.getByPosition(it->second).name
|
|
|
|
+ "' and '" + elem.name
|
2020-08-08 00:47:03 +00:00
|
|
|
+ "' of Nested data structure '" + split.first
|
2018-09-12 21:08:55 +00:00
|
|
|
+ "' (Array columns) have different array sizes.", ErrorCodes::SIZES_OF_ARRAYS_DOESNT_MATCH);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-12-09 11:35:22 +00:00
|
|
|
|
2021-07-01 17:59:28 +00:00
|
|
|
std::unordered_set<String> getAllTableNames(const Block & block)
|
|
|
|
{
|
|
|
|
std::unordered_set<String> nested_table_names;
|
|
|
|
for (auto & name : block.getNames())
|
|
|
|
{
|
|
|
|
auto nested_table_name = Nested::extractTableName(name);
|
|
|
|
if (!nested_table_name.empty())
|
|
|
|
nested_table_names.insert(nested_table_name);
|
|
|
|
}
|
|
|
|
return nested_table_names;
|
|
|
|
}
|
|
|
|
|
2017-07-16 04:08:20 +00:00
|
|
|
}
|
|
|
|
|
2013-07-11 17:35:56 +00:00
|
|
|
}
|