2019-10-02 10:10:45 +00:00
|
|
|
#include <Databases/DatabaseOnDisk.h>
|
2019-10-03 09:31:59 +00:00
|
|
|
|
2019-10-02 10:10:45 +00:00
|
|
|
#include <IO/ReadBufferFromFile.h>
|
|
|
|
#include <IO/ReadHelpers.h>
|
|
|
|
#include <IO/WriteBufferFromFile.h>
|
|
|
|
#include <IO/WriteHelpers.h>
|
2019-10-10 17:33:01 +00:00
|
|
|
#include <Interpreters/Context.h>
|
|
|
|
#include <Interpreters/InterpreterCreateQuery.h>
|
2019-10-02 10:10:45 +00:00
|
|
|
#include <Parsers/ASTCreateQuery.h>
|
2019-10-03 09:31:59 +00:00
|
|
|
#include <Parsers/ParserCreateQuery.h>
|
2019-10-10 17:33:01 +00:00
|
|
|
#include <Parsers/formatAST.h>
|
|
|
|
#include <Parsers/parseQuery.h>
|
2019-10-02 10:10:45 +00:00
|
|
|
#include <Storages/IStorage.h>
|
2019-10-10 17:33:01 +00:00
|
|
|
#include <Storages/StorageFactory.h>
|
|
|
|
#include <TableFunctions/TableFunctionFactory.h>
|
|
|
|
#include <Common/escapeForFileName.h>
|
2019-10-02 10:10:45 +00:00
|
|
|
|
|
|
|
#include <common/logger_useful.h>
|
2019-10-03 09:31:59 +00:00
|
|
|
#include <Poco/DirectoryIterator.h>
|
2019-10-02 10:10:45 +00:00
|
|
|
|
2019-11-26 19:06:19 +00:00
|
|
|
#include <Databases/DatabaseOrdinary.h>
|
|
|
|
#include <Databases/DatabaseAtomic.h>
|
2020-07-06 08:30:11 +00:00
|
|
|
#include <Common/assert_cast.h>
|
2020-03-16 11:38:50 +00:00
|
|
|
|
2019-10-10 17:33:01 +00:00
|
|
|
|
2019-10-02 10:10:45 +00:00
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
2019-10-10 17:33:01 +00:00
|
|
|
static constexpr size_t METADATA_FILE_BUFFER_SIZE = 32768;
|
|
|
|
|
2019-10-02 10:10:45 +00:00
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
2020-02-25 18:02:41 +00:00
|
|
|
extern const int CANNOT_GET_CREATE_TABLE_QUERY;
|
|
|
|
extern const int NOT_IMPLEMENTED;
|
|
|
|
extern const int LOGICAL_ERROR;
|
2019-10-03 08:27:43 +00:00
|
|
|
extern const int FILE_DOESNT_EXIST;
|
|
|
|
extern const int INCORRECT_FILE_NAME;
|
2019-10-02 10:10:45 +00:00
|
|
|
extern const int SYNTAX_ERROR;
|
2019-10-03 08:27:43 +00:00
|
|
|
extern const int TABLE_ALREADY_EXISTS;
|
2019-10-10 17:33:01 +00:00
|
|
|
extern const int DICTIONARY_ALREADY_EXISTS;
|
|
|
|
extern const int EMPTY_LIST_OF_COLUMNS_PASSED;
|
2019-10-02 10:10:45 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2019-10-10 17:33:01 +00:00
|
|
|
std::pair<String, StoragePtr> createTableFromAST(
|
|
|
|
ASTCreateQuery ast_create_query,
|
|
|
|
const String & database_name,
|
2019-11-01 12:47:55 +00:00
|
|
|
const String & table_data_path_relative,
|
2019-10-10 17:33:01 +00:00
|
|
|
Context & context,
|
|
|
|
bool has_force_restore_data_flag)
|
|
|
|
{
|
|
|
|
ast_create_query.attach = true;
|
|
|
|
ast_create_query.database = database_name;
|
|
|
|
|
|
|
|
if (ast_create_query.as_table_function)
|
|
|
|
{
|
|
|
|
const auto & table_function = ast_create_query.as_table_function->as<ASTFunction &>();
|
|
|
|
const auto & factory = TableFunctionFactory::instance();
|
|
|
|
StoragePtr storage = factory.get(table_function.name, context)->execute(ast_create_query.as_table_function, context, ast_create_query.table);
|
2020-04-07 14:05:51 +00:00
|
|
|
storage->renameInMemory(ast_create_query);
|
2019-10-10 17:33:01 +00:00
|
|
|
return {ast_create_query.table, storage};
|
|
|
|
}
|
|
|
|
/// We do not directly use `InterpreterCreateQuery::execute`, because
|
2019-12-02 19:11:18 +00:00
|
|
|
/// - the database has not been loaded yet;
|
2019-10-10 17:33:01 +00:00
|
|
|
/// - the code is simpler, since the query is already brought to a suitable form.
|
|
|
|
if (!ast_create_query.columns_list || !ast_create_query.columns_list->columns)
|
|
|
|
throw Exception("Missing definition of columns.", ErrorCodes::EMPTY_LIST_OF_COLUMNS_PASSED);
|
|
|
|
|
2020-05-04 00:11:49 +00:00
|
|
|
ColumnsDescription columns = InterpreterCreateQuery::getColumnsDescription(*ast_create_query.columns_list->columns, context, false);
|
2019-10-10 17:33:01 +00:00
|
|
|
ConstraintsDescription constraints = InterpreterCreateQuery::getConstraintsDescription(ast_create_query.columns_list->constraints);
|
|
|
|
|
|
|
|
return
|
|
|
|
{
|
|
|
|
ast_create_query.table,
|
|
|
|
StorageFactory::instance().get(
|
|
|
|
ast_create_query,
|
2019-12-04 16:06:55 +00:00
|
|
|
table_data_path_relative,
|
|
|
|
context,
|
|
|
|
context.getGlobalContext(),
|
|
|
|
columns,
|
|
|
|
constraints,
|
|
|
|
has_force_restore_data_flag)
|
2019-10-10 17:33:01 +00:00
|
|
|
};
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
String getObjectDefinitionFromCreateQuery(const ASTPtr & query)
|
|
|
|
{
|
|
|
|
ASTPtr query_clone = query->clone();
|
2019-10-22 10:47:43 +00:00
|
|
|
auto * create = query_clone->as<ASTCreateQuery>();
|
2019-10-10 17:33:01 +00:00
|
|
|
|
2019-10-22 10:47:43 +00:00
|
|
|
if (!create)
|
|
|
|
{
|
|
|
|
std::ostringstream query_stream;
|
2019-11-07 08:52:22 +00:00
|
|
|
formatAST(*query, query_stream, true);
|
2019-10-22 10:47:43 +00:00
|
|
|
throw Exception("Query '" + query_stream.str() + "' is not CREATE query", ErrorCodes::LOGICAL_ERROR);
|
|
|
|
}
|
|
|
|
|
|
|
|
if (!create->is_dictionary)
|
|
|
|
create->attach = true;
|
2019-10-11 15:22:30 +00:00
|
|
|
|
2019-10-10 17:33:01 +00:00
|
|
|
/// We remove everything that is not needed for ATTACH from the query.
|
2019-10-22 10:47:43 +00:00
|
|
|
create->database.clear();
|
|
|
|
create->as_database.clear();
|
|
|
|
create->as_table.clear();
|
|
|
|
create->if_not_exists = false;
|
|
|
|
create->is_populate = false;
|
|
|
|
create->replace_view = false;
|
2019-10-10 17:33:01 +00:00
|
|
|
|
|
|
|
/// For views it is necessary to save the SELECT query itself, for the rest - on the contrary
|
2019-10-22 10:47:43 +00:00
|
|
|
if (!create->is_view && !create->is_materialized_view && !create->is_live_view)
|
|
|
|
create->select = nullptr;
|
2019-10-10 17:33:01 +00:00
|
|
|
|
2019-10-22 10:47:43 +00:00
|
|
|
create->format = nullptr;
|
|
|
|
create->out_file = nullptr;
|
2019-10-10 17:33:01 +00:00
|
|
|
|
2020-01-17 16:07:20 +00:00
|
|
|
if (create->uuid != UUIDHelpers::Nil)
|
2019-12-02 19:11:18 +00:00
|
|
|
create->table = TABLE_WITH_UUID_NAME_PLACEHOLDER;
|
|
|
|
|
2019-10-10 17:33:01 +00:00
|
|
|
std::ostringstream statement_stream;
|
2019-10-22 10:47:43 +00:00
|
|
|
formatAST(*create, statement_stream, false);
|
2019-10-10 17:33:01 +00:00
|
|
|
statement_stream << '\n';
|
|
|
|
return statement_stream.str();
|
|
|
|
}
|
|
|
|
|
2020-06-11 03:24:52 +00:00
|
|
|
DatabaseOnDisk::DatabaseOnDisk(
|
|
|
|
const String & name,
|
|
|
|
const String & metadata_path_,
|
|
|
|
const String & data_path_,
|
|
|
|
const String & logger,
|
|
|
|
const Context & context)
|
2020-05-28 20:10:45 +00:00
|
|
|
: DatabaseWithOwnTablesBase(name, logger, context)
|
2019-11-28 19:40:51 +00:00
|
|
|
, metadata_path(metadata_path_)
|
2020-04-06 23:22:44 +00:00
|
|
|
, data_path(data_path_)
|
2019-11-28 19:40:51 +00:00
|
|
|
{
|
2020-04-06 23:22:44 +00:00
|
|
|
Poco::File(context.getPath() + data_path).createDirectories();
|
|
|
|
Poco::File(metadata_path).createDirectories();
|
2019-11-28 19:40:51 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2019-10-03 07:59:48 +00:00
|
|
|
void DatabaseOnDisk::createTable(
|
|
|
|
const Context & context,
|
|
|
|
const String & table_name,
|
|
|
|
const StoragePtr & table,
|
|
|
|
const ASTPtr & query)
|
|
|
|
{
|
|
|
|
const auto & settings = context.getSettingsRef();
|
2020-03-23 00:12:13 +00:00
|
|
|
const auto & create = query->as<ASTCreateQuery &>();
|
2020-07-07 12:11:58 +00:00
|
|
|
assert(table_name == create.table);
|
2019-10-03 07:59:48 +00:00
|
|
|
|
|
|
|
/// Create a file with metadata if necessary - if the query is not ATTACH.
|
|
|
|
/// Write the query of `ATTACH table` to it.
|
|
|
|
|
|
|
|
/** The code is based on the assumption that all threads share the same order of operations
|
|
|
|
* - creating the .sql.tmp file;
|
|
|
|
* - adding a table to `tables`;
|
|
|
|
* - rename .sql.tmp to .sql.
|
|
|
|
*/
|
|
|
|
|
|
|
|
/// A race condition would be possible if a table with the same name is simultaneously created using CREATE and using ATTACH.
|
|
|
|
/// But there is protection from it - see using DDLGuard in InterpreterCreateQuery.
|
|
|
|
|
2020-04-23 16:51:48 +00:00
|
|
|
if (isDictionaryExist(table_name))
|
2019-11-06 16:05:04 +00:00
|
|
|
throw Exception("Dictionary " + backQuote(getDatabaseName()) + "." + backQuote(table_name) + " already exists.",
|
2019-10-10 17:33:01 +00:00
|
|
|
ErrorCodes::DICTIONARY_ALREADY_EXISTS);
|
|
|
|
|
2020-05-28 20:10:45 +00:00
|
|
|
if (isTableExist(table_name, global_context))
|
2019-11-06 16:05:04 +00:00
|
|
|
throw Exception("Table " + backQuote(getDatabaseName()) + "." + backQuote(table_name) + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS);
|
2019-10-03 07:59:48 +00:00
|
|
|
|
2020-03-23 00:12:13 +00:00
|
|
|
if (create.attach_short_syntax)
|
|
|
|
{
|
|
|
|
/// Metadata already exists, table was detached
|
|
|
|
attachTable(table_name, table, getTableDataPath(create));
|
|
|
|
return;
|
|
|
|
}
|
|
|
|
|
2019-11-06 16:05:04 +00:00
|
|
|
String table_metadata_path = getObjectMetadataPath(table_name);
|
2020-01-22 11:30:11 +00:00
|
|
|
String table_metadata_tmp_path = table_metadata_path + create_suffix;
|
2019-10-03 07:59:48 +00:00
|
|
|
String statement;
|
|
|
|
|
|
|
|
{
|
2019-10-10 17:33:01 +00:00
|
|
|
statement = getObjectDefinitionFromCreateQuery(query);
|
2019-10-03 07:59:48 +00:00
|
|
|
|
|
|
|
/// Exclusive flags guarantees, that table is not created right now in another thread. Otherwise, exception will be thrown.
|
|
|
|
WriteBufferFromFile out(table_metadata_tmp_path, statement.size(), O_WRONLY | O_CREAT | O_EXCL);
|
|
|
|
writeString(statement, out);
|
|
|
|
out.next();
|
|
|
|
if (settings.fsync_metadata)
|
|
|
|
out.sync();
|
|
|
|
out.close();
|
|
|
|
}
|
|
|
|
|
2020-03-23 00:12:13 +00:00
|
|
|
commitCreateTable(create, table, table_metadata_tmp_path, table_metadata_path);
|
|
|
|
}
|
|
|
|
|
|
|
|
void DatabaseOnDisk::commitCreateTable(const ASTCreateQuery & query, const StoragePtr & table,
|
|
|
|
const String & table_metadata_tmp_path, const String & table_metadata_path)
|
|
|
|
{
|
2019-10-03 07:59:48 +00:00
|
|
|
try
|
|
|
|
{
|
|
|
|
/// Add a table to the map of known tables.
|
2020-03-23 00:12:13 +00:00
|
|
|
attachTable(query.table, table, getTableDataPath(query));
|
2019-10-03 07:59:48 +00:00
|
|
|
|
|
|
|
/// 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.
|
|
|
|
Poco::File(table_metadata_tmp_path).renameTo(table_metadata_path);
|
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
|
|
|
Poco::File(table_metadata_tmp_path).remove();
|
|
|
|
throw;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-03-20 00:07:52 +00:00
|
|
|
void DatabaseOnDisk::dropTable(const Context & context, const String & table_name, bool /*no_delay*/)
|
2019-10-03 07:59:48 +00:00
|
|
|
{
|
2019-11-06 16:05:04 +00:00
|
|
|
String table_metadata_path = getObjectMetadataPath(table_name);
|
2020-01-22 11:30:11 +00:00
|
|
|
String table_metadata_path_drop = table_metadata_path + drop_suffix;
|
|
|
|
String table_data_path_relative = getTableDataPath(table_name);
|
2020-07-16 21:41:26 +00:00
|
|
|
if (table_data_path_relative.empty())
|
|
|
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "Path is empty");
|
2019-10-03 07:59:48 +00:00
|
|
|
|
2020-01-22 11:30:11 +00:00
|
|
|
StoragePtr table = detachTable(table_name);
|
|
|
|
bool renamed = false;
|
2019-10-03 07:59:48 +00:00
|
|
|
try
|
|
|
|
{
|
2020-01-22 11:30:11 +00:00
|
|
|
Poco::File(table_metadata_path).renameTo(table_metadata_path_drop);
|
|
|
|
renamed = true;
|
|
|
|
table->drop();
|
|
|
|
table->is_dropped = true;
|
|
|
|
|
|
|
|
Poco::File table_data_dir{context.getPath() + table_data_path_relative};
|
|
|
|
if (table_data_dir.exists())
|
|
|
|
table_data_dir.remove(true);
|
2019-10-03 07:59:48 +00:00
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
2020-05-23 22:24:01 +00:00
|
|
|
LOG_WARNING(log, getCurrentExceptionMessage(__PRETTY_FUNCTION__));
|
2020-01-22 11:30:11 +00:00
|
|
|
attachTable(table_name, table, table_data_path_relative);
|
|
|
|
if (renamed)
|
|
|
|
Poco::File(table_metadata_path_drop).renameTo(table_metadata_path);
|
2019-10-03 07:59:48 +00:00
|
|
|
throw;
|
|
|
|
}
|
2020-01-22 11:30:11 +00:00
|
|
|
|
|
|
|
Poco::File(table_metadata_path_drop).remove();
|
2019-10-03 07:59:48 +00:00
|
|
|
}
|
|
|
|
|
2019-11-06 16:05:04 +00:00
|
|
|
void DatabaseOnDisk::renameTable(
|
|
|
|
const Context & context,
|
|
|
|
const String & table_name,
|
|
|
|
IDatabase & to_database,
|
2020-03-31 20:38:05 +00:00
|
|
|
const String & to_table_name,
|
2020-07-16 21:41:26 +00:00
|
|
|
bool exchange,
|
|
|
|
bool dictionary)
|
2019-10-10 17:33:01 +00:00
|
|
|
{
|
2020-03-31 20:38:05 +00:00
|
|
|
if (exchange)
|
|
|
|
throw Exception("Tables can be exchanged only in Atomic databases", ErrorCodes::NOT_IMPLEMENTED);
|
2020-07-16 21:41:26 +00:00
|
|
|
if (dictionary)
|
|
|
|
throw Exception("Dictionaries can be renamed only in Atomic databases", ErrorCodes::NOT_IMPLEMENTED);
|
2020-03-31 20:38:05 +00:00
|
|
|
|
2019-11-26 19:06:19 +00:00
|
|
|
bool from_ordinary_to_atomic = false;
|
2019-12-02 19:11:18 +00:00
|
|
|
bool from_atomic_to_ordinary = false;
|
2019-11-06 16:05:04 +00:00
|
|
|
if (typeid(*this) != typeid(to_database))
|
2019-11-26 19:06:19 +00:00
|
|
|
{
|
|
|
|
if (typeid_cast<DatabaseOrdinary *>(this) && typeid_cast<DatabaseAtomic *>(&to_database))
|
|
|
|
from_ordinary_to_atomic = true;
|
2019-12-02 19:11:18 +00:00
|
|
|
else if (typeid_cast<DatabaseAtomic *>(this) && typeid_cast<DatabaseOrdinary *>(&to_database))
|
|
|
|
from_atomic_to_ordinary = true;
|
2019-11-26 19:06:19 +00:00
|
|
|
else
|
|
|
|
throw Exception("Moving tables between databases of different engines is not supported", ErrorCodes::NOT_IMPLEMENTED);
|
|
|
|
}
|
2019-11-06 16:05:04 +00:00
|
|
|
|
2020-01-22 11:30:11 +00:00
|
|
|
auto table_data_relative_path = getTableDataPath(table_name);
|
2020-06-18 16:10:47 +00:00
|
|
|
TableExclusiveLockHolder table_lock;
|
2020-01-22 11:30:11 +00:00
|
|
|
String table_metadata_path;
|
|
|
|
ASTPtr attach_query;
|
2020-03-18 17:38:52 +00:00
|
|
|
/// DatabaseLazy::detachTable may return nullptr even if table exists, so we need tryGetTable for this case.
|
2020-05-28 20:10:45 +00:00
|
|
|
StoragePtr table = tryGetTable(table_name, global_context);
|
2020-03-18 17:38:52 +00:00
|
|
|
detachTable(table_name);
|
2019-10-10 17:33:01 +00:00
|
|
|
try
|
|
|
|
{
|
2020-04-12 13:35:44 +00:00
|
|
|
table_lock = table->lockExclusively(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout);
|
2020-01-22 11:30:11 +00:00
|
|
|
|
|
|
|
table_metadata_path = getObjectMetadataPath(table_name);
|
2020-03-19 21:14:52 +00:00
|
|
|
attach_query = parseQueryFromMetadata(log, context, table_metadata_path);
|
2020-01-22 11:30:11 +00:00
|
|
|
auto & create = attach_query->as<ASTCreateQuery &>();
|
2020-03-23 22:40:40 +00:00
|
|
|
create.database = to_database.getDatabaseName();
|
2020-01-22 11:30:11 +00:00
|
|
|
create.table = to_table_name;
|
|
|
|
if (from_ordinary_to_atomic)
|
|
|
|
create.uuid = UUIDHelpers::generateV4();
|
|
|
|
if (from_atomic_to_ordinary)
|
|
|
|
create.uuid = UUIDHelpers::Nil;
|
|
|
|
|
|
|
|
/// Notify the table that it is renamed. It will move data to new path (if it stores data on disk) and update StorageID
|
2020-04-07 14:05:51 +00:00
|
|
|
table->rename(to_database.getTableDataPath(create), StorageID(create));
|
2019-10-10 17:33:01 +00:00
|
|
|
}
|
2019-11-06 16:05:04 +00:00
|
|
|
catch (const Exception &)
|
2019-10-10 17:33:01 +00:00
|
|
|
{
|
2020-01-22 11:30:11 +00:00
|
|
|
attachTable(table_name, table, table_data_relative_path);
|
2019-10-10 17:33:01 +00:00
|
|
|
throw;
|
|
|
|
}
|
2019-11-06 16:05:04 +00:00
|
|
|
catch (const Poco::Exception & e)
|
|
|
|
{
|
2020-01-22 11:30:11 +00:00
|
|
|
attachTable(table_name, table, table_data_relative_path);
|
2019-11-06 16:05:04 +00:00
|
|
|
/// Better diagnostics.
|
2020-06-04 17:49:14 +00:00
|
|
|
throw Exception{Exception::CreateFromPocoTag{}, e};
|
2019-11-06 16:05:04 +00:00
|
|
|
}
|
2019-10-10 17:33:01 +00:00
|
|
|
|
2020-01-22 11:30:11 +00:00
|
|
|
/// Now table data are moved to new database, so we must add metadata and attach table to new database
|
|
|
|
to_database.createTable(context, to_table_name, table, attach_query);
|
|
|
|
|
|
|
|
Poco::File(table_metadata_path).remove();
|
2020-07-06 08:30:11 +00:00
|
|
|
|
|
|
|
/// Special case: usually no actions with symlinks are required when detaching/attaching table,
|
|
|
|
/// but not when moving from Atomic database to Ordinary
|
|
|
|
if (from_atomic_to_ordinary)
|
|
|
|
{
|
|
|
|
auto & atomic_db = assert_cast<DatabaseAtomic &>(*this);
|
|
|
|
atomic_db.tryRemoveSymlink(table_name);
|
|
|
|
}
|
2019-11-06 16:05:04 +00:00
|
|
|
}
|
|
|
|
|
2020-05-28 20:10:45 +00:00
|
|
|
ASTPtr DatabaseOnDisk::getCreateTableQueryImpl(const String & table_name, const Context &, bool throw_on_error) const
|
2019-10-03 07:59:48 +00:00
|
|
|
{
|
|
|
|
ASTPtr ast;
|
2020-05-28 20:10:45 +00:00
|
|
|
bool has_table = tryGetTable(table_name, global_context) != nullptr;
|
2019-11-06 16:05:04 +00:00
|
|
|
auto table_metadata_path = getObjectMetadataPath(table_name);
|
2020-01-29 18:45:03 +00:00
|
|
|
try
|
2019-10-03 07:59:48 +00:00
|
|
|
{
|
2020-04-23 16:51:48 +00:00
|
|
|
ast = getCreateQueryFromMetadata(table_metadata_path, throw_on_error);
|
2020-01-29 18:45:03 +00:00
|
|
|
}
|
|
|
|
catch (const Exception & e)
|
|
|
|
{
|
|
|
|
if (!has_table && e.code() == ErrorCodes::FILE_DOESNT_EXIST && throw_on_error)
|
|
|
|
throw Exception{"Table " + backQuote(table_name) + " doesn't exist",
|
|
|
|
ErrorCodes::CANNOT_GET_CREATE_TABLE_QUERY};
|
|
|
|
else if (throw_on_error)
|
|
|
|
throw;
|
2019-10-03 07:59:48 +00:00
|
|
|
}
|
|
|
|
return ast;
|
|
|
|
}
|
|
|
|
|
2020-04-23 16:51:48 +00:00
|
|
|
ASTPtr DatabaseOnDisk::getCreateDatabaseQuery() const
|
2019-10-03 07:59:48 +00:00
|
|
|
{
|
|
|
|
ASTPtr ast;
|
|
|
|
|
2020-04-23 16:51:48 +00:00
|
|
|
auto settings = global_context.getSettingsRef();
|
2020-07-07 12:11:58 +00:00
|
|
|
{
|
|
|
|
std::lock_guard lock(mutex);
|
|
|
|
auto database_metadata_path = global_context.getPath() + "metadata/" + escapeForFileName(database_name) + ".sql";
|
|
|
|
ast = parseQueryFromMetadata(log, global_context, database_metadata_path, true);
|
|
|
|
auto & ast_create_query = ast->as<ASTCreateQuery &>();
|
|
|
|
ast_create_query.attach = false;
|
|
|
|
ast_create_query.database = database_name;
|
|
|
|
}
|
2019-10-03 07:59:48 +00:00
|
|
|
if (!ast)
|
|
|
|
{
|
|
|
|
/// Handle databases (such as default) for which there are no database.sql files.
|
2019-12-02 19:11:18 +00:00
|
|
|
/// If database.sql doesn't exist, then engine is Ordinary
|
|
|
|
String query = "CREATE DATABASE " + backQuoteIfNeed(getDatabaseName()) + " ENGINE = Ordinary";
|
2019-10-03 07:59:48 +00:00
|
|
|
ParserCreateQuery parser;
|
2020-01-14 11:11:01 +00:00
|
|
|
ast = parseQuery(parser, query.data(), query.data() + query.size(), "", 0, settings.max_parser_depth);
|
2019-10-03 07:59:48 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
return ast;
|
|
|
|
}
|
|
|
|
|
2019-11-06 16:05:04 +00:00
|
|
|
void DatabaseOnDisk::drop(const Context & context)
|
2019-10-03 07:59:48 +00:00
|
|
|
{
|
2019-11-06 16:05:04 +00:00
|
|
|
Poco::File(context.getPath() + getDataPath()).remove(false);
|
|
|
|
Poco::File(getMetadataPath()).remove(false);
|
2019-10-03 07:59:48 +00:00
|
|
|
}
|
|
|
|
|
2020-03-08 22:38:12 +00:00
|
|
|
String DatabaseOnDisk::getObjectMetadataPath(const String & object_name) const
|
2019-10-03 07:59:48 +00:00
|
|
|
{
|
2020-03-08 22:38:12 +00:00
|
|
|
return getMetadataPath() + escapeForFileName(object_name) + ".sql";
|
2019-10-02 10:10:45 +00:00
|
|
|
}
|
|
|
|
|
2020-03-08 22:38:12 +00:00
|
|
|
time_t DatabaseOnDisk::getObjectMetadataModificationTime(const String & object_name) const
|
2019-10-03 08:35:58 +00:00
|
|
|
{
|
2020-03-08 22:38:12 +00:00
|
|
|
String table_metadata_path = getObjectMetadataPath(object_name);
|
2019-10-03 08:35:58 +00:00
|
|
|
Poco::File meta_file(table_metadata_path);
|
|
|
|
|
|
|
|
if (meta_file.exists())
|
|
|
|
return meta_file.getLastModified().epochTime();
|
|
|
|
else
|
|
|
|
return static_cast<time_t>(0);
|
|
|
|
}
|
|
|
|
|
2020-04-06 23:22:44 +00:00
|
|
|
void DatabaseOnDisk::iterateMetadataFiles(const Context & context, const IteratingFunction & process_metadata_file) const
|
2020-01-23 19:10:09 +00:00
|
|
|
{
|
2020-04-06 23:22:44 +00:00
|
|
|
auto process_tmp_drop_metadata_file = [&](const String & file_name)
|
2020-01-23 19:10:09 +00:00
|
|
|
{
|
2020-04-06 23:22:44 +00:00
|
|
|
assert(getEngineName() != "Atomic");
|
2020-01-23 19:10:09 +00:00
|
|
|
static const char * tmp_drop_ext = ".sql.tmp_drop";
|
|
|
|
const std::string object_name = file_name.substr(0, file_name.size() - strlen(tmp_drop_ext));
|
|
|
|
if (Poco::File(context.getPath() + getDataPath() + '/' + object_name).exists())
|
|
|
|
{
|
2020-01-28 19:39:52 +00:00
|
|
|
Poco::File(getMetadataPath() + file_name).renameTo(getMetadataPath() + object_name + ".sql");
|
2020-05-23 22:24:01 +00:00
|
|
|
LOG_WARNING(log, "Object {} was not dropped previously and will be restored", backQuote(object_name));
|
2020-01-23 19:10:09 +00:00
|
|
|
process_metadata_file(object_name + ".sql");
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
2020-05-23 22:24:01 +00:00
|
|
|
LOG_INFO(log, "Removing file {}", getMetadataPath() + file_name);
|
2020-01-23 19:10:09 +00:00
|
|
|
Poco::File(getMetadataPath() + file_name).remove();
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
2020-06-29 23:28:25 +00:00
|
|
|
/// Metadata files to load: name and flag for .tmp_drop files
|
|
|
|
std::set<std::pair<String, bool>> metadata_files;
|
|
|
|
|
2019-10-03 08:27:43 +00:00
|
|
|
Poco::DirectoryIterator dir_end;
|
2019-11-06 16:05:04 +00:00
|
|
|
for (Poco::DirectoryIterator dir_it(getMetadataPath()); dir_it != dir_end; ++dir_it)
|
2019-10-03 08:27:43 +00:00
|
|
|
{
|
|
|
|
/// For '.svn', '.gitignore' directory and similar.
|
|
|
|
if (dir_it.name().at(0) == '.')
|
|
|
|
continue;
|
|
|
|
|
|
|
|
/// There are .sql.bak files - skip them.
|
|
|
|
if (endsWith(dir_it.name(), ".sql.bak"))
|
|
|
|
continue;
|
|
|
|
|
|
|
|
static const char * tmp_drop_ext = ".sql.tmp_drop";
|
|
|
|
if (endsWith(dir_it.name(), tmp_drop_ext))
|
|
|
|
{
|
2020-01-23 19:10:09 +00:00
|
|
|
/// There are files that we tried to delete previously
|
2020-06-29 23:28:25 +00:00
|
|
|
metadata_files.emplace(dir_it.name(), false);
|
2019-10-03 08:27:43 +00:00
|
|
|
}
|
2020-01-23 19:10:09 +00:00
|
|
|
else if (endsWith(dir_it.name(), ".sql.tmp"))
|
2019-10-03 08:27:43 +00:00
|
|
|
{
|
2020-01-23 19:10:09 +00:00
|
|
|
/// There are files .sql.tmp - delete
|
2020-05-23 22:24:01 +00:00
|
|
|
LOG_INFO(log, "Removing file {}", dir_it->path());
|
2019-10-03 08:27:43 +00:00
|
|
|
Poco::File(dir_it->path()).remove();
|
|
|
|
}
|
2020-01-23 19:10:09 +00:00
|
|
|
else if (endsWith(dir_it.name(), ".sql"))
|
2019-10-03 08:27:43 +00:00
|
|
|
{
|
2020-01-23 19:10:09 +00:00
|
|
|
/// The required files have names like `table_name.sql`
|
2020-06-29 23:28:25 +00:00
|
|
|
metadata_files.emplace(dir_it.name(), true);
|
2019-10-03 08:27:43 +00:00
|
|
|
}
|
|
|
|
else
|
2019-11-06 16:05:04 +00:00
|
|
|
throw Exception("Incorrect file extension: " + dir_it.name() + " in metadata directory " + getMetadataPath(),
|
2019-10-03 08:27:43 +00:00
|
|
|
ErrorCodes::INCORRECT_FILE_NAME);
|
|
|
|
}
|
2020-06-29 23:28:25 +00:00
|
|
|
|
|
|
|
/// Read and parse metadata in parallel
|
2020-07-16 23:12:47 +00:00
|
|
|
ThreadPool pool;
|
2020-06-29 23:28:25 +00:00
|
|
|
for (const auto & file : metadata_files)
|
|
|
|
{
|
|
|
|
pool.scheduleOrThrowOnError([&]()
|
|
|
|
{
|
|
|
|
if (file.second)
|
|
|
|
process_metadata_file(file.first);
|
|
|
|
else
|
|
|
|
process_tmp_drop_metadata_file(file.first);
|
|
|
|
});
|
|
|
|
}
|
|
|
|
pool.wait();
|
2019-10-03 08:27:43 +00:00
|
|
|
}
|
|
|
|
|
2020-06-27 19:05:00 +00:00
|
|
|
ASTPtr DatabaseOnDisk::parseQueryFromMetadata(Poco::Logger * logger, const Context & context, const String & metadata_file_path, bool throw_on_error /*= true*/, bool remove_empty /*= false*/)
|
2019-11-06 16:05:04 +00:00
|
|
|
{
|
|
|
|
String query;
|
|
|
|
|
|
|
|
try
|
|
|
|
{
|
2019-12-02 19:11:18 +00:00
|
|
|
ReadBufferFromFile in(metadata_file_path, METADATA_FILE_BUFFER_SIZE);
|
2019-11-06 16:05:04 +00:00
|
|
|
readStringUntilEOF(query, in);
|
|
|
|
}
|
|
|
|
catch (const Exception & e)
|
|
|
|
{
|
|
|
|
if (!throw_on_error && e.code() == ErrorCodes::FILE_DOESNT_EXIST)
|
|
|
|
return nullptr;
|
|
|
|
else
|
|
|
|
throw;
|
|
|
|
}
|
|
|
|
|
2019-12-02 19:11:18 +00:00
|
|
|
/** Empty files with metadata are generated after a rough restart of the server.
|
|
|
|
* Remove these files to slightly reduce the work of the admins on startup.
|
|
|
|
*/
|
|
|
|
if (remove_empty && query.empty())
|
|
|
|
{
|
2020-07-06 08:30:11 +00:00
|
|
|
if (logger)
|
|
|
|
LOG_ERROR(logger, "File {} is empty. Removing.", metadata_file_path);
|
2019-12-02 19:11:18 +00:00
|
|
|
Poco::File(metadata_file_path).remove();
|
|
|
|
return nullptr;
|
|
|
|
}
|
|
|
|
|
2020-01-14 11:11:01 +00:00
|
|
|
auto settings = context.getSettingsRef();
|
2019-11-06 16:05:04 +00:00
|
|
|
ParserCreateQuery parser;
|
|
|
|
const char * pos = query.data();
|
|
|
|
std::string error_message;
|
|
|
|
auto ast = tryParseQuery(parser, pos, pos + query.size(), error_message, /* hilite = */ false,
|
2020-03-19 21:14:52 +00:00
|
|
|
"in file " + metadata_file_path, /* allow_multi_statements = */ false, 0, settings.max_parser_depth);
|
2019-11-06 16:05:04 +00:00
|
|
|
|
|
|
|
if (!ast && throw_on_error)
|
|
|
|
throw Exception(error_message, ErrorCodes::SYNTAX_ERROR);
|
2019-12-02 19:11:18 +00:00
|
|
|
else if (!ast)
|
|
|
|
return nullptr;
|
|
|
|
|
|
|
|
auto & create = ast->as<ASTCreateQuery &>();
|
2020-07-02 20:39:31 +00:00
|
|
|
if (!create.table.empty() && create.uuid != UUIDHelpers::Nil)
|
2019-12-02 19:11:18 +00:00
|
|
|
{
|
|
|
|
String table_name = Poco::Path(metadata_file_path).makeFile().getBaseName();
|
2020-03-18 17:38:52 +00:00
|
|
|
table_name = unescapeForFileName(table_name);
|
2020-01-23 19:10:09 +00:00
|
|
|
|
2020-07-06 08:30:11 +00:00
|
|
|
if (create.table != TABLE_WITH_UUID_NAME_PLACEHOLDER && logger)
|
2020-06-27 19:05:00 +00:00
|
|
|
LOG_WARNING(logger, "File {} contains both UUID and table name. Will use name `{}` instead of `{}`", metadata_file_path, table_name, create.table);
|
2019-12-02 19:11:18 +00:00
|
|
|
create.table = table_name;
|
|
|
|
}
|
2019-11-06 16:05:04 +00:00
|
|
|
|
|
|
|
return ast;
|
|
|
|
}
|
|
|
|
|
2020-04-23 16:51:48 +00:00
|
|
|
ASTPtr DatabaseOnDisk::getCreateQueryFromMetadata(const String & database_metadata_path, bool throw_on_error) const
|
2019-11-06 16:05:04 +00:00
|
|
|
{
|
2020-04-23 16:51:48 +00:00
|
|
|
ASTPtr ast = parseQueryFromMetadata(log, global_context, database_metadata_path, throw_on_error);
|
2019-11-06 16:05:04 +00:00
|
|
|
|
|
|
|
if (ast)
|
|
|
|
{
|
|
|
|
auto & ast_create_query = ast->as<ASTCreateQuery &>();
|
|
|
|
ast_create_query.attach = false;
|
2020-07-07 12:11:58 +00:00
|
|
|
ast_create_query.database = getDatabaseName();
|
2019-11-06 16:05:04 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
return ast;
|
|
|
|
}
|
|
|
|
|
2019-10-05 10:21:47 +00:00
|
|
|
}
|