ClickHouse/dbms/src/Storages/StorageFactory.cpp

151 lines
5.2 KiB
C++
Raw Normal View History

#include <Storages/StorageFactory.h>
#include <Interpreters/Context.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTCreateQuery.h>
#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>
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(
ASTCreateQuery & query,
const String & data_path,
const String & table_name,
const String & database_name,
Context & local_context,
Context & context,
const ColumnsDescription & columns,
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
{
/// 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);
}
Data Skipping Indices (#4143) * made index parser * added index parsing * some fixes * added index interface and factory * fixed compilation * ptrs * added indexParts * indextypes * index condition * IndexCondition * added indexes in selectexecutor * fix * changed comment * fix * added granularity * comments * fix * fix * added writing indexes * removed indexpart class * fix * added setSkipIndexes * add rw for MergeTreeIndexes * fixes * upd error * fix * fix * reading * test index * fixed nullptr error * fixed * fix * unique names * asts -> exprlist * minmax index * fix * fixed select * fixed merging * fixed mutation * working minmax * removed test index * fixed style * added indexes to checkDataPart * added tests for minmax index * fixed constructor * fix style * fixed includes * fixed setSkipIndexes * added indexes meta to zookeeper * added parsing * removed throw * alter cmds parse * fix * added alter * fix * alters fix * fix alters * fix "after" * fixed alter * alter fix + test * fixes * upd setSkipIndexes * fixed alter bug with drop all indices * fix metadata editing * new test and repl fix * rm test files * fixed repl alter * fix * fix * indices * MTReadStream * upd test for bug * fix * added useful parsers and ast classes * fix * fix comments * replaced columns * fix * fixed parsing * fixed printing * fix err * basic IndicesDescription * go to IndicesDescr * moved indices * go to indicesDescr * fix test minmax_index* * fixed MT alter * fixed bug with replMT indices storing in zk * rename * refactoring * docs ru * docs ru * docs en * refactor * rename tests * fix docs * refactoring * fix * fix * fix * fixed style * unique idx * unique * fix * better minmax calculation * upd * added getBlock * unique_condition * added termForAST * unique * fixed not * uniqueCondition::mayBeTrueOnGranule * fix * fixed bug with double column * is always true * fix * key set * spaces * test * tests * fix * unique * fix * fix * fixed bug with duplicate column * removed unused data * fix * fixes * __bitSwapLastTwo * fix
2019-02-05 14:50:25 +00:00
if ((storage_def->partition_by || storage_def->primary_key || storage_def->order_by || storage_def->sample_by ||
(query.columns_list && query.columns_list->indices && !query.columns_list->indices->children.empty()))
&& !endsWith(name, "MergeTree"))
{
throw Exception(
Data Skipping Indices (#4143) * made index parser * added index parsing * some fixes * added index interface and factory * fixed compilation * ptrs * added indexParts * indextypes * index condition * IndexCondition * added indexes in selectexecutor * fix * changed comment * fix * added granularity * comments * fix * fix * added writing indexes * removed indexpart class * fix * added setSkipIndexes * add rw for MergeTreeIndexes * fixes * upd error * fix * fix * reading * test index * fixed nullptr error * fixed * fix * unique names * asts -> exprlist * minmax index * fix * fixed select * fixed merging * fixed mutation * working minmax * removed test index * fixed style * added indexes to checkDataPart * added tests for minmax index * fixed constructor * fix style * fixed includes * fixed setSkipIndexes * added indexes meta to zookeeper * added parsing * removed throw * alter cmds parse * fix * added alter * fix * alters fix * fix alters * fix "after" * fixed alter * alter fix + test * fixes * upd setSkipIndexes * fixed alter bug with drop all indices * fix metadata editing * new test and repl fix * rm test files * fixed repl alter * fix * fix * indices * MTReadStream * upd test for bug * fix * added useful parsers and ast classes * fix * fix comments * replaced columns * fix * fixed parsing * fixed printing * fix err * basic IndicesDescription * go to IndicesDescr * moved indices * go to indicesDescr * fix test minmax_index* * fixed MT alter * fixed bug with replMT indices storing in zk * rename * refactoring * docs ru * docs ru * docs en * refactor * rename tests * fix docs * refactoring * fix * fix * fix * fixed style * unique idx * unique * fix * better minmax calculation * upd * added getBlock * unique_condition * added termForAST * unique * fixed not * uniqueCondition::mayBeTrueOnGranule * fix * fixed bug with double column * is always true * fix * key set * spaces * test * tests * fix * unique * fix * fix * fixed bug with duplicate column * removed unused data * fix * fixes * __bitSwapLastTwo * fix
2019-02-05 14:50:25 +00:00
"Engine " + name + " doesn't support PARTITION BY, PRIMARY KEY, ORDER BY 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);
}
}
}
auto it = storages.find(name);
if (it == storages.end())
2019-04-03 11:13:22 +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);
}
Arguments arguments
{
.engine_name = name,
.engine_args = args,
.storage_def = storage_def,
.query = query,
.data_path = data_path,
.table_name = table_name,
.database_name = database_name,
.local_context = local_context,
.context = context,
.columns = columns,
.attach = attach,
.has_force_restore_data_flag = has_force_restore_data_flag
};
return it->second(arguments);
2011-10-31 17:30:44 +00:00
}
}