mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 15:42:02 +00:00
Merge pull request #2033 from yandex/show-create-database
Show create database
This commit is contained in:
commit
ca5fd1e9de
@ -3,15 +3,21 @@
|
||||
#include <Interpreters/ExternalDictionaries.h>
|
||||
#include <Storages/StorageDictionary.h>
|
||||
#include <common/logger_useful.h>
|
||||
#include <Parsers/IAST.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <Parsers/ParserCreateQuery.h>
|
||||
#include <Parsers/parseQuery.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int TABLE_ALREADY_EXISTS;
|
||||
extern const int UNKNOWN_TABLE;
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int CANNOT_GET_CREATE_TABLE_QUERY;
|
||||
extern const int TABLE_ALREADY_EXISTS;
|
||||
extern const int UNKNOWN_TABLE;
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int CANNOT_GET_CREATE_TABLE_QUERY;
|
||||
extern const int SYNTAX_ERROR;
|
||||
}
|
||||
|
||||
DatabaseDictionary::DatabaseDictionary(const String & name_, const Context & context)
|
||||
@ -153,11 +159,54 @@ time_t DatabaseDictionary::getTableMetadataModificationTime(
|
||||
return static_cast<time_t>(0);
|
||||
}
|
||||
|
||||
ASTPtr DatabaseDictionary::getCreateQuery(
|
||||
const Context &,
|
||||
const String &) const
|
||||
ASTPtr DatabaseDictionary::getCreateTableQueryImpl(const Context & context,
|
||||
const String & table_name, bool throw_on_error) const
|
||||
{
|
||||
throw Exception("There is no CREATE TABLE query for DatabaseDictionary tables", ErrorCodes::CANNOT_GET_CREATE_TABLE_QUERY);
|
||||
String query;
|
||||
{
|
||||
WriteBufferFromString buffer(query);
|
||||
|
||||
const auto & dictionaries = context.getExternalDictionaries();
|
||||
auto dictionary = throw_on_error ? dictionaries.getDictionary(table_name)
|
||||
: dictionaries.tryGetDictionary(table_name);
|
||||
|
||||
auto names_and_types = StorageDictionary::getNamesAndTypes(dictionary->getStructure());
|
||||
buffer << "CREATE TABLE " << backQuoteIfNeed(name) << '.' << backQuoteIfNeed(table_name) << " (";
|
||||
buffer << StorageDictionary::generateNamesAndTypesDescription(names_and_types.begin(), names_and_types.end());
|
||||
buffer << ") Engine = Dictionary(" << backQuoteIfNeed(table_name) << ")";
|
||||
}
|
||||
|
||||
ParserCreateQuery parser;
|
||||
const char * pos = query.data();
|
||||
std::string error_message;
|
||||
auto ast = tryParseQuery(parser, pos, pos + query.size(), error_message,
|
||||
/* hilite = */ false, "", /* allow_multi_statements = */ false);
|
||||
|
||||
if (!ast && throw_on_error)
|
||||
throw Exception(error_message, ErrorCodes::SYNTAX_ERROR);
|
||||
|
||||
return ast;
|
||||
}
|
||||
|
||||
ASTPtr DatabaseDictionary::getCreateTableQuery(const Context & context, const String & table_name) const
|
||||
{
|
||||
return getCreateTableQueryImpl(context, table_name, true);
|
||||
}
|
||||
|
||||
ASTPtr DatabaseDictionary::tryGetCreateTableQuery(const Context & context, const String & table_name) const
|
||||
{
|
||||
return getCreateTableQueryImpl(context, table_name, false);
|
||||
}
|
||||
|
||||
ASTPtr DatabaseDictionary::getCreateDatabaseQuery(const Context & /*context*/) const
|
||||
{
|
||||
String query;
|
||||
{
|
||||
WriteBufferFromString buffer(query);
|
||||
buffer << "CREATE DATABASE " << backQuoteIfNeed(name) << " ENGINE = Dictionary";
|
||||
}
|
||||
ParserCreateQuery parser;
|
||||
return parseQuery(parser, query.data(), query.data() + query.size(), "");
|
||||
}
|
||||
|
||||
void DatabaseDictionary::shutdown()
|
||||
|
@ -22,16 +22,6 @@ class ExternalDictionaries;
|
||||
*/
|
||||
class DatabaseDictionary : public IDatabase
|
||||
{
|
||||
private:
|
||||
const String name;
|
||||
mutable std::mutex mutex;
|
||||
const ExternalDictionaries & external_dictionaries;
|
||||
std::unordered_set<String> deleted_tables;
|
||||
|
||||
Poco::Logger * log;
|
||||
|
||||
Tables loadTables();
|
||||
|
||||
public:
|
||||
DatabaseDictionary(const String & name_, const Context & context);
|
||||
|
||||
@ -86,12 +76,30 @@ public:
|
||||
const Context & context,
|
||||
const String & table_name) override;
|
||||
|
||||
ASTPtr getCreateQuery(
|
||||
ASTPtr getCreateTableQuery(
|
||||
const Context & context,
|
||||
const String & table_name) const override;
|
||||
|
||||
ASTPtr tryGetCreateTableQuery(
|
||||
const Context & context,
|
||||
const String & table_name) const override;
|
||||
|
||||
ASTPtr getCreateDatabaseQuery(const Context & context) const override;
|
||||
|
||||
void shutdown() override;
|
||||
void drop() override;
|
||||
|
||||
private:
|
||||
const String name;
|
||||
mutable std::mutex mutex;
|
||||
const ExternalDictionaries & external_dictionaries;
|
||||
std::unordered_set<String> deleted_tables;
|
||||
|
||||
Poco::Logger * log;
|
||||
|
||||
Tables loadTables();
|
||||
|
||||
ASTPtr getCreateTableQueryImpl(const Context & context, const String & table_name, bool throw_on_error) const;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -118,13 +118,18 @@ time_t DatabaseMemory::getTableMetadataModificationTime(
|
||||
return static_cast<time_t>(0);
|
||||
}
|
||||
|
||||
ASTPtr DatabaseMemory::getCreateQuery(
|
||||
ASTPtr DatabaseMemory::getCreateTableQuery(
|
||||
const Context &,
|
||||
const String &) const
|
||||
{
|
||||
throw Exception("There is no CREATE TABLE query for DatabaseMemory tables", ErrorCodes::CANNOT_GET_CREATE_TABLE_QUERY);
|
||||
}
|
||||
|
||||
ASTPtr DatabaseMemory::getCreateDatabaseQuery(const Context &) const
|
||||
{
|
||||
throw Exception("There is no CREATE DATABASE query for DatabaseMemory", ErrorCodes::CANNOT_GET_CREATE_TABLE_QUERY);
|
||||
}
|
||||
|
||||
void DatabaseMemory::shutdown()
|
||||
{
|
||||
/// You can not hold a lock during shutdown.
|
||||
|
@ -77,9 +77,10 @@ public:
|
||||
const Context & context,
|
||||
const String & table_name) override;
|
||||
|
||||
ASTPtr getCreateQuery(
|
||||
const Context & context,
|
||||
const String & table_name) const override;
|
||||
ASTPtr getCreateTableQuery(const Context & context, const String & table_name) const override;
|
||||
ASTPtr tryGetCreateTableQuery(const Context &, const String &) const override { return nullptr; }
|
||||
|
||||
ASTPtr getCreateDatabaseQuery(const Context & context) const override;
|
||||
|
||||
void shutdown() override;
|
||||
void drop() override;
|
||||
|
@ -31,6 +31,7 @@ namespace ErrorCodes
|
||||
extern const int FILE_DOESNT_EXIST;
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int CANNOT_GET_CREATE_TABLE_QUERY;
|
||||
extern const int SYNTAX_ERROR;
|
||||
}
|
||||
|
||||
|
||||
@ -45,6 +46,12 @@ namespace detail
|
||||
{
|
||||
return base_path + (endsWith(base_path, "/") ? "" : "/") + escapeForFileName(table_name) + ".sql";
|
||||
}
|
||||
|
||||
String getDatabaseMetadataPath(const String & base_path)
|
||||
{
|
||||
return (endsWith(base_path, "/") ? base_path.substr(0, base_path.size() - 1) : base_path) + ".sql";
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
static void loadTable(
|
||||
@ -329,19 +336,42 @@ void DatabaseOrdinary::removeTable(
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
static ASTPtr getCreateQueryImpl(const String & path, const String & table_name)
|
||||
static ASTPtr getQueryFromMetadata(const String & metadata_path, bool throw_on_error = true)
|
||||
{
|
||||
String table_metadata_path = detail::getTableMetadataPath(path, table_name);
|
||||
if (!Poco::File(metadata_path).exists())
|
||||
return nullptr;
|
||||
|
||||
String query;
|
||||
|
||||
{
|
||||
ReadBufferFromFile in(table_metadata_path, 4096);
|
||||
ReadBufferFromFile in(metadata_path, 4096);
|
||||
readStringUntilEOF(query, in);
|
||||
}
|
||||
|
||||
ParserCreateQuery parser;
|
||||
return parseQuery(parser, query.data(), query.data() + query.size(), "in file " + table_metadata_path);
|
||||
const char * pos = query.data();
|
||||
std::string error_message;
|
||||
auto ast = tryParseQuery(parser, pos, pos + query.size(), error_message, /* hilite = */ false,
|
||||
"in file " + metadata_path, /* allow_multi_statements = */ false);
|
||||
|
||||
if (!ast && throw_on_error)
|
||||
throw Exception(error_message, ErrorCodes::SYNTAX_ERROR);
|
||||
|
||||
return ast;
|
||||
}
|
||||
|
||||
static ASTPtr getCreateQueryFromMetadata(const String & metadata_path, const String & database, bool throw_on_error)
|
||||
{
|
||||
ASTPtr ast = getQueryFromMetadata(metadata_path, throw_on_error);
|
||||
|
||||
if (ast)
|
||||
{
|
||||
ASTCreateQuery & ast_create_query = typeid_cast<ASTCreateQuery &>(*ast);
|
||||
ast_create_query.attach = false;
|
||||
ast_create_query.database = database;
|
||||
}
|
||||
|
||||
return ast;
|
||||
}
|
||||
|
||||
|
||||
@ -378,7 +408,9 @@ void DatabaseOrdinary::renameTable(
|
||||
throw Exception{e};
|
||||
}
|
||||
|
||||
ASTPtr ast = getCreateQueryImpl(metadata_path, table_name);
|
||||
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);
|
||||
ASTCreateQuery & ast_create_query = typeid_cast<ASTCreateQuery &>(*ast);
|
||||
ast_create_query.table = to_table_name;
|
||||
|
||||
@ -405,28 +437,51 @@ time_t DatabaseOrdinary::getTableMetadataModificationTime(
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
ASTPtr DatabaseOrdinary::getCreateQuery(
|
||||
const Context & context,
|
||||
const String & table_name) const
|
||||
ASTPtr DatabaseOrdinary::getCreateTableQueryImpl(const Context & context,
|
||||
const String & table_name, bool throw_on_error) const
|
||||
{
|
||||
ASTPtr ast;
|
||||
try
|
||||
{
|
||||
ast = getCreateQueryImpl(metadata_path, table_name);
|
||||
}
|
||||
catch (const Exception & e)
|
||||
{
|
||||
/// Handle system.* tables for which there are no table.sql files
|
||||
if (e.code() == ErrorCodes::FILE_DOESNT_EXIST && tryGetTable(context, table_name) != nullptr)
|
||||
throw Exception("There is no CREATE TABLE query for table " + table_name, ErrorCodes::CANNOT_GET_CREATE_TABLE_QUERY);
|
||||
|
||||
throw;
|
||||
auto table_metadata_path = detail::getTableMetadataPath(metadata_path, table_name);
|
||||
ast = getCreateQueryFromMetadata(table_metadata_path, name, throw_on_error);
|
||||
if (!ast && throw_on_error)
|
||||
{
|
||||
/// Handle system.* tables for which there are no table.sql files.
|
||||
bool has_table = tryGetTable(context, table_name) != nullptr;
|
||||
|
||||
auto msg = has_table
|
||||
? "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);
|
||||
}
|
||||
|
||||
ASTCreateQuery & ast_create_query = typeid_cast<ASTCreateQuery &>(*ast);
|
||||
ast_create_query.attach = false;
|
||||
ast_create_query.database = name;
|
||||
return ast;
|
||||
}
|
||||
|
||||
ASTPtr DatabaseOrdinary::getCreateTableQuery(const Context & context, const String & table_name) const
|
||||
{
|
||||
return getCreateTableQueryImpl(context, table_name, true);
|
||||
}
|
||||
|
||||
ASTPtr DatabaseOrdinary::tryGetCreateTableQuery(const Context & context, const String & table_name) const
|
||||
{
|
||||
return getCreateTableQueryImpl(context, table_name, false);
|
||||
}
|
||||
|
||||
ASTPtr DatabaseOrdinary::getCreateDatabaseQuery(const Context & /*context*/) const
|
||||
{
|
||||
ASTPtr ast;
|
||||
|
||||
auto database_metadata_path = detail::getDatabaseMetadataPath(metadata_path);
|
||||
ast = getCreateQueryFromMetadata(database_metadata_path, name, true);
|
||||
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;
|
||||
ast = parseQuery(parser, query.data(), query.data() + query.size(), "");
|
||||
}
|
||||
|
||||
return ast;
|
||||
}
|
||||
|
@ -52,10 +52,16 @@ public:
|
||||
const Context & context,
|
||||
const String & table_name) override;
|
||||
|
||||
ASTPtr getCreateQuery(
|
||||
ASTPtr getCreateTableQuery(
|
||||
const Context & context,
|
||||
const String & table_name) const override;
|
||||
|
||||
ASTPtr tryGetCreateTableQuery(
|
||||
const Context & context,
|
||||
const String & table_name) const override;
|
||||
|
||||
ASTPtr getCreateDatabaseQuery(const Context & context) const override;
|
||||
|
||||
String getDataPath() const override;
|
||||
String getMetadataPath() const override;
|
||||
String getTableMetadataPath(const String & table_name) const override;
|
||||
@ -65,6 +71,8 @@ public:
|
||||
|
||||
private:
|
||||
void startupTables(ThreadPool * thread_pool);
|
||||
|
||||
ASTPtr getCreateTableQueryImpl(const Context & context, const String & table_name, bool throw_on_error) const;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -122,9 +122,15 @@ public:
|
||||
const String & name) = 0;
|
||||
|
||||
/// Get the CREATE TABLE query for the table. It can also provide information for detached tables for which there is metadata.
|
||||
virtual ASTPtr getCreateQuery(
|
||||
const Context & context,
|
||||
const String & name) const = 0;
|
||||
virtual ASTPtr tryGetCreateTableQuery(const Context & context, const String & name) const = 0;
|
||||
|
||||
virtual ASTPtr getCreateTableQuery(const Context & context, const String & name) const
|
||||
{
|
||||
return tryGetCreateTableQuery(context, name);
|
||||
}
|
||||
|
||||
/// Get the CREATE DATABASE query for current database.
|
||||
virtual ASTPtr getCreateDatabaseQuery(const Context & context) const = 0;
|
||||
|
||||
/// Returns path for persistent data storage if the database supports it, empty string otherwise
|
||||
virtual String getDataPath() const { return {}; }
|
||||
|
@ -82,6 +82,7 @@ namespace ErrorCodes
|
||||
extern const int TABLE_SIZE_EXCEEDS_MAX_DROP_SIZE_LIMIT;
|
||||
extern const int SESSION_NOT_FOUND;
|
||||
extern const int SESSION_IS_LOCKED;
|
||||
extern const int CANNOT_GET_CREATE_TABLE_QUERY;
|
||||
}
|
||||
|
||||
|
||||
@ -912,17 +913,17 @@ DatabasePtr Context::detachDatabase(const String & database_name)
|
||||
}
|
||||
|
||||
|
||||
ASTPtr Context::getCreateQuery(const String & database_name, const String & table_name) const
|
||||
ASTPtr Context::getCreateTableQuery(const String & database_name, const String & table_name) const
|
||||
{
|
||||
auto lock = getLock();
|
||||
|
||||
String db = resolveDatabase(database_name, current_database);
|
||||
assertDatabaseExists(db);
|
||||
|
||||
return shared->databases[db]->getCreateQuery(*this, table_name);
|
||||
return shared->databases[db]->getCreateTableQuery(*this, table_name);
|
||||
}
|
||||
|
||||
ASTPtr Context::getCreateExternalQuery(const String & table_name) const
|
||||
ASTPtr Context::getCreateExternalTableQuery(const String & table_name) const
|
||||
{
|
||||
TableAndCreateASTs::const_iterator jt = external_tables.find(table_name);
|
||||
if (external_tables.end() == jt)
|
||||
@ -931,6 +932,15 @@ ASTPtr Context::getCreateExternalQuery(const String & table_name) const
|
||||
return jt->second.second;
|
||||
}
|
||||
|
||||
ASTPtr Context::getCreateDatabaseQuery(const String & database_name) const
|
||||
{
|
||||
auto lock = getLock();
|
||||
|
||||
String db = resolveDatabase(database_name, current_database);
|
||||
assertDatabaseExists(db);
|
||||
|
||||
return shared->databases[db]->getCreateDatabaseQuery(*this);
|
||||
}
|
||||
|
||||
Settings Context::getSettings() const
|
||||
{
|
||||
|
@ -241,8 +241,9 @@ public:
|
||||
UInt16 getTCPPort() const;
|
||||
|
||||
/// Get query for the CREATE table.
|
||||
ASTPtr getCreateQuery(const String & database_name, const String & table_name) const;
|
||||
ASTPtr getCreateExternalQuery(const String & table_name) const;
|
||||
ASTPtr getCreateTableQuery(const String & database_name, const String & table_name) const;
|
||||
ASTPtr getCreateExternalTableQuery(const String & table_name) const;
|
||||
ASTPtr getCreateDatabaseQuery(const String & database_name) const;
|
||||
|
||||
const DatabasePtr getDatabase(const String & database_name) const;
|
||||
DatabasePtr getDatabase(const String & database_name);
|
||||
|
@ -31,6 +31,11 @@ public:
|
||||
return std::static_pointer_cast<IDictionaryBase>(getLoadable(name));
|
||||
}
|
||||
|
||||
DictPtr tryGetDictionary(const std::string & name) const
|
||||
{
|
||||
return std::static_pointer_cast<IDictionaryBase>(tryGetLoadable(name));
|
||||
}
|
||||
|
||||
protected:
|
||||
|
||||
std::unique_ptr<IExternalLoadable> create(const std::string & name, const Configuration & config,
|
||||
|
@ -385,21 +385,35 @@ void ExternalLoader::reload(const std::string & name)
|
||||
throw Exception("Failed to load " + object_name + " '" + name + "' during the reload process", ErrorCodes::BAD_ARGUMENTS);
|
||||
}
|
||||
|
||||
ExternalLoader::LoadablePtr ExternalLoader::getLoadable(const std::string & name) const
|
||||
ExternalLoader::LoadablePtr ExternalLoader::getLoadableImpl(const std::string & name, bool throw_on_error) const
|
||||
{
|
||||
const std::lock_guard<std::mutex> lock{map_mutex};
|
||||
|
||||
const auto it = loadable_objects.find(name);
|
||||
if (it == std::end(loadable_objects))
|
||||
throw Exception("No such " + object_name + ": " + name, ErrorCodes::BAD_ARGUMENTS);
|
||||
{
|
||||
if (throw_on_error)
|
||||
throw Exception("No such " + object_name + ": " + name, ErrorCodes::BAD_ARGUMENTS);
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
if (!it->second.loadable)
|
||||
it->second.exception ? std::rethrow_exception(it->second.exception) :
|
||||
throw Exception{object_name + " '" + name + "' is not loaded", ErrorCodes::LOGICAL_ERROR};
|
||||
if (!it->second.loadable && throw_on_error)
|
||||
it->second.exception ? std::rethrow_exception(it->second.exception)
|
||||
: throw Exception{object_name + " '" + name + "' is not loaded", ErrorCodes::LOGICAL_ERROR};
|
||||
|
||||
return it->second.loadable;
|
||||
}
|
||||
|
||||
ExternalLoader::LoadablePtr ExternalLoader::getLoadable(const std::string & name) const
|
||||
{
|
||||
return getLoadableImpl(name, true);
|
||||
}
|
||||
|
||||
ExternalLoader::LoadablePtr ExternalLoader::tryGetLoadable(const std::string & name) const
|
||||
{
|
||||
return getLoadableImpl(name, false);
|
||||
}
|
||||
|
||||
ExternalLoader::LockedObjectsMap ExternalLoader::getObjectsMap() const
|
||||
{
|
||||
return LockedObjectsMap(map_mutex, loadable_objects);
|
||||
|
@ -104,6 +104,7 @@ public:
|
||||
void reload(const std::string & name);
|
||||
|
||||
LoadablePtr getLoadable(const std::string & name) const;
|
||||
LoadablePtr tryGetLoadable(const std::string & name) const;
|
||||
|
||||
protected:
|
||||
virtual std::unique_ptr<IExternalLoadable> create(const std::string & name, const Configuration & config,
|
||||
@ -172,6 +173,8 @@ private:
|
||||
void reloadAndUpdate(bool throw_on_error = false);
|
||||
|
||||
void reloadPeriodically();
|
||||
|
||||
LoadablePtr getLoadableImpl(const std::string & name, bool throw_on_error) const;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -425,7 +425,7 @@ void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const
|
||||
String as_database_name = create.as_database.empty() ? context.getCurrentDatabase() : create.as_database;
|
||||
String as_table_name = create.as_table;
|
||||
|
||||
ASTPtr as_create_ptr = context.getCreateQuery(as_database_name, as_table_name);
|
||||
ASTPtr as_create_ptr = context.getCreateTableQuery(as_database_name, as_table_name);
|
||||
const auto & as_create = typeid_cast<const ASTCreateQuery &>(*as_create_ptr);
|
||||
|
||||
if (as_create.is_view)
|
||||
@ -454,7 +454,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create)
|
||||
if (create.attach && !create.storage && !create.columns)
|
||||
{
|
||||
// Table SQL definition is available even if the table is detached
|
||||
auto query = context.getCreateQuery(database_name, table_name);
|
||||
auto query = context.getCreateTableQuery(database_name, table_name);
|
||||
auto & as_create = typeid_cast<const ASTCreateQuery &>(*query);
|
||||
create = as_create; // Copy the saved create query, but use ATTACH instead of CREATE
|
||||
create.attach = true;
|
||||
|
@ -107,7 +107,11 @@ std::unique_ptr<IInterpreter> InterpreterFactory::get(ASTPtr & query, Context &
|
||||
{
|
||||
return std::make_unique<InterpreterExistsQuery>(query, context);
|
||||
}
|
||||
else if (typeid_cast<ASTShowCreateQuery *>(query.get()))
|
||||
else if (typeid_cast<ASTShowCreateTableQuery *>(query.get()))
|
||||
{
|
||||
return std::make_unique<InterpreterShowCreateQuery>(query, context);
|
||||
}
|
||||
else if (typeid_cast<ASTShowCreateDatabaseQuery *>(query.get()))
|
||||
{
|
||||
return std::make_unique<InterpreterShowCreateQuery>(query, context);
|
||||
}
|
||||
|
@ -39,13 +39,18 @@ Block InterpreterShowCreateQuery::getSampleBlock()
|
||||
|
||||
BlockInputStreamPtr InterpreterShowCreateQuery::executeImpl()
|
||||
{
|
||||
const ASTShowCreateQuery & ast = typeid_cast<const ASTShowCreateQuery &>(*query_ptr);
|
||||
const auto & ast = dynamic_cast<const ASTQueryWithTableAndOutput &>(*query_ptr);
|
||||
|
||||
if (ast.temporary && !ast.database.empty())
|
||||
throw Exception("Temporary databases are not possible.", ErrorCodes::SYNTAX_ERROR);
|
||||
|
||||
ASTPtr create_query = (ast.temporary ? context.getCreateExternalQuery(ast.table) :
|
||||
context.getCreateQuery(ast.database, ast.table));
|
||||
ASTPtr create_query;
|
||||
if (ast.temporary)
|
||||
create_query = context.getCreateExternalTableQuery(ast.table);
|
||||
else if (ast.table.empty())
|
||||
create_query = context.getCreateDatabaseQuery(ast.database);
|
||||
else
|
||||
create_query = context.getCreateTableQuery(ast.database, ast.table);
|
||||
|
||||
if (!create_query && ast.temporary)
|
||||
throw Exception("Unable to show the create query of " + ast.table + ". Maybe it was created by the system.", ErrorCodes::THERE_IS_NO_QUERY);
|
||||
|
@ -19,6 +19,7 @@ bool ParserTablePropertiesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected &
|
||||
ParserKeyword s_desc("DESC");
|
||||
ParserKeyword s_show("SHOW");
|
||||
ParserKeyword s_create("CREATE");
|
||||
ParserKeyword s_database("DATABASE");
|
||||
ParserKeyword s_table("TABLE");
|
||||
ParserToken s_dot(TokenType::Dot);
|
||||
ParserIdentifier name_p;
|
||||
@ -27,6 +28,8 @@ bool ParserTablePropertiesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected &
|
||||
ASTPtr table;
|
||||
std::shared_ptr<ASTQueryWithTableAndOutput> query;
|
||||
|
||||
bool parse_only_database_name = false;
|
||||
|
||||
if (s_exists.ignore(pos, expected))
|
||||
{
|
||||
query = std::make_shared<ASTExistsQuery>();
|
||||
@ -36,26 +39,40 @@ bool ParserTablePropertiesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected &
|
||||
if (!s_create.ignore(pos, expected))
|
||||
return false;
|
||||
|
||||
query = std::make_shared<ASTShowCreateQuery>();
|
||||
if (s_database.ignore(pos, expected))
|
||||
{
|
||||
parse_only_database_name = true;
|
||||
query = std::make_shared<ASTShowCreateDatabaseQuery>();
|
||||
}
|
||||
else
|
||||
query = std::make_shared<ASTShowCreateTableQuery>();
|
||||
}
|
||||
else
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
if (s_temporary.ignore(pos, expected))
|
||||
query->temporary = true;
|
||||
|
||||
s_table.ignore(pos, expected);
|
||||
|
||||
if (!name_p.parse(pos, table, expected))
|
||||
return false;
|
||||
|
||||
if (s_dot.ignore(pos, expected))
|
||||
if (parse_only_database_name)
|
||||
{
|
||||
database = table;
|
||||
if (!name_p.parse(pos, database, expected))
|
||||
return false;
|
||||
}
|
||||
else
|
||||
{
|
||||
if (s_temporary.ignore(pos, expected))
|
||||
query->temporary = true;
|
||||
|
||||
s_table.ignore(pos, expected);
|
||||
|
||||
if (!name_p.parse(pos, table, expected))
|
||||
return false;
|
||||
|
||||
if (s_dot.ignore(pos, expected))
|
||||
{
|
||||
database = table;
|
||||
if (!name_p.parse(pos, table, expected))
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
if (database)
|
||||
|
@ -12,12 +12,18 @@ struct ASTExistsQueryIDAndQueryNames
|
||||
static constexpr auto Query = "EXISTS TABLE";
|
||||
};
|
||||
|
||||
struct ASTShowCreateQueryIDAndQueryNames
|
||||
struct ASTShowCreateTableQueryIDAndQueryNames
|
||||
{
|
||||
static constexpr auto ID = "ShowCreateQuery";
|
||||
static constexpr auto ID = "ShowCreateTableQuery";
|
||||
static constexpr auto Query = "SHOW CREATE TABLE";
|
||||
};
|
||||
|
||||
struct ASTShowCreateDatabaseQueryIDAndQueryNames
|
||||
{
|
||||
static constexpr auto ID = "ShowCreateDatabaseQuery";
|
||||
static constexpr auto Query = "SHOW CREATE DATABASE";
|
||||
};
|
||||
|
||||
struct ASTDescribeQueryExistsQueryIDAndQueryNames
|
||||
{
|
||||
static constexpr auto ID = "DescribeQuery";
|
||||
@ -25,7 +31,17 @@ struct ASTDescribeQueryExistsQueryIDAndQueryNames
|
||||
};
|
||||
|
||||
using ASTExistsQuery = ASTQueryWithTableAndOutputImpl<ASTExistsQueryIDAndQueryNames>;
|
||||
using ASTShowCreateQuery = ASTQueryWithTableAndOutputImpl<ASTShowCreateQueryIDAndQueryNames>;
|
||||
using ASTShowCreateTableQuery = ASTQueryWithTableAndOutputImpl<ASTShowCreateTableQueryIDAndQueryNames>;
|
||||
|
||||
class ASTShowCreateDatabaseQuery : public ASTQueryWithTableAndOutputImpl<ASTShowCreateDatabaseQueryIDAndQueryNames>
|
||||
{
|
||||
protected:
|
||||
void formatQueryImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << ASTShowCreateDatabaseQueryIDAndQueryNames::Query
|
||||
<< " " << (settings.hilite ? hilite_none : "") << backQuoteIfNeed(database);
|
||||
}
|
||||
};
|
||||
|
||||
class ASTDescribeQuery : public ASTQueryWithOutput
|
||||
{
|
||||
|
@ -86,7 +86,6 @@ void StorageDictionary::checkNamesAndTypesCompatibleWithDictionary(const Diction
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void registerStorageDictionary(StorageFactory & factory)
|
||||
{
|
||||
factory.registerStorage("Dictionary", [](const StorageFactory::Arguments & args)
|
||||
|
@ -4,6 +4,8 @@
|
||||
#include <Core/Defines.h>
|
||||
#include <common/MultiVersion.h>
|
||||
#include <ext/shared_ptr_helper.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
|
||||
namespace Poco
|
||||
@ -32,6 +34,26 @@ public:
|
||||
void drop() override {}
|
||||
static NamesAndTypesList getNamesAndTypes(const DictionaryStructure & dictionary_structure);
|
||||
|
||||
template <typename ForwardIterator>
|
||||
static std::string generateNamesAndTypesDescription(ForwardIterator begin, ForwardIterator end)
|
||||
{
|
||||
std::string description;
|
||||
{
|
||||
WriteBufferFromString buffer(description);
|
||||
bool first = true;
|
||||
for (; begin != end; ++begin)
|
||||
{
|
||||
if (!first)
|
||||
buffer << ", ";
|
||||
first = false;
|
||||
|
||||
buffer << begin->name << ' ' << begin->type->getName();
|
||||
}
|
||||
}
|
||||
|
||||
return description;
|
||||
}
|
||||
|
||||
private:
|
||||
using Ptr = MultiVersion<IDictionaryBase>::Version;
|
||||
|
||||
@ -41,24 +63,6 @@ private:
|
||||
|
||||
void checkNamesAndTypesCompatibleWithDictionary(const DictionaryStructure & dictionary_structure) const;
|
||||
|
||||
template <typename ForwardIterator>
|
||||
std::string generateNamesAndTypesDescription(ForwardIterator begin, ForwardIterator end) const
|
||||
{
|
||||
if (begin == end)
|
||||
{
|
||||
return "";
|
||||
}
|
||||
std::string description;
|
||||
for (; begin != end; ++begin)
|
||||
{
|
||||
description += ", ";
|
||||
description += begin->name;
|
||||
description += ' ';
|
||||
description += begin->type->getName();
|
||||
}
|
||||
return description.substr(2, description.size());
|
||||
}
|
||||
|
||||
protected:
|
||||
StorageDictionary(const String & table_name_,
|
||||
const ColumnsDescription & columns_,
|
||||
|
@ -200,17 +200,7 @@ BlockInputStreams StorageSystemTables::read(
|
||||
|
||||
if (has_create_table_query || has_engine_full)
|
||||
{
|
||||
ASTPtr ast;
|
||||
|
||||
try
|
||||
{
|
||||
ast = database->getCreateQuery(context, table_name);
|
||||
}
|
||||
catch (const Exception & e)
|
||||
{
|
||||
if (e.code() != ErrorCodes::CANNOT_GET_CREATE_TABLE_QUERY)
|
||||
throw;
|
||||
}
|
||||
ASTPtr ast = database->tryGetCreateTableQuery(context, table_name);
|
||||
|
||||
if (has_create_table_query)
|
||||
res_columns[j++]->insert(ast ? queryToString(ast) : "");
|
||||
|
@ -0,0 +1 @@
|
||||
CREATE DATABASE test ENGINE = Ordinary
|
@ -0,0 +1,2 @@
|
||||
create database if not exists test;
|
||||
show create database test;
|
Loading…
Reference in New Issue
Block a user