Merge pull request #1824 from zhang2014/fix/ISSUES-117

ISSUES-117 support temporary table management
This commit is contained in:
alexey-milovidov 2018-02-08 22:42:10 +03:00 committed by GitHub
commit d6b7233aa3
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
13 changed files with 85 additions and 32 deletions

View File

@ -46,11 +46,16 @@ BlockIO InterpreterDropQuery::execute()
}
/// Drop temporary table.
if (drop.database.empty())
if (drop.database.empty() || drop.temporary)
{
StoragePtr table = (context.hasSessionContext() ? context.getSessionContext() : context).tryRemoveExternalTable(drop.table);
if (table)
{
if (drop.database.empty() && !drop.temporary)
{
LOG_WARNING((&Logger::get("InterpreterDropQuery")),
"It is recommended to use `DROP TEMPORARY TABLE` to delete temporary tables");
}
table->shutdown();
/// If table was already dropped by anyone, an exception will be thrown
auto table_lock = table->lockForAlter(__PRETTY_FUNCTION__);

View File

@ -30,6 +30,9 @@ String InterpreterShowTablesQuery::getRewrittenQuery()
if (query.databases)
return "SELECT name FROM system.databases";
if (query.temporary && !query.from.empty())
throw Exception("The `FROM` and `TEMPORARY` cannot be used together in `SHOW TABLES`", ErrorCodes::SYNTAX_ERROR);
String database = query.from.empty() ? context.getCurrentDatabase() : query.from;
/** The parameter check_database_access_rights is reset when the SHOW TABLES query is processed,
@ -39,7 +42,12 @@ String InterpreterShowTablesQuery::getRewrittenQuery()
context.assertDatabaseExists(database, false);
std::stringstream rewritten_query;
rewritten_query << "SELECT name FROM system.tables WHERE database = " << std::quoted(database, '\'');
rewritten_query << "SELECT name FROM system.tables WHERE ";
if (query.temporary)
rewritten_query << "is_temporary";
else
rewritten_query << "database = " << std::quoted(database, '\'');
if (!query.like.empty())
rewritten_query << " AND name " << (query.not_like ? "NOT " : "") << "LIKE " << std::quoted(query.like, '\'');

View File

@ -15,6 +15,7 @@ class ASTDropQuery : public ASTQueryWithOutput, public ASTQueryWithOnCluster
public:
bool detach{false}; /// DETACH query, not DROP.
bool if_exists{false};
bool temporary{false};
String database;
String table;

View File

@ -15,6 +15,7 @@ class ASTShowTablesQuery : public ASTQueryWithOutput
{
public:
bool databases{false};
bool temporary{false};
String from;
String like;
bool not_like{false};

View File

@ -17,6 +17,7 @@ bool ParserDropQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
ParserKeyword s_drop("DROP");
ParserKeyword s_detach("DETACH");
ParserKeyword s_temporary("TEMPORARY");
ParserKeyword s_table("TABLE");
ParserKeyword s_database("DATABASE");
ParserToken s_dot(TokenType::Dot);
@ -28,6 +29,7 @@ bool ParserDropQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
String cluster_str;
bool detach = false;
bool if_exists = false;
bool temporary = false;
if (!s_drop.ignore(pos, expected))
{
@ -53,6 +55,9 @@ bool ParserDropQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
}
else
{
if (s_temporary.ignore(pos, expected))
temporary = true;
if (!s_table.ignore(pos, expected))
return false;
@ -81,6 +86,7 @@ bool ParserDropQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
query->detach = detach;
query->if_exists = if_exists;
query->temporary = temporary;
if (database)
query->database = typeid_cast<ASTIdentifier &>(*database).name;
if (table)

View File

@ -18,6 +18,7 @@ bool ParserShowTablesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
Pos begin = pos;
ParserKeyword s_show("SHOW");
ParserKeyword s_temporary("TEMPORARY");
ParserKeyword s_tables("TABLES");
ParserKeyword s_databases("DATABASES");
ParserKeyword s_from("FROM");
@ -38,27 +39,33 @@ bool ParserShowTablesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
{
query->databases = true;
}
else if (s_tables.ignore(pos, expected))
else
{
if (s_from.ignore(pos, expected))
if (s_temporary.ignore(pos))
query->temporary = true;
if (s_tables.ignore(pos, expected))
{
if (!name_p.parse(pos, database, expected))
if (s_from.ignore(pos, expected))
{
if (!name_p.parse(pos, database, expected))
return false;
}
if (s_not.ignore(pos, expected))
query->not_like = true;
if (s_like.ignore(pos, expected))
{
if (!like_p.parse(pos, like, expected))
return false;
}
else if (query->not_like)
return false;
}
if (s_not.ignore(pos, expected))
query->not_like = true;
if (s_like.ignore(pos, expected))
{
if (!like_p.parse(pos, like, expected))
return false;
}
else if (query->not_like)
else
return false;
}
else
return false;
query->range = StringRange(begin, pos);

View File

@ -14,7 +14,7 @@ namespace DB
class ParserShowTablesQuery : public IParserBase
{
protected:
const char * getName() const { return "SHOW TABLES|DATABASES query"; }
const char * getName() const { return "SHOW [TEMPORARY] TABLES|DATABASES [[NOT] LIKE 'str']"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected);
};

View File

@ -7,7 +7,7 @@
#include <Storages/VirtualColumnUtils.h>
#include <Databases/IDatabase.h>
#include <Interpreters/Context.h>
#include <DataTypes/DataTypesNumber.h>
namespace DB
{
@ -19,7 +19,8 @@ StorageSystemTables::StorageSystemTables(const std::string & name_)
{"database", std::make_shared<DataTypeString>()},
{"name", std::make_shared<DataTypeString>()},
{"engine", std::make_shared<DataTypeString>()},
{"metadata_modification_time", std::make_shared<DataTypeDateTime>()}
{"metadata_modification_time", std::make_shared<DataTypeDateTime>()},
{"is_temporary", std::make_shared<DataTypeUInt8>()}
};
}
@ -54,6 +55,7 @@ BlockInputStreams StorageSystemTables::read(
for (size_t row_number = 0; row_number < filtered_databases_column->size(); ++row_number)
{
std::string database_name = filtered_databases_column->getDataAt(row_number).toString();
auto database = context.tryGetDatabase(database_name);
if (!database)
@ -68,10 +70,28 @@ BlockInputStreams StorageSystemTables::read(
res_columns[0]->insert(database_name);
res_columns[1]->insert(table_name);
res_columns[2]->insert(iterator->table()->getName());
res_columns[3]->insert(static_cast<UInt64>(database->getTableMetadataModificationTime(context, table_name)));
res_columns[3]->insert(
static_cast<UInt64>(database->getTableMetadataModificationTime(context, table_name)));
res_columns[4]->insert(UInt64(0));
}
}
if (context.hasSessionContext())
{
Tables external_tables = context.getSessionContext().getExternalTables();
for (auto table : external_tables)
{
res_columns[0]->insert(String{});
res_columns[1]->insert(table.first);
res_columns[2]->insert(table.second->getName());
res_columns[3]->insert(UInt64(0));
res_columns[4]->insert(UInt64(1));
}
}
return BlockInputStreams(1, std::make_shared<OneBlockInputStream>(getSampleBlock().cloneWithColumns(std::move(res_columns))));
}

View File

@ -1,8 +1,8 @@
drop table if exists temp_tab;
create temporary table temp_tab (number UInt64);
insert into temp_tab select number from system.numbers limit 1;
select number from temp_tab;
drop table temp_tab;
create temporary table temp_tab (number UInt64);
select number from temp_tab;
drop table temp_tab;
DROP TEMPORARY TABLE IF EXISTS temp_tab;
CREATE TEMPORARY TABLE temp_tab (number UInt64);
INSERT INTO temp_tab SELECT number FROM system.numbers LIMIT 1;
SELECT number FROM temp_tab;
DROP TABLE temp_tab;
CREATE TEMPORARY TABLE temp_tab (number UInt64);
SELECT number FROM temp_tab;
DROP TEMPORARY TABLE temp_tab;

View File

@ -16,4 +16,4 @@ SELECT '---';
SELECT ID FROM readonly00542 ORDER BY ID;
DROP TABLE readonly00542;
DROP TEMPORARY TABLE readonly00542;

View File

@ -51,7 +51,7 @@ $CLICKHOUSE_CLIENT -n --query="
ID Int
) Engine=Memory;
INSERT INTO readonly (ID) VALUES (1);
DROP TABLE readonly;
DROP TEMPORARY TABLE readonly;
" 2> /dev/null;
CODE=$?;
[ "$CODE" -ne "0" ] && echo "Fail" && exit $CODE;
@ -83,7 +83,7 @@ $CLICKHOUSE_CLIENT -n --query="
ID Int
) Engine=Memory;
INSERT INTO readonly (ID) VALUES (1);
DROP TABLE readonly;
DROP TEMPORARY TABLE readonly;
" 2> /dev/null;
CODE=$?;
[ "$CODE" -ne "0" ] && echo "Fail" && exit $CODE;

View File

@ -0,0 +1 @@
temp_tab

View File

@ -0,0 +1,4 @@
DROP TEMPORARY TABLE IF EXISTS temp_tab;
CREATE TEMPORARY TABLE temp_tab (number UInt64);
SHOW TEMPORARY TABLES LIKE 'temp_tab';
DROP TEMPORARY TABLE temp_tab;