implement ATTACH TABLE name FROM 'path/to/data/

This commit is contained in:
Alexander Tokmakov 2020-12-08 17:13:35 +03:00
parent 57f40e4260
commit 04ceaa66f2
7 changed files with 128 additions and 9 deletions

View File

@ -7,6 +7,7 @@
#include <Common/typeid_cast.h>
#include <Common/Macros.h>
#include <Common/randomSeed.h>
#include <Common/renameat2.h>
#include <Core/Defines.h>
#include <Core/Settings.h>
@ -56,6 +57,7 @@
#include <Interpreters/addTypeConversionToAST.h>
#include <TableFunctions/TableFunctionFactory.h>
#include <common/logger_useful.h>
namespace DB
@ -76,6 +78,7 @@ namespace ErrorCodes
extern const int ILLEGAL_SYNTAX_FOR_DATA_TYPE;
extern const int ILLEGAL_COLUMN;
extern const int LOGICAL_ERROR;
extern const int PATH_ACCESS_DENIED;
}
namespace fs = std::filesystem;
@ -143,7 +146,8 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create)
if (create.storage->engine->name == "Atomic")
{
if (create.attach && create.uuid == UUIDHelpers::Nil)
throw Exception("UUID must be specified for ATTACH", ErrorCodes::INCORRECT_QUERY);
throw Exception(ErrorCodes::INCORRECT_QUERY, "UUID must be specified for ATTACH. "
"If you want to attach existing database, use just ATTACH DATABASE {};", create.database);
else if (create.uuid == UUIDHelpers::Nil)
create.uuid = UUIDHelpers::generateV4();
@ -209,7 +213,8 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create)
if (need_write_metadata)
{
fs::rename(metadata_file_tmp_path, metadata_file_path);
/// Prevents from overwriting metadata of detached database
renameNoReplace(metadata_file_tmp_path, metadata_file_path);
renamed = true;
}
@ -647,13 +652,27 @@ void InterpreterCreateQuery::assertOrSetUUID(ASTCreateQuery & create, const Data
const auto * kind = create.is_dictionary ? "Dictionary" : "Table";
const auto * kind_upper = create.is_dictionary ? "DICTIONARY" : "TABLE";
bool from_path = create.attach_from_path.has_value();
if (database->getEngineName() == "Atomic")
{
if (create.attach && create.uuid == UUIDHelpers::Nil)
if (create.attach && !from_path && create.uuid == UUIDHelpers::Nil)
{
throw Exception(ErrorCodes::INCORRECT_QUERY,
"UUID must be specified in ATTACH {} query for Atomic database engine",
kind_upper);
if (!create.attach && create.uuid == UUIDHelpers::Nil)
"Incorrect ATTACH {} query for Atomic database engine. "
"Use one of the following queries instead:\n"
"1. ATTACH {} {};\n"
"2. CREATE {} {} <table definition>;\n"
"3. ATTACH {} {} FROM '/path/to/data/' <table definition>;\n"
"4. ATTACH {} {} UUID '<uuid>' <table definition>;",
kind_upper,
kind_upper, create.table,
kind_upper, create.table,
kind_upper, create.table,
kind_upper, create.table);
}
if (create.uuid == UUIDHelpers::Nil)
create.uuid = UUIDHelpers::generateV4();
}
else
@ -696,7 +715,32 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create)
create.attach_short_syntax = true;
create.if_not_exists = if_not_exists;
}
/// TODO maybe assert table structure if create.attach_short_syntax is false?
/// TODO throw exception if !create.attach_short_syntax && !create.attach_from_path && !internal
if (create.attach_from_path)
{
fs::path data_path = fs::path(*create.attach_from_path).lexically_normal();
fs::path user_files = fs::path(context.getUserFilesPath()).lexically_normal();
if (data_path.is_relative())
data_path = (user_files / data_path).lexically_normal();
if (!startsWith(data_path, user_files))
throw Exception(ErrorCodes::PATH_ACCESS_DENIED,
"Data directory {} must be inside {} to attach it", String(data_path), String(user_files));
fs::path root_path = fs::path(context.getPath()).lexically_normal();
/// Data path must be relative to root_path
create.attach_from_path = fs::relative(data_path, root_path) / "";
}
else if (create.attach && !create.attach_short_syntax)
{
auto * log = &Poco::Logger::get("InterpreterCreateQuery");
LOG_WARNING(log, "ATTACH TABLE query with full table definition is not recommended: "
"use either ATTACH TABLE {}; to attach existing table "
"or CREATE TABLE {} <table definition>; to create new table "
"or ATTACH TABLE {} FROM '/path/to/data/' <table definition>; to create new table and attach data.",
create.table, create.table, create.table);
}
if (!create.temporary && create.database.empty())
create.database = current_database;
@ -775,6 +819,18 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create,
return true;
}
bool from_path = create.attach_from_path.has_value();
String actual_data_path = data_path;
if (from_path)
{
if (data_path.empty())
throw Exception(ErrorCodes::NOT_IMPLEMENTED,
"ATTACH ... FROM ... query is not supported for {} database engine", database->getEngineName());
/// We will try to create Storage instance with provided data path
data_path = *create.attach_from_path;
create.attach_from_path = std::nullopt;
}
StoragePtr res;
/// NOTE: CREATE query may be rewritten by Storage creator or table function
if (create.as_table_function)
@ -786,7 +842,7 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create,
else
{
res = StorageFactory::instance().get(create,
database ? database->getTableDataPath(create) : "",
data_path,
context,
context.getGlobalContext(),
properties.columns,
@ -794,8 +850,18 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create,
false);
}
if (from_path && !res->storesDataOnDisk())
throw Exception(ErrorCodes::NOT_IMPLEMENTED,
"ATTACH ... FROM ... query is not supported for {} table engine, "
"because such tables do not store any data on disk. Use CREATE instead.", res->getName());
database->createTable(context, table_name, res, query_ptr);
/// Move table data to the proper place. Wo do not move data earlier to avoid situations
/// when data directory moved, but table has not been created due to some error.
if (from_path)
res->rename(actual_data_path, {create.database, create.table, create.uuid});
/// We must call "startup" and "shutdown" while holding DDLGuard.
/// Because otherwise method "shutdown" (from InterpreterDropQuery) can be called before startup
/// (in case when table was created and instantly dropped before started up)

View File

@ -251,6 +251,12 @@ void ASTCreateQuery::formatQueryImpl(const FormatSettings & settings, FormatStat
if (uuid != UUIDHelpers::Nil)
settings.ostr << (settings.hilite ? hilite_keyword : "") << " UUID " << (settings.hilite ? hilite_none : "")
<< quoteString(toString(uuid));
assert(attach || !attach_from_path);
if (attach_from_path)
settings.ostr << (settings.hilite ? hilite_keyword : "") << " FROM " << (settings.hilite ? hilite_none : "")
<< quoteString(*attach_from_path);
if (live_view_timeout)
settings.ostr << (settings.hilite ? hilite_keyword : "") << " WITH TIMEOUT " << (settings.hilite ? hilite_none : "")
<< *live_view_timeout;

View File

@ -79,6 +79,8 @@ public:
std::optional<UInt64> live_view_timeout; /// For CREATE LIVE VIEW ... WITH TIMEOUT ...
bool attach_short_syntax{false};
std::optional<String> attach_from_path = std::nullopt;
/** Get the text that identifies this element. */
String getID(char delim) const override { return (attach ? "AttachQuery" : "CreateQuery") + (delim + database) + delim + table; }

View File

@ -359,6 +359,7 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe
ParserKeyword s_table("TABLE");
ParserKeyword s_if_not_exists("IF NOT EXISTS");
ParserCompoundIdentifier table_name_p(true);
ParserKeyword s_from("FROM");
ParserKeyword s_on("ON");
ParserKeyword s_as("AS");
ParserToken s_dot(TokenType::Dot);
@ -378,6 +379,7 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe
ASTPtr as_table;
ASTPtr as_table_function;
ASTPtr select;
ASTPtr from_path;
String cluster_str;
bool attach = false;
@ -405,6 +407,13 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe
if (!table_name_p.parse(pos, table, expected))
return false;
if (attach && s_from.ignore(pos, expected))
{
ParserLiteral from_path_p;
if (!from_path_p.parse(pos, from_path, expected))
return false;
}
if (s_on.ignore(pos, expected))
{
if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected))
@ -414,7 +423,8 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe
StorageID table_id = getTableIdentifier(table);
// Shortcut for ATTACH a previously detached table
if (attach && (!pos.isValid() || pos.get().type == TokenType::Semicolon))
bool short_attach = attach && !from_path;
if (short_attach && (!pos.isValid() || pos.get().type == TokenType::Semicolon))
{
auto query = std::make_shared<ASTCreateQuery>();
node = query;
@ -509,6 +519,9 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe
tryGetIdentifierNameInto(as_table, query->as_table);
query->set(query->select, select);
if (from_path)
query->attach_from_path = from_path->as<ASTLiteral &>().value.get<String>();
return true;
}

View File

@ -528,7 +528,10 @@ BlockOutputStreamPtr StorageFile::write(
std::string path;
if (!paths.empty())
{
path = paths[0];
Poco::File(Poco::Path(path).makeParent()).createDirectories();
}
return std::make_shared<StorageFileBlockOutputStream>(*this, metadata_snapshot,
chooseCompressionMethod(path, compression_method), context,

View File

@ -0,0 +1,4 @@
file 42
file 42
42 mt
42 mt

View File

@ -0,0 +1,25 @@
drop table if exists test;
drop table if exists file;
drop table if exists mt;
attach table test from 'some/path' (n UInt8) engine=Memory; -- { serverError 48 }
attach table test from '/etc/passwd' (s String) engine=File(TSVRaw); -- { serverError 481 }
attach table test from '../../../../../../../../../etc/passwd' (s String) engine=File(TSVRaw); -- { serverError 481 }
insert into table function file('01188_attach/file/data.TSV', 'TSV', 's String, n UInt8') values ('file', 42);
attach table file from '01188_attach/file' (s String, n UInt8) engine=File(TSV);
select * from file;
detach table file;
attach table file;
select * from file;
attach table mt from '01188_attach/file' (n UInt8, s String) engine=MergeTree order by n;
select * from mt;
insert into mt values (42, 'mt');
select * from mt;
detach table mt;
attach table mt;
select * from mt;
drop table file;
drop table mt;