mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-17 13:13:36 +00:00
use UUIDs
This commit is contained in:
parent
075edf03ca
commit
ab4d452145
@ -1,21 +1,66 @@
|
||||
#include <Databases/DatabaseAtomic.h>
|
||||
#include <Databases/DatabaseOnDisk.h>
|
||||
#include <Poco/File.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int UNKNOWN_TABLE;
|
||||
extern const int TABLE_ALREADY_EXISTS;
|
||||
}
|
||||
|
||||
DatabaseAtomic::DatabaseAtomic(String name_, String metadata_path_, const Context & context_)
|
||||
: DatabaseOrdinary(name_, metadata_path_, context_)
|
||||
{
|
||||
data_path = "store/";
|
||||
}
|
||||
|
||||
//void DatabaseAtomic::renameTable(const Context & context, const String & table_name, IDatabase & to_database,
|
||||
// const String & to_table_name, TableStructureWriteLockHolder & lock)
|
||||
//{
|
||||
// //TODO
|
||||
// DatabaseOnDisk::renameTable<DatabaseAtomic>(*this, context, table_name, to_database, to_table_name, lock);
|
||||
//}
|
||||
String DatabaseAtomic::getDataPath(const String & table_name) const
|
||||
{
|
||||
auto it = table_name_to_path.find(table_name);
|
||||
if (it == table_name_to_path.end())
|
||||
throw Exception("Table " + table_name + " not found in database " + getDatabaseName(), ErrorCodes::UNKNOWN_TABLE);
|
||||
return data_path + it->second;
|
||||
}
|
||||
|
||||
String DatabaseAtomic::getDataPath(const ASTCreateQuery & query) const
|
||||
{
|
||||
stringToUUID(query.uuid); /// Check UUID is valid
|
||||
const size_t uuid_prefix_len = 3;
|
||||
return data_path + query.uuid.substr(0, uuid_prefix_len) + '/' + query.uuid + '/';
|
||||
}
|
||||
|
||||
void DatabaseAtomic::drop(const Context &)
|
||||
{
|
||||
Poco::File(getMetadataPath()).remove(false);
|
||||
}
|
||||
|
||||
void DatabaseAtomic::createTable(const Context & context, const String & table_name, const StoragePtr & table,
|
||||
const ASTPtr & query)
|
||||
{
|
||||
String relative_table_path = getDataPath(query->as<ASTCreateQuery &>());
|
||||
DatabaseOnDisk::createTable(context, table_name, table, query);
|
||||
|
||||
}
|
||||
|
||||
void DatabaseAtomic::attachTable(const String & name, const StoragePtr & table, const String & relative_table_path)
|
||||
{
|
||||
DatabaseWithDictionaries::attachTable(name, table, relative_table_path);
|
||||
std::lock_guard lock(mutex);
|
||||
table_name_to_path.emplace(std::make_pair(table->getTableName(), relative_table_path));
|
||||
}
|
||||
|
||||
StoragePtr DatabaseAtomic::detachTable(const String & name)
|
||||
{
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
table_name_to_path.erase(name);
|
||||
}
|
||||
return DatabaseWithDictionaries::detachTable(name);
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
@ -7,7 +7,7 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class DatabaseAtomic : /* public DatabaseWithOwnTablesBase */ public DatabaseOrdinary
|
||||
class DatabaseAtomic : public DatabaseOrdinary
|
||||
{
|
||||
public:
|
||||
|
||||
@ -15,12 +15,23 @@ public:
|
||||
|
||||
String getEngineName() const override { return "Atomic"; }
|
||||
|
||||
//void renameTable(const Context & context,
|
||||
// const String & table_name,
|
||||
// IDatabase & to_database,
|
||||
// const String & to_table_name,
|
||||
// TableStructureWriteLockHolder &) override;
|
||||
void createTable(
|
||||
const Context & context,
|
||||
const String & table_name,
|
||||
const StoragePtr & table,
|
||||
const ASTPtr & query) override;
|
||||
|
||||
void attachTable(const String & name, const StoragePtr & table, const String & relative_table_path = {}) override;
|
||||
StoragePtr detachTable(const String & name) override;
|
||||
|
||||
String getDataPath(const String & table_name) const override;
|
||||
String getDataPath(const ASTCreateQuery & query) const override;
|
||||
|
||||
void drop(const Context & /*context*/) override;
|
||||
|
||||
private:
|
||||
//TODO store path in DatabaseWithOwnTables::tables
|
||||
std::map<String, String> table_name_to_path;
|
||||
|
||||
};
|
||||
|
||||
|
@ -160,7 +160,7 @@ bool DatabaseLazy::empty(const Context & /* context */) const
|
||||
return tables_cache.empty();
|
||||
}
|
||||
|
||||
void DatabaseLazy::attachTable(const String & table_name, const StoragePtr & table)
|
||||
void DatabaseLazy::attachTable(const String & table_name, const StoragePtr & table, const String &)
|
||||
{
|
||||
LOG_DEBUG(log, "Attach table " << backQuote(table_name) << ".");
|
||||
std::lock_guard lock(mutex);
|
||||
@ -239,7 +239,7 @@ StoragePtr DatabaseLazy::loadTable(const Context & context, const String & table
|
||||
if (ast)
|
||||
{
|
||||
auto & ast_create = ast->as<const ASTCreateQuery &>();
|
||||
String table_data_path_relative = getDataPath() + escapeForFileName(ast_create.table) + '/';
|
||||
String table_data_path_relative = getDataPath(table_name);
|
||||
table = createTableFromAST(ast_create, database_name, table_data_path_relative, context_copy, false).second;
|
||||
}
|
||||
|
||||
|
@ -66,7 +66,7 @@ public:
|
||||
|
||||
DatabaseTablesIteratorPtr getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_table_name = {}) override;
|
||||
|
||||
void attachTable(const String & table_name, const StoragePtr & table) override;
|
||||
void attachTable(const String & table_name, const StoragePtr & table, const String & relative_table_path = {}) override;
|
||||
|
||||
StoragePtr detachTable(const String & table_name) override;
|
||||
|
||||
|
@ -185,7 +185,7 @@ void DatabaseOnDisk::createTable(
|
||||
try
|
||||
{
|
||||
/// Add a table to the map of known tables.
|
||||
attachTable(table_name, table);
|
||||
attachTable(table_name, table, getDataPath(query->as<ASTCreateQuery &>()));
|
||||
|
||||
/// If it was ATTACH query and file with table metadata already exist
|
||||
/// (so, ATTACH is done after DETACH), then rename atomically replaces old file with new one.
|
||||
@ -200,6 +200,8 @@ void DatabaseOnDisk::createTable(
|
||||
|
||||
void DatabaseOnDisk::removeTable(const Context & /* context */, const String & table_name)
|
||||
{
|
||||
String table_data = getDataPath(table_name);
|
||||
|
||||
StoragePtr res = detachTable(table_name);
|
||||
|
||||
String table_metadata_path = getObjectMetadataPath(table_name);
|
||||
@ -219,7 +221,7 @@ void DatabaseOnDisk::removeTable(const Context & /* context */, const String & t
|
||||
{
|
||||
LOG_WARNING(log, getCurrentExceptionMessage(__PRETTY_FUNCTION__));
|
||||
}
|
||||
attachTable(table_name, res);
|
||||
attachTable(table_name, res, data_path);
|
||||
throw;
|
||||
}
|
||||
}
|
||||
@ -329,7 +331,7 @@ time_t DatabaseOnDisk::getObjectMetadataModificationTime(const String & table_na
|
||||
return static_cast<time_t>(0);
|
||||
}
|
||||
|
||||
void DatabaseOnDisk::iterateMetadataFiles(const Context & context, const IteratingFunction & iterating_function) const
|
||||
void DatabaseOnDisk::iterateMetadataFiles(const Context & /*context*/, const IteratingFunction & iterating_function) const
|
||||
{
|
||||
Poco::DirectoryIterator dir_end;
|
||||
for (Poco::DirectoryIterator dir_it(getMetadataPath()); dir_it != dir_end; ++dir_it)
|
||||
@ -346,17 +348,19 @@ void DatabaseOnDisk::iterateMetadataFiles(const Context & context, const Iterati
|
||||
static const char * tmp_drop_ext = ".sql.tmp_drop";
|
||||
if (endsWith(dir_it.name(), tmp_drop_ext))
|
||||
{
|
||||
const std::string object_name = dir_it.name().substr(0, dir_it.name().size() - strlen(tmp_drop_ext));
|
||||
if (Poco::File(context.getPath() + getDataPath() + '/' + object_name).exists())
|
||||
{
|
||||
Poco::File(dir_it->path()).renameTo(object_name + ".sql");
|
||||
LOG_WARNING(log, "Object " << backQuote(object_name) << " was not dropped previously");
|
||||
}
|
||||
else
|
||||
{
|
||||
LOG_INFO(log, "Removing file " << dir_it->path());
|
||||
Poco::File(dir_it->path()).remove();
|
||||
}
|
||||
//const std::string object_name = dir_it.name().substr(0, dir_it.name().size() - strlen(tmp_drop_ext));
|
||||
|
||||
//FIXME
|
||||
//if (Poco::File(context.getPath() + getDataPath(object_name)).exists())
|
||||
//{
|
||||
// Poco::File(dir_it->path()).renameTo(object_name + ".sql");
|
||||
// LOG_WARNING(log, "Object " << backQuote(object_name) << " was not dropped previously");
|
||||
//}
|
||||
//else
|
||||
//{
|
||||
// LOG_INFO(log, "Removing file " << dir_it->path());
|
||||
// Poco::File(dir_it->path()).remove();
|
||||
//}
|
||||
continue;
|
||||
}
|
||||
|
||||
@ -428,4 +432,14 @@ ASTPtr DatabaseOnDisk::getCreateQueryFromMetadata(const String & database_metada
|
||||
return ast;
|
||||
}
|
||||
|
||||
String DatabaseOnDisk::getDataPath(const String & table_name) const
|
||||
{
|
||||
return data_path + escapeForFileName(table_name) + "/";
|
||||
}
|
||||
|
||||
String DatabaseOnDisk::getDataPath(const ASTCreateQuery & query) const
|
||||
{
|
||||
return getDataPath(query.table);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -63,6 +63,8 @@ public:
|
||||
time_t getObjectMetadataModificationTime(const String & object_name) const override;
|
||||
|
||||
String getDataPath() const override { return data_path; }
|
||||
String getDataPath(const String & table_name) const override;
|
||||
String getDataPath(const ASTCreateQuery & query) const override;
|
||||
String getMetadataPath() const override { return metadata_path; }
|
||||
|
||||
protected:
|
||||
@ -80,7 +82,7 @@ protected:
|
||||
|
||||
|
||||
const String metadata_path;
|
||||
const String data_path;
|
||||
/*const*/ String data_path;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -52,7 +52,7 @@ void loadObject(
|
||||
Context & context,
|
||||
const ASTCreateQuery & query,
|
||||
DatabaseOrdinary & database,
|
||||
const String & database_data_path_relative,
|
||||
const String & /*database_data_path_relative*/,
|
||||
const String & database_name,
|
||||
bool has_force_restore_data_flag)
|
||||
try
|
||||
@ -64,9 +64,11 @@ try
|
||||
}
|
||||
else
|
||||
{
|
||||
String table_data_path_relative = database_data_path_relative + escapeForFileName(query.table) + '/';
|
||||
//FIMXE
|
||||
//String table_data_path_relative = database_data_path_relative + escapeForFileName(query.table) + '/';
|
||||
String table_data_path_relative = database.getDataPath(query);
|
||||
auto [table_name, table] = createTableFromAST(query, database_name, table_data_path_relative, context, has_force_restore_data_flag);
|
||||
database.attachTable(table_name, table);
|
||||
database.attachTable(table_name, table, database.getDataPath(query));
|
||||
}
|
||||
}
|
||||
catch (const Exception & e)
|
||||
@ -141,7 +143,8 @@ void DatabaseOrdinary::loadStoredObjects(
|
||||
|
||||
auto loadOneObject = [&](const ASTCreateQuery & query)
|
||||
{
|
||||
loadObject(context, query, *this, getDataPath(), getDatabaseName(), has_force_restore_data_flag);
|
||||
//FIXME
|
||||
loadObject(context, query, *this, getDataPath(query), getDatabaseName(), has_force_restore_data_flag);
|
||||
|
||||
/// Messages, so that it's not boring to wait for the server to load for a long time.
|
||||
if (query.is_dictionary)
|
||||
|
@ -83,7 +83,7 @@ StoragePtr DatabaseWithOwnTablesBase::detachTable(const String & table_name)
|
||||
return res;
|
||||
}
|
||||
|
||||
void DatabaseWithOwnTablesBase::attachTable(const String & table_name, const StoragePtr & table)
|
||||
void DatabaseWithOwnTablesBase::attachTable(const String & table_name, const StoragePtr & table, const String &)
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
if (!tables.emplace(table_name, table).second)
|
||||
|
@ -30,7 +30,7 @@ public:
|
||||
|
||||
bool empty(const Context & context) const override;
|
||||
|
||||
void attachTable(const String & table_name, const StoragePtr & table) override;
|
||||
void attachTable(const String & table_name, const StoragePtr & table, const String & relative_table_path = {}) override;
|
||||
|
||||
StoragePtr detachTable(const String & table_name) override;
|
||||
|
||||
|
@ -21,6 +21,7 @@ class ColumnsDescription;
|
||||
struct IndicesDescription;
|
||||
struct TableStructureWriteLockHolder;
|
||||
using Dictionaries = std::set<String>;
|
||||
class ASTCreateQuery;
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
@ -184,7 +185,7 @@ public:
|
||||
}
|
||||
|
||||
/// Add a table to the database, but do not add it to the metadata. The database may not support this method.
|
||||
virtual void attachTable(const String & /*name*/, const StoragePtr & /*table*/)
|
||||
virtual void attachTable(const String & /*name*/, const StoragePtr & /*table*/, [[maybe_unused]] const String & relative_table_path = {})
|
||||
{
|
||||
throw Exception("There is no ATTACH TABLE query for Database" + getEngineName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
@ -269,6 +270,10 @@ public:
|
||||
String getDatabaseName() const { return database_name; }
|
||||
/// Returns path for persistent data storage if the database supports it, empty string otherwise
|
||||
virtual String getDataPath() const { return {}; }
|
||||
/// Returns path for persistent data storage for table if the database supports it, empty string otherwise
|
||||
virtual String getDataPath(const String & /*table_name*/) const { return {}; }
|
||||
/// Returns path for persistent data storage for CREATE/ATTACH query if the database supports it, empty string otherwise
|
||||
virtual String getDataPath(const ASTCreateQuery & /*query*/) const { return {}; }
|
||||
/// Returns metadata path if the database supports it, empty string otherwise
|
||||
virtual String getMetadataPath() const { return {}; }
|
||||
/// Returns metadata path of a concrete table if the database supports it, empty string otherwise
|
||||
|
@ -514,6 +514,7 @@ DatabasePtr Context::tryGetDatabase(const String & database_name)
|
||||
|
||||
String Context::getPath() const
|
||||
{
|
||||
//FIXME there is no <path> in clickhouse-local, so context->getPath() + relative_data_path may not work correctly (it may create StorageFile for example)
|
||||
auto lock = getLock();
|
||||
return shared->path;
|
||||
}
|
||||
|
@ -48,6 +48,8 @@
|
||||
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
|
||||
#include <boost/uuid/uuid_generators.hpp>
|
||||
#include <boost/uuid/uuid_io.hpp>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -545,6 +547,9 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create)
|
||||
if (!create.to_table.empty())
|
||||
databases.emplace(create.to_database);
|
||||
|
||||
/// NOTE: if it's CREATE query and create.database is DatabaseAtomic, different UUIDs will be generated on all servers.
|
||||
/// However, it allows to use UUID as replica name.
|
||||
|
||||
return executeDDLQueryOnCluster(query_ptr, context, std::move(databases));
|
||||
}
|
||||
|
||||
@ -587,7 +592,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create)
|
||||
return fillTableIfNeeded(create, database_name);
|
||||
}
|
||||
|
||||
bool InterpreterCreateQuery::doCreateTable(const ASTCreateQuery & create,
|
||||
bool InterpreterCreateQuery::doCreateTable(/*const*/ ASTCreateQuery & create,
|
||||
const InterpreterCreateQuery::TableProperties & properties,
|
||||
const String & database_name)
|
||||
{
|
||||
@ -601,17 +606,33 @@ bool InterpreterCreateQuery::doCreateTable(const ASTCreateQuery & create,
|
||||
if (need_add_to_database)
|
||||
{
|
||||
database = context.getDatabase(database_name);
|
||||
if (!create.uuid.empty() && database->getEngineName() != "Atomic")
|
||||
throw Exception("Table UUID specified, but engine of database " + database_name + " is not Atomic",
|
||||
ErrorCodes::INCORRECT_QUERY);
|
||||
if (database->getEngineName() == "Atomic")
|
||||
{
|
||||
//TODO implement ATTACH FROM 'path/to/data': generate UUID and move table data to store/
|
||||
if (create.attach && create.uuid.empty())
|
||||
throw Exception("UUID must be specified in ATTACH TABLE query for Atomic database engine", ErrorCodes::INCORRECT_QUERY);
|
||||
if (!create.attach && create.uuid.empty())
|
||||
create.uuid = boost::uuids::to_string(boost::uuids::random_generator()());
|
||||
}
|
||||
else
|
||||
{
|
||||
if (!create.uuid.empty())
|
||||
throw Exception("Table UUID specified, but engine of database " + database_name + " is not Atomic", ErrorCodes::INCORRECT_QUERY);
|
||||
}
|
||||
|
||||
data_path = database->getDataPath();
|
||||
if (!create.attach && create.uuid.empty() && database->getEngineName() == "Atomic")
|
||||
create.uuid = boost::uuids::to_string(boost::uuids::random_generator()());
|
||||
|
||||
data_path = database->getDataPath(create);
|
||||
|
||||
/** If the request specifies IF NOT EXISTS, we allow concurrent CREATE queries (which do nothing).
|
||||
* If table doesnt exist, one thread is creating table, while others wait in DDLGuard.
|
||||
*/
|
||||
guard = context.getDDLGuard(database_name, table_name);
|
||||
|
||||
if (!create.attach && !data_path.empty() && Poco::File(context.getPath() + data_path).exists())
|
||||
throw Exception("Directory for table data " + data_path + " already exists", ErrorCodes::TABLE_ALREADY_EXISTS);
|
||||
|
||||
/// Table can be created before or it can be created concurrently in another thread, while we were waiting in DDLGuard.
|
||||
if (database->isTableExist(context, table_name))
|
||||
{
|
||||
@ -646,7 +667,7 @@ bool InterpreterCreateQuery::doCreateTable(const ASTCreateQuery & create,
|
||||
else
|
||||
{
|
||||
res = StorageFactory::instance().get(create,
|
||||
data_path + escapeForFileName(table_name) + "/",
|
||||
data_path,
|
||||
table_name,
|
||||
database_name,
|
||||
context,
|
||||
|
@ -67,7 +67,7 @@ private:
|
||||
void checkAccess(const ASTCreateQuery & create);
|
||||
|
||||
/// Create IStorage and add it to database. If table already exists and IF NOT EXISTS specified, do nothing and return false.
|
||||
bool doCreateTable(const ASTCreateQuery & create, const TableProperties & properties, const String & database_name);
|
||||
bool doCreateTable(/*const*/ ASTCreateQuery & create, const TableProperties & properties, const String & database_name);
|
||||
/// Inserts data in created table if it's CREATE ... SELECT
|
||||
BlockIO fillTableIfNeeded(const ASTCreateQuery & create, const String & database_name);
|
||||
|
||||
|
@ -113,6 +113,7 @@ BlockIO InterpreterDropQuery::executeToTable(
|
||||
const auto drop_metadata_name = metadata_file_without_extension + ".sql.tmp_drop";
|
||||
|
||||
/// Try to rename metadata file and delete the data
|
||||
//TODO move this logic to DatabaseOnDisk
|
||||
try
|
||||
{
|
||||
/// There some kind of tables that have no metadata - ignore renaming
|
||||
@ -128,16 +129,16 @@ BlockIO InterpreterDropQuery::executeToTable(
|
||||
throw;
|
||||
}
|
||||
|
||||
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.second->is_dropped = true;
|
||||
|
||||
String database_data_path = context.getPath() + database_and_table.first->getDataPath();
|
||||
|
||||
/// If it is not virtual database like Dictionary then drop remaining data dir
|
||||
if (!database_data_path.empty())
|
||||
if (!table_data_path.empty())
|
||||
{
|
||||
String table_data_path = database_data_path + "/" + escapeForFileName(database_and_table.second->getTableName());
|
||||
table_data_path = context.getPath() + table_data_path;
|
||||
|
||||
if (Poco::File(table_data_path).exists())
|
||||
Poco::File(table_data_path).remove(true);
|
||||
|
@ -14,6 +14,8 @@
|
||||
#include <Interpreters/InterpreterShowCreateQuery.h>
|
||||
|
||||
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
@ -67,6 +69,10 @@ BlockInputStreamPtr InterpreterShowCreateQuery::executeImpl()
|
||||
if (!create_query && show_query->temporary)
|
||||
throw Exception("Unable to show the create query of " + show_query->table + ". Maybe it was created by the system.", ErrorCodes::THERE_IS_NO_QUERY);
|
||||
|
||||
//FIXME temporary print create query without UUID for tests (remove it)
|
||||
auto & create = create_query->as<ASTCreateQuery &>();
|
||||
create.uuid.clear();
|
||||
|
||||
std::stringstream stream;
|
||||
formatAST(*create_query, stream, false, true);
|
||||
String res = stream.str();
|
||||
|
@ -285,12 +285,12 @@ StoragePtr InterpreterSystemQuery::tryRestartReplica(const String & database_nam
|
||||
auto & create = create_ast->as<ASTCreateQuery &>();
|
||||
create.attach = true;
|
||||
|
||||
std::string data_path = database->getDataPath();
|
||||
std::string data_path = database->getDataPath(create);
|
||||
auto columns = InterpreterCreateQuery::getColumnsDescription(*create.columns_list->columns, system_context);
|
||||
auto constraints = InterpreterCreateQuery::getConstraintsDescription(create.columns_list->constraints);
|
||||
|
||||
StoragePtr table = StorageFactory::instance().get(create,
|
||||
data_path + escapeForFileName(table_name) + "/",
|
||||
data_path,
|
||||
table_name,
|
||||
database_name,
|
||||
system_context,
|
||||
|
@ -233,7 +233,8 @@ void ASTCreateQuery::formatQueryImpl(const FormatSettings & settings, FormatStat
|
||||
<< what << " "
|
||||
<< (if_not_exists ? "IF NOT EXISTS " : "")
|
||||
<< (settings.hilite ? hilite_none : "")
|
||||
<< (!database.empty() ? backQuoteIfNeed(database) + "." : "") << backQuoteIfNeed(table);
|
||||
<< (!database.empty() ? backQuoteIfNeed(database) + "." : "") << backQuoteIfNeed(table)
|
||||
<< (!uuid.empty() ? " UUID " + quoteString(uuid) : "");
|
||||
formatOnCluster(settings);
|
||||
}
|
||||
else
|
||||
|
@ -401,7 +401,6 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe
|
||||
|
||||
if (attach && s_uuid.ignore(pos, expected))
|
||||
{
|
||||
/// For CREATE query uuid will be generated
|
||||
if (!uuid_p.parse(pos, uuid, expected))
|
||||
return false;
|
||||
}
|
||||
@ -505,6 +504,7 @@ bool ParserCreateLiveViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & e
|
||||
ParserKeyword s_temporary("TEMPORARY");
|
||||
ParserKeyword s_attach("ATTACH");
|
||||
ParserKeyword s_if_not_exists("IF NOT EXISTS");
|
||||
ParserKeyword s_uuid("UUID");
|
||||
ParserKeyword s_as("AS");
|
||||
ParserKeyword s_view("VIEW");
|
||||
ParserKeyword s_live("LIVE");
|
||||
@ -515,9 +515,11 @@ bool ParserCreateLiveViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & e
|
||||
ParserIdentifier name_p;
|
||||
ParserTablePropertiesDeclarationList table_properties_p;
|
||||
ParserSelectWithUnionQuery select_p;
|
||||
ParserStringLiteral uuid_p;
|
||||
|
||||
ASTPtr database;
|
||||
ASTPtr table;
|
||||
ASTPtr uuid;
|
||||
ASTPtr columns_list;
|
||||
ASTPtr to_database;
|
||||
ASTPtr to_table;
|
||||
@ -563,6 +565,12 @@ bool ParserCreateLiveViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & e
|
||||
return false;
|
||||
}
|
||||
|
||||
if (attach && s_uuid.ignore(pos, expected))
|
||||
{
|
||||
if (!uuid_p.parse(pos, uuid, expected))
|
||||
return false;
|
||||
}
|
||||
|
||||
if (ParserKeyword{"ON"}.ignore(pos, expected))
|
||||
{
|
||||
if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected))
|
||||
@ -615,6 +623,8 @@ bool ParserCreateLiveViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & e
|
||||
|
||||
tryGetIdentifierNameInto(to_database, query->to_database);
|
||||
tryGetIdentifierNameInto(to_table, query->to_table);
|
||||
if (uuid)
|
||||
query->uuid = uuid->as<ASTLiteral>()->value.get<String>();
|
||||
|
||||
query->set(query->columns_list, columns_list);
|
||||
|
||||
@ -687,6 +697,7 @@ bool ParserCreateViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
|
||||
ParserKeyword s_temporary("TEMPORARY");
|
||||
ParserKeyword s_attach("ATTACH");
|
||||
ParserKeyword s_if_not_exists("IF NOT EXISTS");
|
||||
ParserKeyword s_uuid("UUID");
|
||||
ParserKeyword s_as("AS");
|
||||
ParserKeyword s_view("VIEW");
|
||||
ParserKeyword s_materialized("MATERIALIZED");
|
||||
@ -700,9 +711,11 @@ bool ParserCreateViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
|
||||
ParserTablePropertiesDeclarationList table_properties_p;
|
||||
ParserSelectWithUnionQuery select_p;
|
||||
ParserNameList names_p;
|
||||
ParserStringLiteral uuid_p;
|
||||
|
||||
ASTPtr database;
|
||||
ASTPtr table;
|
||||
ASTPtr uuid;
|
||||
ASTPtr columns_list;
|
||||
ASTPtr to_database;
|
||||
ASTPtr to_table;
|
||||
@ -756,6 +769,12 @@ bool ParserCreateViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
|
||||
return false;
|
||||
}
|
||||
|
||||
if (attach && s_uuid.ignore(pos, expected))
|
||||
{
|
||||
if (!uuid_p.parse(pos, uuid, expected))
|
||||
return false;
|
||||
}
|
||||
|
||||
if (ParserKeyword{"ON"}.ignore(pos, expected))
|
||||
{
|
||||
if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected))
|
||||
@ -816,6 +835,8 @@ bool ParserCreateViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
|
||||
|
||||
tryGetIdentifierNameInto(database, query->database);
|
||||
tryGetIdentifierNameInto(table, query->table);
|
||||
if (uuid)
|
||||
query->uuid = uuid->as<ASTLiteral>()->value.get<String>();
|
||||
query->cluster = cluster_str;
|
||||
|
||||
tryGetIdentifierNameInto(to_database, query->to_database);
|
||||
@ -898,10 +919,8 @@ bool ParserCreateDictionaryQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, E
|
||||
query->is_dictionary = true;
|
||||
query->attach = attach;
|
||||
|
||||
if (database)
|
||||
query->database = typeid_cast<const ASTIdentifier &>(*database).name;
|
||||
|
||||
query->table = typeid_cast<const ASTIdentifier &>(*name).name;
|
||||
tryGetIdentifierNameInto(database, query->database);
|
||||
tryGetIdentifierNameInto(name, query->table);
|
||||
|
||||
query->if_not_exists = if_not_exists;
|
||||
query->set(query->dictionary_attributes_list, attributes);
|
||||
|
@ -312,7 +312,7 @@ protected:
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected);
|
||||
};
|
||||
|
||||
/// CREATE|ATTACH LIVE VIEW [IF NOT EXISTS] [db.]name [TO [db.]name] AS SELECT ...
|
||||
/// CREATE|ATTACH LIVE VIEW [IF NOT EXISTS] [db.]name [UUID 'uuid'] [TO [db.]name] AS SELECT ...
|
||||
class ParserCreateLiveViewQuery : public IParserBase
|
||||
{
|
||||
protected:
|
||||
@ -328,7 +328,7 @@ protected:
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected);
|
||||
};
|
||||
|
||||
/// CREATE[OR REPLACE]|ATTACH [[MATERIALIZED] VIEW] | [VIEW]] [IF NOT EXISTS] [db.]name [TO [db.]name] [ENGINE = engine] [POPULATE] AS SELECT ...
|
||||
/// CREATE[OR REPLACE]|ATTACH [[MATERIALIZED] VIEW] | [VIEW]] [IF NOT EXISTS] [db.]name [UUID 'uuid'] [TO [db.]name] [ENGINE = engine] [POPULATE] AS SELECT ...
|
||||
class ParserCreateViewQuery : public IParserBase
|
||||
{
|
||||
protected:
|
||||
|
@ -214,12 +214,7 @@ StorageStripeLog::StorageStripeLog(
|
||||
throw Exception("Storage " + getName() + " requires data path", ErrorCodes::INCORRECT_FILE_NAME);
|
||||
|
||||
if (!attach)
|
||||
{
|
||||
/// create files if they do not exist
|
||||
if (0 != mkdir(path.c_str(), S_IRWXU | S_IRWXG | S_IRWXO) && errno != EEXIST)
|
||||
throwFromErrnoWithPath("Cannot create directory " + path, path,
|
||||
ErrorCodes::CANNOT_CREATE_DIRECTORY);
|
||||
}
|
||||
Poco::File(path).createDirectories();
|
||||
}
|
||||
|
||||
|
||||
|
@ -343,12 +343,7 @@ StorageTinyLog::StorageTinyLog(
|
||||
throw Exception("Storage " + getName() + " requires data path", ErrorCodes::INCORRECT_FILE_NAME);
|
||||
|
||||
if (!attach)
|
||||
{
|
||||
/// create files if they do not exist
|
||||
if (0 != mkdir(path.c_str(), S_IRWXU | S_IRWXG | S_IRWXO) && errno != EEXIST)
|
||||
throwFromErrnoWithPath("Cannot create directory " + path, path,
|
||||
ErrorCodes::CANNOT_CREATE_DIRECTORY);
|
||||
}
|
||||
Poco::File(path).createDirectories();
|
||||
|
||||
for (const auto & col : getColumns().getAllPhysical())
|
||||
addFiles(col.name, *col.type);
|
||||
|
@ -41,6 +41,7 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
//FIXME cannot store system tables in DatabaseAtomic
|
||||
void attachSystemTablesLocal(IDatabase & system_database)
|
||||
{
|
||||
system_database.attachTable("one", StorageSystemOne::create("one"));
|
||||
|
@ -19,7 +19,7 @@ INSERT INTO set2 VALUES ('abc'), ('World');
|
||||
SELECT arrayJoin(['Hello', 'test', 'World', 'world', 'abc', 'xyz']) AS s WHERE s IN set2;
|
||||
|
||||
DETACH TABLE set2;
|
||||
ATTACH TABLE set2 (x String) ENGINE = Set;
|
||||
ATTACH TABLE set2; -- (x String) ENGINE = Set;
|
||||
|
||||
SELECT arrayJoin(['Hello', 'test', 'World', 'world', 'abc', 'xyz']) AS s WHERE s IN set2;
|
||||
|
||||
|
@ -6,7 +6,8 @@ CREATE TABLE t_00180 (x UInt8) ENGINE = Null;
|
||||
CREATE MATERIALIZED VIEW mv_00180 ENGINE = Null AS SELECT * FROM t_00180;
|
||||
|
||||
DETACH TABLE mv_00180;
|
||||
ATTACH MATERIALIZED VIEW mv_00180 ENGINE = Null AS SELECT * FROM t_00180;
|
||||
--ATTACH MATERIALIZED VIEW mv_00180 ENGINE = Null AS SELECT * FROM t_00180;
|
||||
ATTACH TABLE mv_00180;
|
||||
|
||||
DROP TABLE t_00180;
|
||||
DROP TABLE mv_00180;
|
||||
|
@ -11,7 +11,7 @@ INSERT INTO array_pk VALUES ([5, 6], 'ghi', 6);
|
||||
SELECT * FROM array_pk ORDER BY n;
|
||||
|
||||
DETACH TABLE array_pk;
|
||||
ATTACH TABLE array_pk (key Array(UInt8), s String, n UInt64, d Date MATERIALIZED '2000-01-01') ENGINE = MergeTree(d, (key, s, n), 1);
|
||||
ATTACH TABLE array_pk; -- (key Array(UInt8), s String, n UInt64, d Date MATERIALIZED '2000-01-01') ENGINE = MergeTree(d, (key, s, n), 1);
|
||||
|
||||
SELECT * FROM array_pk ORDER BY n;
|
||||
|
||||
|
@ -5,7 +5,7 @@ SELECT * FROM log LIMIT 1;
|
||||
SELECT * FROM log;
|
||||
|
||||
DETACH TABLE log;
|
||||
ATTACH TABLE log (s String) ENGINE = Log;
|
||||
ATTACH TABLE log; -- (s String) ENGINE = Log;
|
||||
|
||||
SELECT * FROM log;
|
||||
SELECT * FROM log LIMIT 1;
|
||||
@ -15,13 +15,13 @@ INSERT INTO log VALUES ('Hello'), ('World');
|
||||
SELECT * FROM log LIMIT 1;
|
||||
|
||||
DETACH TABLE log;
|
||||
ATTACH TABLE log (s String) ENGINE = Log;
|
||||
ATTACH TABLE log; -- (s String) ENGINE = Log;
|
||||
|
||||
SELECT * FROM log LIMIT 1;
|
||||
SELECT * FROM log;
|
||||
|
||||
DETACH TABLE log;
|
||||
ATTACH TABLE log (s String) ENGINE = Log;
|
||||
ATTACH TABLE log; -- (s String) ENGINE = Log;
|
||||
|
||||
SELECT * FROM log;
|
||||
SELECT * FROM log LIMIT 1;
|
||||
|
Loading…
Reference in New Issue
Block a user