提交 bcfde7c6 编写于 作者: A Alexey Milovidov

Get rid of DataTypeNested, part 2 [#CLICKHOUSE-2].

上级 8bc30c8b
Subproject commit 3a986afbb977fa13582991ce8f2c0b2045ffaa33
Subproject commit 3401fa1e45605b5ae806f94905c92f5f546a607b
Subproject commit f3a8bd553a865c59f1bd6e1f68bf182cf75a8f00
Subproject commit f4340f46b2387bc8de7d5320c0b83bb1499933ad
#include <DataStreams/AddingDefaultBlockOutputStream.h>
#include <Common/typeid_cast.h>
#include <DataTypes/DataTypeNested.h>
#include <DataTypes/NestedUtils.h>
#include <DataTypes/DataTypeArray.h>
#include <Columns/ColumnArray.h>
#include <Core/Block.h>
......@@ -36,7 +36,7 @@ void AddingDefaultBlockOutputStream::write(const DB::Block & block)
if (const ColumnArray * array = typeid_cast<const ColumnArray *>(&*elem.column))
{
String offsets_name = DataTypeNested::extractNestedTableName(elem.name);
String offsets_name = Nested::extractTableName(elem.name);
auto & offsets_column = offset_columns[offsets_name];
/// If for some reason there are different offset columns for one nested structure, then we take nonempty.
......@@ -54,7 +54,7 @@ void AddingDefaultBlockOutputStream::write(const DB::Block & block)
column_to_add.name = requested_column.name;
column_to_add.type = requested_column.type;
String offsets_name = DataTypeNested::extractNestedTableName(column_to_add.name);
String offsets_name = Nested::extractTableName(column_to_add.name);
if (offset_columns.count(offsets_name))
{
ColumnPtr offsets_column = offset_columns[offsets_name];
......
#include <DataStreams/SummingSortedBlockInputStream.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeNested.h>
#include <DataTypes/NestedUtils.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypeArray.h>
#include <Columns/ColumnTuple.h>
......@@ -148,7 +148,7 @@ Block SummingSortedBlockInputStream::readImpl()
/// Discover nested Maps and find columns for summation
if (typeid_cast<const DataTypeArray *>(column.type.get()))
{
const auto map_name = DataTypeNested::extractNestedTableName(column.name);
const auto map_name = Nested::extractTableName(column.name);
/// if nested table name ends with `Map` it is a possible candidate for special handling
if (map_name == column.name || !endsWith(map_name, "Map"))
{
......
#include <Common/StringUtils.h>
#include <Columns/ColumnTuple.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeFactory.h>
#include <Parsers/IAST.h>
#include <Parsers/ASTNameTypePair.h>
......@@ -445,4 +446,13 @@ void registerDataTypeTuple(DataTypeFactory & factory)
factory.registerDataType("Tuple", create);
}
void registerDataTypeNested(DataTypeFactory & factory)
{
/// 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));
});
}
}
......@@ -9,7 +9,7 @@
#include <IO/WriteHelpers.h>
#include <DataTypes/IDataType.h>
#include <DataTypes/DataTypeNested.h>
#include <DataTypes/NestedUtils.h>
namespace DB
......@@ -70,7 +70,7 @@ size_t IDataType::getSizeOfValueInMemory() const
String IDataType::getFileNameForStream(const String & column_name, const IDataType::SubstreamPath & path)
{
String nested_table_name = DataTypeNested::extractNestedTableName(column_name);
String nested_table_name = Nested::extractTableName(column_name);
bool is_sizes_of_nested_type = !path.empty() && path.back().type == IDataType::Substream::ArraySizes
&& nested_table_name != column_name;
......
......@@ -4,7 +4,7 @@
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypeNested.h>
#include <DataTypes/NestedUtils.h>
#include <DataTypes/DataTypeFactory.h>
#include <Parsers/IAST.h>
......@@ -18,14 +18,16 @@ namespace ErrorCodes
extern const int INVALID_NESTED_NAME;
}
namespace Nested
{
std::string DataTypeNested::concatenateNestedName(const std::string & nested_table_name, const std::string & nested_field_name)
std::string concatenateName(const std::string & nested_table_name, const std::string & nested_field_name)
{
return nested_table_name + "." + nested_field_name;
}
std::string DataTypeNested::extractNestedTableName(const std::string & nested_name)
std::string extractTableName(const std::string & nested_name)
{
const char * first_pos = strchr(nested_name.data(), '.');
const char * last_pos = strrchr(nested_name.data(), '.');
......@@ -35,7 +37,7 @@ std::string DataTypeNested::extractNestedTableName(const std::string & nested_na
}
std::string DataTypeNested::extractNestedColumnName(const std::string & nested_name)
std::string extractElementName(const std::string & nested_name)
{
const char * first_pos = strchr(nested_name.data(), '.');
const char * last_pos = strrchr(nested_name.data(), '.');
......@@ -45,7 +47,7 @@ std::string DataTypeNested::extractNestedColumnName(const std::string & nested_n
}
NamesAndTypesListPtr DataTypeNested::expandNestedColumns(const NamesAndTypesList & names_and_types)
NamesAndTypesListPtr flatten(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)
......@@ -60,7 +62,7 @@ NamesAndTypesListPtr DataTypeNested::expandNestedColumns(const NamesAndTypesList
for (size_t i = 0; i < tuple_size; ++i)
{
String nested_name = DataTypeNested::concatenateNestedName(it->name, names[i]);
String nested_name = concatenateName(it->name, names[i]);
columns->push_back(NameAndTypePair(nested_name, std::make_shared<DataTypeArray>(elements[i])));
}
}
......@@ -73,14 +75,6 @@ NamesAndTypesListPtr DataTypeNested::expandNestedColumns(const NamesAndTypesList
return columns;
}
void registerDataTypeNested(DataTypeFactory & factory)
{
/// 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));
});
}
}
......@@ -6,16 +6,17 @@
namespace DB
{
namespace DataTypeNested
namespace Nested
{
std::string concatenateNestedName(const std::string & nested_table_name, const std::string & nested_field_name);
std::string concatenateName(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.
std::string extractNestedTableName(const std::string & nested_name);
std::string extractTableName(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.
std::string extractNestedColumnName(const std::string & nested_name);
std::string extractElementName(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`
NamesAndTypesListPtr expandNestedColumns(const NamesAndTypesList & names_and_types);
NamesAndTypesListPtr flatten(const NamesAndTypesList & names_and_types);
};
}
......@@ -18,7 +18,7 @@
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypeExpression.h>
#include <DataTypes/DataTypeNested.h>
#include <DataTypes/NestedUtils.h>
#include <DataTypes/DataTypesNumber.h>
#include <Columns/ColumnSet.h>
......@@ -1912,11 +1912,11 @@ void ExpressionAnalyzer::getArrayJoinedColumns()
bool found = false;
for (const auto & column_name_type : columns)
{
String table_name = DataTypeNested::extractNestedTableName(column_name_type.name);
String column_name = DataTypeNested::extractNestedColumnName(column_name_type.name);
String table_name = Nested::extractTableName(column_name_type.name);
String column_name = Nested::extractElementName(column_name_type.name);
if (table_name == source_name)
{
array_join_result_to_source[DataTypeNested::concatenateNestedName(result_name, column_name)] = column_name_type.name;
array_join_result_to_source[Nested::concatenateName(result_name, column_name)] = column_name_type.name;
found = true;
break;
}
......@@ -1939,7 +1939,7 @@ void ExpressionAnalyzer::getArrayJoinedColumnsImpl(const ASTPtr & ast)
{
if (node->kind == ASTIdentifier::Column)
{
String table_name = DataTypeNested::extractNestedTableName(node->name);
String table_name = Nested::extractTableName(node->name);
if (array_join_alias_to_name.count(node->name))
{
......@@ -1949,9 +1949,9 @@ void ExpressionAnalyzer::getArrayJoinedColumnsImpl(const ASTPtr & ast)
else if (array_join_alias_to_name.count(table_name))
{
/// ARRAY JOIN was written with a nested table. Example: SELECT PP.KEY1 FROM ... ARRAY JOIN ParsedParams AS PP
String nested_column = DataTypeNested::extractNestedColumnName(node->name); /// Key1
String nested_column = Nested::extractElementName(node->name); /// Key1
array_join_result_to_source[node->name] /// PP.Key1 -> ParsedParams.Key1
= DataTypeNested::concatenateNestedName(array_join_alias_to_name[table_name], nested_column);
= Nested::concatenateName(array_join_alias_to_name[table_name], nested_column);
}
else if (array_join_name_to_alias.count(node->name))
{
......@@ -1959,7 +1959,7 @@ void ExpressionAnalyzer::getArrayJoinedColumnsImpl(const ASTPtr & ast)
* That is, the query uses the original array, replicated by itself.
*/
String nested_column = DataTypeNested::extractNestedColumnName(node->name); /// Key1
String nested_column = Nested::extractElementName(node->name); /// Key1
array_join_result_to_source[ /// PP.Key1 -> ParsedParams.Key1
array_join_name_to_alias[node->name]] = node->name;
}
......@@ -1968,9 +1968,9 @@ void ExpressionAnalyzer::getArrayJoinedColumnsImpl(const ASTPtr & ast)
/** Example: SELECT ParsedParams.Key1 FROM ... ARRAY JOIN ParsedParams AS PP.
*/
String nested_column = DataTypeNested::extractNestedColumnName(node->name); /// Key1
String nested_column = Nested::extractElementName(node->name); /// Key1
array_join_result_to_source[ /// PP.Key1 -> ParsedParams.Key1
DataTypeNested::concatenateNestedName(array_join_name_to_alias[table_name], nested_column)] = node->name;
Nested::concatenateName(array_join_name_to_alias[table_name], nested_column)] = node->name;
}
}
}
......@@ -2877,7 +2877,7 @@ void ExpressionAnalyzer::getRequiredColumnsImpl(const ASTPtr & ast,
{
if (node->kind == ASTIdentifier::Column
&& !ignored_names.count(node->name)
&& !ignored_names.count(DataTypeNested::extractNestedTableName(node->name)))
&& !ignored_names.count(Nested::extractTableName(node->name)))
{
if (!available_joined_columns.count(node->name)
|| available_columns.count(node->name)) /// Read column from left table if has.
......
......@@ -34,7 +34,7 @@
#include <Interpreters/InterpreterSelectQuery.h>
#include <DataTypes/DataTypeFactory.h>
#include <DataTypes/DataTypeNested.h>
#include <DataTypes/NestedUtils.h>
#include <DataTypes/DataTypesNumber.h>
#include <Databases/DatabaseFactory.h>
......@@ -255,7 +255,7 @@ static ColumnsAndDefaults parseColumns(const ASTExpressionList & column_list_ast
}
}
return {*DataTypeNested::expandNestedColumns(columns), defaults};
return {*Nested::flatten(columns), defaults};
}
......
#include <Storages/AlterCommands.h>
#include <Storages/IStorage.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeNested.h>
#include <DataTypes/NestedUtils.h>
#include <Interpreters/Context.h>
#include <Interpreters/ExpressionAnalyzer.h>
#include <Interpreters/ExpressionActions.h>
......@@ -80,7 +80,7 @@ void AlterCommand::apply(
column_defaults.emplace(column_name, ColumnDefault{default_type, default_expression});
/// Slow, because each time a list is copied
columns = *DataTypeNested::expandNestedColumns(columns);
columns = *Nested::flatten(columns);
}
else if (type == DROP_COLUMN)
{
......
......@@ -24,7 +24,7 @@
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeEnum.h>
#include <DataTypes/DataTypeNested.h>
#include <DataTypes/NestedUtils.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeNullable.h>
#include <Functions/FunctionFactory.h>
......@@ -300,7 +300,7 @@ void MergeTreeData::MergingParams::check(const NamesAndTypesList & columns) cons
for (const auto & column_to_sum : columns_to_sum)
if (columns.end() == std::find_if(columns.begin(), columns.end(),
[&](const NameAndTypePair & name_and_type) { return column_to_sum == DataTypeNested::extractNestedTableName(name_and_type.name); }))
[&](const NameAndTypePair & name_and_type) { return column_to_sum == Nested::extractTableName(name_and_type.name); }))
throw Exception("Column " + column_to_sum + " listed in columns to sum does not exist in table declaration.");
}
......
......@@ -18,7 +18,7 @@
#include <DataStreams/ColumnGathererStream.h>
#include <IO/CompressedWriteBuffer.h>
#include <IO/CompressedReadBufferFromFile.h>
#include <DataTypes/DataTypeNested.h>
#include <DataTypes/NestedUtils.h>
#include <DataTypes/DataTypeArray.h>
#include <Storages/MergeTree/BackgroundProcessingPool.h>
#include <Common/SimpleIncrement.h>
......@@ -681,7 +681,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMerger::mergePartsToTemporaryPart
{
const String & column_name = it_name_and_type->name;
const DataTypePtr & column_type = it_name_and_type->type;
const String offset_column_name = DataTypeNested::extractNestedTableName(column_name);
const String offset_column_name = Nested::extractTableName(column_name);
Names column_name_{column_name};
Float64 progress_before = merge_entry->progress;
bool offset_written = offset_columns_written.count(offset_column_name);
......
#include <DataTypes/DataTypeNested.h>
#include <DataTypes/NestedUtils.h>
#include <DataTypes/DataTypeArray.h>
#include <Common/escapeForFileName.h>
#include <Common/MemoryTracker.h>
......@@ -97,7 +97,7 @@ size_t MergeTreeReader::readRows(size_t from_mark, bool continue_reading, size_t
/// For nested data structures collect pointers to offset columns.
if (const DataTypeArray * type_arr = typeid_cast<const DataTypeArray *>(it.type.get()))
{
String name = DataTypeNested::extractNestedTableName(it.name);
String name = Nested::extractTableName(it.name);
auto it_inserted = offset_columns.emplace(name, nullptr);
......@@ -432,7 +432,7 @@ void MergeTreeReader::fillMissingColumns(Block & res, const Names & ordered_name
if (const ColumnArray * array = typeid_cast<const ColumnArray *>(column.column.get()))
{
String offsets_name = DataTypeNested::extractNestedTableName(column.name);
String offsets_name = Nested::extractTableName(column.name);
auto & offsets_column = offset_columns[offsets_name];
/// If for some reason multiple offsets columns are present for the same nested data structure,
......@@ -474,7 +474,7 @@ void MergeTreeReader::fillMissingColumns(Block & res, const Names & ordered_name
column_to_add.name = requested_column.name;
column_to_add.type = requested_column.type;
String offsets_name = DataTypeNested::extractNestedTableName(column_to_add.name);
String offsets_name = Nested::extractTableName(column_to_add.name);
if (offset_columns.count(offsets_name))
{
ColumnPtr offsets_column = offset_columns[offsets_name];
......
......@@ -10,7 +10,7 @@
#include <Parsers/formatAST.h>
#include <Common/escapeForFileName.h>
#include <Common/typeid_cast.h>
#include <DataTypes/DataTypeNested.h>
#include <DataTypes/NestedUtils.h>
#include <ext/scope_guard.h>
#include <ext/map.h>
#include <memory>
......@@ -384,7 +384,7 @@ bool MergeTreeWhereOptimizer::cannotBeMoved(const IAST * ptr) const
/// disallow moving result of ARRAY JOIN to PREWHERE
if (identifier_ptr->kind == ASTIdentifier::Column)
if (array_joined_names.count(identifier_ptr->name) ||
array_joined_names.count(DataTypeNested::extractNestedTableName(identifier_ptr->name)))
array_joined_names.count(Nested::extractTableName(identifier_ptr->name)))
return true;
}
......
#include <Storages/MergeTree/MergedBlockOutputStream.h>
#include <IO/createWriteBufferFromFileBase.h>
#include <Common/escapeForFileName.h>
#include <DataTypes/DataTypeNested.h>
#include <DataTypes/NestedUtils.h>
#include <Common/StringUtils.h>
#include <Common/typeid_cast.h>
#include <Common/MemoryTracker.h>
......
......@@ -12,7 +12,7 @@
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypeNested.h>
#include <DataTypes/NestedUtils.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataStreams/IProfilingBlockInputStream.h>
......@@ -212,7 +212,7 @@ Block LogBlockInputStream::readImpl()
/// For nested structures, remember pointers to columns with offsets
if (const DataTypeArray * type_arr = typeid_cast<const DataTypeArray *>(column_types[i].get()))
{
String nested_name = DataTypeNested::extractNestedTableName(name);
String nested_name = Nested::extractTableName(name);
if (offset_columns.count(nested_name) == 0)
offset_columns[nested_name] = ColumnArray::ColumnOffsets::create();
......
......@@ -18,7 +18,7 @@
#include <IO/WriteHelpers.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeNested.h>
#include <DataTypes/NestedUtils.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataStreams/IProfilingBlockInputStream.h>
......@@ -197,7 +197,7 @@ Block TinyLogBlockInputStream::readImpl()
/// For nested structures, remember pointers to columns with offsets
if (const DataTypeArray * type_arr = typeid_cast<const DataTypeArray *>(column_types[i].get()))
{
String nested_name = DataTypeNested::extractNestedTableName(name);
String nested_name = Nested::extractTableName(name);
if (offset_columns.count(nested_name) == 0)
offset_columns[nested_name] = ColumnArray::ColumnOffsets::create();
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册