Adding DETACH TABLE PERMANENTLY

This commit is contained in:
Mikhail Filimonov 2020-11-30 18:52:32 +01:00
parent dd2ae6926d
commit f6097cf818
No known key found for this signature in database
GPG Key ID: 6E49C2E9AF1220BE
12 changed files with 261 additions and 8 deletions

View File

@ -170,14 +170,19 @@ void DatabaseOnDisk::createTable(
if (isTableExist(table_name, global_context))
throw Exception("Table " + backQuote(getDatabaseName()) + "." + backQuote(table_name) + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS);
if (create.attach_short_syntax)
String table_metadata_path = getObjectMetadataPath(table_name);
if (create.attach_short_syntax && Poco::File(table_metadata_path).exists())
{
/// Metadata already exists, table was detached
/// Metadata already exists, table was detached (not permanently)
attachTable(table_name, table, getTableDataPath(create));
return;
/// if the table was detached permanently, then usual metadata file doesn't exists
/// (.sql_detached instead) and we use longer, but safer way of attaching that back
/// with recreating the metadata file.
}
String table_metadata_path = getObjectMetadataPath(table_name);
String table_metadata_tmp_path = table_metadata_path + create_suffix;
String statement;
@ -213,8 +218,47 @@ void DatabaseOnDisk::commitCreateTable(const ASTCreateQuery & query, const Stora
Poco::File(table_metadata_tmp_path).remove();
throw;
}
try
{
/// If the table was detached permanently we will have a file with
/// .sql_detached suffix, which is not needed anymore since we attached the table back
auto table_metadata_file_detached = Poco::File(table_metadata_path + detached_suffix);
if (table_metadata_file_detached.exists())
table_metadata_file_detached.remove();
}
catch (...)
{
// It's not a big issue if we can't remove the .sql_detached file.
LOG_WARNING(log, getCurrentExceptionMessage(__PRETTY_FUNCTION__));
}
}
void DatabaseOnDisk::detachTablePermanently(const String & table_name)
{
StoragePtr table = detachTable(table_name);
/// This is possible for Lazy database.
if (!table)
return;
String table_metadata_path = getObjectMetadataPath(table_name);
String table_metadata_path_detached = table_metadata_path + detached_suffix;
try
{
/// it will silently overwrite the file if exists, and it's ok
Poco::File(table_metadata_path).renameTo(table_metadata_path_detached);
}
catch (Exception & e)
{
e.addMessage("while trying to detach table {} permanently.", table_name);
throw;
}
}
void DatabaseOnDisk::dropTable(const Context & context, const String & table_name, bool /*no_delay*/)
{
String table_metadata_path = getObjectMetadataPath(table_name);
@ -328,14 +372,22 @@ void DatabaseOnDisk::renameTable(
}
}
/// It returns create table statement (even if table is detached permanently)
ASTPtr DatabaseOnDisk::getCreateTableQueryImpl(const String & table_name, const Context &, bool throw_on_error) const
{
ASTPtr ast;
bool has_table = tryGetTable(table_name, global_context) != nullptr;
auto table_metadata_path = getObjectMetadataPath(table_name);
try
{
ast = getCreateQueryFromMetadata(table_metadata_path, throw_on_error);
if (Poco::File(table_metadata_path).exists())
ast = getCreateQueryFromMetadata(table_metadata_path, throw_on_error);
else if (Poco::File(table_metadata_path + detached_suffix).exists())
ast = getCreateQueryFromMetadata(table_metadata_path + detached_suffix, throw_on_error);
else if (throw_on_error)
throw Exception("Metadata file does not exist", ErrorCodes::FILE_DOESNT_EXIST);
}
catch (const Exception & e)
{
@ -430,6 +482,10 @@ void DatabaseOnDisk::iterateMetadataFiles(const Context & context, const Iterati
if (endsWith(dir_it.name(), ".sql.bak"))
continue;
/// Permanently detached tables are not attached automatically
if (endsWith(dir_it.name(), ".sql_detached"))
continue;
static const char * tmp_drop_ext = ".sql.tmp_drop";
if (endsWith(dir_it.name(), tmp_drop_ext))
{
@ -510,6 +566,8 @@ ASTPtr DatabaseOnDisk::parseQueryFromMetadata(Poco::Logger * logger, const Conte
auto & create = ast->as<ASTCreateQuery &>();
if (!create.table.empty() && create.uuid != UUIDHelpers::Nil)
{
/// if the table is detached permanently getBaseName will still return a proper name
/// because we use table_name.sql_detached naming
String table_name = Poco::Path(metadata_file_path).makeFile().getBaseName();
table_name = unescapeForFileName(table_name);

View File

@ -39,6 +39,8 @@ public:
const StoragePtr & table,
const ASTPtr & query) override;
void detachTablePermanently(const String & table_name) override;
void dropTable(
const Context & context,
const String & table_name,
@ -71,6 +73,11 @@ protected:
static constexpr const char * create_suffix = ".tmp";
static constexpr const char * drop_suffix = ".tmp_drop";
/// engine=Atomic takes the table name from basename of metadata file (also for detached table)
/// in case of double dots (table_name.sql.detached) it would extract 'table_name.sql'
/// so we use simpler option "table_name.sql_detached" and get 'table_name' correctly.
static constexpr const char * detached_suffix = "_detached";
using IteratingFunction = std::function<void(const String &)>;
void iterateMetadataFiles(const Context & context, const IteratingFunction & process_metadata_file) const;

View File

@ -221,6 +221,8 @@ public:
}
/// Add a table to the database, but do not add it to the metadata. The database may not support this method.
///
/// Note: ATTACH TABLE statement actually uses createTable method.
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);
@ -245,6 +247,13 @@ public:
throw Exception("There is no DETACH DICTIONARY query for Database" + getEngineName(), ErrorCodes::NOT_IMPLEMENTED);
}
/// Forget about the table without deleting it's data, but rename metadata file to prevent reloading it
/// with next restart. The database may not support this method.
virtual void detachTablePermanently(const String & /*name*/)
{
throw Exception("There is no DETACH TABLE PERMANENTLY query for Database" + getEngineName(), ErrorCodes::NOT_IMPLEMENTED);
}
/// Rename the table and possibly move the table to another database.
virtual void renameTable(
const Context & /*context*/,

View File

@ -734,7 +734,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create)
auto database = DatabaseCatalog::instance().getDatabase(database_name);
bool if_not_exists = create.if_not_exists;
// Table SQL definition is available even if the table is detached
// Table SQL definition is available even if the table is detached (even permanently)
auto query = database->getCreateTableQuery(create.table, context);
create = query->as<ASTCreateQuery &>(); // Copy the saved create query, but use ATTACH instead of CREATE
if (create.is_dictionary)

View File

@ -128,8 +128,18 @@ BlockIO InterpreterDropQuery::executeToTableImpl(const ASTDropQuery & query, Dat
TableExclusiveLockHolder table_lock;
if (database->getUUID() == UUIDHelpers::Nil)
table_lock = table->lockExclusively(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout);
/// Drop table from memory, don't touch data and metadata
database->detachTable(table_id.table_name);
if (query.permanently)
{
/// Drop table from memory, don't touch data, metadata file renamed and will be skipped during server restart
database->detachTablePermanently(table_id.table_name);
}
else
{
/// Drop table from memory, don't touch data and metadata
database->detachTable(table_id.table_name);
}
}
else if (query.kind == ASTDropQuery::Kind::Truncate)
{

View File

@ -108,6 +108,7 @@ void loadMetadata(Context & context, const String & default_database_name)
if (!it->isDirectory())
{
/// TODO: DETACH DATABASE PERMANENTLY ?
if (endsWith(it.name(), ".sql"))
{
String db_name = it.name().substr(0, it.name().size() - 4);

View File

@ -67,6 +67,9 @@ void ASTDropQuery::formatQueryImpl(const FormatSettings & settings, FormatState
formatOnCluster(settings);
if (permanently)
settings.ostr << " PERMANENTLY";
if (no_delay)
settings.ostr << (settings.hilite ? hilite_keyword : "") << " NO DELAY" << (settings.hilite ? hilite_none : "");
}

View File

@ -33,6 +33,9 @@ public:
bool no_delay{false};
// We detach the object permanently, so it will not be reattached back during server restart.
bool permanently{false};
/** Get the text that identifies this element. */
String getID(char) const override;
ASTPtr clone() const override;

View File

@ -21,6 +21,7 @@ bool parseDropQuery(IParser::Pos & pos, ASTPtr & node, Expected & expected, bool
ParserToken s_dot(TokenType::Dot);
ParserKeyword s_if_exists("IF EXISTS");
ParserIdentifier name_p;
ParserKeyword s_permanently("PERMANENTLY");
ParserKeyword s_no_delay("NO DELAY");
ParserKeyword s_sync("SYNC");
@ -32,6 +33,7 @@ bool parseDropQuery(IParser::Pos & pos, ASTPtr & node, Expected & expected, bool
bool is_dictionary = false;
bool is_view = false;
bool no_delay = false;
bool permanently = false;
if (s_database.ignore(pos, expected))
{
@ -83,6 +85,9 @@ bool parseDropQuery(IParser::Pos & pos, ASTPtr & node, Expected & expected, bool
return false;
}
if (s_permanently.ignore(pos, expected))
permanently = true;
if (s_no_delay.ignore(pos, expected) || s_sync.ignore(pos, expected))
no_delay = true;
}
@ -96,6 +101,7 @@ bool parseDropQuery(IParser::Pos & pos, ASTPtr & node, Expected & expected, bool
query->is_dictionary = is_dictionary;
query->is_view = is_view;
query->no_delay = no_delay;
query->permanently = permanently;
tryGetIdentifierNameInto(database, query->database);
tryGetIdentifierNameInto(table, query->table);

View File

@ -8,7 +8,7 @@ namespace DB
{
/** Query like this:
* DROP|DETACH|TRUNCATE TABLE [IF EXISTS] [db.]name
* DROP|DETACH|TRUNCATE TABLE [IF EXISTS] [db.]name [PERMANENTLY]
*
* Or:
* DROP DATABASE [IF EXISTS] db

View File

@ -0,0 +1,84 @@
#####
db_ordinary.log_table 1
>table detached!
db_ordinary.log_table 2
>Table is back after restart
10
db_ordinary.log_table 3
>table detached (permanently)!
db_ordinary.log_table 4
>table is still detached (after restart)!
db_ordinary.log_table 5
>Table is back after attach
db_ordinary.log_table 6
10
#####
db_ordinary.mt_table 1
>table detached!
db_ordinary.mt_table 2
>Table is back after restart
10
db_ordinary.mt_table 3
>table detached (permanently)!
db_ordinary.mt_table 4
>table is still detached (after restart)!
db_ordinary.mt_table 5
>Table is back after attach
db_ordinary.mt_table 6
10
#####
db_ordinary.null_table 1
>table detached!
db_ordinary.null_table 2
>Table is back after restart
0
db_ordinary.null_table 3
>table detached (permanently)!
db_ordinary.null_table 4
>table is still detached (after restart)!
db_ordinary.null_table 5
>Table is back after attach
db_ordinary.null_table 6
0
#####
db_atomic.log_table 1
>table detached!
db_atomic.log_table 2
>Table is back after restart
10
db_atomic.log_table 3
>table detached (permanently)!
db_atomic.log_table 4
>table is still detached (after restart)!
db_atomic.log_table 5
>Table is back after attach
db_atomic.log_table 6
10
#####
db_atomic.mt_table 1
>table detached!
db_atomic.mt_table 2
>Table is back after restart
10
db_atomic.mt_table 3
>table detached (permanently)!
db_atomic.mt_table 4
>table is still detached (after restart)!
db_atomic.mt_table 5
>Table is back after attach
db_atomic.mt_table 6
10
#####
db_atomic.null_table 1
>table detached!
db_atomic.null_table 2
>Table is back after restart
0
db_atomic.null_table 3
>table detached (permanently)!
db_atomic.null_table 4
>table is still detached (after restart)!
db_atomic.null_table 5
>Table is back after attach
db_atomic.null_table 6
0

View File

@ -0,0 +1,72 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. "$CURDIR"/../shell_config.sh
## tests with real clickhouse restart would be a bit to heavy,
## to ensure the table will not reappear back clickhose-local is enough.
# TODO: clean it...
# CLICKHOUSE_LOCAL="/home/mfilimonov/workspace/ClickHouse-detach-permanently/build/programs/clickhouse-local"
# CLICKHOUSE_TMP=$(pwd)
WORKING_FOLDER_01600="${CLICKHOUSE_TMP}/01600_detach_permanently"
rm -rf "${WORKING_FOLDER_01600}"
mkdir -p "${WORKING_FOLDER_01600}"
clickhouse_local() {
local query="$1"
shift
${CLICKHOUSE_LOCAL} --query "$query" $@ -- --path="${WORKING_FOLDER_01600}"
}
test_detach_attach_sequence() {
local db="$1"
local table="$2"
echo "#####"
echo "${db}.${table} 1"
# normal DETACH - while process is running (clickhouse-local here, same for server) table is detached.
clickhouse_local "DETACH TABLE ${db}.${table}; SELECT if( count() = 0, '>table detached!', '>Fail') FROM system.tables WHERE database='${db}' AND name='${table}';"
# but once we restart the precess (either clickhouse-local either clickhouse server) the table is back.
echo "${db}.${table} 2"
clickhouse_local "SELECT if(name='${table}', '>Table is back after restart', '>fail') FROM system.tables WHERE database='${db}' AND name='${table}'; SELECT count() FROM ${db}.${table};"
# permanent DETACH - table is detached, and metadata file renamed, prevening further attach
echo "${db}.${table} 3"
clickhouse_local "DETACH TABLE ${db}.${table} PERMANENTLY; SELECT if( count() = 0, '>table detached (permanently)!', '>Fail') FROM system.tables WHERE database='${db}' AND name='${table}';"
# still detached after restart
echo "${db}.${table} 4"
clickhouse_local "SELECT if( count() = 0, '>table is still detached (after restart)!', '>Fail') FROM system.tables WHERE database='${db}' AND name='${table}';"
# but can be reattached
echo "${db}.${table} 5"
clickhouse_local "ATTACH TABLE ${db}.${table}; SELECT if(name='${table}', '>Table is back after attach', '>fail') FROM system.tables WHERE database='${db}' AND name='${table}';"
echo "${db}.${table} 6"
clickhouse_local "SELECT count() FROM ${db}.${table};"
}
clickhouse_local "DROP DATABASE IF EXISTS db_ordinary SYNC;"
clickhouse_local "DROP DATABASE IF EXISTS db_atomic SYNC;"
clickhouse_local "CREATE DATABASE db_ordinary Engine=Ordinary"
clickhouse_local "CREATE DATABASE db_atomic Engine=Atomic"
clickhouse_local "CREATE TABLE db_ordinary.log_table Engine=Log AS SELECT * FROM numbers(10)"
clickhouse_local "CREATE TABLE db_ordinary.mt_table Engine=MergeTree ORDER BY tuple() AS SELECT * FROM numbers(10)"
clickhouse_local "CREATE TABLE db_ordinary.null_table Engine=Null AS SELECT * FROM numbers(10)"
clickhouse_local "CREATE TABLE db_atomic.log_table Engine=Log AS SELECT * FROM numbers(10)"
clickhouse_local "CREATE TABLE db_atomic.mt_table Engine=MergeTree ORDER BY tuple() AS SELECT * FROM numbers(10)"
clickhouse_local "CREATE TABLE db_atomic.null_table Engine=Null AS SELECT * FROM numbers(10)"
test_detach_attach_sequence "db_ordinary" "log_table"
test_detach_attach_sequence "db_ordinary" "mt_table"
test_detach_attach_sequence "db_ordinary" "null_table"
test_detach_attach_sequence "db_atomic" "log_table"
test_detach_attach_sequence "db_atomic" "mt_table"
test_detach_attach_sequence "db_atomic" "null_table"