2018-06-05 19:46:49 +00:00
|
|
|
#include <iomanip>
|
|
|
|
|
2018-10-17 01:31:28 +00:00
|
|
|
#include <Poco/Event.h>
|
2016-03-19 01:18:49 +00:00
|
|
|
#include <Poco/DirectoryIterator.h>
|
2017-01-21 04:24:28 +00:00
|
|
|
#include <common/logger_useful.h>
|
2016-03-19 01:18:49 +00:00
|
|
|
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Databases/DatabaseOrdinary.h>
|
|
|
|
#include <Databases/DatabaseMemory.h>
|
|
|
|
#include <Databases/DatabasesCommon.h>
|
|
|
|
#include <Common/escapeForFileName.h>
|
2018-01-15 19:07:47 +00:00
|
|
|
#include <Common/StringUtils/StringUtils.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Common/Stopwatch.h>
|
2017-06-23 20:22:35 +00:00
|
|
|
#include <common/ThreadPool.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Parsers/ASTCreateQuery.h>
|
|
|
|
#include <Parsers/parseQuery.h>
|
|
|
|
#include <Parsers/ParserCreateQuery.h>
|
|
|
|
#include <Interpreters/Context.h>
|
|
|
|
#include <Interpreters/Settings.h>
|
|
|
|
#include <Interpreters/InterpreterCreateQuery.h>
|
|
|
|
#include <IO/WriteBufferFromFile.h>
|
|
|
|
#include <IO/ReadBufferFromFile.h>
|
2018-06-05 19:46:49 +00:00
|
|
|
#include <IO/WriteHelpers.h>
|
|
|
|
#include <IO/ReadHelpers.h>
|
2018-12-05 11:20:55 +00:00
|
|
|
#include <ext/scope_guard.h>
|
2016-03-19 01:18:49 +00:00
|
|
|
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
extern const int TABLE_ALREADY_EXISTS;
|
|
|
|
extern const int UNKNOWN_TABLE;
|
|
|
|
extern const int CANNOT_CREATE_TABLE_FROM_METADATA;
|
|
|
|
extern const int INCORRECT_FILE_NAME;
|
2018-01-30 17:47:04 +00:00
|
|
|
extern const int FILE_DOESNT_EXIST;
|
2017-04-01 07:20:54 +00:00
|
|
|
extern const int LOGICAL_ERROR;
|
2018-01-30 17:47:04 +00:00
|
|
|
extern const int CANNOT_GET_CREATE_TABLE_QUERY;
|
2018-03-23 19:56:24 +00:00
|
|
|
extern const int SYNTAX_ERROR;
|
2016-03-19 01:18:49 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
static constexpr size_t PRINT_MESSAGE_EACH_N_TABLES = 256;
|
|
|
|
static constexpr size_t PRINT_MESSAGE_EACH_N_SECONDS = 5;
|
|
|
|
static constexpr size_t METADATA_FILE_BUFFER_SIZE = 32768;
|
|
|
|
|
2018-02-21 19:26:59 +00:00
|
|
|
namespace detail
|
2016-09-02 13:04:11 +00:00
|
|
|
{
|
2018-02-21 19:26:59 +00:00
|
|
|
String getTableMetadataPath(const String & base_path, const String & table_name)
|
|
|
|
{
|
|
|
|
return base_path + (endsWith(base_path, "/") ? "" : "/") + escapeForFileName(table_name) + ".sql";
|
|
|
|
}
|
2018-03-12 14:14:56 +00:00
|
|
|
|
|
|
|
String getDatabaseMetadataPath(const String & base_path)
|
|
|
|
{
|
|
|
|
return (endsWith(base_path, "/") ? base_path.substr(0, base_path.size() - 1) : base_path) + ".sql";
|
|
|
|
}
|
|
|
|
|
2016-09-02 13:04:11 +00:00
|
|
|
}
|
|
|
|
|
2016-03-19 01:18:49 +00:00
|
|
|
static void loadTable(
|
2017-04-01 07:20:54 +00:00
|
|
|
Context & context,
|
|
|
|
const String & database_metadata_path,
|
|
|
|
DatabaseOrdinary & database,
|
|
|
|
const String & database_name,
|
|
|
|
const String & database_data_path,
|
|
|
|
const String & file_name,
|
|
|
|
bool has_force_restore_data_flag)
|
2016-03-19 01:18:49 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
Logger * log = &Logger::get("loadTable");
|
|
|
|
|
|
|
|
const String table_metadata_path = database_metadata_path + "/" + file_name;
|
|
|
|
|
|
|
|
String s;
|
|
|
|
{
|
|
|
|
char in_buf[METADATA_FILE_BUFFER_SIZE];
|
|
|
|
ReadBufferFromFile in(table_metadata_path, METADATA_FILE_BUFFER_SIZE, -1, in_buf);
|
|
|
|
readStringUntilEOF(s, in);
|
|
|
|
}
|
|
|
|
|
|
|
|
/** 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 (s.empty())
|
|
|
|
{
|
|
|
|
LOG_ERROR(log, "File " << table_metadata_path << " is empty. Removing.");
|
|
|
|
Poco::File(table_metadata_path).remove();
|
|
|
|
return;
|
|
|
|
}
|
|
|
|
|
|
|
|
try
|
|
|
|
{
|
|
|
|
String table_name;
|
|
|
|
StoragePtr table;
|
|
|
|
std::tie(table_name, table) = createTableFromDefinition(
|
|
|
|
s, database_name, database_data_path, context, has_force_restore_data_flag, "in file " + table_metadata_path);
|
|
|
|
database.attachTable(table_name, table);
|
|
|
|
}
|
|
|
|
catch (const Exception & e)
|
|
|
|
{
|
|
|
|
throw Exception("Cannot create table from metadata file " + table_metadata_path + ", error: " + e.displayText() +
|
|
|
|
", stack trace:\n" + e.getStackTrace().toString(),
|
|
|
|
ErrorCodes::CANNOT_CREATE_TABLE_FROM_METADATA);
|
|
|
|
}
|
2016-03-19 01:18:49 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2018-03-23 20:46:43 +00:00
|
|
|
DatabaseOrdinary::DatabaseOrdinary(String name_, const String & metadata_path_, const Context & context)
|
|
|
|
: DatabaseWithOwnTablesBase(std::move(name_))
|
|
|
|
, metadata_path(metadata_path_)
|
|
|
|
, data_path(context.getPath() + "data/" + escapeForFileName(name) + "/")
|
|
|
|
, log(&Logger::get("DatabaseOrdinary (" + name + ")"))
|
2016-06-10 20:46:58 +00:00
|
|
|
{
|
2018-02-02 17:29:45 +00:00
|
|
|
Poco::File(data_path).createDirectories();
|
2016-06-10 20:46:58 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2017-09-11 12:39:01 +00:00
|
|
|
void DatabaseOrdinary::loadTables(
|
|
|
|
Context & context,
|
|
|
|
ThreadPool * thread_pool,
|
|
|
|
bool has_force_restore_data_flag)
|
2016-03-19 01:18:49 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
using FileNames = std::vector<std::string>;
|
|
|
|
FileNames file_names;
|
|
|
|
|
|
|
|
Poco::DirectoryIterator dir_end;
|
2017-11-03 19:53:10 +00:00
|
|
|
for (Poco::DirectoryIterator dir_it(metadata_path); dir_it != dir_end; ++dir_it)
|
2017-04-01 07:20:54 +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;
|
|
|
|
|
|
|
|
/// There are files .sql.tmp - delete.
|
|
|
|
if (endsWith(dir_it.name(), ".sql.tmp"))
|
|
|
|
{
|
|
|
|
LOG_INFO(log, "Removing file " << dir_it->path());
|
|
|
|
Poco::File(dir_it->path()).remove();
|
|
|
|
continue;
|
|
|
|
}
|
|
|
|
|
|
|
|
/// The required files have names like `table_name.sql`
|
|
|
|
if (endsWith(dir_it.name(), ".sql"))
|
|
|
|
file_names.push_back(dir_it.name());
|
|
|
|
else
|
2017-11-03 19:53:10 +00:00
|
|
|
throw Exception("Incorrect file extension: " + dir_it.name() + " in metadata directory " + metadata_path,
|
2017-04-01 07:20:54 +00:00
|
|
|
ErrorCodes::INCORRECT_FILE_NAME);
|
|
|
|
}
|
|
|
|
|
2018-10-17 02:34:14 +00:00
|
|
|
if (file_names.empty())
|
|
|
|
return;
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
/** Tables load faster if they are loaded in sorted (by name) order.
|
2017-06-21 19:07:08 +00:00
|
|
|
* Otherwise (for the ext4 filesystem), `DirectoryIterator` iterates through them in some order,
|
2017-04-01 07:20:54 +00:00
|
|
|
* which does not correspond to order tables creation and does not correspond to order of their location on disk.
|
|
|
|
*/
|
|
|
|
std::sort(file_names.begin(), file_names.end());
|
|
|
|
|
|
|
|
size_t total_tables = file_names.size();
|
|
|
|
LOG_INFO(log, "Total " << total_tables << " tables.");
|
|
|
|
|
2018-02-25 02:43:27 +00:00
|
|
|
AtomicStopwatch watch;
|
2017-04-01 07:20:54 +00:00
|
|
|
std::atomic<size_t> tables_processed {0};
|
2018-10-17 01:31:28 +00:00
|
|
|
Poco::Event all_tables_processed;
|
2018-12-05 12:46:32 +00:00
|
|
|
ExceptionHandler exception_handler;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-10-17 01:59:24 +00:00
|
|
|
auto task_function = [&](const String & table)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2018-12-05 11:20:55 +00:00
|
|
|
SCOPE_EXIT(
|
2018-12-07 04:10:22 +00:00
|
|
|
if (++tables_processed == total_tables)
|
|
|
|
all_tables_processed.set()
|
2018-12-05 11:20:55 +00:00
|
|
|
);
|
|
|
|
|
2018-10-17 01:59:24 +00:00
|
|
|
/// Messages, so that it's not boring to wait for the server to load for a long time.
|
|
|
|
if ((tables_processed + 1) % PRINT_MESSAGE_EACH_N_TABLES == 0
|
|
|
|
|| watch.compareAndRestart(PRINT_MESSAGE_EACH_N_SECONDS))
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2018-10-17 01:59:24 +00:00
|
|
|
LOG_INFO(log, std::fixed << std::setprecision(2) << tables_processed * 100.0 / total_tables << "%");
|
|
|
|
watch.restart();
|
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-10-17 01:59:24 +00:00
|
|
|
loadTable(context, metadata_path, *this, name, data_path, table, has_force_restore_data_flag);
|
2017-04-01 07:20:54 +00:00
|
|
|
};
|
|
|
|
|
2018-10-17 01:59:24 +00:00
|
|
|
for (const auto & filename : file_names)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2018-12-05 12:46:32 +00:00
|
|
|
auto task = createExceptionHandledJob(std::bind(task_function, filename), exception_handler);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
if (thread_pool)
|
|
|
|
thread_pool->schedule(task);
|
|
|
|
else
|
|
|
|
task();
|
|
|
|
}
|
|
|
|
|
2017-06-06 17:06:14 +00:00
|
|
|
if (thread_pool)
|
2018-10-17 01:31:28 +00:00
|
|
|
all_tables_processed.wait();
|
2017-06-06 17:06:14 +00:00
|
|
|
|
2018-12-05 12:46:32 +00:00
|
|
|
exception_handler.throwIfException();
|
|
|
|
|
2017-06-06 17:06:14 +00:00
|
|
|
/// After all tables was basically initialized, startup them.
|
|
|
|
startupTables(thread_pool);
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
void DatabaseOrdinary::startupTables(ThreadPool * thread_pool)
|
|
|
|
{
|
|
|
|
LOG_INFO(log, "Starting up tables.");
|
|
|
|
|
2018-02-25 02:43:27 +00:00
|
|
|
AtomicStopwatch watch;
|
2017-06-06 17:06:14 +00:00
|
|
|
std::atomic<size_t> tables_processed {0};
|
|
|
|
size_t total_tables = tables.size();
|
2018-10-17 01:31:28 +00:00
|
|
|
Poco::Event all_tables_processed;
|
2018-12-05 12:46:32 +00:00
|
|
|
ExceptionHandler exception_handler;
|
2017-06-06 17:06:14 +00:00
|
|
|
|
2018-10-17 02:34:14 +00:00
|
|
|
if (!total_tables)
|
|
|
|
return;
|
|
|
|
|
2018-10-17 01:59:24 +00:00
|
|
|
auto task_function = [&](const StoragePtr & table)
|
2017-06-06 17:06:14 +00:00
|
|
|
{
|
2018-12-05 12:46:32 +00:00
|
|
|
SCOPE_EXIT(
|
2018-12-07 04:11:11 +00:00
|
|
|
if (++tables_processed == total_tables)
|
|
|
|
all_tables_processed.set()
|
2018-12-05 12:46:32 +00:00
|
|
|
);
|
|
|
|
|
2018-10-17 01:59:24 +00:00
|
|
|
if ((tables_processed + 1) % PRINT_MESSAGE_EACH_N_TABLES == 0
|
|
|
|
|| watch.compareAndRestart(PRINT_MESSAGE_EACH_N_SECONDS))
|
2017-06-06 17:06:14 +00:00
|
|
|
{
|
2018-10-17 01:59:24 +00:00
|
|
|
LOG_INFO(log, std::fixed << std::setprecision(2) << tables_processed * 100.0 / total_tables << "%");
|
|
|
|
watch.restart();
|
|
|
|
}
|
2017-06-06 17:06:14 +00:00
|
|
|
|
2018-10-17 01:59:24 +00:00
|
|
|
table->startup();
|
2017-06-06 17:06:14 +00:00
|
|
|
};
|
|
|
|
|
2018-10-17 01:59:24 +00:00
|
|
|
for (const auto & name_storage : tables)
|
2017-06-06 17:06:14 +00:00
|
|
|
{
|
2018-12-05 12:46:32 +00:00
|
|
|
auto task = createExceptionHandledJob(std::bind(task_function, name_storage.second), exception_handler);
|
2017-06-06 17:06:14 +00:00
|
|
|
|
|
|
|
if (thread_pool)
|
|
|
|
thread_pool->schedule(task);
|
|
|
|
else
|
|
|
|
task();
|
|
|
|
}
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
if (thread_pool)
|
2018-10-17 01:31:28 +00:00
|
|
|
all_tables_processed.wait();
|
2018-12-05 12:46:32 +00:00
|
|
|
|
|
|
|
exception_handler.throwIfException();
|
2016-03-19 01:18:49 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2017-01-23 18:05:07 +00:00
|
|
|
void DatabaseOrdinary::createTable(
|
2017-09-11 12:39:01 +00:00
|
|
|
const Context & context,
|
|
|
|
const String & table_name,
|
|
|
|
const StoragePtr & table,
|
2017-10-25 19:52:32 +00:00
|
|
|
const ASTPtr & query)
|
2016-03-19 01:18:49 +00:00
|
|
|
{
|
2017-09-11 12:39:01 +00:00
|
|
|
const auto & settings = context.getSettingsRef();
|
|
|
|
|
2017-04-01 07:20:54 +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.
|
|
|
|
|
|
|
|
{
|
2019-01-02 06:44:36 +00:00
|
|
|
std::lock_guard lock(mutex);
|
2018-01-30 17:47:04 +00:00
|
|
|
if (tables.find(table_name) != tables.end())
|
2017-04-01 07:20:54 +00:00
|
|
|
throw Exception("Table " + name + "." + table_name + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS);
|
|
|
|
}
|
|
|
|
|
2018-02-22 09:09:14 +00:00
|
|
|
String table_metadata_path = getTableMetadataPath(table_name);
|
2017-04-01 07:20:54 +00:00
|
|
|
String table_metadata_tmp_path = table_metadata_path + ".tmp";
|
|
|
|
String statement;
|
|
|
|
|
|
|
|
{
|
|
|
|
statement = getTableDefinitionFromCreateQuery(query);
|
|
|
|
|
|
|
|
/// 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();
|
|
|
|
}
|
|
|
|
|
|
|
|
try
|
|
|
|
{
|
|
|
|
/// Add a table to the map of known tables.
|
|
|
|
{
|
2019-01-02 06:44:36 +00:00
|
|
|
std::lock_guard lock(mutex);
|
2017-04-01 07:20:54 +00:00
|
|
|
if (!tables.emplace(table_name, table).second)
|
|
|
|
throw Exception("Table " + name + "." + table_name + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS);
|
|
|
|
}
|
|
|
|
|
|
|
|
/// 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;
|
|
|
|
}
|
2016-03-19 01:18:49 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2017-09-11 12:39:01 +00:00
|
|
|
void DatabaseOrdinary::removeTable(
|
2017-12-01 21:13:25 +00:00
|
|
|
const Context & /*context*/,
|
2017-09-11 12:39:01 +00:00
|
|
|
const String & table_name)
|
2016-03-19 01:18:49 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
StoragePtr res = detachTable(table_name);
|
|
|
|
|
2018-02-22 09:09:14 +00:00
|
|
|
String table_metadata_path = getTableMetadataPath(table_name);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
try
|
|
|
|
{
|
|
|
|
Poco::File(table_metadata_path).remove();
|
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
|
|
|
attachTable(table_name, res);
|
|
|
|
throw;
|
|
|
|
}
|
2016-03-19 01:18:49 +00:00
|
|
|
}
|
|
|
|
|
2018-03-23 19:56:24 +00:00
|
|
|
static ASTPtr getQueryFromMetadata(const String & metadata_path, bool throw_on_error = true)
|
2016-03-19 01:18:49 +00:00
|
|
|
{
|
2018-03-13 13:28:32 +00:00
|
|
|
if (!Poco::File(metadata_path).exists())
|
|
|
|
return nullptr;
|
2016-03-19 01:18:49 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
String query;
|
2016-03-19 01:18:49 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2018-03-12 14:14:56 +00:00
|
|
|
ReadBufferFromFile in(metadata_path, 4096);
|
2017-04-01 07:20:54 +00:00
|
|
|
readStringUntilEOF(query, in);
|
|
|
|
}
|
2016-03-19 01:18:49 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
ParserCreateQuery parser;
|
2018-03-23 19:56:24 +00:00
|
|
|
const char * pos = query.data();
|
|
|
|
std::string error_message;
|
|
|
|
auto ast = tryParseQuery(parser, pos, pos + query.size(), error_message, /* hilite = */ false,
|
2018-04-16 15:35:11 +00:00
|
|
|
"in file " + metadata_path, /* allow_multi_statements = */ false, 0);
|
2018-03-23 19:56:24 +00:00
|
|
|
|
|
|
|
if (!ast && throw_on_error)
|
|
|
|
throw Exception(error_message, ErrorCodes::SYNTAX_ERROR);
|
|
|
|
|
|
|
|
return ast;
|
2016-03-19 01:18:49 +00:00
|
|
|
}
|
|
|
|
|
2018-03-23 19:56:24 +00:00
|
|
|
static ASTPtr getCreateQueryFromMetadata(const String & metadata_path, const String & database, bool throw_on_error)
|
2018-03-13 13:28:32 +00:00
|
|
|
{
|
2018-03-23 19:56:24 +00:00
|
|
|
ASTPtr ast = getQueryFromMetadata(metadata_path, throw_on_error);
|
2018-03-13 13:28:32 +00:00
|
|
|
|
|
|
|
if (ast)
|
|
|
|
{
|
|
|
|
ASTCreateQuery & ast_create_query = typeid_cast<ASTCreateQuery &>(*ast);
|
|
|
|
ast_create_query.attach = false;
|
|
|
|
ast_create_query.database = database;
|
|
|
|
}
|
|
|
|
|
|
|
|
return ast;
|
2016-03-19 01:18:49 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
void DatabaseOrdinary::renameTable(
|
2017-09-11 12:39:01 +00:00
|
|
|
const Context & context,
|
|
|
|
const String & table_name,
|
|
|
|
IDatabase & to_database,
|
|
|
|
const String & to_table_name)
|
2016-03-19 01:18:49 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
DatabaseOrdinary * to_database_concrete = typeid_cast<DatabaseOrdinary *>(&to_database);
|
|
|
|
|
|
|
|
if (!to_database_concrete)
|
|
|
|
throw Exception("Moving tables between databases of different engines is not supported", ErrorCodes::NOT_IMPLEMENTED);
|
|
|
|
|
2017-09-11 12:39:01 +00:00
|
|
|
StoragePtr table = tryGetTable(context, table_name);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
if (!table)
|
2017-06-23 15:55:45 +00:00
|
|
|
throw Exception("Table " + name + "." + table_name + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
/// Notify the table that it is renamed. If the table does not support renaming, exception is thrown.
|
|
|
|
try
|
|
|
|
{
|
|
|
|
table->rename(context.getPath() + "/data/" + escapeForFileName(to_database_concrete->name) + "/",
|
|
|
|
to_database_concrete->name,
|
|
|
|
to_table_name);
|
|
|
|
}
|
2018-06-03 17:43:56 +00:00
|
|
|
catch (const Exception &)
|
2017-08-03 17:56:25 +00:00
|
|
|
{
|
|
|
|
throw;
|
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
catch (const Poco::Exception & e)
|
|
|
|
{
|
2018-01-30 17:47:04 +00:00
|
|
|
/// Better diagnostics.
|
2018-11-22 21:19:58 +00:00
|
|
|
throw Exception{Exception::CreateFromPoco, e};
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
2018-03-13 13:28:32 +00:00
|
|
|
ASTPtr ast = getQueryFromMetadata(detail::getTableMetadataPath(metadata_path, table_name));
|
|
|
|
if (!ast)
|
|
|
|
throw Exception("There is no metadata file for table " + table_name, ErrorCodes::FILE_DOESNT_EXIST);
|
2017-04-01 07:20:54 +00:00
|
|
|
ASTCreateQuery & ast_create_query = typeid_cast<ASTCreateQuery &>(*ast);
|
|
|
|
ast_create_query.table = to_table_name;
|
|
|
|
|
|
|
|
/// NOTE Non-atomic.
|
2017-10-25 19:52:32 +00:00
|
|
|
to_database_concrete->createTable(context, to_table_name, table, ast);
|
2017-09-11 12:39:01 +00:00
|
|
|
removeTable(context, table_name);
|
2016-03-19 01:18:49 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2017-09-11 12:39:01 +00:00
|
|
|
time_t DatabaseOrdinary::getTableMetadataModificationTime(
|
2017-12-01 21:13:25 +00:00
|
|
|
const Context & /*context*/,
|
2017-09-11 12:39:01 +00:00
|
|
|
const String & table_name)
|
2016-09-02 13:04:11 +00:00
|
|
|
{
|
2018-02-22 09:09:14 +00:00
|
|
|
String table_metadata_path = getTableMetadataPath(table_name);
|
2017-04-01 07:20:54 +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);
|
|
|
|
}
|
2016-09-02 13:04:11 +00:00
|
|
|
}
|
|
|
|
|
2018-03-23 19:56:24 +00:00
|
|
|
ASTPtr DatabaseOrdinary::getCreateTableQueryImpl(const Context & context,
|
|
|
|
const String & table_name, bool throw_on_error) const
|
2016-03-19 01:18:49 +00:00
|
|
|
{
|
2018-01-30 17:47:04 +00:00
|
|
|
ASTPtr ast;
|
2018-03-13 13:28:32 +00:00
|
|
|
|
|
|
|
auto table_metadata_path = detail::getTableMetadataPath(metadata_path, table_name);
|
2018-03-23 19:56:24 +00:00
|
|
|
ast = getCreateQueryFromMetadata(table_metadata_path, name, throw_on_error);
|
|
|
|
if (!ast && throw_on_error)
|
2018-01-30 17:47:04 +00:00
|
|
|
{
|
2018-03-13 13:28:32 +00:00
|
|
|
/// Handle system.* tables for which there are no table.sql files.
|
2018-03-14 19:56:44 +00:00
|
|
|
bool has_table = tryGetTable(context, table_name) != nullptr;
|
2018-01-30 17:47:04 +00:00
|
|
|
|
2018-03-14 19:56:44 +00:00
|
|
|
auto msg = has_table
|
2018-03-13 13:28:32 +00:00
|
|
|
? "There is no CREATE TABLE query for table "
|
|
|
|
: "There is no metadata file for table ";
|
2018-01-30 17:47:04 +00:00
|
|
|
|
2018-03-13 13:28:32 +00:00
|
|
|
throw Exception(msg + table_name, ErrorCodes::CANNOT_GET_CREATE_TABLE_QUERY);
|
2018-01-30 17:47:04 +00:00
|
|
|
}
|
2016-03-19 01:18:49 +00:00
|
|
|
|
2018-03-13 13:28:32 +00:00
|
|
|
return ast;
|
|
|
|
}
|
|
|
|
|
2018-03-14 19:56:44 +00:00
|
|
|
ASTPtr DatabaseOrdinary::getCreateTableQuery(const Context & context, const String & table_name) const
|
|
|
|
{
|
2018-03-23 19:56:24 +00:00
|
|
|
return getCreateTableQueryImpl(context, table_name, true);
|
2018-03-14 19:56:44 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
ASTPtr DatabaseOrdinary::tryGetCreateTableQuery(const Context & context, const String & table_name) const
|
|
|
|
{
|
2018-03-23 19:56:24 +00:00
|
|
|
return getCreateTableQueryImpl(context, table_name, false);
|
2018-03-14 19:56:44 +00:00
|
|
|
}
|
|
|
|
|
2018-03-13 13:28:32 +00:00
|
|
|
ASTPtr DatabaseOrdinary::getCreateDatabaseQuery(const Context & /*context*/) const
|
|
|
|
{
|
|
|
|
ASTPtr ast;
|
|
|
|
|
|
|
|
auto database_metadata_path = detail::getDatabaseMetadataPath(metadata_path);
|
2018-03-23 19:56:24 +00:00
|
|
|
ast = getCreateQueryFromMetadata(database_metadata_path, name, true);
|
2018-03-13 13:28:32 +00:00
|
|
|
if (!ast)
|
|
|
|
{
|
|
|
|
/// Handle databases (such as default) for which there are no database.sql files.
|
|
|
|
String query = "CREATE DATABASE " + backQuoteIfNeed(name) + " ENGINE = Ordinary";
|
|
|
|
ParserCreateQuery parser;
|
2018-04-16 15:39:12 +00:00
|
|
|
ast = parseQuery(parser, query.data(), query.data() + query.size(), "", 0);
|
2018-03-13 13:28:32 +00:00
|
|
|
}
|
2016-03-19 01:18:49 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
return ast;
|
2016-03-19 01:18:49 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
void DatabaseOrdinary::shutdown()
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
/// You can not hold a lock during shutdown.
|
|
|
|
/// Because inside `shutdown` function the tables can work with database, and mutex is not recursive.
|
2016-03-26 04:13:15 +00:00
|
|
|
|
2017-09-11 12:39:01 +00:00
|
|
|
Tables tables_snapshot;
|
|
|
|
{
|
2019-01-02 06:44:36 +00:00
|
|
|
std::lock_guard lock(mutex);
|
2017-09-11 12:39:01 +00:00
|
|
|
tables_snapshot = tables;
|
|
|
|
}
|
|
|
|
|
2018-01-10 00:04:08 +00:00
|
|
|
for (const auto & kv: tables_snapshot)
|
|
|
|
{
|
2017-09-11 12:39:01 +00:00
|
|
|
kv.second->shutdown();
|
|
|
|
}
|
2016-03-19 01:18:49 +00:00
|
|
|
|
2019-01-02 06:44:36 +00:00
|
|
|
std::lock_guard lock(mutex);
|
2017-04-01 07:20:54 +00:00
|
|
|
tables.clear();
|
2016-03-19 01:18:49 +00:00
|
|
|
}
|
|
|
|
|
2016-05-13 21:08:19 +00:00
|
|
|
void DatabaseOrdinary::alterTable(
|
2017-04-01 07:20:54 +00:00
|
|
|
const Context & context,
|
|
|
|
const String & name,
|
2018-03-06 20:18:34 +00:00
|
|
|
const ColumnsDescription & columns,
|
2017-09-17 18:49:43 +00:00
|
|
|
const ASTModifier & storage_modifier)
|
2016-05-13 21:08:19 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
/// Read the definition of the table and replace the necessary parts with new ones.
|
|
|
|
|
|
|
|
String table_name_escaped = escapeForFileName(name);
|
2017-11-03 19:53:10 +00:00
|
|
|
String table_metadata_tmp_path = metadata_path + "/" + table_name_escaped + ".sql.tmp";
|
|
|
|
String table_metadata_path = metadata_path + "/" + table_name_escaped + ".sql";
|
2017-04-01 07:20:54 +00:00
|
|
|
String statement;
|
|
|
|
|
|
|
|
{
|
|
|
|
char in_buf[METADATA_FILE_BUFFER_SIZE];
|
|
|
|
ReadBufferFromFile in(table_metadata_path, METADATA_FILE_BUFFER_SIZE, -1, in_buf);
|
|
|
|
readStringUntilEOF(statement, in);
|
|
|
|
}
|
|
|
|
|
|
|
|
ParserCreateQuery parser;
|
2018-04-16 15:11:13 +00:00
|
|
|
ASTPtr ast = parseQuery(parser, statement.data(), statement.data() + statement.size(), "in file " + table_metadata_path, 0);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
ASTCreateQuery & ast_create_query = typeid_cast<ASTCreateQuery &>(*ast);
|
|
|
|
|
2018-03-06 20:18:34 +00:00
|
|
|
ASTPtr new_columns = InterpreterCreateQuery::formatColumns(columns);
|
2017-09-17 18:49:43 +00:00
|
|
|
ast_create_query.replace(ast_create_query.columns, new_columns);
|
|
|
|
|
|
|
|
if (storage_modifier)
|
|
|
|
storage_modifier(*ast_create_query.storage);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
statement = getTableDefinitionFromCreateQuery(ast);
|
|
|
|
|
|
|
|
{
|
|
|
|
WriteBufferFromFile out(table_metadata_tmp_path, statement.size(), O_WRONLY | O_CREAT | O_EXCL);
|
|
|
|
writeString(statement, out);
|
|
|
|
out.next();
|
|
|
|
if (context.getSettingsRef().fsync_metadata)
|
|
|
|
out.sync();
|
|
|
|
out.close();
|
|
|
|
}
|
|
|
|
|
|
|
|
try
|
|
|
|
{
|
|
|
|
/// rename atomically replaces the old file with the new one.
|
|
|
|
Poco::File(table_metadata_tmp_path).renameTo(table_metadata_path);
|
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
|
|
|
Poco::File(table_metadata_tmp_path).remove();
|
|
|
|
throw;
|
|
|
|
}
|
2016-05-13 21:08:19 +00:00
|
|
|
}
|
|
|
|
|
2018-06-09 15:48:22 +00:00
|
|
|
|
|
|
|
void DatabaseOrdinary::drop()
|
|
|
|
{
|
|
|
|
Poco::File(data_path).remove(false);
|
|
|
|
Poco::File(metadata_path).remove(false);
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2018-02-21 19:26:59 +00:00
|
|
|
String DatabaseOrdinary::getDataPath() const
|
2017-11-03 19:53:10 +00:00
|
|
|
{
|
|
|
|
return data_path;
|
|
|
|
}
|
|
|
|
|
2018-02-21 19:26:59 +00:00
|
|
|
String DatabaseOrdinary::getMetadataPath() const
|
|
|
|
{
|
|
|
|
return metadata_path;
|
|
|
|
}
|
|
|
|
|
2018-05-10 15:56:38 +00:00
|
|
|
String DatabaseOrdinary::getDatabaseName() const
|
|
|
|
{
|
|
|
|
return name;
|
|
|
|
}
|
|
|
|
|
2018-02-21 19:26:59 +00:00
|
|
|
String DatabaseOrdinary::getTableMetadataPath(const String & table_name) const
|
|
|
|
{
|
|
|
|
return detail::getTableMetadataPath(metadata_path, table_name);
|
|
|
|
}
|
|
|
|
|
2016-03-19 01:18:49 +00:00
|
|
|
}
|