diff --git a/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp index 971c7a28d49..f13219dc454 100644 --- a/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -58,7 +58,7 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream( query = materialized_view->getInnerQuery(); std::unique_ptr insert = std::make_unique(); insert->database = inner_table_id.database_name; - insert->table = inner_table_id.table_name; //FIXME add uuid + insert->table = inner_table_id.table_name; ASTPtr insert_query_ptr(insert.release()); InterpreterInsertQuery interpreter(insert_query_ptr, *views_context); BlockIO io = interpreter.execute(); diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index 48c193a34e9..4ffc712f6f0 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -979,7 +979,6 @@ StoragePtr Context::getTableImpl(const StorageID & table_id, Exception * excepti return res; } - //FIXME what if table was moved to another database? db = resolveDatabase(table_id.database_name, current_database); checkDatabaseAccessRightsImpl(db); diff --git a/dbms/src/Interpreters/Context.h b/dbms/src/Interpreters/Context.h index 53d77adf94e..11af06f2b09 100644 --- a/dbms/src/Interpreters/Context.h +++ b/dbms/src/Interpreters/Context.h @@ -124,7 +124,6 @@ using IHostContextPtr = std::shared_ptr; * * Everything is encapsulated for all sorts of checks and locks. */ -///TODO remove syntax sugar and legacy methods from Context (e.g. getInputFormat(...) which just returns object from factory) class Context { private: @@ -249,8 +248,6 @@ public: ClientInfo & getClientInfo() { return client_info; } const ClientInfo & getClientInfo() const { return client_info; } - void setQuota(const String & name, const String & quota_key, const String & user_name, const Poco::Net::IPAddress & address); - void addDependency(const StorageID & from, const StorageID & where); void removeDependency(const StorageID & from, const StorageID & where); Dependencies getDependencies(const StorageID & from) const; diff --git a/dbms/src/Parsers/ASTAlterQuery.h b/dbms/src/Parsers/ASTAlterQuery.h index 2262624a1a4..02e76d5555c 100644 --- a/dbms/src/Parsers/ASTAlterQuery.h +++ b/dbms/src/Parsers/ASTAlterQuery.h @@ -142,7 +142,6 @@ public: String with_name; /// REPLACE(ATTACH) PARTITION partition FROM db.table - //FIXME use StorageID String from_database; String from_table; /// To distinguish REPLACE and ATTACH PARTITION partition FROM db.table diff --git a/dbms/src/Parsers/ASTCreateQuery.h b/dbms/src/Parsers/ASTCreateQuery.h index 3beab213161..41847c28b1e 100644 --- a/dbms/src/Parsers/ASTCreateQuery.h +++ b/dbms/src/Parsers/ASTCreateQuery.h @@ -4,7 +4,6 @@ #include #include #include -#include namespace DB diff --git a/dbms/src/Parsers/ExpressionElementParsers.cpp b/dbms/src/Parsers/ExpressionElementParsers.cpp index 4ab56d3e83a..eb77d77a5c8 100644 --- a/dbms/src/Parsers/ExpressionElementParsers.cpp +++ b/dbms/src/Parsers/ExpressionElementParsers.cpp @@ -30,7 +30,7 @@ #include #include -#include +#include #include "ASTColumnsMatcher.h" @@ -198,40 +198,6 @@ bool ParserCompoundIdentifier::parseImpl(Pos & pos, ASTPtr & node, Expected & ex } -bool parseStorageID(IParser::Pos & pos, StorageID & res, Expected & expected) -{ - ParserKeyword s_uuid("UUID"); - ParserIdentifier name_p; - ParserStringLiteral uuid_p; - ParserToken s_dot(TokenType::Dot); - - ASTPtr database; - ASTPtr table; - ASTPtr uuid; - - if (!name_p.parse(pos, table, expected)) - return false; - - if (s_dot.ignore(pos, expected)) - { - database = table; - if (!name_p.parse(pos, table, expected)) - return false; - } - - if (s_uuid.ignore(pos, expected)) - { - if (!uuid_p.parse(pos, uuid, expected)) - return false; - } - - tryGetIdentifierNameInto(database, res.database_name); - tryGetIdentifierNameInto(table, res.table_name); - res.uuid = uuid ? uuid->as()->value.get() : ""; - return true; -} - - bool ParserFunction::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { ParserIdentifier id_parser; diff --git a/dbms/src/Parsers/ExpressionElementParsers.h b/dbms/src/Parsers/ExpressionElementParsers.h index 9a1e090d8d6..fbcaeeb3d45 100644 --- a/dbms/src/Parsers/ExpressionElementParsers.h +++ b/dbms/src/Parsers/ExpressionElementParsers.h @@ -56,12 +56,6 @@ protected: bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected); }; -struct StorageID; -/// Table name, possibly with database name and UUID as string literal -/// [db_name.]table_name [UUID 'xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx'] -//TODO replace with class -bool parseStorageID(IParser::Pos & pos, StorageID & res, Expected & expected); - /// Just * class ParserAsterisk : public IParserBase { diff --git a/dbms/src/Parsers/ParserCreateQuery.cpp b/dbms/src/Parsers/ParserCreateQuery.cpp index db2b42f5ea5..43e5d274994 100644 --- a/dbms/src/Parsers/ParserCreateQuery.cpp +++ b/dbms/src/Parsers/ParserCreateQuery.cpp @@ -13,7 +13,6 @@ #include #include #include -#include namespace DB diff --git a/dbms/src/Storages/IStorage.cpp b/dbms/src/Storages/IStorage.cpp index cc572226a2d..3faee4d841a 100644 --- a/dbms/src/Storages/IStorage.cpp +++ b/dbms/src/Storages/IStorage.cpp @@ -29,7 +29,7 @@ namespace ErrorCodes extern const int TABLE_IS_DROPPED; } -IStorage::IStorage(StorageID id_, ColumnsDescription virtuals_) : id(std::move(id_)), virtuals(std::move(virtuals_)) +IStorage::IStorage(StorageID storage_id_, ColumnsDescription virtuals_) : storage_id(std::move(storage_id_)), virtuals(std::move(virtuals_)) { } @@ -412,7 +412,7 @@ void IStorage::alter( SettingsChanges new_changes; params.applyForSettingsOnly(new_changes); IDatabase::ASTModifier settings_modifier = getSettingsModifier(new_changes); - context.getDatabase(table_id.database_name)->alterTable(context, table_id.table_name, getColumns(), getIndices(), getConstraints(), settings_modifier); //FIXME + context.getDatabase(table_id.database_name)->alterTable(context, table_id.table_name, getColumns(), getIndices(), getConstraints(), settings_modifier); } else { @@ -421,7 +421,7 @@ void IStorage::alter( auto new_indices = getIndices(); auto new_constraints = getConstraints(); params.applyForColumnsOnly(new_columns); - context.getDatabase(table_id.database_name)->alterTable(context, table_id.table_name, new_columns, new_indices, new_constraints, {}); //FIXME + context.getDatabase(table_id.database_name)->alterTable(context, table_id.table_name, new_columns, new_indices, new_constraints, {}); setColumns(std::move(new_columns)); } } @@ -469,7 +469,7 @@ StorageID IStorage::getStorageID(std::unique_lock * id_lock) const lock = std::unique_lock(id_mutex); else if (!*id_lock) *id_lock = std::unique_lock(id_mutex); - return id; + return storage_id; } void IStorage::renameInMemory(const String & new_database_name, const String & new_table_name, @@ -480,8 +480,8 @@ void IStorage::renameInMemory(const String & new_database_name, const String & n lock = std::unique_lock(id_mutex); else if (!*id_lock) *id_lock = std::unique_lock(id_mutex); - id.database_name = new_database_name; - id.table_name = new_table_name; + storage_id.database_name = new_database_name; + storage_id.table_name = new_table_name; } } diff --git a/dbms/src/Storages/IStorage.h b/dbms/src/Storages/IStorage.h index 559c903822e..5fc164c77b2 100644 --- a/dbms/src/Storages/IStorage.h +++ b/dbms/src/Storages/IStorage.h @@ -77,7 +77,7 @@ class IStorage : public std::enable_shared_from_this, public TypePromo { public: IStorage() = delete; - explicit IStorage(StorageID id_) : id(std::move(id_)) {} + explicit IStorage(StorageID storage_id_) : storage_id(std::move(storage_id_)) {} IStorage(StorageID id_, ColumnsDescription virtuals_); virtual ~IStorage() = default; @@ -167,7 +167,7 @@ protected: /// still thread-unsafe part. IDatabase::ASTModifier getSettingsModifier(const SettingsChanges & new_changes) const; private: - StorageID id; + StorageID storage_id; mutable std::mutex id_mutex; ColumnsDescription columns; /// combined real and virtual columns const ColumnsDescription virtuals = {}; @@ -315,7 +315,7 @@ public: /** * Just updates names of database and table without moving any data on disk - * Can be called only from DatabaseAtomic. + * Can be called directly only from DatabaseAtomic. */ virtual void renameInMemory(const String & new_database_name, const String & new_table_name, std::unique_lock * id_lock = nullptr); diff --git a/dbms/src/Storages/Kafka/StorageKafka.cpp b/dbms/src/Storages/Kafka/StorageKafka.cpp index 2a11e14a602..56cf62ce342 100644 --- a/dbms/src/Storages/Kafka/StorageKafka.cpp +++ b/dbms/src/Storages/Kafka/StorageKafka.cpp @@ -312,7 +312,7 @@ bool StorageKafka::checkDependencies(const StorageID & table_id) return false; // Check all its dependencies - if (!checkDependencies(StorageID(db_tab.database_name, db_tab.table_name))) + if (!checkDependencies(db_tab)) return false; } @@ -360,7 +360,6 @@ bool StorageKafka::streamToViews() // Create an INSERT query for streaming data auto insert = std::make_shared(); - //FIXME use uid if not empty insert->database = table_id.database_name; insert->table = table_id.table_name; diff --git a/dbms/src/Storages/LiveView/StorageLiveView.cpp b/dbms/src/Storages/LiveView/StorageLiveView.cpp index ea89a21ca68..fd98dd243cd 100644 --- a/dbms/src/Storages/LiveView/StorageLiveView.cpp +++ b/dbms/src/Storages/LiveView/StorageLiveView.cpp @@ -177,7 +177,6 @@ StorageLiveView::StorageLiveView( select_table_id = extractDependentTableFromSelectQuery(select_query, global_context, true); /// If the table is not specified - use the table `system.one` - //FIXME why? if (select_table_id.empty()) select_table_id = StorageID("system", "one"); diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index 7dae60a78f4..6e1d62f20c9 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -1152,7 +1152,7 @@ void MergeTreeData::removePartsFinally(const MergeTreeData::DataPartsVector & pa /// NOTE: There is no need to log parts deletion somewhere else, all deleting parts pass through this function and pass away auto table_id = getStorageID(); - if (auto part_log = global_context.getPartLog(table_id.database_name)) //FIXME + if (auto part_log = global_context.getPartLog(table_id.database_name)) { PartLogElement part_log_elem; diff --git a/dbms/src/Storages/StorageBuffer.cpp b/dbms/src/Storages/StorageBuffer.cpp index 10d667115f6..462fc58c0c2 100644 --- a/dbms/src/Storages/StorageBuffer.cpp +++ b/dbms/src/Storages/StorageBuffer.cpp @@ -722,7 +722,7 @@ void StorageBuffer::alter(const AlterCommands & params, const Context & context, auto new_indices = getIndices(); auto new_constraints = getConstraints(); params.applyForColumnsOnly(new_columns); - context.getDatabase(table_id.database_name)->alterTable(context, table_id.table_name, new_columns, new_indices, new_constraints, {}); //FIXME + 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/StorageDistributed.cpp b/dbms/src/Storages/StorageDistributed.cpp index 4346dfa395d..4ced2694361 100644 --- a/dbms/src/Storages/StorageDistributed.cpp +++ b/dbms/src/Storages/StorageDistributed.cpp @@ -345,18 +345,18 @@ BlockInputStreams StorageDistributed::read( if (has_sharding_key) { auto smaller_cluster = skipUnusedShards(cluster, query_info); - auto storage_id = getStorageID(); + auto table_id = getStorageID(); if (smaller_cluster) { cluster = smaller_cluster; - LOG_DEBUG(log, "Reading from " << storage_id.getNameForLogs() << ": " + LOG_DEBUG(log, "Reading from " << table_id.getNameForLogs() << ": " "Skipping irrelevant shards - the query will be sent to the following shards of the cluster (shard numbers): " " " << makeFormattedListOfShards(cluster)); } else { - LOG_DEBUG(log, "Reading from " << storage_id.getNameForLogs() << ": " + LOG_DEBUG(log, "Reading from " << table_id.getNameForLogs() << ": " "Unable to figure out irrelevant shards from WHERE/PREWHERE clauses - the query will be sent to all shards of the cluster"); } } diff --git a/dbms/src/Storages/StorageMaterializedView.cpp b/dbms/src/Storages/StorageMaterializedView.cpp index e31a64ba596..6b34ac62ab8 100644 --- a/dbms/src/Storages/StorageMaterializedView.cpp +++ b/dbms/src/Storages/StorageMaterializedView.cpp @@ -47,11 +47,10 @@ StorageID extractDependentTableFromSelectQuery(ASTSelectQuery & query, Context & ASTPtr subquery = extractTableExpression(query, 0); if (!db_and_table && !subquery) - return {}; //FIXME in which cases we cannot get table name? + return {}; if (db_and_table) { - //TODO uuid return StorageID(db_and_table->database, db_and_table->table/*, db_and_table->uuid*/); } else if (auto * ast_select = subquery->as()) @@ -126,7 +125,6 @@ StorageMaterializedView::StorageMaterializedView( else if (attach_) { /// If there is an ATTACH request, then the internal table must already be created. - //TODO use uuid target_table_id = StorageID(table_id_.database_name, generateInnerTableName(table_id_.table_name)); } else diff --git a/dbms/src/Storages/System/StorageSystemGraphite.cpp b/dbms/src/Storages/System/StorageSystemGraphite.cpp index c622cfbe235..0af21f4dbc2 100644 --- a/dbms/src/Storages/System/StorageSystemGraphite.cpp +++ b/dbms/src/Storages/System/StorageSystemGraphite.cpp @@ -56,14 +56,14 @@ StorageSystemGraphite::Configs StorageSystemGraphite::getConfigs(const Context & { table_data->merging_params.graphite_params, { table_id.database_name }, - { table_id.database_name }, + { table_id.table_name }, }; graphite_configs.emplace(config_name, new_config); } else { graphite_configs[config_name].databases.emplace_back(table_id.database_name); - graphite_configs[config_name].tables.emplace_back(table_id.database_name); + graphite_configs[config_name].tables.emplace_back(table_id.table_name); } } } diff --git a/dbms/tests/integration/test_filesystem_layout/test.py b/dbms/tests/integration/test_filesystem_layout/test.py index 16d63cbf2b7..83389b3d9bd 100644 --- a/dbms/tests/integration/test_filesystem_layout/test.py +++ b/dbms/tests/integration/test_filesystem_layout/test.py @@ -16,7 +16,7 @@ def started_cluster(): def test_file_path_escaping(started_cluster): - node.query('CREATE DATABASE IF NOT EXISTS test') + node.query('CREATE DATABASE IF NOT EXISTS test ENGINE = Ordinary') node.query(''' CREATE TABLE test.`T.a_b,l-e!` (`~Id` UInt32) ENGINE = MergeTree() PARTITION BY `~Id` ORDER BY `~Id`; diff --git a/dbms/tests/integration/test_partition/test.py b/dbms/tests/integration/test_partition/test.py index 659b9a89069..21b1e55f666 100644 --- a/dbms/tests/integration/test_partition/test.py +++ b/dbms/tests/integration/test_partition/test.py @@ -14,7 +14,7 @@ path_to_data = '/var/lib/clickhouse/' def started_cluster(): try: cluster.start() - q('CREATE DATABASE test') + q('CREATE DATABASE test ENGINE = Ordinary') yield cluster diff --git a/dbms/tests/queries/0_stateless/00226_zookeeper_deduplication_and_unexpected_parts.sql b/dbms/tests/queries/0_stateless/00226_zookeeper_deduplication_and_unexpected_parts.sql index 6b29d0a8cd3..1abe7d54610 100644 --- a/dbms/tests/queries/0_stateless/00226_zookeeper_deduplication_and_unexpected_parts.sql +++ b/dbms/tests/queries/0_stateless/00226_zookeeper_deduplication_and_unexpected_parts.sql @@ -21,7 +21,7 @@ INSERT INTO deduplication (x) VALUES (1); SELECT * FROM deduplication; DETACH TABLE deduplication; -ATTACH TABLE deduplication (d Date DEFAULT '2015-01-01', x Int8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/deduplication', 'r1', d, x, 1); +ATTACH TABLE deduplication;-- (d Date DEFAULT '2015-01-01', x Int8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/deduplication', 'r1', d, x, 1); SELECT * FROM deduplication; diff --git a/dbms/tests/queries/0_stateless/00611_zookeeper_different_checksums_formats.sql b/dbms/tests/queries/0_stateless/00611_zookeeper_different_checksums_formats.sql index 3fbb0d400f3..4f0dfb54906 100644 --- a/dbms/tests/queries/0_stateless/00611_zookeeper_different_checksums_formats.sql +++ b/dbms/tests/queries/0_stateless/00611_zookeeper_different_checksums_formats.sql @@ -1,7 +1,7 @@ DROP TABLE IF EXISTS table_old; DROP TABLE IF EXISTS table_new; -CREATE TABLE table_old (k UInt64, d Array(String)) ENGINE = ReplicatedMergeTree('/clickhouse/test/tables/checksums_test', 'old') ORDER BY k SETTINGS use_minimalistic_checksums_in_zookeeper=0; +CREATE TABLE table_old UUID '00000611-abcd-4000-8000-123456789ab6c' (k UInt64, d Array(String)) ENGINE = ReplicatedMergeTree('/clickhouse/test/tables/checksums_test', 'old') ORDER BY k SETTINGS use_minimalistic_checksums_in_zookeeper=0; CREATE TABLE table_new (k UInt64, d Array(String)) ENGINE = ReplicatedMergeTree('/clickhouse/test/tables/checksums_test', 'new') ORDER BY k SETTINGS use_minimalistic_checksums_in_zookeeper=1; SET insert_quorum=2; @@ -17,8 +17,8 @@ SELECT * FROM table_new ORDER BY k; SELECT 'DETACH'; DETACH TABLE table_old; -ATTACH TABLE table_old (k UInt64, d Array(String)) ENGINE = ReplicatedMergeTree('/clickhouse/test/tables/checksums_test', 'old') ORDER BY k SETTINGS use_minimalistic_checksums_in_zookeeper=1; +ATTACH TABLE table_old UUID '00000611-abcd-4000-8000-123456789ab6c' (k UInt64, d Array(String)) ENGINE = ReplicatedMergeTree('/clickhouse/test/tables/checksums_test', 'old') ORDER BY k SETTINGS use_minimalistic_checksums_in_zookeeper=1; SELECT * FROM table_old ORDER BY k; DROP TABLE IF EXISTS table_old; -DROP TABLE IF EXISTS table_new; \ No newline at end of file +DROP TABLE IF EXISTS table_new; diff --git a/dbms/tests/queries/1_stateful/00065_loyalty_with_storage_join.sql b/dbms/tests/queries/1_stateful/00065_loyalty_with_storage_join.sql index d3e73faa7be..b03013c11fe 100644 --- a/dbms/tests/queries/1_stateful/00065_loyalty_with_storage_join.sql +++ b/dbms/tests/queries/1_stateful/00065_loyalty_with_storage_join.sql @@ -24,7 +24,7 @@ GROUP BY loyalty ORDER BY loyalty ASC; DETACH TABLE join; -ATTACH TABLE join (UserID UInt64, loyalty Int8) ENGINE = Join(ANY, INNER, UserID); +ATTACH TABLE join;-- (UserID UInt64, loyalty Int8) ENGINE = Join(ANY, INNER, UserID); SELECT loyalty,