From 3a4ee4a07c62bac4d36c4684f9cd4c5b31d0b289 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 10 Dec 2019 23:47:05 +0300 Subject: [PATCH] remove separate getTableName() and getDatabaseName() --- dbms/src/Core/iostream_debug_helpers.cpp | 3 ++- .../PushingToViewsBlockOutputStream.cpp | 5 +++-- dbms/src/Interpreters/InterpreterDropQuery.cpp | 7 ++++--- dbms/src/Interpreters/InterpreterSelectQuery.cpp | 4 +++- dbms/src/Interpreters/PartLog.cpp | 7 ++++--- .../Distributed/DistributedBlockOutputStream.cpp | 4 ++-- dbms/src/Storages/IStorage.cpp | 13 ++++++------- dbms/src/Storages/IStorage.h | 12 ------------ dbms/src/Storages/LiveView/ProxyStorage.h | 2 +- dbms/src/Storages/MergeTree/MergeTreeData.cpp | 4 +--- .../MergeTree/StorageFromMergeTreeDataPart.h | 12 +++++++----- dbms/src/Storages/StorageBuffer.cpp | 5 ++--- dbms/src/Storages/StorageDistributed.cpp | 5 ++--- dbms/src/Storages/StorageMerge.cpp | 15 ++++++++++----- dbms/src/Storages/StorageMergeTree.cpp | 7 +++---- dbms/src/Storages/StorageReplicatedMergeTree.cpp | 11 +++++------ .../src/Storages/System/StorageSystemGraphite.cpp | 10 ++++++---- 17 files changed, 61 insertions(+), 65 deletions(-) diff --git a/dbms/src/Core/iostream_debug_helpers.cpp b/dbms/src/Core/iostream_debug_helpers.cpp index eea8694dfb0..07a478961a1 100644 --- a/dbms/src/Core/iostream_debug_helpers.cpp +++ b/dbms/src/Core/iostream_debug_helpers.cpp @@ -47,7 +47,8 @@ std::ostream & operator<<(std::ostream & stream, const IDataType & what) std::ostream & operator<<(std::ostream & stream, const IStorage & what) { - stream << "IStorage(name = " << what.getName() << ", tableName = " << what.getTableName() << ") {" + auto table_id = what.getStorageID(); + stream << "IStorage(name = " << what.getName() << ", tableName = " << table_id.table_name << ") {" << what.getColumns().getAllPhysical().toString() << "}"; return stream; } diff --git a/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp index 1e197d4d182..971c7a28d49 100644 --- a/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -54,10 +54,11 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream( if (auto * materialized_view = dynamic_cast(dependent_table.get())) { StoragePtr inner_table = materialized_view->getTargetTable(); + auto inner_table_id = inner_table->getStorageID(); query = materialized_view->getInnerQuery(); std::unique_ptr insert = std::make_unique(); - insert->database = inner_table->getDatabaseName(); - insert->table = inner_table->getTableName(); + insert->database = inner_table_id.database_name; + insert->table = inner_table_id.table_name; //FIXME add uuid ASTPtr insert_query_ptr(insert.release()); InterpreterInsertQuery interpreter(insert_query_ptr, *views_context); BlockIO io = interpreter.execute(); diff --git a/dbms/src/Interpreters/InterpreterDropQuery.cpp b/dbms/src/Interpreters/InterpreterDropQuery.cpp index 2f7240b5ddc..b9bb2aef4dc 100644 --- a/dbms/src/Interpreters/InterpreterDropQuery.cpp +++ b/dbms/src/Interpreters/InterpreterDropQuery.cpp @@ -79,13 +79,14 @@ BlockIO InterpreterDropQuery::executeToTable( if (database_and_table.first && database_and_table.second) { + auto table_id = database_and_table.second->getStorageID(); if (kind == ASTDropQuery::Kind::Detach) { database_and_table.second->shutdown(); /// If table was already dropped by anyone, an exception will be thrown auto table_lock = database_and_table.second->lockExclusively(context.getCurrentQueryId()); /// Drop table from memory, don't touch data and metadata - database_and_table.first->detachTable(database_and_table.second->getTableName()); + database_and_table.first->detachTable(table_id.table_name); } else if (kind == ASTDropQuery::Kind::Truncate) { @@ -107,7 +108,7 @@ BlockIO InterpreterDropQuery::executeToTable( const std::string metadata_file_without_extension = database_and_table.first->getMetadataPath() - + escapeForFileName(database_and_table.second->getTableName()); + + escapeForFileName(table_id.table_name); const auto prev_metadata_name = metadata_file_without_extension + ".sql"; const auto drop_metadata_name = metadata_file_without_extension + ".sql.tmp_drop"; @@ -132,7 +133,7 @@ BlockIO InterpreterDropQuery::executeToTable( String table_data_path = database_and_table.first->getDataPath(table_name); /// Delete table metadata and table itself from memory - database_and_table.first->removeTable(context, database_and_table.second->getTableName()); + database_and_table.first->removeTable(context, table_id.table_name); database_and_table.second->is_dropped = true; /// If it is not virtual database like Dictionary then drop remaining data dir diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index efc4541c162..5216aaeab34 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -295,7 +295,9 @@ InterpreterSelectQuery::InterpreterSelectQuery( if (auto view_source = context->getViewSource()) { auto & storage_values = static_cast(*view_source); - if (storage_values.getDatabaseName() == database_name && storage_values.getTableName() == table_name) + auto tmp_table_id = storage_values.getStorageID(); + //FIXME maybe add uuid? + if (tmp_table_id.database_name == database_name && tmp_table_id.table_name == table_name) { /// Read from view source. storage = context->getViewSource(); diff --git a/dbms/src/Interpreters/PartLog.cpp b/dbms/src/Interpreters/PartLog.cpp index c860f3212c7..66315dc4eec 100644 --- a/dbms/src/Interpreters/PartLog.cpp +++ b/dbms/src/Interpreters/PartLog.cpp @@ -112,7 +112,8 @@ bool PartLog::addNewParts(Context & current_context, const PartLog::MutableDataP try { - part_log = current_context.getPartLog(parts.front()->storage.getDatabaseName()); // assume parts belong to the same table + auto table_id = parts.front()->storage.getStorageID(); + part_log = current_context.getPartLog(table_id.database_name); // assume parts belong to the same table if (!part_log) return false; @@ -124,8 +125,8 @@ bool PartLog::addNewParts(Context & current_context, const PartLog::MutableDataP elem.event_time = time(nullptr); elem.duration_ms = elapsed_ns / 1000000; - elem.database_name = part->storage.getDatabaseName(); - elem.table_name = part->storage.getTableName(); + elem.database_name = table_id.database_name; + elem.table_name = table_id.table_name; elem.partition_id = part->info.partition_id; elem.part_name = part->name; elem.path_on_disk = part->getFullPath(); diff --git a/dbms/src/Storages/Distributed/DistributedBlockOutputStream.cpp b/dbms/src/Storages/Distributed/DistributedBlockOutputStream.cpp index 61cb10cc38e..1fd5d1c5311 100644 --- a/dbms/src/Storages/Distributed/DistributedBlockOutputStream.cpp +++ b/dbms/src/Storages/Distributed/DistributedBlockOutputStream.cpp @@ -90,7 +90,7 @@ void DistributedBlockOutputStream::write(const Block & block) if (ordinary_block.has(col.name)) { ordinary_block.erase(col.name); - LOG_DEBUG(log, storage.getTableName() + LOG_DEBUG(log, storage.getStorageID().getNameForLogs() << ": column " + col.name + " will be removed, " << "because it is MATERIALIZED"); } @@ -515,7 +515,7 @@ void DistributedBlockOutputStream::writeAsyncImpl(const Block & block, const siz else { if (shard_info.dir_name_for_internal_replication.empty()) - throw Exception("Directory name for async inserts is empty, table " + storage.getTableName(), ErrorCodes::LOGICAL_ERROR); + throw Exception("Directory name for async inserts is empty, table " + storage.getStorageID().getNameForLogs(), ErrorCodes::LOGICAL_ERROR); writeToShard(block, {shard_info.dir_name_for_internal_replication}); } diff --git a/dbms/src/Storages/IStorage.cpp b/dbms/src/Storages/IStorage.cpp index 4ab4e149cdd..a3f97337835 100644 --- a/dbms/src/Storages/IStorage.cpp +++ b/dbms/src/Storages/IStorage.cpp @@ -176,7 +176,7 @@ void IStorage::check(const Names & column_names, bool include_virtuals) const { if (columns_map.end() == columns_map.find(name)) throw Exception( - "There is no column with name " + backQuote(name) + " in table " + getTableName() + ". There are columns: " + list_of_columns, + "There is no column with name " + backQuote(name) + " in table " + getStorageID().getNameForLogs() + ". There are columns: " + list_of_columns, ErrorCodes::NO_SUCH_COLUMN_IN_TABLE); if (unique_names.end() != unique_names.find(name)) @@ -344,7 +344,7 @@ TableStructureWriteLockHolder IStorage::lockAlterIntention(const String & query_ void IStorage::lockNewDataStructureExclusively(TableStructureWriteLockHolder & lock_holder, const String & query_id) { if (!lock_holder.alter_intention_lock) - throw Exception("Alter intention lock for table " + getTableName() + " was not taken. This is a bug.", ErrorCodes::LOGICAL_ERROR); + throw Exception("Alter intention lock for table " + getStorageID().getNameForLogs() + " was not taken. This is a bug.", ErrorCodes::LOGICAL_ERROR); lock_holder.new_data_structure_lock = new_data_structure_lock->getLock(RWLockImpl::Write, query_id); } @@ -352,7 +352,7 @@ void IStorage::lockNewDataStructureExclusively(TableStructureWriteLockHolder & l void IStorage::lockStructureExclusively(TableStructureWriteLockHolder & lock_holder, const String & query_id) { if (!lock_holder.alter_intention_lock) - throw Exception("Alter intention lock for table " + getTableName() + " was not taken. This is a bug.", ErrorCodes::LOGICAL_ERROR); + throw Exception("Alter intention lock for table " + getStorageID().getNameForLogs() + " was not taken. This is a bug.", ErrorCodes::LOGICAL_ERROR); if (!lock_holder.new_data_structure_lock) lock_holder.new_data_structure_lock = new_data_structure_lock->getLock(RWLockImpl::Write, query_id); @@ -405,15 +405,14 @@ void IStorage::alter( if (params.isMutable()) throw Exception("Method alter supports only change comment of column for storage " + getName(), ErrorCodes::NOT_IMPLEMENTED); - const String database_name = getDatabaseName(); - const String table_name = getTableName(); + auto table_id = getStorageID(); if (params.isSettingsAlter()) { SettingsChanges new_changes; params.applyForSettingsOnly(new_changes); IDatabase::ASTModifier settings_modifier = getSettingsModifier(new_changes); - context.getDatabase(database_name)->alterTable(context, table_name, getColumns(), getIndices(), getConstraints(), settings_modifier); + context.getDatabase(table_id.database_name)->alterTable(context, table_id.table_name, getColumns(), getIndices(), getConstraints(), settings_modifier); //FIXME } else { @@ -422,7 +421,7 @@ void IStorage::alter( auto new_indices = getIndices(); auto new_constraints = getConstraints(); params.applyForColumnsOnly(new_columns); - context.getDatabase(database_name)->alterTable(context, table_name, new_columns, new_indices, new_constraints, {}); + context.getDatabase(table_id.database_name)->alterTable(context, table_id.table_name, new_columns, new_indices, new_constraints, {}); //FIXME setColumns(std::move(new_columns)); } } diff --git a/dbms/src/Storages/IStorage.h b/dbms/src/Storages/IStorage.h index f069d50f8b3..ec84547061e 100644 --- a/dbms/src/Storages/IStorage.h +++ b/dbms/src/Storages/IStorage.h @@ -90,18 +90,6 @@ public: /// The name of the table. StorageID getStorageID(std::unique_lock * lock = nullptr) const; - // FIXME remove those methods - std::string getTableName() const - { - std::lock_guard lock(id_mutex); - return id.table_name; - } - std::string getDatabaseName() const - { - std::lock_guard lock(id_mutex); - return id.database_name; - } - /// Returns true if the storage receives data from a remote server or servers. virtual bool isRemote() const { return false; } diff --git a/dbms/src/Storages/LiveView/ProxyStorage.h b/dbms/src/Storages/LiveView/ProxyStorage.h index df8aade6540..ea5d32f4264 100644 --- a/dbms/src/Storages/LiveView/ProxyStorage.h +++ b/dbms/src/Storages/LiveView/ProxyStorage.h @@ -9,7 +9,7 @@ class ProxyStorage : public IStorage { public: ProxyStorage(StoragePtr storage_, BlockInputStreams streams_, QueryProcessingStage::Enum to_stage_) - : IStorage({"", storage_->getTableName()}), storage(std::move(storage_)), streams(std::move(streams_)), to_stage(to_stage_) {} + : IStorage({"", storage_->getStorageID().table_name}), storage(std::move(storage_)), streams(std::move(streams_)), to_stage(to_stage_) {} public: std::string getName() const override { return "ProxyStorage(" + storage->getName() + ")"; } diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index 4072ca706ed..d9db7755a05 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -3355,9 +3355,7 @@ void MergeTreeData::freezePartitionsByMatcher(MatcherFn matcher, const String & String part_absolute_path = Poco::Path(part->getFullPath()).absolute().toString(); String backup_part_absolute_path = backup_path - + "data/" - + escapeForFileName(getDatabaseName()) + "/" - + escapeForFileName(getTableName()) + "/" + + relative_data_path + part->relative_path; localBackup(part_absolute_path, backup_part_absolute_path); part->is_frozen.store(true, std::memory_order_relaxed); diff --git a/dbms/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h b/dbms/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h index e04749e86cc..1326c61b1f6 100644 --- a/dbms/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h +++ b/dbms/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h @@ -49,11 +49,7 @@ public: protected: StorageFromMergeTreeDataPart(const MergeTreeData::DataPartPtr & part_) - : IStorage({ - part_->storage.getDatabaseName(), - part_->storage.getTableName() + " (part " + part_->name + ")" - } - , part_->storage.getVirtuals()) + : IStorage(getIDFromPart(part_), part_->storage.getVirtuals()) , part(part_) { setColumns(part_->storage.getColumns()); @@ -62,6 +58,12 @@ protected: private: MergeTreeData::DataPartPtr part; + + static StorageID getIDFromPart(const MergeTreeData::DataPartPtr & part_) + { + auto table_id = part_->storage.getStorageID(); + return StorageID(table_id.database_name, table_id.table_name + " (part " + part_->name + ")"); + } }; } diff --git a/dbms/src/Storages/StorageBuffer.cpp b/dbms/src/Storages/StorageBuffer.cpp index cbdd4236071..9bff477b762 100644 --- a/dbms/src/Storages/StorageBuffer.cpp +++ b/dbms/src/Storages/StorageBuffer.cpp @@ -710,8 +710,7 @@ void StorageBuffer::alter(const AlterCommands & params, const Context & context, { lockStructureExclusively(table_lock_holder, context.getCurrentQueryId()); - const String database_name_ = getDatabaseName(); - const String table_name_ = getTableName(); + auto table_id = getStorageID(); /// So that no blocks of the old structure remain. optimize({} /*query*/, {} /*partition_id*/, false /*final*/, false /*deduplicate*/, context); @@ -720,7 +719,7 @@ void StorageBuffer::alter(const AlterCommands & params, const Context & context, auto new_indices = getIndices(); auto new_constraints = getConstraints(); params.applyForColumnsOnly(new_columns); - context.getDatabase(database_name_)->alterTable(context, table_name_, new_columns, new_indices, new_constraints, {}); + context.getDatabase(table_id.database_name)->alterTable(context, table_id.table_name, new_columns, new_indices, new_constraints, {}); //FIXME setColumns(std::move(new_columns)); } diff --git a/dbms/src/Storages/StorageDistributed.cpp b/dbms/src/Storages/StorageDistributed.cpp index a47afda5ecf..4af368cbd43 100644 --- a/dbms/src/Storages/StorageDistributed.cpp +++ b/dbms/src/Storages/StorageDistributed.cpp @@ -402,14 +402,13 @@ void StorageDistributed::alter( { lockStructureExclusively(table_lock_holder, context.getCurrentQueryId()); - const String current_database_name = getDatabaseName(); - const String current_table_name = getTableName(); + auto table_id = getStorageID(); auto new_columns = getColumns(); auto new_indices = getIndices(); auto new_constraints = getConstraints(); params.applyForColumnsOnly(new_columns); - context.getDatabase(current_database_name)->alterTable(context, current_table_name, new_columns, new_indices, new_constraints, {}); + context.getDatabase(table_id.database_name)->alterTable(context, table_id.table_name, new_columns, new_indices, new_constraints, {}); setColumns(std::move(new_columns)); } diff --git a/dbms/src/Storages/StorageMerge.cpp b/dbms/src/Storages/StorageMerge.cpp index 8c955e27ace..427af8c6bd0 100644 --- a/dbms/src/Storages/StorageMerge.cpp +++ b/dbms/src/Storages/StorageMerge.cpp @@ -263,7 +263,10 @@ BlockInputStreams StorageMerge::createSourceStreams(const SelectQueryInfo & quer SelectQueryInfo modified_query_info = query_info; modified_query_info.query = query_info.query->clone(); - VirtualColumnUtils::rewriteEntityInAst(modified_query_info.query, "_table", storage ? storage->getTableName() : ""); + StorageID table_id; + if (storage) + table_id = storage->getStorageID(); + VirtualColumnUtils::rewriteEntityInAst(modified_query_info.query, "_table", table_id.table_name); if (!storage) return BlockInputStreams{ @@ -283,7 +286,7 @@ BlockInputStreams StorageMerge::createSourceStreams(const SelectQueryInfo & quer } else if (processed_stage > storage->getQueryProcessingStage(modified_context)) { - modified_query_info.query->as()->replaceDatabaseAndTable(source_database, storage->getTableName()); + modified_query_info.query->as()->replaceDatabaseAndTable(source_database, table_id.table_name); /// Maximum permissible parallelism is streams_num modified_context.getSettingsRef().max_threads = UInt64(streams_num); @@ -314,7 +317,7 @@ BlockInputStreams StorageMerge::createSourceStreams(const SelectQueryInfo & quer { if (has_table_virtual_column) source_stream = std::make_shared>( - source_stream, std::make_shared(), storage->getTableName(), "_table"); + source_stream, std::make_shared(), table_id.table_name, "_table"); /// Subordinary tables could have different but convertible types, like numeric types of different width. /// We must return streams with structure equals to structure of Merge table. @@ -363,7 +366,8 @@ StorageMerge::StorageListWithLocks StorageMerge::getSelectedTables(const ASTPtr if (storage.get() != this) { selected_tables.emplace_back(storage, get_lock ? storage->lockStructureForShare(false, query_id) : TableStructureReadLockHolder{}); - virtual_column->insert(storage->getTableName()); + auto table_id = storage->getStorageID(); + virtual_column->insert(table_id.table_name); } iterator->next(); @@ -376,7 +380,8 @@ StorageMerge::StorageListWithLocks StorageMerge::getSelectedTables(const ASTPtr auto values = VirtualColumnUtils::extractSingleValueFromBlock(virtual_columns_block, "_table"); /// Remove unused tables from the list - selected_tables.remove_if([&] (const auto & elem) { return values.find(elem.first->getTableName()) == values.end(); }); + //FIXME table name can be changed, use StorageID + selected_tables.remove_if([&] (const auto & elem) { return values.find(elem.first->getStorageID().table_name) == values.end(); }); } return selected_tables; diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index 14189f6ef6f..c9f685e75e2 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -260,8 +260,7 @@ void StorageMergeTree::alter( const Context & context, TableStructureWriteLockHolder & table_lock_holder) { - const String current_database_name = getDatabaseName(); - const String current_table_name = getTableName(); + auto table_id = getStorageID(); if (!params.isMutable()) { @@ -279,7 +278,7 @@ void StorageMergeTree::alter( changeSettings(new_changes, table_lock_holder); IDatabase::ASTModifier settings_modifier = getSettingsModifier(new_changes); - context.getDatabase(current_database_name)->alterTable(context, current_table_name, new_columns, new_indices, new_constraints, settings_modifier); + context.getDatabase(table_id.database_name)->alterTable(context, table_id.table_name, new_columns, new_indices, new_constraints, settings_modifier); setColumns(std::move(new_columns)); return; } @@ -326,7 +325,7 @@ void StorageMergeTree::alter( changeSettings(new_changes, table_lock_holder); - context.getDatabase(current_database_name)->alterTable(context, current_table_name, new_columns, new_indices, new_constraints, storage_modifier); + context.getDatabase(table_id.database_name)->alterTable(context, table_id.table_name, new_columns, new_indices, new_constraints, storage_modifier); /// Reinitialize primary key because primary key column types might have changed. diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index ea437a73b88..761c05cbc73 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -3173,8 +3173,7 @@ void StorageReplicatedMergeTree::alter( LOG_DEBUG(log, "Doing ALTER"); - const String current_database_name = getDatabaseName(); - const String current_table_name = getTableName(); + auto table_id = getStorageID(); if (params.isSettingsAlter()) { @@ -3187,8 +3186,8 @@ void StorageReplicatedMergeTree::alter( changeSettings(new_changes, table_lock_holder); IDatabase::ASTModifier settings_modifier = getSettingsModifier(new_changes); - global_context.getDatabase(current_database_name)->alterTable( - query_context, current_table_name, getColumns(), getIndices(), getConstraints(), settings_modifier); + global_context.getDatabase(table_id.database_name)->alterTable( + query_context, table_id.table_name, getColumns(), getIndices(), getConstraints(), settings_modifier); return; } @@ -3268,8 +3267,8 @@ void StorageReplicatedMergeTree::alter( changeSettings(new_changes, table_lock_holder); - global_context.getDatabase(current_database_name)->alterTable( - query_context, current_table_name, getColumns(), getIndices(), getConstraints(), settings_modifier); + global_context.getDatabase(table_id.database_name)->alterTable( + query_context, table_id.table_name, getColumns(), getIndices(), getConstraints(), settings_modifier); } diff --git a/dbms/src/Storages/System/StorageSystemGraphite.cpp b/dbms/src/Storages/System/StorageSystemGraphite.cpp index 36839e06196..c622cfbe235 100644 --- a/dbms/src/Storages/System/StorageSystemGraphite.cpp +++ b/dbms/src/Storages/System/StorageSystemGraphite.cpp @@ -48,20 +48,22 @@ StorageSystemGraphite::Configs StorageSystemGraphite::getConfigs(const Context & { const String & config_name = table_data->merging_params.graphite_params.config_name; + auto table_id = table_data->getStorageID(); if (!graphite_configs.count(config_name)) { + //TODO add uuid Config new_config = { table_data->merging_params.graphite_params, - { table_data->getDatabaseName() }, - { table_data->getTableName() }, + { table_id.database_name }, + { table_id.database_name }, }; graphite_configs.emplace(config_name, new_config); } else { - graphite_configs[config_name].databases.emplace_back(table_data->getDatabaseName()); - graphite_configs[config_name].tables.emplace_back(table_data->getTableName()); + graphite_configs[config_name].databases.emplace_back(table_id.database_name); + graphite_configs[config_name].tables.emplace_back(table_id.database_name); } } }