This commit is contained in:
Vasilyev Nikita 2019-10-08 11:10:55 +03:00
parent 5fb84b55a6
commit 030aa1cf10
6 changed files with 39 additions and 27 deletions

View File

@ -1,5 +1,3 @@
#include <iomanip>
#include <Core/Settings.h>
#include <Databases/DatabaseLazy.h>
#include <Databases/DatabaseOnDisk.h>
@ -13,6 +11,8 @@
#include <Storages/IStorage.h>
#include <common/logger_useful.h>
#include <ext/scope_guard.h>
#include <iomanip>
#include <Poco/File.h>
@ -61,7 +61,7 @@ void DatabaseLazy::createTable(
const StoragePtr & table,
const ASTPtr & query)
{
clearExpiredTables();
SCOPE_EXIT({ clearExpiredTables(); });
if (!endsWith(table->getName(), "Log"))
throw Exception("Lazy engine can be used only with *Log tables.", ErrorCodes::UNSUPPORTED_METHOD);
DatabaseOnDisk::createTable(*this, context, table_name, table, query);
@ -78,7 +78,7 @@ void DatabaseLazy::removeTable(
const Context & context,
const String & table_name)
{
clearExpiredTables();
SCOPE_EXIT({ clearExpiredTables(); });
DatabaseOnDisk::removeTable(*this, context, table_name, log);
}
@ -89,7 +89,7 @@ void DatabaseLazy::renameTable(
const String & to_table_name,
TableStructureWriteLockHolder & lock)
{
clearExpiredTables();
SCOPE_EXIT({ clearExpiredTables(); });
DatabaseOnDisk::renameTable<DatabaseLazy>(*this, context, table_name, to_database, to_table_name, lock);
}
@ -103,7 +103,7 @@ time_t DatabaseLazy::getTableMetadataModificationTime(
if (it != tables_cache.end())
return it->second.metadata_modification_time;
else
throw Exception("Table " + getDatabaseName() + "." + table_name + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE);
throw Exception("Table " + backQuote(getDatabaseName()) + "." + backQuote(table_name) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE);
}
ASTPtr DatabaseLazy::getCreateTableQuery(const Context & context, const String & table_name) const
@ -129,7 +129,7 @@ void DatabaseLazy::alterTable(
const ConstraintsDescription & /* constraints */,
const ASTModifier & /* storage_modifier */)
{
clearExpiredTables();
SCOPE_EXIT({ clearExpiredTables(); });
throw Exception("ALTER query is not supported for Lazy database.", ErrorCodes::UNSUPPORTED_METHOD);
}
@ -143,7 +143,7 @@ bool DatabaseLazy::isTableExist(
const Context & /* context */,
const String & table_name) const
{
clearExpiredTables();
SCOPE_EXIT({ clearExpiredTables(); });
std::lock_guard lock(tables_mutex);
return tables_cache.find(table_name) != tables_cache.end();
}
@ -152,12 +152,12 @@ StoragePtr DatabaseLazy::tryGetTable(
const Context & context,
const String & table_name) const
{
clearExpiredTables();
SCOPE_EXIT({ clearExpiredTables(); });
{
std::lock_guard lock(tables_mutex);
auto it = tables_cache.find(table_name);
if (it == tables_cache.end())
throw Exception("Table " + getDatabaseName() + "." + table_name + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE);
throw Exception("Table " + backQuote(getDatabaseName()) + "." + backQuote(table_name) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE);
if (it->second.table)
{
@ -192,7 +192,7 @@ bool DatabaseLazy::empty(const Context & /* context */) const
void DatabaseLazy::attachTable(const String & table_name, const StoragePtr & table)
{
LOG_DEBUG(log, "attach table" << table_name);
LOG_DEBUG(log, "Attach table " << backQuote(table_name) << ".");
std::lock_guard lock(tables_mutex);
time_t current_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now());
@ -200,7 +200,7 @@ void DatabaseLazy::attachTable(const String & table_name, const StoragePtr & tab
std::forward_as_tuple(table_name),
std::forward_as_tuple(table, current_time, DatabaseOnDisk::getTableMetadataModificationTime(*this, table_name)));
if (!inserted)
throw Exception("Table " + getDatabaseName() + "." + table_name + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS);
throw Exception("Table " + backQuote(getDatabaseName()) + "." + backQuote(table_name) + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS);
it->second.expiration_iterator = cache_expiration_queue.emplace(cache_expiration_queue.end(), current_time, table_name);
}
@ -209,11 +209,11 @@ StoragePtr DatabaseLazy::detachTable(const String & table_name)
{
StoragePtr res;
{
LOG_DEBUG(log, "detach table" << table_name);
LOG_DEBUG(log, "Detach table " << backQuote(table_name) << ".");
std::lock_guard lock(tables_mutex);
auto it = tables_cache.find(table_name);
if (it == tables_cache.end())
throw Exception("Table " + getDatabaseName() + "." + table_name + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE);
throw Exception("Table " + backQuote(getDatabaseName()) + "." + backQuote(table_name) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE);
res = it->second.table;
if (it->second.expiration_iterator != cache_expiration_queue.end())
cache_expiration_queue.erase(it->second.expiration_iterator);
@ -274,11 +274,11 @@ String DatabaseLazy::getTableMetadataPath(const String & table_name) const
StoragePtr DatabaseLazy::loadTable(const Context & context, const String & table_name) const
{
clearExpiredTables();
SCOPE_EXIT({ clearExpiredTables(); });
LOG_DEBUG(log, "Load table '" << table_name << "' to cache.");
LOG_DEBUG(log, "Load table " << backQuote(table_name) << " to cache.");
const String table_metadata_path = getMetadataPath() + "/" + table_name + ".sql";
const String table_metadata_path = getMetadataPath() + "/" + escapeForFileName(table_name) + ".sql";
String s;
{
@ -292,8 +292,7 @@ StoragePtr DatabaseLazy::loadTable(const Context & context, const String & table
*/
if (s.empty())
{
// TODO: exception
LOG_ERROR(log, "loadTable: File " << table_metadata_path << " is empty. Removing.");
LOG_ERROR(log, "LoadTable: File " << table_metadata_path << " is empty. Removing.");
Poco::File(table_metadata_path).remove();
return nullptr;
}
@ -311,7 +310,7 @@ StoragePtr DatabaseLazy::loadTable(const Context & context, const String & table
std::lock_guard lock(tables_mutex);
auto it = tables_cache.find(table_name);
if (it == tables_cache.end())
throw Exception("Table " + getDatabaseName() + "." + table_name + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE);
throw Exception("Table " + backQuote(getDatabaseName()) + "." + backQuote(table_name) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE);
if (it->second.expiration_iterator != cache_expiration_queue.end())
cache_expiration_queue.erase(it->second.expiration_iterator);
@ -350,14 +349,14 @@ void DatabaseLazy::clearExpiredTables() const
if (!it->second.table || it->second.table.unique())
{
LOG_DEBUG(log, "Drop table '" << it->first << "' from cache.");
LOG_DEBUG(log, "Drop table " << backQuote(it->first) << " from cache.");
it->second.table.reset();
expired_tables.erase(it->second.expiration_iterator);
it->second.expiration_iterator = cache_expiration_queue.end();
}
else
{
LOG_DEBUG(log, "Table '" << it->first << "' is busy.");
LOG_DEBUG(log, "Table " << backQuote(it->first) << " is busy.");
busy_tables.splice(busy_tables.end(), expired_tables, it->second.expiration_iterator);
}
}

View File

@ -96,7 +96,7 @@ private:
time_t last_touched;
String table_name;
CacheExpirationQueueElement(time_t last_touched_, String table_name_)
CacheExpirationQueueElement(time_t last_touched_, const String & table_name_)
: last_touched(last_touched_), table_name(table_name_) {}
};

View File

@ -107,7 +107,7 @@ void DatabaseOnDisk::createTable(
/// But there is protection from it - see using DDLGuard in InterpreterCreateQuery.
if (database.isTableExist(context, table_name))
throw Exception("Table " + database.getDatabaseName() + "." + table_name + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS);
throw Exception("Table " + backQuote(database.getDatabaseName()) + "." + backQuote(table_name) + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS);
String table_metadata_path = database.getTableMetadataPath(table_name);
String table_metadata_tmp_path = table_metadata_path + ".tmp";
@ -187,7 +187,7 @@ ASTPtr DatabaseOnDisk::getCreateTableQueryImpl(const IDatabase & database, const
? "There is no CREATE TABLE query for table "
: "There is no metadata file for table ";
throw Exception(msg + table_name, ErrorCodes::CANNOT_GET_CREATE_TABLE_QUERY);
throw Exception(msg + backQuote(table_name), ErrorCodes::CANNOT_GET_CREATE_TABLE_QUERY);
}
return ast;

View File

@ -106,7 +106,7 @@ void DatabaseOnDisk::renameTable(
StoragePtr table = database.tryGetTable(context, table_name);
if (!table)
throw Exception("Table " + database.getDatabaseName() + "." + table_name + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE);
throw Exception("Table " + backQuote(database.getDatabaseName()) + "." + backQuote(table_name) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE);
/// Notify the table that it is renamed. If the table does not support renaming, exception is thrown.
try
@ -127,7 +127,7 @@ void DatabaseOnDisk::renameTable(
ASTPtr ast = detail::getQueryFromMetadata(detail::getTableMetadataPath(database.getMetadataPath(), table_name));
if (!ast)
throw Exception("There is no metadata file for table " + table_name, ErrorCodes::FILE_DOESNT_EXIST);
throw Exception("There is no metadata file for table " + backQuote(table_name) + ".", ErrorCodes::FILE_DOESNT_EXIST);
ast->as<ASTCreateQuery &>().table = to_table_name;
/// NOTE Non-atomic.

View File

@ -0,0 +1,2 @@
1 1
1 1

View File

@ -0,0 +1,11 @@
CREATE TABLE `таблица_со_странным_названием` (a UInt64, b UInt64) ENGINE = Log;
INSERT INTO `таблица_со_странным_названием` VALUES (1, 1);
SELECT * FROM `таблица_со_странным_названием`;
DROP TABLE `таблица_со_странным_названием`;
CREATE DATABASE testlazy ENGINE = Lazy(1);
CREATE TABLE testlazy.`таблица_со_странным_названием` (a UInt64, b UInt64) ENGINE = Log;
INSERT INTO testlazy.`таблица_со_странным_названием` VALUES (1, 1);
SELECT * FROM testlazy.`таблица_со_странным_названием`;
DROP TABLE testlazy.`таблица_со_странным_названием`;
DROP DATABASE testlazy