2017-12-28 21:36:27 +00:00
|
|
|
#include <Storages/StorageFactory.h>
|
|
|
|
#include <Interpreters/Context.h>
|
|
|
|
#include <Parsers/ASTFunction.h>
|
2017-12-30 00:36:06 +00:00
|
|
|
#include <Parsers/ASTCreateQuery.h>
|
2017-12-28 21:36:27 +00:00
|
|
|
#include <Common/Exception.h>
|
2018-06-05 19:46:49 +00:00
|
|
|
#include <Common/StringUtils/StringUtils.h>
|
2019-04-03 11:13:22 +00:00
|
|
|
#include <IO/WriteHelpers.h>
|
2020-03-13 10:30:55 +00:00
|
|
|
#include <Interpreters/StorageID.h>
|
2017-12-28 21:36:27 +00:00
|
|
|
|
|
|
|
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;
|
2017-12-30 00:36:06 +00:00
|
|
|
extern const int DATA_TYPE_CANNOT_BE_USED_IN_TABLES;
|
2017-12-28 21:36:27 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
/// 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);
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2021-04-10 23:33:54 +00:00
|
|
|
ContextPtr StorageFactory::Arguments::getContext() const
|
|
|
|
{
|
|
|
|
auto ptr = context.lock();
|
|
|
|
if (!ptr)
|
|
|
|
throw Exception("Context has expired", ErrorCodes::LOGICAL_ERROR);
|
|
|
|
return ptr;
|
|
|
|
}
|
|
|
|
|
|
|
|
ContextPtr StorageFactory::Arguments::getLocalContext() const
|
|
|
|
{
|
|
|
|
auto ptr = local_context.lock();
|
|
|
|
if (!ptr)
|
|
|
|
throw Exception("Context has expired", ErrorCodes::LOGICAL_ERROR);
|
|
|
|
return ptr;
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2020-01-25 19:14:33 +00:00
|
|
|
void StorageFactory::registerStorage(const std::string & name, CreatorFn creator_fn, StorageFeatures features)
|
2017-12-28 21:36:27 +00:00
|
|
|
{
|
2020-01-25 19:14:33 +00:00
|
|
|
if (!storages.emplace(name, Creator{std::move(creator_fn), features}).second)
|
2017-12-28 21:36:27 +00:00
|
|
|
throw Exception("TableFunctionFactory: the table function name '" + name + "' is not unique",
|
|
|
|
ErrorCodes::LOGICAL_ERROR);
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
StoragePtr StorageFactory::get(
|
2019-12-16 18:10:35 +00:00
|
|
|
const ASTCreateQuery & query,
|
2019-10-25 19:07:47 +00:00
|
|
|
const String & relative_data_path,
|
2021-04-10 23:33:54 +00:00
|
|
|
ContextPtr local_context,
|
|
|
|
ContextPtr context,
|
2018-03-06 20:18:34 +00:00
|
|
|
const ColumnsDescription & columns,
|
2019-08-24 21:20:20 +00:00
|
|
|
const ConstraintsDescription & constraints,
|
2017-12-28 21:36:27 +00:00
|
|
|
bool has_force_restore_data_flag) const
|
|
|
|
{
|
|
|
|
String name;
|
2017-12-30 00:36:06 +00:00
|
|
|
ASTStorage * storage_def = query.storage;
|
2017-12-28 21:36:27 +00:00
|
|
|
|
2020-02-19 18:58:29 +00:00
|
|
|
bool has_engine_args = false;
|
|
|
|
|
2020-12-16 03:19:38 +00:00
|
|
|
if (query.is_ordinary_view)
|
2017-12-28 21:36:27 +00:00
|
|
|
{
|
|
|
|
if (query.storage)
|
|
|
|
throw Exception("Specifying ENGINE is not allowed for a View", ErrorCodes::INCORRECT_QUERY);
|
|
|
|
|
|
|
|
name = "View";
|
|
|
|
}
|
2019-05-28 21:17:48 +00:00
|
|
|
else if (query.is_live_view)
|
|
|
|
{
|
|
|
|
if (query.storage)
|
|
|
|
throw Exception("Specifying ENGINE is not allowed for a LiveView", ErrorCodes::INCORRECT_QUERY);
|
|
|
|
|
|
|
|
name = "LiveView";
|
|
|
|
}
|
2021-04-21 13:45:13 +00:00
|
|
|
else if (query.is_dictionary)
|
|
|
|
{
|
|
|
|
if (query.storage)
|
|
|
|
throw Exception("Specifying ENGINE is not allowed for a Dictionary", ErrorCodes::INCORRECT_QUERY);
|
|
|
|
|
|
|
|
name = "Dictionary";
|
|
|
|
}
|
2017-12-28 21:36:27 +00:00
|
|
|
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.
|
2018-03-10 17:03:57 +00:00
|
|
|
checkAllTypesAreAllowedInTable(columns.getAll());
|
2017-12-28 21:36:27 +00:00
|
|
|
|
|
|
|
if (query.is_materialized_view)
|
|
|
|
{
|
|
|
|
name = "MaterializedView";
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
2017-12-30 00:36:06 +00:00
|
|
|
if (!storage_def)
|
2017-12-28 21:36:27 +00:00
|
|
|
throw Exception("Incorrect CREATE query: ENGINE required", ErrorCodes::ENGINE_REQUIRED);
|
|
|
|
|
2017-12-30 00:36:06 +00:00
|
|
|
const ASTFunction & engine_def = *storage_def->engine;
|
2017-12-28 21:36:27 +00:00
|
|
|
|
|
|
|
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)
|
2020-02-19 18:58:29 +00:00
|
|
|
has_engine_args = true;
|
2017-12-28 21:36:27 +00:00
|
|
|
|
|
|
|
name = engine_def.name;
|
|
|
|
|
|
|
|
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);
|
|
|
|
}
|
2019-05-28 21:17:48 +00:00
|
|
|
else if (name == "LiveView")
|
|
|
|
{
|
|
|
|
throw Exception(
|
|
|
|
"Direct creation of tables with ENGINE LiveView is not supported, use CREATE LIVE VIEW statement",
|
|
|
|
ErrorCodes::INCORRECT_QUERY);
|
|
|
|
}
|
2017-12-28 21:36:27 +00:00
|
|
|
|
2020-01-27 18:28:27 +00:00
|
|
|
auto it = storages.find(name);
|
|
|
|
if (it == storages.end())
|
2020-01-25 19:14:33 +00:00
|
|
|
{
|
2020-01-27 18:28:27 +00:00
|
|
|
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);
|
2020-01-25 19:14:33 +00:00
|
|
|
}
|
|
|
|
|
2020-03-23 02:12:31 +00:00
|
|
|
auto check_feature = [&](String feature_description, FeatureMatcherFn feature_matcher_fn)
|
2020-01-27 18:28:27 +00:00
|
|
|
{
|
|
|
|
if (!feature_matcher_fn(it->second.features))
|
|
|
|
{
|
|
|
|
String msg = "Engine " + name + " doesn't support " + feature_description + ". "
|
|
|
|
"Currently only the following engines have support for the feature: [";
|
|
|
|
auto supporting_engines = getAllRegisteredNamesByFeatureMatcherFn(feature_matcher_fn);
|
|
|
|
for (size_t index = 0; index < supporting_engines.size(); ++index)
|
|
|
|
{
|
|
|
|
if (index)
|
|
|
|
msg += ", ";
|
|
|
|
msg += supporting_engines[index];
|
|
|
|
}
|
|
|
|
msg += "]";
|
|
|
|
throw Exception(msg, ErrorCodes::BAD_ARGUMENTS);
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
|
|
|
if (storage_def->settings)
|
2020-03-23 02:12:31 +00:00
|
|
|
check_feature(
|
2020-01-27 18:28:27 +00:00
|
|
|
"SETTINGS clause",
|
|
|
|
[](StorageFeatures features) { return features.supports_settings; });
|
|
|
|
|
|
|
|
if (storage_def->partition_by || storage_def->primary_key || storage_def->order_by || storage_def->sample_by)
|
2020-03-23 02:12:31 +00:00
|
|
|
check_feature(
|
2020-01-27 18:28:27 +00:00
|
|
|
"PARTITION_BY, PRIMARY_KEY, ORDER_BY or SAMPLE_BY clauses",
|
|
|
|
[](StorageFeatures features) { return features.supports_sort_order; });
|
|
|
|
|
|
|
|
if (storage_def->ttl_table || !columns.getColumnTTLs().empty())
|
2020-03-23 02:12:31 +00:00
|
|
|
check_feature(
|
2020-01-27 18:28:27 +00:00
|
|
|
"TTL clause",
|
|
|
|
[](StorageFeatures features) { return features.supports_ttl; });
|
|
|
|
|
|
|
|
if (query.columns_list && query.columns_list->indices && !query.columns_list->indices->children.empty())
|
2020-03-23 02:12:31 +00:00
|
|
|
check_feature(
|
2020-01-27 18:28:27 +00:00
|
|
|
"skipping indices",
|
|
|
|
[](StorageFeatures features) { return features.supports_skipping_indices; });
|
2021-02-10 14:12:49 +00:00
|
|
|
|
|
|
|
if (query.columns_list && query.columns_list->projections && !query.columns_list->projections->children.empty())
|
|
|
|
check_feature(
|
|
|
|
"projections",
|
|
|
|
[](StorageFeatures features) { return features.supports_projections; });
|
2020-01-27 18:28:27 +00:00
|
|
|
}
|
2020-01-27 13:04:01 +00:00
|
|
|
}
|
2020-01-25 19:14:33 +00:00
|
|
|
|
2021-04-23 12:18:23 +00:00
|
|
|
String comment;
|
|
|
|
|
2021-04-23 12:25:09 +00:00
|
|
|
if (storage_def && storage_def->comment)
|
2021-04-23 12:18:23 +00:00
|
|
|
{
|
|
|
|
comment = storage_def->comment->as<ASTLiteral &>().value.get<String>();
|
|
|
|
}
|
|
|
|
|
2020-02-19 18:58:29 +00:00
|
|
|
ASTs empty_engine_args;
|
2017-12-30 00:36:06 +00:00
|
|
|
Arguments arguments
|
|
|
|
{
|
|
|
|
.engine_name = name,
|
2020-02-19 18:58:29 +00:00
|
|
|
.engine_args = has_engine_args ? storage_def->engine->arguments->children : empty_engine_args,
|
2017-12-30 00:36:06 +00:00
|
|
|
.storage_def = storage_def,
|
|
|
|
.query = query,
|
2019-10-25 19:07:47 +00:00
|
|
|
.relative_data_path = relative_data_path,
|
2019-12-04 16:06:55 +00:00
|
|
|
.table_id = StorageID(query.database, query.table, query.uuid),
|
2017-12-30 00:36:06 +00:00
|
|
|
.local_context = local_context,
|
|
|
|
.context = context,
|
|
|
|
.columns = columns,
|
2019-08-24 21:20:20 +00:00
|
|
|
.constraints = constraints,
|
2019-12-04 16:06:55 +00:00
|
|
|
.attach = query.attach,
|
2021-04-23 12:18:23 +00:00
|
|
|
.has_force_restore_data_flag = has_force_restore_data_flag,
|
|
|
|
.comment = comment
|
2017-09-18 14:18:29 +00:00
|
|
|
};
|
2021-04-10 23:33:54 +00:00
|
|
|
assert(arguments.getContext() == arguments.getContext()->getGlobalContext());
|
2017-09-18 14:18:29 +00:00
|
|
|
|
2020-09-26 19:18:28 +00:00
|
|
|
auto res = storages.at(name).creator_fn(arguments);
|
2021-04-25 14:05:48 +00:00
|
|
|
if (!empty_engine_args.empty()) //-V547
|
2020-09-26 19:18:28 +00:00
|
|
|
{
|
|
|
|
/// Storage creator modified empty arguments list, so we should modify the query
|
|
|
|
assert(storage_def && storage_def->engine && !storage_def->engine->arguments);
|
|
|
|
storage_def->engine->arguments = std::make_shared<ASTExpressionList>();
|
|
|
|
storage_def->engine->children.push_back(storage_def->engine->arguments);
|
|
|
|
storage_def->engine->arguments->children = empty_engine_args;
|
|
|
|
}
|
2021-01-20 10:54:11 +00:00
|
|
|
|
2021-04-10 23:33:54 +00:00
|
|
|
if (local_context->hasQueryContext() && context->getSettingsRef().log_queries)
|
|
|
|
local_context->getQueryContext()->addQueryFactoriesInfo(Context::QueryLogFactories::Storage, name);
|
2021-01-20 10:54:11 +00:00
|
|
|
|
2020-09-26 19:18:28 +00:00
|
|
|
return res;
|
2011-10-31 17:30:44 +00:00
|
|
|
}
|
|
|
|
|
2019-08-22 03:24:05 +00:00
|
|
|
StorageFactory & StorageFactory::instance()
|
|
|
|
{
|
|
|
|
static StorageFactory ret;
|
|
|
|
return ret;
|
|
|
|
}
|
|
|
|
|
2020-04-06 05:19:40 +00:00
|
|
|
|
|
|
|
AccessType StorageFactory::getSourceAccessType(const String & table_engine) const
|
|
|
|
{
|
|
|
|
auto it = storages.find(table_engine);
|
|
|
|
if (it == storages.end())
|
|
|
|
return AccessType::NONE;
|
|
|
|
return it->second.features.source_access_type;
|
|
|
|
}
|
|
|
|
|
2011-10-31 17:30:44 +00:00
|
|
|
}
|