This commit is contained in:
Alexander Tokmakov 2019-12-12 15:30:31 +03:00
parent 42296cc80a
commit 8987b027ba
22 changed files with 27 additions and 78 deletions

View File

@ -58,7 +58,7 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream(
query = materialized_view->getInnerQuery();
std::unique_ptr<ASTInsertQuery> insert = std::make_unique<ASTInsertQuery>();
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();

View File

@ -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);

View File

@ -124,7 +124,6 @@ using IHostContextPtr = std::shared_ptr<IHostContext>;
*
* 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;

View File

@ -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

View File

@ -4,7 +4,6 @@
#include <Parsers/ASTQueryWithOnCluster.h>
#include <Parsers/ASTDictionary.h>
#include <Parsers/ASTDictionaryAttributeDeclaration.h>
#include <Storages/StorageID.h>
namespace DB

View File

@ -30,7 +30,7 @@
#include <Parsers/ParserCreateQuery.h>
#include <Parsers/queryToString.h>
#include <Storages/StorageID.h>
#include <boost/algorithm/string.hpp>
#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<ASTLiteral>()->value.get<String>() : "";
return true;
}
bool ParserFunction::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserIdentifier id_parser;

View File

@ -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
{

View File

@ -13,7 +13,6 @@
#include <Parsers/ASTConstraintDeclaration.h>
#include <Parsers/ParserDictionary.h>
#include <Parsers/ParserDictionaryAttributeDeclaration.h>
#include <Storages/StorageID.h>
namespace DB

View File

@ -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<std::mutex> * 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;
}
}

View File

@ -77,7 +77,7 @@ class IStorage : public std::enable_shared_from_this<IStorage>, 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<std::mutex> * id_lock = nullptr);

View File

@ -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<ASTInsertQuery>();
//FIXME use uid if not empty
insert->database = table_id.database_name;
insert->table = table_id.table_name;

View File

@ -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");

View File

@ -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;

View File

@ -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));
}

View File

@ -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");
}
}

View File

@ -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<ASTSelectWithUnionQuery>())
@ -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

View File

@ -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);
}
}
}

View File

@ -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`;

View File

@ -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

View File

@ -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;

View File

@ -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,7 +17,7 @@ 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;

View File

@ -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,