dbms: updated all usages of NamesAndTypesPair to new field names. [#METR-10202]

This commit is contained in:
Michael Kolupaev 2014-07-09 15:45:51 +04:00
parent d848cfec91
commit 0a67d151bd
38 changed files with 158 additions and 156 deletions

View File

@ -32,7 +32,7 @@ public:
{
Names res(arguments.size());
for (size_t i = 0; i < arguments.size(); ++i)
res[i] = arguments[i].first;
res[i] = arguments[i].name;
return res;
}

View File

@ -19,6 +19,9 @@ struct NameAndTypePair
String name;
DataTypePtr type;
NameAndTypePair() {}
NameAndTypePair(const String & name_, const DataTypePtr & type_) : name(name_), type(type_) {}
bool operator<(const NameAndTypePair & rhs) const
{
return std::make_pair(name, type->getName()) < std::make_pair(rhs.name, rhs.type->getName());

View File

@ -33,7 +33,7 @@ public:
res << "AddingDefault(" << children.back()->getID();
for (NamesAndTypesList::const_iterator it = required_columns->begin(); it != required_columns->end(); ++it)
res << ", " << it->first << ", " << it->second->getName();
res << ", " << it->name << ", " << it->type->getName();
res << ")";
return res.str();

View File

@ -41,14 +41,12 @@ protected:
virtual void writeTotals();
virtual void writeExtremes();
typedef std::vector<NameAndTypePair> NamesAndTypesVector;
WriteBufferValidUTF8 ostr;
size_t field_number;
size_t row_count;
bool applied_limit;
size_t rows_before_limit;
NamesAndTypesVector fields;
NamesAndTypes fields;
Block totals;
Block extremes;
};

View File

@ -436,8 +436,8 @@ public:
for (size_t i = 0; i < expression_arguments.size(); ++i)
{
const std::string & argument_name = expression_arguments[i].first;
DataTypePtr argument_type = expression_arguments[i].second;
const std::string & argument_name = expression_arguments[i].name;
DataTypePtr argument_type = expression_arguments[i].type;
ColumnPtr column_array_ptr = block.getByPosition(arguments[i + 1]).column;
const ColumnArray * column_array = typeid_cast<const ColumnArray *>(&*column_array_ptr);

View File

@ -169,7 +169,7 @@ public:
{
for (NamesAndTypesList::iterator it = input_columns.begin(); it != input_columns.end(); ++it)
{
sample_block.insert(ColumnWithNameAndType(nullptr, it->second, it->first));
sample_block.insert(ColumnWithNameAndType(nullptr, it->type, it->name));
}
}
@ -220,7 +220,7 @@ public:
{
Names names;
for (NamesAndTypesList::const_iterator it = input_columns.begin(); it != input_columns.end(); ++it)
names.push_back(it->first);
names.push_back(it->name);
return names;
}

View File

@ -90,8 +90,8 @@ private:
for (NamesAndTypesList::iterator it = columns.begin(); it != columns.end(); ++it)
{
name_column->insert(it->first);
type_column->insert(it->second->getName());
name_column->insert(it->name);
type_column->insert(it->type->getName());
}
return new OneBlockInputStream(block);

View File

@ -261,7 +261,7 @@ public:
{
/// Размер - в количестве засечек.
if (!size)
size = Poco::File(storage.full_path + name + "/" + escapeForFileName(storage.columns->front().first) + ".mrk")
size = Poco::File(storage.full_path + name + "/" + escapeForFileName(storage.columns->front().name) + ".mrk")
.getSize() / MERGE_TREE_MARK_SIZE;
size_t key_size = storage.sort_descr.size();
@ -315,7 +315,7 @@ public:
ssize_t marks_size = -1;
for (NamesAndTypesList::const_iterator it = storage.columns->begin(); it != storage.columns->end(); ++it)
{
Poco::File marks_file(path + "/" + escapeForFileName(it->first) + ".mrk");
Poco::File marks_file(path + "/" + escapeForFileName(it->name) + ".mrk");
/// При добавлении нового столбца в таблицу файлы .mrk не создаются. Не будем ничего удалять.
if (!marks_file.exists())

View File

@ -372,8 +372,8 @@ private:
for (size_t i = 0; i < column_nested.getData().size(); ++i, ++it)
{
readData(
DataTypeNested::concatenateNestedName(name, it->first),
*it->second,
DataTypeNested::concatenateNestedName(name, it->name),
*it->type,
*column_nested.getData()[i],
from_mark,
column_nested.getOffsets()[column.size() - 1] - column_nested.getData()[i]->size(),

View File

@ -213,7 +213,7 @@ public:
columns_list = storage.getColumnsList();
for (const auto & it : columns_list)
addStream(part_path, it.first, *it.second);
addStream(part_path, it.name, *it.type);
}
void write(const Block & block)
@ -247,7 +247,7 @@ public:
/// Теперь пишем данные.
for (const auto & it : columns_list)
{
const ColumnWithNameAndType & column = block.getByName(it.first);
const ColumnWithNameAndType & column = block.getByName(it.name);
writeData(column.name, *column.type, *column.column, offset_columns);
}

View File

@ -105,9 +105,7 @@ private:
String column_name = (*name)[i].get<const String &>();
String data_type_name = (*type)[i].get<const String &>();
res.emplace_back(std::piecewise_construct,
std::forward_as_tuple(column_name),
std::forward_as_tuple(context.getDataTypeFactory().get(data_type_name)));
res.emplace_back(column_name, context.getDataTypeFactory().get(data_type_name));
}
}

View File

@ -26,7 +26,7 @@ String chooseSuffix(const NamesAndTypesList & columns, const String & name)
{
bool done = true;
for (const auto & it : columns)
if (it.first == name + current_suffix)
if (it.name == name + current_suffix)
{
done = false;
break;
@ -49,7 +49,7 @@ String chooseSuffixForSet(const NamesAndTypesList & columns, const std::vector<S
{
for (size_t i = 0; i < names.size(); ++i)
{
if (it.first == names[i] + current_suffix)
if (it.name == names[i] + current_suffix)
{
done = false;
break;
@ -153,7 +153,7 @@ BlockInputStreamPtr getVirtualColumnsBlocks(ASTPtr query, const Block & input, c
/// Вычисляем имена виртуальных столбцов
std::vector<String> columns;
for (const auto & it : input.getColumnsList())
columns.push_back(it.first);
columns.push_back(it.name);
/// Формируем запрос и записываем имена виртуальных столбцов
ASTSelectQuery & new_select = typeid_cast<ASTSelectQuery & >(*new_query);

View File

@ -23,13 +23,13 @@ void Block::addDefaults(NamesAndTypesListPtr required_columns)
{
for (NamesAndTypesList::const_iterator it = required_columns->begin(); it != required_columns->end(); ++it)
{
if (!has(it->first))
if (!has(it->name))
{
ColumnWithNameAndType col;
col.name = it->first;
col.type = it->second;
col.column = dynamic_cast<IColumnConst &>(*it->second->createConstColumn(
rows(), it->second->getDefault())).convertToFullColumn();
col.name = it->name;
col.type = it->type;
col.column = dynamic_cast<IColumnConst &>(*it->type->createConstColumn(
rows(), it->type->getDefault())).convertToFullColumn();
insert(col);
}
}

View File

@ -17,7 +17,7 @@ JSONCompactRowOutputStream::JSONCompactRowOutputStream(WriteBuffer & ostr_, cons
void JSONCompactRowOutputStream::writeField(const Field & field)
{
fields[field_number].second->serializeTextJSON(field, ostr);
fields[field_number].type->serializeTextJSON(field, ostr);
++field_number;
}

View File

@ -28,10 +28,10 @@ void JSONRowOutputStream::writePrefix()
writeCString("\t\t{\n", ostr);
writeCString("\t\t\t\"name\": ", ostr);
writeDoubleQuotedString(fields[i].first, ostr);
writeDoubleQuotedString(fields[i].name, ostr);
writeCString(",\n", ostr);
writeCString("\t\t\t\"type\": ", ostr);
writeDoubleQuotedString(fields[i].second->getName(), ostr);
writeDoubleQuotedString(fields[i].type->getName(), ostr);
writeChar('\n', ostr);
writeCString("\t\t}", ostr);
@ -52,9 +52,9 @@ void JSONRowOutputStream::writePrefix()
void JSONRowOutputStream::writeField(const Field & field)
{
writeCString("\t\t\t", ostr);
writeDoubleQuotedString(fields[field_number].first, ostr);
writeDoubleQuotedString(fields[field_number].name, ostr);
writeCString(": ", ostr);
fields[field_number].second->serializeTextJSON(field, ostr);
fields[field_number].type->serializeTextJSON(field, ostr);
++field_number;
}

View File

@ -50,7 +50,7 @@ static void readData(const IDataType & type, IColumn & column, ReadBuffer & istr
for (size_t i = 0; i < column_nested.getData().size(); ++i, ++it)
{
readData(
*it->second,
*it->type,
*column_nested.getData()[i],
istr,
column_nested.getOffsets()[rows - 1]);

View File

@ -39,7 +39,7 @@ static void writeData(const IDataType & type, const IColumn & column, WriteBuffe
{
if (column_nested.getData()[i]->empty())
break;
writeData(*it->second, *column_nested.getData()[i], ostr);
writeData(*it->type, *column_nested.getData()[i], ostr);
}
}
else

View File

@ -91,7 +91,7 @@ int main(int argc, char ** argv)
SharedPtr<DataTypes> data_types = new DataTypes;
for (NamesAndTypesList::const_iterator it = names_and_types_list->begin(); it != names_and_types_list->end(); ++it)
data_types->push_back(it->second);
data_types->push_back(it->type);
/// создаём описание, как читать данные из tab separated дампа
@ -99,8 +99,8 @@ int main(int argc, char ** argv)
for (NamesAndTypesList::const_iterator it = names_and_types_list->begin(); it != names_and_types_list->end(); ++it)
{
ColumnWithNameAndType elem;
elem.name = it->first;
elem.type = it->second;
elem.name = it->name;
elem.type = it->type;
elem.column = elem.type->createColumn();
sample.insert(elem);
}

View File

@ -46,14 +46,14 @@ int main(int argc, char ** argv)
SharedPtr<DataTypes> data_types = new DataTypes;
for (NamesAndTypesList::const_iterator it = names_and_types_list->begin(); it != names_and_types_list->end(); ++it)
data_types->push_back(it->second);
data_types->push_back(it->type);
Block sample;
for (NamesAndTypesList::const_iterator it = names_and_types_list->begin(); it != names_and_types_list->end(); ++it)
{
ColumnWithNameAndType elem;
elem.name = it->first;
elem.type = it->second;
elem.name = it->name;
elem.type = it->type;
elem.column = elem.type->createColumn();
sample.insert(elem);
}

View File

@ -92,7 +92,7 @@ int main(int argc, char ** argv)
Names column_names;
for (NamesAndTypesList::const_iterator it = names_and_types_list->begin(); it != names_and_types_list->end(); ++it)
column_names.push_back(it->first);
column_names.push_back(it->name);
/// создаём объект существующей таблицы хит лога

View File

@ -99,7 +99,7 @@ int main(int argc, char ** argv)
SharedPtr<NamesAndTypesMap> names_and_types_map = new NamesAndTypesMap;
for (NamesAndTypesList::const_iterator it = names_and_types_list->begin(); it != names_and_types_list->end(); ++it)
names_and_types_map->insert(*it);
names_and_types_map->insert(std::make_pair(it->name, it->type));
ParserSelectQuery parser;
ASTPtr ast;

View File

@ -56,9 +56,9 @@ std::string DataTypeNested::getName() const
{
if (it != nested->begin())
writeCString(", ", out);
writeString(it->first, out);
writeString(it->name, out);
writeChar(' ', out);
writeString(it->second->getName(), out);
writeString(it->type->getName(), out);
}
writeChar(')', out);
@ -106,7 +106,7 @@ void DataTypeNested::serializeBinary(const IColumn & column, WriteBuffer & ostr,
{
NamesAndTypesList::const_iterator it = nested->begin();
for (size_t i = 0; i < nested->size(); ++i, ++it)
it->second->serializeBinary(*column_nested.getData()[i], ostr, nested_offset, nested_limit);
it->type->serializeBinary(*column_nested.getData()[i], ostr, nested_offset, nested_limit);
}
}
@ -125,7 +125,7 @@ void DataTypeNested::deserializeBinary(IColumn & column, ReadBuffer & istr, size
NamesAndTypesList::const_iterator it = nested->begin();
for (size_t i = 0; i < nested->size(); ++i, ++it)
{
it->second->deserializeBinary(*column_nested.getData()[i], istr, nested_limit);
it->type->deserializeBinary(*column_nested.getData()[i], istr, nested_limit);
if (column_nested.getData()[i]->size() != last_offset)
throw Exception("Cannot read all nested column values", ErrorCodes::CANNOT_READ_ALL_DATA);
}
@ -225,7 +225,7 @@ ColumnPtr DataTypeNested::createColumn() const
Columns columns;
columns.reserve(nested->size());
for (NamesAndTypesList::const_iterator it = nested->begin(); it != nested->end(); ++it)
columns.push_back(it->second->createColumn());
columns.push_back(it->type->createColumn());
return new ColumnNested(columns);
}
@ -240,13 +240,13 @@ NamesAndTypesListPtr DataTypeNested::expandNestedColumns(const NamesAndTypesList
NamesAndTypesListPtr columns = new 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->second))
if (const DataTypeNested * type_nested = typeid_cast<const DataTypeNested *>(&*it->type))
{
const NamesAndTypesList & nested = *type_nested->getNestedTypesList();
for (NamesAndTypesList::const_iterator jt = nested.begin(); jt != nested.end(); ++jt)
{
String nested_name = DataTypeNested::concatenateNestedName(it->first, jt->first);
columns->push_back(NameAndTypePair(nested_name, new DataTypeArray(jt->second)));
String nested_name = DataTypeNested::concatenateNestedName(it->name, jt->name);
columns->push_back(NameAndTypePair(nested_name, new DataTypeArray(jt->type)));
}
}
else

View File

@ -150,7 +150,7 @@ void ExpressionAction::prepare(Block & sample_block)
else if (type == JOIN)
{
for (const auto & col : columns_added_by_join)
sample_block.insert(ColumnWithNameAndType(col.second->createColumn(), col.second, col.first));
sample_block.insert(ColumnWithNameAndType(col.type->createColumn(), col.type, col.name));
std::cerr << sample_block.dumpNames() << std::endl;
}
@ -342,7 +342,7 @@ std::string ExpressionAction::toString() const
{
if (it != columns_added_by_join.begin())
ss << ", ";
ss << it->first;
ss << it->name;
}
break;
@ -399,7 +399,7 @@ void ExpressionActions::addInput(const ColumnWithNameAndType & column)
void ExpressionActions::addInput(const NameAndTypePair & column)
{
addInput(ColumnWithNameAndType(nullptr, column.second, column.first));
addInput(ColumnWithNameAndType(nullptr, column.type, column.name));
}
void ExpressionActions::add(const ExpressionAction & action, Names & out_new_columns)
@ -453,10 +453,10 @@ void ExpressionActions::prependArrayJoin(const ExpressionAction & action, const
NameSet array_join_set(action.array_joined_columns.begin(), action.array_joined_columns.end());
for (auto & it : input_columns)
{
if (array_join_set.count(it.first))
if (array_join_set.count(it.name))
{
array_join_set.erase(it.first);
it.second = new DataTypeArray(it.second);
array_join_set.erase(it.name);
it.type = new DataTypeArray(it.type);
}
}
for (const std::string & name : array_join_set)
@ -505,15 +505,15 @@ std::string ExpressionActions::getSmallestColumn(const NamesAndTypesList & colum
if (it == columns.end())
throw Exception("No available columns", ErrorCodes::LOGICAL_ERROR);
size_t min_size = it->second->isNumeric() ? it->second->getSizeOfField() : 100;
String res = it->first;
size_t min_size = it->type->isNumeric() ? it->type->getSizeOfField() : 100;
String res = it->name;
for (; it != columns.end(); ++it)
{
size_t current_size = it->second->isNumeric() ? it->second->getSizeOfField() : 100;
size_t current_size = it->type->isNumeric() ? it->type->getSizeOfField() : 100;
if (current_size < min_size)
{
min_size = current_size;
res = it->first;
res = it->name;
}
}
@ -540,7 +540,7 @@ void ExpressionActions::finalize(const Names & output_columns)
{
NamesAndTypesList sample_columns = sample_block.getColumnsList();
for (NamesAndTypesList::iterator it = sample_columns.begin(); it != sample_columns.end(); ++it)
unmodified_columns.insert(it->first);
unmodified_columns.insert(it->name);
}
/// Будем идти с конца и поодерживать множество нужных на данном этапе столбцов.
@ -624,10 +624,10 @@ void ExpressionActions::finalize(const Names & output_columns)
{
NamesAndTypesList::iterator it0 = it;
++it;
if (!needed_columns.count(it0->first))
if (!needed_columns.count(it0->name))
{
if (unmodified_columns.count(it0->first))
sample_block.erase(it0->first);
if (unmodified_columns.count(it0->name))
sample_block.erase(it0->name);
input_columns.erase(it0);
}
}
@ -675,7 +675,7 @@ std::string ExpressionActions::getID() const
{
if (it != output_columns.begin())
ss << ", ";
ss << it->first;
ss << it->name;
}
ss << "}";
@ -688,7 +688,7 @@ std::string ExpressionActions::dumpActions() const
ss << "input:\n";
for (NamesAndTypesList::const_iterator it = input_columns.begin(); it != input_columns.end(); ++it)
ss << it->first << " " << it->second->getName() << "\n";
ss << it->name << " " << it->type->getName() << "\n";
ss << "\nactions:\n";
for (size_t i = 0; i < actions.size(); ++i)
@ -697,7 +697,7 @@ std::string ExpressionActions::dumpActions() const
ss << "\noutput:\n";
NamesAndTypesList output_columns = sample_block.getColumnsList();
for (NamesAndTypesList::const_iterator it = output_columns.begin(); it != output_columns.end(); ++it)
ss << it->first << " " << it->second->getName() << "\n";
ss << it->name << " " << it->type->getName() << "\n";
return ss.str();
}
@ -807,7 +807,7 @@ void ExpressionActionsChain::finalize()
if (i + 1 < static_cast<int>(steps.size()))
{
for (const auto & it : steps[i + 1].actions->getRequiredColumnsWithTypes())
required_output.push_back(it.first);
required_output.push_back(it.name);
}
steps[i].actions->finalize(required_output);
}

View File

@ -104,17 +104,17 @@ void ExpressionAnalyzer::analyzeAggregation()
getRootActions(group_asts[i], true, false, temp_actions);
NameAndTypePair key;
key.first = group_asts[i]->getColumnName();
key.second = temp_actions->getSampleBlock().has(key.first)
? temp_actions->getSampleBlock().getByName(key.first).type
: throw Exception("Unknown identifier (in GROUP BY): " + key.first, ErrorCodes::UNKNOWN_IDENTIFIER);
key.name = group_asts[i]->getColumnName();
key.type = temp_actions->getSampleBlock().has(key.name)
? temp_actions->getSampleBlock().getByName(key.name).type
: throw Exception("Unknown identifier (in GROUP BY): " + key.name, ErrorCodes::UNKNOWN_IDENTIFIER);
aggregation_keys.push_back(key);
if (!unique_keys.count(key.first))
if (!unique_keys.count(key.name))
{
aggregated_columns.push_back(key);
unique_keys.insert(key.first);
unique_keys.insert(key.name);
}
}
}
@ -185,7 +185,7 @@ void ExpressionAnalyzer::findExternalTables(ASTPtr & ast)
NamesAndTypesList::iterator ExpressionAnalyzer::findColumn(const String & name, NamesAndTypesList & cols)
{
return std::find_if(cols.begin(), cols.end(),
[&](const NamesAndTypesList::value_type & val) { return val.first == name; });
[&](const NamesAndTypesList::value_type & val) { return val.name == name; });
}
@ -326,7 +326,7 @@ void ExpressionAnalyzer::normalizeTreeImpl(ASTPtr & ast, MapOfASTs & finished_as
{
ASTs all_columns;
for (const auto & column_name_type : columns)
all_columns.emplace_back(new ASTIdentifier(asterisk->range, column_name_type.first));
all_columns.emplace_back(new ASTIdentifier(asterisk->range, column_name_type.name));
asts.erase(asts.begin() + i);
asts.insert(asts.begin() + i, all_columns.begin(), all_columns.end());
@ -695,9 +695,9 @@ struct ExpressionAnalyzer::ScopeStack
for (NamesAndTypesList::const_iterator it = input_columns.begin(); it != input_columns.end(); ++it)
{
all_columns.emplace_back(nullptr, it->second, it->first);
new_names.insert(it->first);
stack.back().new_columns.insert(it->first);
all_columns.emplace_back(nullptr, it->type, it->name);
new_names.insert(it->name);
stack.back().new_columns.insert(it->name);
}
const Block & prev_sample_block = prev.actions->getSampleBlock();
@ -810,11 +810,11 @@ void ExpressionAnalyzer::getArrayJoinedColumns()
bool found = false;
for (const auto & column_name_type : columns)
{
String table_name = DataTypeNested::extractNestedTableName(column_name_type.first);
String column_name = DataTypeNested::extractNestedColumnName(column_name_type.first);
String table_name = DataTypeNested::extractNestedTableName(column_name_type.name);
String column_name = DataTypeNested::extractNestedColumnName(column_name_type.name);
if (table_name == source_name)
{
array_join_result_to_source[DataTypeNested::concatenateNestedName(result_name, column_name)] = column_name_type.first;
array_join_result_to_source[DataTypeNested::concatenateNestedName(result_name, column_name)] = column_name_type.name;
found = true;
break;
}
@ -870,7 +870,7 @@ void ExpressionAnalyzer::getActionsImpl(ASTPtr ast, bool no_subqueries, bool onl
bool found = false;
for (const auto & column_name_type : columns)
if (column_name_type.first == name)
if (column_name_type.name == name)
found = true;
if (found)
@ -1246,11 +1246,11 @@ bool ExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, bool only_ty
JoinPtr join = new Join(join_key_names_left, join_key_names_right, settings.limits, ast_join.kind, ast_join.strictness);
for (const auto & name_type : columns_added_by_join)
std::cerr << "! Column added by JOIN: " << name_type.first << std::endl;
std::cerr << "! Column added by JOIN: " << name_type.name << std::endl;
Names required_joined_columns(join_key_names_right.begin(), join_key_names_right.end());
for (const auto & name_type : columns_added_by_join)
required_joined_columns.push_back(name_type.first);
required_joined_columns.push_back(name_type.name);
/** Для GLOBAL JOIN-ов происходит следующее:
* - в функции addExternalStorage подзапрос JOIN (SELECT ...) заменяется на JOIN _data1,
@ -1471,7 +1471,7 @@ ExpressionActionsPtr ExpressionAnalyzer::getActions(bool project_result)
{
/// Не будем удалять исходные столбцы.
for (const auto & column_name_type : columns)
result_names.push_back(column_name_type.first);
result_names.push_back(column_name_type.name);
}
actions->finalize(result_names);
@ -1492,7 +1492,7 @@ ExpressionActionsPtr ExpressionAnalyzer::getConstActions()
void ExpressionAnalyzer::getAggregateInfo(Names & key_names, AggregateDescriptions & aggregates)
{
for (NamesAndTypesList::iterator it = aggregation_keys.begin(); it != aggregation_keys.end(); ++it)
key_names.push_back(it->first);
key_names.push_back(it->name);
aggregates = aggregate_descriptions;
}
@ -1539,7 +1539,7 @@ void ExpressionAnalyzer::collectUsedColumns()
for (NamesAndTypesList::iterator it = columns_added_by_join.begin(); it != columns_added_by_join.end();)
{
if (required_joined_columns.count(it->first))
if (required_joined_columns.count(it->name))
++it;
else
columns_added_by_join.erase(it++);
@ -1555,8 +1555,8 @@ void ExpressionAnalyzer::collectUsedColumns()
array_join_sources.insert(result_source.second);
for (const auto & column_name_type : columns)
if (array_join_sources.count(column_name_type.first))
required.insert(column_name_type.first);
if (array_join_sources.count(column_name_type.name))
required.insert(column_name_type.name);
/// Нужно прочитать хоть один столбец, чтобы узнать количество строк.
if (required.empty())
@ -1566,11 +1566,11 @@ void ExpressionAnalyzer::collectUsedColumns()
for (NamesAndTypesList::iterator it = columns.begin(); it != columns.end();)
{
unknown_required_columns.erase(it->first);
unknown_required_columns.erase(it->name);
if (!required.count(it->first))
if (!required.count(it->name))
{
required.erase(it->first);
required.erase(it->name);
columns.erase(it++);
}
else
@ -1643,7 +1643,7 @@ Names ExpressionAnalyzer::getRequiredColumns()
Names res;
for (const auto & column_name_type : columns)
res.push_back(column_name_type.first);
res.push_back(column_name_type.name);
return res;
}

View File

@ -145,8 +145,8 @@ StoragePtr InterpreterCreateQuery::execute(bool assume_metadata_exists)
{
ASTPtr name_and_type_pair_ptr = new ASTNameTypePair;
ASTNameTypePair & name_and_type_pair = typeid_cast<ASTNameTypePair &>(*name_and_type_pair_ptr);
name_and_type_pair.name = it->first;
StringPtr type_name = new String(it->second->getName());
name_and_type_pair.name = it->name;
StringPtr type_name = new String(it->type->getName());
ParserIdentifierWithOptionalParameters storage_p;
Expected expected = "";

View File

@ -159,7 +159,7 @@ DataTypes InterpreterSelectQuery::getReturnTypes()
NamesAndTypesList columns = query_analyzer->getSelectSampleBlock().getColumnsList();
for (NamesAndTypesList::iterator it = columns.begin(); it != columns.end(); ++it)
{
res.push_back(it->second);
res.push_back(it->type);
}
return res;
}

View File

@ -785,9 +785,9 @@ String formatColumnsForCreateQuery(NamesAndTypesList & columns)
{
if (it != columns.begin())
res += ", ";
res += backQuoteIfNeed(it->first);
res += backQuoteIfNeed(it->name);
res += " ";
res += it->second->getName();
res += it->type->getName();
}
res += ")";
return res;

View File

@ -15,7 +15,7 @@ bool ITableDeclaration::hasRealColumn(const String & column_name) const
{
const NamesAndTypesList & real_columns = getColumnsList();
for (auto & it : real_columns)
if (it.first == column_name)
if (it.name == column_name)
return true;
return false;
}
@ -26,7 +26,7 @@ Names ITableDeclaration::getColumnNamesList() const
const NamesAndTypesList & real_columns = getColumnsList();
Names res;
for (auto & it : real_columns)
res.push_back(it.first);
res.push_back(it.name);
return res;
}
@ -35,7 +35,7 @@ NameAndTypePair ITableDeclaration::getRealColumn(const String & column_name) con
{
const NamesAndTypesList & real_columns = getColumnsList();
for (auto & it : real_columns)
if (it.first == column_name)
if (it.name == column_name)
return it;
throw Exception("There is no column " + column_name + " in table.", ErrorCodes::NO_SUCH_COLUMN_IN_TABLE);
}
@ -57,8 +57,8 @@ const DataTypePtr ITableDeclaration::getDataTypeByName(const String & column_nam
{
const NamesAndTypesList & names_and_types = getColumnsList();
for (NamesAndTypesList::const_iterator it = names_and_types.begin(); it != names_and_types.end(); ++it)
if (it->first == column_name)
return it->second;
if (it->name == column_name)
return it->type;
throw Exception("There is no column " + column_name + " in table.", ErrorCodes::NO_SUCH_COLUMN_IN_TABLE);
}
@ -72,8 +72,8 @@ Block ITableDeclaration::getSampleBlock() const
for (NamesAndTypesList::const_iterator it = names_and_types.begin(); it != names_and_types.end(); ++it)
{
ColumnWithNameAndType col;
col.name = it->first;
col.type = it->second;
col.name = it->name;
col.type = it->type;
col.column = col.type->createColumn();
res.insert(col);
}
@ -89,7 +89,7 @@ static std::string listOfColumns(const NamesAndTypesList & available_columns)
{
if (it != available_columns.begin())
s << ", ";
s << it->first;
s << it->name;
}
return s.str();
}
@ -104,7 +104,7 @@ static NamesAndTypesMap getColumnsMap(const NamesAndTypesList & available_column
res.set_empty_key(StringRef());
for (NamesAndTypesList::const_iterator it = available_columns.begin(); it != available_columns.end(); ++it)
res.insert(NamesAndTypesMap::value_type(it->first, &*it->second));
res.insert(NamesAndTypesMap::value_type(it->name, &*it->type));
return res;
}
@ -170,8 +170,8 @@ void ITableDeclaration::check(const Block & block, bool need_all) const
{
for (NamesAndTypesList::const_iterator it = available_columns.begin(); it != available_columns.end(); ++it)
{
if (!names_in_block.count(it->first))
throw Exception("Expected column " + it->first, ErrorCodes::NOT_FOUND_COLUMN_IN_BLOCK);
if (!names_in_block.count(it->name))
throw Exception("Expected column " + it->name, ErrorCodes::NOT_FOUND_COLUMN_IN_BLOCK);
}
}
}
@ -180,7 +180,7 @@ void ITableDeclaration::check(const Block & block, bool need_all) const
static bool namesEqual(const String & name_without_dot, const DB::NameAndTypePair & name_type)
{
String name_with_dot = name_without_dot + ".";
return (name_with_dot == name_type.first.substr(0, name_without_dot.length() + 1) || name_without_dot == name_type.first);
return (name_with_dot == name_type.name.substr(0, name_without_dot.length() + 1) || name_without_dot == name_type.name);
}
void ITableDeclaration::alterColumns(const ASTAlterQuery::Parameters & params, NamesAndTypesListPtr & columns, const Context & context)
@ -250,7 +250,7 @@ void ITableDeclaration::alterColumns(const ASTAlterQuery::Parameters & params, N
NamesAndTypesList::iterator column_it = std::find_if(columns->begin(), columns->end(), boost::bind(namesEqual, ast_name_type.name, _1) );
if (column_it == columns->end())
throw Exception("Wrong column name. Cannot find column " + ast_name_type.name + " to modify.", DB::ErrorCodes::ILLEGAL_COLUMN);
column_it->second = data_type;
column_it->type = data_type;
}
else
throw Exception("Wrong parameter type in ALTER query", ErrorCodes::LOGICAL_ERROR);

View File

@ -397,8 +397,8 @@ static DataTypePtr getDataTypeByName(const String & name, const NamesAndTypesLis
{
for (const auto & it : columns)
{
if (it.first == name)
return it.second;
if (it.name == name)
return it.type;
}
throw Exception("No column " + name + " in table", ErrorCodes::NO_SUCH_COLUMN_IN_TABLE);
}
@ -406,7 +406,7 @@ static DataTypePtr getDataTypeByName(const String & name, const NamesAndTypesLis
/// одинаковыми считаются имена, вида "name.*"
static bool namesWithDotEqual(const String & name_with_dot, const NameAndTypePair & name_type)
{
return (name_with_dot == name_type.first.substr(0, name_with_dot.length()));
return (name_with_dot == name_type.name.substr(0, name_with_dot.length()));
}
void MergeTreeData::alter(const ASTAlterQuery::Parameters & params)

View File

@ -256,7 +256,7 @@ MergeTreeData::DataPartPtr MergeTreeDataMerger::mergeParts(
Names all_column_names;
NamesAndTypesList columns_list = data.getColumnsList();
for (const auto & it : columns_list)
all_column_names.push_back(it.first);
all_column_names.push_back(it.name);
MergeTreeData::MutableDataPartPtr new_data_part = std::make_shared<MergeTreeData::DataPart>(data);
ActiveDataPartSet::parsePartName(merged_name, *new_data_part);

View File

@ -41,7 +41,7 @@ StoragePtr StorageChunkMerger::create(
NameAndTypePair StorageChunkMerger::getColumn(const String &column_name) const
{
if (column_name == _table_column_name) return std::make_pair(_table_column_name, new DataTypeString);
if (column_name == _table_column_name) return NameAndTypePair(_table_column_name, new DataTypeString);
return getRealColumn(column_name);
}
@ -361,7 +361,10 @@ bool StorageChunkMerger::mergeChunks(const Storages & chunks)
}
/// Объединим множества столбцов сливаемых чанков.
ColumnsMap known_columns_types(columns->begin(), columns->end());
ColumnsMap known_columns_types;
for (const NameAndTypePair & column : *columns)
known_columns_types.insert(std::make_pair(column.name, column.type));
NamesAndTypesListPtr required_columns = new NamesAndTypesList;
*required_columns = *columns;
@ -371,8 +374,8 @@ bool StorageChunkMerger::mergeChunks(const Storages & chunks)
for (NamesAndTypesList::const_iterator it = current_columns.begin(); it != current_columns.end(); ++it)
{
const std::string & name = it->first;
const DataTypePtr & type = it->second;
const std::string & name = it->name;
const DataTypePtr & type = it->type;
if (known_columns_types.count(name))
{
String current_type_name = type->getName();
@ -445,8 +448,8 @@ bool StorageChunkMerger::mergeChunks(const Storages & chunks)
select_query->select_expression_list = select_list;
for (NamesAndTypesList::const_iterator it = src_columns.begin(); it != src_columns.end(); ++it)
{
src_column_names.push_back(it->first);
select_list->children.push_back(newIdentifier(it->first, ASTIdentifier::Column));
src_column_names.push_back(it->name);
select_list->children.push_back(newIdentifier(it->name, ASTIdentifier::Column));
}
QueryProcessingStage::Enum processed_stage = QueryProcessingStage::Complete;

View File

@ -168,7 +168,7 @@ StorageChunks::StorageChunks(
NameAndTypePair StorageChunks::getColumn(const String &column_name) const
{
if (column_name == _table_column_name) return std::make_pair(_table_column_name, new DataTypeString);
if (column_name == _table_column_name) return NameAndTypePair(_table_column_name, new DataTypeString);
return getRealColumn(column_name);
}

View File

@ -315,7 +315,7 @@ StoragePtr StorageFactory::get(
}
else if (name == "SystemNumbers")
{
if (columns->size() != 1 || columns->begin()->first != "number" || columns->begin()->second->getName() != "UInt64")
if (columns->size() != 1 || columns->begin()->name != "number" || columns->begin()->type->getName() != "UInt64")
throw Exception("Storage SystemNumbers only allows one column with name 'number' and type 'UInt64'",
ErrorCodes::ILLEGAL_COLUMN);
@ -323,7 +323,7 @@ StoragePtr StorageFactory::get(
}
else if (name == "SystemOne")
{
if (columns->size() != 1 || columns->begin()->first != "dummy" || columns->begin()->second->getName() != "UInt8")
if (columns->size() != 1 || columns->begin()->name != "dummy" || columns->begin()->type->getName() != "UInt8")
throw Exception("Storage SystemOne only allows one column with name 'dummy' and type 'UInt8'",
ErrorCodes::ILLEGAL_COLUMN);

View File

@ -185,7 +185,7 @@ void LogBlockInputStream::addStream(const String & name, const IDataType & type,
const NamesAndTypesList & columns = *type_nested->getNestedTypesList();
for (NamesAndTypesList::const_iterator it = columns.begin(); it != columns.end(); ++it)
addStream(DataTypeNested::concatenateNestedName(name, it->first), *it->second, level + 1);
addStream(DataTypeNested::concatenateNestedName(name, it->name), *it->type, level + 1);
}
else
streams[name].reset(new Stream(
@ -233,8 +233,8 @@ void LogBlockInputStream::readData(const String & name, const IDataType & type,
for (size_t i = 0; i < column_nested.getData().size(); ++i, ++it)
{
readData(
DataTypeNested::concatenateNestedName(name, it->first),
*it->second,
DataTypeNested::concatenateNestedName(name, it->name),
*it->type,
*column_nested.getData()[i],
column_nested.getOffsets()[column.size() - 1],
level + 1);
@ -251,7 +251,7 @@ LogBlockOutputStream::LogBlockOutputStream(StorageLog & storage_)
lock(storage.rwlock), marks_stream(storage.marks_file.path(), 4096, O_APPEND | O_CREAT | O_WRONLY)
{
for (NamesAndTypesList::const_iterator it = storage.columns->begin(); it != storage.columns->end(); ++it)
addStream(it->first, *it->second);
addStream(it->name, *it->type);
}
@ -304,7 +304,7 @@ void LogBlockOutputStream::addStream(const String & name, const IDataType & type
const NamesAndTypesList & columns = *type_nested->getNestedTypesList();
for (NamesAndTypesList::const_iterator it = columns.begin(); it != columns.end(); ++it)
addStream(DataTypeNested::concatenateNestedName(name, it->first), *it->second, level + 1);
addStream(DataTypeNested::concatenateNestedName(name, it->name), *it->type, level + 1);
}
else
streams[name].reset(new Stream(storage.files[name].data_file.path(), storage.max_compress_block_size));
@ -354,8 +354,8 @@ void LogBlockOutputStream::writeData(const String & name, const IDataType & type
for (size_t i = 0; i < column_nested.getData().size(); ++i, ++it)
{
writeData(
DataTypeNested::concatenateNestedName(name, it->first),
*it->second,
DataTypeNested::concatenateNestedName(name, it->name),
*it->type,
*column_nested.getData()[i],
out_marks,
offset_columns,
@ -412,7 +412,7 @@ StorageLog::StorageLog(const std::string & path_, const std::string & name_, Nam
Poco::File(path + escapeForFileName(name) + '/').createDirectories();
for (NamesAndTypesList::const_iterator it = columns->begin(); it != columns->end(); ++it)
addFile(it->first, *it->second);
addFile(it->name, *it->type);
marks_file = Poco::File(path + escapeForFileName(name) + '/' + DBMS_STORAGE_LOG_MARKS_FILE_NAME);
}
@ -459,7 +459,7 @@ void StorageLog::addFile(const String & column_name, const IDataType & type, siz
const NamesAndTypesList & columns = *type_nested->getNestedTypesList();
for (NamesAndTypesList::const_iterator it = columns.begin(); it != columns.end(); ++it)
addFile(DataTypeNested::concatenateNestedName(name, it->first), *it->second, level + 1);
addFile(DataTypeNested::concatenateNestedName(name, it->name), *it->type, level + 1);
}
else
{
@ -544,8 +544,8 @@ void StorageLog::rename(const String & new_path_to_db, const String & new_name)
const Marks & StorageLog::getMarksWithRealRowCount() const
{
const String & column_name = columns->front().first;
const IDataType & column_type = *columns->front().second;
const String & column_name = columns->front().name;
const IDataType & column_type = *columns->front().type;
String file_name;
/** Засечки достаём из первого столбца.

View File

@ -28,7 +28,7 @@ StoragePtr StorageMerge::create(
NameAndTypePair StorageMerge::getColumn(const String &column_name) const
{
if (column_name == _table_column_name) return std::make_pair(_table_column_name, new DataTypeString);
if (column_name == _table_column_name) return NameAndTypePair(_table_column_name, new DataTypeString);
return getRealColumn(column_name);
}

View File

@ -144,9 +144,9 @@ void StorageReplicatedMergeTree::createTable()
WriteBufferFromOStream buf(metadata);
for (auto & it : data.getColumnsList())
{
writeBackQuotedString(it.first, buf);
writeBackQuotedString(it.name, buf);
writeChar(' ', buf);
writeString(it.second->getName(), buf);
writeString(it.type->getName(), buf);
writeChar('\n', buf);
}
buf.next();
@ -185,11 +185,11 @@ void StorageReplicatedMergeTree::checkTableStructure()
{
String name;
readBackQuotedString(name, buf);
if (name != it.first)
throw Exception("Unexpected column name in ZooKeeper: expected " + it.first + ", found " + name,
if (name != it.name)
throw Exception("Unexpected column name in ZooKeeper: expected " + it.name + ", found " + name,
ErrorCodes::UNKNOWN_IDENTIFIER);
assertString(" ", buf);
assertString(it.second->getName(), buf);
assertString(it.type->getName(), buf);
assertString("\n", buf);
}
assertEOF(buf);

View File

@ -127,7 +127,7 @@ void TinyLogBlockInputStream::addStream(const String & name, const IDataType & t
const NamesAndTypesList & columns = *type_nested->getNestedTypesList();
for (NamesAndTypesList::const_iterator it = columns.begin(); it != columns.end(); ++it)
addStream(DataTypeNested::concatenateNestedName(name, it->first), *it->second, level + 1);
addStream(DataTypeNested::concatenateNestedName(name, it->name), *it->type, level + 1);
}
else
streams[name].reset(new Stream(storage.files[name].data_file.path()));
@ -172,8 +172,8 @@ void TinyLogBlockInputStream::readData(const String & name, const IDataType & ty
for (size_t i = 0; i < column_nested.getData().size(); ++i, ++it)
{
readData(
DataTypeNested::concatenateNestedName(name, it->first),
*it->second,
DataTypeNested::concatenateNestedName(name, it->name),
*it->type,
*column_nested.getData()[i],
column_nested.getOffsets()[column.size() - 1],
level + 1);
@ -189,7 +189,7 @@ TinyLogBlockOutputStream::TinyLogBlockOutputStream(StorageTinyLog & storage_)
: storage(storage_)
{
for (NamesAndTypesList::const_iterator it = storage.columns->begin(); it != storage.columns->end(); ++it)
addStream(it->first, *it->second);
addStream(it->name, *it->type);
}
@ -211,7 +211,7 @@ void TinyLogBlockOutputStream::addStream(const String & name, const IDataType &
const NamesAndTypesList & columns = *type_nested->getNestedTypesList();
for (NamesAndTypesList::const_iterator it = columns.begin(); it != columns.end(); ++it)
addStream(DataTypeNested::concatenateNestedName(name, it->first), *it->second, level + 1);
addStream(DataTypeNested::concatenateNestedName(name, it->name), *it->type, level + 1);
}
else
streams[name].reset(new Stream(storage.files[name].data_file.path(), storage.max_compress_block_size));
@ -248,8 +248,8 @@ void TinyLogBlockOutputStream::writeData(const String & name, const IDataType &
for (size_t i = 0; i < column_nested.getData().size(); ++i, ++it)
{
writeData(
DataTypeNested::concatenateNestedName(name, it->first),
*it->second,
DataTypeNested::concatenateNestedName(name, it->name),
*it->type,
*column_nested.getData()[i],
offset_columns,
level + 1);
@ -300,7 +300,7 @@ StorageTinyLog::StorageTinyLog(const std::string & path_, const std::string & na
}
for (NamesAndTypesList::const_iterator it = columns->begin(); it != columns->end(); ++it)
addFile(it->first, *it->second);
addFile(it->name, *it->type);
}
StoragePtr StorageTinyLog::create(const std::string & path_, const std::string & name_, NamesAndTypesListPtr columns_, bool attach, size_t max_compress_block_size_)
@ -341,7 +341,7 @@ void StorageTinyLog::addFile(const String & column_name, const IDataType & type,
const NamesAndTypesList & columns = *type_nested->getNestedTypesList();
for (NamesAndTypesList::const_iterator it = columns.begin(); it != columns.end(); ++it)
addFile(DataTypeNested::concatenateNestedName(name, it->first), *it->second, level + 1);
addFile(DataTypeNested::concatenateNestedName(name, it->name), *it->type, level + 1);
}
else
{

View File

@ -94,8 +94,8 @@ int main(int argc, char ** argv)
for (NamesAndTypesList::const_iterator it = names_and_types_list->begin(); it != names_and_types_list->end(); ++it)
{
data_types->push_back(it->second);
column_names.push_back(it->first);
data_types->push_back(it->type);
column_names.push_back(it->name);
}
/// создаём таблицу хит лога
@ -108,8 +108,8 @@ int main(int argc, char ** argv)
for (NamesAndTypesList::const_iterator it = names_and_types_list->begin(); it != names_and_types_list->end(); ++it)
{
ColumnWithNameAndType elem;
elem.name = it->first;
elem.type = it->second;
elem.name = it->name;
elem.type = it->type;
elem.column = elem.type->createColumn();
sample.insert(elem);
}