This commit is contained in:
Alexander Tokmakov 2019-12-26 21:51:54 +03:00
parent 3b710feeb6
commit a930697cd7
10 changed files with 5 additions and 53 deletions

View File

@ -979,7 +979,6 @@ StoragePtr Context::getTableImpl(const StorageID & table_id, Exception * excepti
return res; return res;
} }
//FIXME what if table was moved to another database?
db = resolveDatabase(table_id.database_name, current_database); db = resolveDatabase(table_id.database_name, current_database);
checkDatabaseAccessRightsImpl(db); checkDatabaseAccessRightsImpl(db);

View File

@ -124,7 +124,6 @@ using IHostContextPtr = std::shared_ptr<IHostContext>;
* *
* Everything is encapsulated for all sorts of checks and locks. * 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 class Context
{ {
private: private:
@ -249,8 +248,6 @@ public:
ClientInfo & getClientInfo() { return client_info; } ClientInfo & getClientInfo() { return client_info; }
const ClientInfo & getClientInfo() const { 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 addDependency(const StorageID & from, const StorageID & where);
void removeDependency(const StorageID & from, const StorageID & where); void removeDependency(const StorageID & from, const StorageID & where);
Dependencies getDependencies(const StorageID & from) const; Dependencies getDependencies(const StorageID & from) const;

View File

@ -142,7 +142,6 @@ public:
String with_name; String with_name;
/// REPLACE(ATTACH) PARTITION partition FROM db.table /// REPLACE(ATTACH) PARTITION partition FROM db.table
//FIXME use StorageID
String from_database; String from_database;
String from_table; String from_table;
/// To distinguish REPLACE and ATTACH PARTITION partition FROM db.table /// To distinguish REPLACE and ATTACH PARTITION partition FROM db.table

View File

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

View File

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

View File

@ -56,12 +56,6 @@ protected:
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected); 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 * /// Just *
class ParserAsterisk : public IParserBase class ParserAsterisk : public IParserBase
{ {

View File

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

View File

@ -412,7 +412,7 @@ void IStorage::alter(
SettingsChanges new_changes; SettingsChanges new_changes;
params.applyForSettingsOnly(new_changes); params.applyForSettingsOnly(new_changes);
IDatabase::ASTModifier settings_modifier = getSettingsModifier(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 else
{ {
@ -421,7 +421,7 @@ void IStorage::alter(
auto new_indices = getIndices(); auto new_indices = getIndices();
auto new_constraints = getConstraints(); auto new_constraints = getConstraints();
params.applyForColumnsOnly(new_columns); 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)); setColumns(std::move(new_columns));
} }
} }

View File

@ -315,7 +315,7 @@ public:
/** /**
* Just updates names of database and table without moving any data on disk * 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); 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; return false;
// Check all its dependencies // Check all its dependencies
if (!checkDependencies(StorageID(db_tab.database_name, db_tab.table_name))) if (!checkDependencies(db_tab))
return false; return false;
} }
@ -360,7 +360,6 @@ bool StorageKafka::streamToViews()
// Create an INSERT query for streaming data // Create an INSERT query for streaming data
auto insert = std::make_shared<ASTInsertQuery>(); auto insert = std::make_shared<ASTInsertQuery>();
//FIXME use uid if not empty
insert->database = table_id.database_name; insert->database = table_id.database_name;
insert->table = table_id.table_name; insert->table = table_id.table_name;