mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-01 12:01:58 +00:00
Get rid of DataTypeNested, part 2 [#CLICKHOUSE-2].
This commit is contained in:
parent
8bc30c8b91
commit
bcfde7c6c0
2
contrib/librdkafka
vendored
2
contrib/librdkafka
vendored
@ -1 +1 @@
|
|||||||
Subproject commit 3a986afbb977fa13582991ce8f2c0b2045ffaa33
|
Subproject commit 3401fa1e45605b5ae806f94905c92f5f546a607b
|
2
contrib/zstd
vendored
2
contrib/zstd
vendored
@ -1 +1 @@
|
|||||||
Subproject commit f3a8bd553a865c59f1bd6e1f68bf182cf75a8f00
|
Subproject commit f4340f46b2387bc8de7d5320c0b83bb1499933ad
|
@ -1,7 +1,7 @@
|
|||||||
#include <DataStreams/AddingDefaultBlockOutputStream.h>
|
#include <DataStreams/AddingDefaultBlockOutputStream.h>
|
||||||
|
|
||||||
#include <Common/typeid_cast.h>
|
#include <Common/typeid_cast.h>
|
||||||
#include <DataTypes/DataTypeNested.h>
|
#include <DataTypes/NestedUtils.h>
|
||||||
#include <DataTypes/DataTypeArray.h>
|
#include <DataTypes/DataTypeArray.h>
|
||||||
#include <Columns/ColumnArray.h>
|
#include <Columns/ColumnArray.h>
|
||||||
#include <Core/Block.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))
|
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];
|
auto & offsets_column = offset_columns[offsets_name];
|
||||||
|
|
||||||
/// If for some reason there are different offset columns for one nested structure, then we take nonempty.
|
/// 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.name = requested_column.name;
|
||||||
column_to_add.type = requested_column.type;
|
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))
|
if (offset_columns.count(offsets_name))
|
||||||
{
|
{
|
||||||
ColumnPtr offsets_column = offset_columns[offsets_name];
|
ColumnPtr offsets_column = offset_columns[offsets_name];
|
||||||
|
@ -1,6 +1,6 @@
|
|||||||
#include <DataStreams/SummingSortedBlockInputStream.h>
|
#include <DataStreams/SummingSortedBlockInputStream.h>
|
||||||
#include <DataTypes/DataTypesNumber.h>
|
#include <DataTypes/DataTypesNumber.h>
|
||||||
#include <DataTypes/DataTypeNested.h>
|
#include <DataTypes/NestedUtils.h>
|
||||||
#include <DataTypes/DataTypeTuple.h>
|
#include <DataTypes/DataTypeTuple.h>
|
||||||
#include <DataTypes/DataTypeArray.h>
|
#include <DataTypes/DataTypeArray.h>
|
||||||
#include <Columns/ColumnTuple.h>
|
#include <Columns/ColumnTuple.h>
|
||||||
@ -148,7 +148,7 @@ Block SummingSortedBlockInputStream::readImpl()
|
|||||||
/// Discover nested Maps and find columns for summation
|
/// Discover nested Maps and find columns for summation
|
||||||
if (typeid_cast<const DataTypeArray *>(column.type.get()))
|
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 nested table name ends with `Map` it is a possible candidate for special handling
|
||||||
if (map_name == column.name || !endsWith(map_name, "Map"))
|
if (map_name == column.name || !endsWith(map_name, "Map"))
|
||||||
{
|
{
|
||||||
|
@ -1,6 +1,7 @@
|
|||||||
#include <Common/StringUtils.h>
|
#include <Common/StringUtils.h>
|
||||||
#include <Columns/ColumnTuple.h>
|
#include <Columns/ColumnTuple.h>
|
||||||
#include <DataTypes/DataTypeTuple.h>
|
#include <DataTypes/DataTypeTuple.h>
|
||||||
|
#include <DataTypes/DataTypeArray.h>
|
||||||
#include <DataTypes/DataTypeFactory.h>
|
#include <DataTypes/DataTypeFactory.h>
|
||||||
#include <Parsers/IAST.h>
|
#include <Parsers/IAST.h>
|
||||||
#include <Parsers/ASTNameTypePair.h>
|
#include <Parsers/ASTNameTypePair.h>
|
||||||
@ -445,4 +446,13 @@ void registerDataTypeTuple(DataTypeFactory & factory)
|
|||||||
factory.registerDataType("Tuple", create);
|
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 <IO/WriteHelpers.h>
|
||||||
|
|
||||||
#include <DataTypes/IDataType.h>
|
#include <DataTypes/IDataType.h>
|
||||||
#include <DataTypes/DataTypeNested.h>
|
#include <DataTypes/NestedUtils.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
@ -70,7 +70,7 @@ size_t IDataType::getSizeOfValueInMemory() const
|
|||||||
|
|
||||||
String IDataType::getFileNameForStream(const String & column_name, const IDataType::SubstreamPath & path)
|
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
|
bool is_sizes_of_nested_type = !path.empty() && path.back().type == IDataType::Substream::ArraySizes
|
||||||
&& nested_table_name != column_name;
|
&& nested_table_name != column_name;
|
||||||
|
|
||||||
|
@ -4,7 +4,7 @@
|
|||||||
|
|
||||||
#include <DataTypes/DataTypeArray.h>
|
#include <DataTypes/DataTypeArray.h>
|
||||||
#include <DataTypes/DataTypeTuple.h>
|
#include <DataTypes/DataTypeTuple.h>
|
||||||
#include <DataTypes/DataTypeNested.h>
|
#include <DataTypes/NestedUtils.h>
|
||||||
#include <DataTypes/DataTypeFactory.h>
|
#include <DataTypes/DataTypeFactory.h>
|
||||||
|
|
||||||
#include <Parsers/IAST.h>
|
#include <Parsers/IAST.h>
|
||||||
@ -18,14 +18,16 @@ namespace ErrorCodes
|
|||||||
extern const int INVALID_NESTED_NAME;
|
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;
|
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 * first_pos = strchr(nested_name.data(), '.');
|
||||||
const char * last_pos = strrchr(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 * first_pos = strchr(nested_name.data(), '.');
|
||||||
const char * last_pos = strrchr(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>();
|
NamesAndTypesListPtr columns = std::make_shared<NamesAndTypesList>();
|
||||||
for (NamesAndTypesList::const_iterator it = names_and_types.begin(); it != names_and_types.end(); ++it)
|
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)
|
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])));
|
columns->push_back(NameAndTypePair(nested_name, std::make_shared<DataTypeArray>(elements[i])));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -73,14 +75,6 @@ NamesAndTypesListPtr DataTypeNested::expandNestedColumns(const NamesAndTypesList
|
|||||||
return columns;
|
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 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.
|
/// 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.
|
/// 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`
|
/// 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/DataTypeNullable.h>
|
||||||
#include <DataTypes/DataTypeTuple.h>
|
#include <DataTypes/DataTypeTuple.h>
|
||||||
#include <DataTypes/DataTypeExpression.h>
|
#include <DataTypes/DataTypeExpression.h>
|
||||||
#include <DataTypes/DataTypeNested.h>
|
#include <DataTypes/NestedUtils.h>
|
||||||
#include <DataTypes/DataTypesNumber.h>
|
#include <DataTypes/DataTypesNumber.h>
|
||||||
|
|
||||||
#include <Columns/ColumnSet.h>
|
#include <Columns/ColumnSet.h>
|
||||||
@ -1912,11 +1912,11 @@ void ExpressionAnalyzer::getArrayJoinedColumns()
|
|||||||
bool found = false;
|
bool found = false;
|
||||||
for (const auto & column_name_type : columns)
|
for (const auto & column_name_type : columns)
|
||||||
{
|
{
|
||||||
String table_name = DataTypeNested::extractNestedTableName(column_name_type.name);
|
String table_name = Nested::extractTableName(column_name_type.name);
|
||||||
String column_name = DataTypeNested::extractNestedColumnName(column_name_type.name);
|
String column_name = Nested::extractElementName(column_name_type.name);
|
||||||
if (table_name == source_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;
|
found = true;
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
@ -1939,7 +1939,7 @@ void ExpressionAnalyzer::getArrayJoinedColumnsImpl(const ASTPtr & ast)
|
|||||||
{
|
{
|
||||||
if (node->kind == ASTIdentifier::Column)
|
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))
|
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))
|
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
|
/// 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
|
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))
|
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.
|
* 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_result_to_source[ /// PP.Key1 -> ParsedParams.Key1
|
||||||
array_join_name_to_alias[node->name]] = node->name;
|
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.
|
/** 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
|
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
|
if (node->kind == ASTIdentifier::Column
|
||||||
&& !ignored_names.count(node->name)
|
&& !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)
|
if (!available_joined_columns.count(node->name)
|
||||||
|| available_columns.count(node->name)) /// Read column from left table if has.
|
|| available_columns.count(node->name)) /// Read column from left table if has.
|
||||||
|
@ -34,7 +34,7 @@
|
|||||||
#include <Interpreters/InterpreterSelectQuery.h>
|
#include <Interpreters/InterpreterSelectQuery.h>
|
||||||
|
|
||||||
#include <DataTypes/DataTypeFactory.h>
|
#include <DataTypes/DataTypeFactory.h>
|
||||||
#include <DataTypes/DataTypeNested.h>
|
#include <DataTypes/NestedUtils.h>
|
||||||
#include <DataTypes/DataTypesNumber.h>
|
#include <DataTypes/DataTypesNumber.h>
|
||||||
|
|
||||||
#include <Databases/DatabaseFactory.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};
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@ -1,7 +1,7 @@
|
|||||||
#include <Storages/AlterCommands.h>
|
#include <Storages/AlterCommands.h>
|
||||||
#include <Storages/IStorage.h>
|
#include <Storages/IStorage.h>
|
||||||
#include <DataTypes/DataTypesNumber.h>
|
#include <DataTypes/DataTypesNumber.h>
|
||||||
#include <DataTypes/DataTypeNested.h>
|
#include <DataTypes/NestedUtils.h>
|
||||||
#include <Interpreters/Context.h>
|
#include <Interpreters/Context.h>
|
||||||
#include <Interpreters/ExpressionAnalyzer.h>
|
#include <Interpreters/ExpressionAnalyzer.h>
|
||||||
#include <Interpreters/ExpressionActions.h>
|
#include <Interpreters/ExpressionActions.h>
|
||||||
@ -80,7 +80,7 @@ void AlterCommand::apply(
|
|||||||
column_defaults.emplace(column_name, ColumnDefault{default_type, default_expression});
|
column_defaults.emplace(column_name, ColumnDefault{default_type, default_expression});
|
||||||
|
|
||||||
/// Slow, because each time a list is copied
|
/// Slow, because each time a list is copied
|
||||||
columns = *DataTypeNested::expandNestedColumns(columns);
|
columns = *Nested::flatten(columns);
|
||||||
}
|
}
|
||||||
else if (type == DROP_COLUMN)
|
else if (type == DROP_COLUMN)
|
||||||
{
|
{
|
||||||
|
@ -24,7 +24,7 @@
|
|||||||
#include <DataTypes/DataTypeDate.h>
|
#include <DataTypes/DataTypeDate.h>
|
||||||
#include <DataTypes/DataTypeDateTime.h>
|
#include <DataTypes/DataTypeDateTime.h>
|
||||||
#include <DataTypes/DataTypeEnum.h>
|
#include <DataTypes/DataTypeEnum.h>
|
||||||
#include <DataTypes/DataTypeNested.h>
|
#include <DataTypes/NestedUtils.h>
|
||||||
#include <DataTypes/DataTypeArray.h>
|
#include <DataTypes/DataTypeArray.h>
|
||||||
#include <DataTypes/DataTypeNullable.h>
|
#include <DataTypes/DataTypeNullable.h>
|
||||||
#include <Functions/FunctionFactory.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)
|
for (const auto & column_to_sum : columns_to_sum)
|
||||||
if (columns.end() == std::find_if(columns.begin(), columns.end(),
|
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.");
|
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 <DataStreams/ColumnGathererStream.h>
|
||||||
#include <IO/CompressedWriteBuffer.h>
|
#include <IO/CompressedWriteBuffer.h>
|
||||||
#include <IO/CompressedReadBufferFromFile.h>
|
#include <IO/CompressedReadBufferFromFile.h>
|
||||||
#include <DataTypes/DataTypeNested.h>
|
#include <DataTypes/NestedUtils.h>
|
||||||
#include <DataTypes/DataTypeArray.h>
|
#include <DataTypes/DataTypeArray.h>
|
||||||
#include <Storages/MergeTree/BackgroundProcessingPool.h>
|
#include <Storages/MergeTree/BackgroundProcessingPool.h>
|
||||||
#include <Common/SimpleIncrement.h>
|
#include <Common/SimpleIncrement.h>
|
||||||
@ -681,7 +681,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMerger::mergePartsToTemporaryPart
|
|||||||
{
|
{
|
||||||
const String & column_name = it_name_and_type->name;
|
const String & column_name = it_name_and_type->name;
|
||||||
const DataTypePtr & column_type = it_name_and_type->type;
|
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};
|
Names column_name_{column_name};
|
||||||
Float64 progress_before = merge_entry->progress;
|
Float64 progress_before = merge_entry->progress;
|
||||||
bool offset_written = offset_columns_written.count(offset_column_name);
|
bool offset_written = offset_columns_written.count(offset_column_name);
|
||||||
|
@ -1,4 +1,4 @@
|
|||||||
#include <DataTypes/DataTypeNested.h>
|
#include <DataTypes/NestedUtils.h>
|
||||||
#include <DataTypes/DataTypeArray.h>
|
#include <DataTypes/DataTypeArray.h>
|
||||||
#include <Common/escapeForFileName.h>
|
#include <Common/escapeForFileName.h>
|
||||||
#include <Common/MemoryTracker.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.
|
/// For nested data structures collect pointers to offset columns.
|
||||||
if (const DataTypeArray * type_arr = typeid_cast<const DataTypeArray *>(it.type.get()))
|
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);
|
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()))
|
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];
|
auto & offsets_column = offset_columns[offsets_name];
|
||||||
|
|
||||||
/// If for some reason multiple offsets columns are present for the same nested data structure,
|
/// 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.name = requested_column.name;
|
||||||
column_to_add.type = requested_column.type;
|
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))
|
if (offset_columns.count(offsets_name))
|
||||||
{
|
{
|
||||||
ColumnPtr offsets_column = offset_columns[offsets_name];
|
ColumnPtr offsets_column = offset_columns[offsets_name];
|
||||||
|
@ -10,7 +10,7 @@
|
|||||||
#include <Parsers/formatAST.h>
|
#include <Parsers/formatAST.h>
|
||||||
#include <Common/escapeForFileName.h>
|
#include <Common/escapeForFileName.h>
|
||||||
#include <Common/typeid_cast.h>
|
#include <Common/typeid_cast.h>
|
||||||
#include <DataTypes/DataTypeNested.h>
|
#include <DataTypes/NestedUtils.h>
|
||||||
#include <ext/scope_guard.h>
|
#include <ext/scope_guard.h>
|
||||||
#include <ext/map.h>
|
#include <ext/map.h>
|
||||||
#include <memory>
|
#include <memory>
|
||||||
@ -384,7 +384,7 @@ bool MergeTreeWhereOptimizer::cannotBeMoved(const IAST * ptr) const
|
|||||||
/// disallow moving result of ARRAY JOIN to PREWHERE
|
/// disallow moving result of ARRAY JOIN to PREWHERE
|
||||||
if (identifier_ptr->kind == ASTIdentifier::Column)
|
if (identifier_ptr->kind == ASTIdentifier::Column)
|
||||||
if (array_joined_names.count(identifier_ptr->name) ||
|
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;
|
return true;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1,7 +1,7 @@
|
|||||||
#include <Storages/MergeTree/MergedBlockOutputStream.h>
|
#include <Storages/MergeTree/MergedBlockOutputStream.h>
|
||||||
#include <IO/createWriteBufferFromFileBase.h>
|
#include <IO/createWriteBufferFromFileBase.h>
|
||||||
#include <Common/escapeForFileName.h>
|
#include <Common/escapeForFileName.h>
|
||||||
#include <DataTypes/DataTypeNested.h>
|
#include <DataTypes/NestedUtils.h>
|
||||||
#include <Common/StringUtils.h>
|
#include <Common/StringUtils.h>
|
||||||
#include <Common/typeid_cast.h>
|
#include <Common/typeid_cast.h>
|
||||||
#include <Common/MemoryTracker.h>
|
#include <Common/MemoryTracker.h>
|
||||||
|
@ -12,7 +12,7 @@
|
|||||||
|
|
||||||
#include <DataTypes/DataTypeArray.h>
|
#include <DataTypes/DataTypeArray.h>
|
||||||
#include <DataTypes/DataTypeString.h>
|
#include <DataTypes/DataTypeString.h>
|
||||||
#include <DataTypes/DataTypeNested.h>
|
#include <DataTypes/NestedUtils.h>
|
||||||
#include <DataTypes/DataTypesNumber.h>
|
#include <DataTypes/DataTypesNumber.h>
|
||||||
|
|
||||||
#include <DataStreams/IProfilingBlockInputStream.h>
|
#include <DataStreams/IProfilingBlockInputStream.h>
|
||||||
@ -212,7 +212,7 @@ Block LogBlockInputStream::readImpl()
|
|||||||
/// For nested structures, remember pointers to columns with offsets
|
/// For nested structures, remember pointers to columns with offsets
|
||||||
if (const DataTypeArray * type_arr = typeid_cast<const DataTypeArray *>(column_types[i].get()))
|
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)
|
if (offset_columns.count(nested_name) == 0)
|
||||||
offset_columns[nested_name] = ColumnArray::ColumnOffsets::create();
|
offset_columns[nested_name] = ColumnArray::ColumnOffsets::create();
|
||||||
|
@ -18,7 +18,7 @@
|
|||||||
#include <IO/WriteHelpers.h>
|
#include <IO/WriteHelpers.h>
|
||||||
|
|
||||||
#include <DataTypes/DataTypeArray.h>
|
#include <DataTypes/DataTypeArray.h>
|
||||||
#include <DataTypes/DataTypeNested.h>
|
#include <DataTypes/NestedUtils.h>
|
||||||
#include <DataTypes/DataTypesNumber.h>
|
#include <DataTypes/DataTypesNumber.h>
|
||||||
|
|
||||||
#include <DataStreams/IProfilingBlockInputStream.h>
|
#include <DataStreams/IProfilingBlockInputStream.h>
|
||||||
@ -197,7 +197,7 @@ Block TinyLogBlockInputStream::readImpl()
|
|||||||
/// For nested structures, remember pointers to columns with offsets
|
/// For nested structures, remember pointers to columns with offsets
|
||||||
if (const DataTypeArray * type_arr = typeid_cast<const DataTypeArray *>(column_types[i].get()))
|
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)
|
if (offset_columns.count(nested_name) == 0)
|
||||||
offset_columns[nested_name] = ColumnArray::ColumnOffsets::create();
|
offset_columns[nested_name] = ColumnArray::ColumnOffsets::create();
|
||||||
|
Loading…
Reference in New Issue
Block a user