mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-15 12:14:18 +00:00
174 lines
5.9 KiB
C++
174 lines
5.9 KiB
C++
#include <Storages/StorageFactory.h>
|
|
#include <Interpreters/Context.h>
|
|
#include <Parsers/ASTFunction.h>
|
|
#include <Parsers/ASTCreateQuery.h>
|
|
#include <Common/Exception.h>
|
|
#include <Common/StringUtils/StringUtils.h>
|
|
#include <IO/WriteHelpers.h>
|
|
|
|
namespace DB
|
|
{
|
|
|
|
namespace ErrorCodes
|
|
{
|
|
extern const int UNKNOWN_STORAGE;
|
|
extern const int LOGICAL_ERROR;
|
|
extern const int INCORRECT_QUERY;
|
|
extern const int ENGINE_REQUIRED;
|
|
extern const int FUNCTION_CANNOT_HAVE_PARAMETERS;
|
|
extern const int BAD_ARGUMENTS;
|
|
extern const int DATA_TYPE_CANNOT_BE_USED_IN_TABLES;
|
|
}
|
|
|
|
|
|
/// Some types are only for intermediate values of expressions and cannot be used in tables.
|
|
static void checkAllTypesAreAllowedInTable(const NamesAndTypesList & names_and_types)
|
|
{
|
|
for (const auto & elem : names_and_types)
|
|
if (elem.type->cannotBeStoredInTables())
|
|
throw Exception("Data type " + elem.type->getName() + " cannot be used in tables", ErrorCodes::DATA_TYPE_CANNOT_BE_USED_IN_TABLES);
|
|
}
|
|
|
|
|
|
void StorageFactory::registerStorage(const std::string & name, Creator creator)
|
|
{
|
|
if (!storages.emplace(name, std::move(creator)).second)
|
|
throw Exception("TableFunctionFactory: the table function name '" + name + "' is not unique",
|
|
ErrorCodes::LOGICAL_ERROR);
|
|
}
|
|
|
|
|
|
StoragePtr StorageFactory::get(
|
|
const ASTCreateQuery & query,
|
|
const String & relative_data_path,
|
|
const String & table_name,
|
|
const String & database_name,
|
|
Context & local_context,
|
|
Context & context,
|
|
const ColumnsDescription & columns,
|
|
const ConstraintsDescription & constraints,
|
|
bool attach,
|
|
bool has_force_restore_data_flag) const
|
|
{
|
|
String name;
|
|
ASTs args;
|
|
ASTStorage * storage_def = query.storage;
|
|
|
|
if (query.is_view)
|
|
{
|
|
if (query.storage)
|
|
throw Exception("Specifying ENGINE is not allowed for a View", ErrorCodes::INCORRECT_QUERY);
|
|
|
|
name = "View";
|
|
}
|
|
else if (query.is_live_view)
|
|
{
|
|
|
|
if (query.storage)
|
|
throw Exception("Specifying ENGINE is not allowed for a LiveView", ErrorCodes::INCORRECT_QUERY);
|
|
|
|
name = "LiveView";
|
|
}
|
|
else
|
|
{
|
|
/// Check for some special types, that are not allowed to be stored in tables. Example: NULL data type.
|
|
/// Exception: any type is allowed in View, because plain (non-materialized) View does not store anything itself.
|
|
checkAllTypesAreAllowedInTable(columns.getAll());
|
|
|
|
if (query.is_materialized_view)
|
|
{
|
|
name = "MaterializedView";
|
|
}
|
|
else
|
|
{
|
|
if (!storage_def)
|
|
throw Exception("Incorrect CREATE query: ENGINE required", ErrorCodes::ENGINE_REQUIRED);
|
|
|
|
const ASTFunction & engine_def = *storage_def->engine;
|
|
|
|
if (engine_def.parameters)
|
|
throw Exception(
|
|
"Engine definition cannot take the form of a parametric function", ErrorCodes::FUNCTION_CANNOT_HAVE_PARAMETERS);
|
|
|
|
if (engine_def.arguments)
|
|
args = engine_def.arguments->children;
|
|
|
|
name = engine_def.name;
|
|
|
|
if (storage_def->settings && !endsWith(name, "MergeTree") && name != "Kafka" && name != "Join")
|
|
{
|
|
throw Exception(
|
|
"Engine " + name + " doesn't support SETTINGS clause. "
|
|
"Currently only the MergeTree family of engines, Kafka engine and Join engine support it",
|
|
ErrorCodes::BAD_ARGUMENTS);
|
|
}
|
|
|
|
if ((storage_def->partition_by || storage_def->primary_key || storage_def->order_by || storage_def->sample_by ||
|
|
storage_def->ttl_table || !columns.getColumnTTLs().empty() ||
|
|
(query.columns_list && query.columns_list->indices && !query.columns_list->indices->children.empty()))
|
|
&& !endsWith(name, "MergeTree"))
|
|
{
|
|
throw Exception(
|
|
"Engine " + name + " doesn't support PARTITION BY, PRIMARY KEY, ORDER BY, TTL or SAMPLE BY clauses and skipping indices. "
|
|
"Currently only the MergeTree family of engines supports them", ErrorCodes::BAD_ARGUMENTS);
|
|
}
|
|
|
|
if (name == "View")
|
|
{
|
|
throw Exception(
|
|
"Direct creation of tables with ENGINE View is not supported, use CREATE VIEW statement",
|
|
ErrorCodes::INCORRECT_QUERY);
|
|
}
|
|
else if (name == "MaterializedView")
|
|
{
|
|
throw Exception(
|
|
"Direct creation of tables with ENGINE MaterializedView is not supported, use CREATE MATERIALIZED VIEW statement",
|
|
ErrorCodes::INCORRECT_QUERY);
|
|
}
|
|
else if (name == "LiveView")
|
|
{
|
|
throw Exception(
|
|
"Direct creation of tables with ENGINE LiveView is not supported, use CREATE LIVE VIEW statement",
|
|
ErrorCodes::INCORRECT_QUERY);
|
|
}
|
|
}
|
|
}
|
|
|
|
auto it = storages.find(name);
|
|
if (it == storages.end())
|
|
{
|
|
auto hints = getHints(name);
|
|
if (!hints.empty())
|
|
throw Exception("Unknown table engine " + name + ". Maybe you meant: " + toString(hints), ErrorCodes::UNKNOWN_STORAGE);
|
|
else
|
|
throw Exception("Unknown table engine " + name, ErrorCodes::UNKNOWN_STORAGE);
|
|
}
|
|
|
|
Arguments arguments
|
|
{
|
|
.engine_name = name,
|
|
.engine_args = args,
|
|
.storage_def = storage_def,
|
|
.query = query,
|
|
.relative_data_path = relative_data_path,
|
|
.table_name = table_name,
|
|
.database_name = database_name,
|
|
.local_context = local_context,
|
|
.context = context,
|
|
.columns = columns,
|
|
.constraints = constraints,
|
|
.attach = attach,
|
|
.has_force_restore_data_flag = has_force_restore_data_flag
|
|
};
|
|
|
|
return it->second(arguments);
|
|
}
|
|
|
|
StorageFactory & StorageFactory::instance()
|
|
{
|
|
static StorageFactory ret;
|
|
return ret;
|
|
}
|
|
|
|
}
|