mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
validation and left over fixes
This commit is contained in:
parent
af534db4d9
commit
906745feaa
@ -2,6 +2,7 @@
|
||||
#include <Common/ZooKeeper/ZooKeeper.h>
|
||||
#include <Common/StatusFile.h>
|
||||
#include <Common/TerminalSize.h>
|
||||
#include <Databases/registerDatabases.h>
|
||||
#include <IO/ConnectionTimeouts.h>
|
||||
#include <Formats/registerFormats.h>
|
||||
#include <Common/scope_guard_safe.h>
|
||||
@ -159,6 +160,7 @@ void ClusterCopierApp::mainImpl()
|
||||
registerFunctions();
|
||||
registerAggregateFunctions();
|
||||
registerTableFunctions();
|
||||
registerDatabases();
|
||||
registerStorages();
|
||||
registerDictionaries();
|
||||
registerDisks(/* global_skip_access_check= */ true);
|
||||
|
@ -17,6 +17,7 @@
|
||||
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Databases/registerDatabases.h>
|
||||
#include <Functions/registerFunctions.h>
|
||||
#include <AggregateFunctions/AggregateFunctionFactory.h>
|
||||
#include <AggregateFunctions/registerAggregateFunctions.h>
|
||||
@ -130,6 +131,7 @@ int mainEntryClickHouseFormat(int argc, char ** argv)
|
||||
registerFunctions();
|
||||
registerAggregateFunctions();
|
||||
registerTableFunctions();
|
||||
registerDatabases();
|
||||
registerStorages();
|
||||
registerFormats();
|
||||
|
||||
|
@ -10,18 +10,17 @@
|
||||
#include <Poco/Logger.h>
|
||||
#include <Poco/NullChannel.h>
|
||||
#include <Poco/SimpleFileChannel.h>
|
||||
#include <Databases/registerDatabases.h>
|
||||
#include <Databases/DatabaseFilesystem.h>
|
||||
#include <Databases/DatabaseMemory.h>
|
||||
#include <Databases/DatabasesOverlay.h>
|
||||
#include <Storages/System/attachSystemTables.h>
|
||||
#include <Storages/System/attachInformationSchemaTables.h>
|
||||
#include <Interpreters/DatabaseCatalog.h>
|
||||
#include <Interpreters/JIT/CompiledExpressionCache.h>
|
||||
#include <Interpreters/ProcessList.h>
|
||||
#include <Interpreters/loadMetadata.h>
|
||||
#include <base/getFQDNOrHostName.h>
|
||||
#include <Common/scope_guard_safe.h>
|
||||
#include <Interpreters/Session.h>
|
||||
#include <Access/AccessControl.h>
|
||||
#include <Common/PoolId.h>
|
||||
#include <Common/Exception.h>
|
||||
@ -32,11 +31,9 @@
|
||||
#include <Common/quoteString.h>
|
||||
#include <Common/randomSeed.h>
|
||||
#include <Common/ThreadPool.h>
|
||||
#include <Loggers/Loggers.h>
|
||||
#include <Loggers/OwnFormattingChannel.h>
|
||||
#include <Loggers/OwnPatternFormatter.h>
|
||||
#include <IO/ReadBufferFromFile.h>
|
||||
#include <IO/ReadBufferFromString.h>
|
||||
#include <IO/WriteBufferFromFileDescriptor.h>
|
||||
#include <IO/UseSSL.h>
|
||||
#include <IO/SharedThreadPools.h>
|
||||
@ -489,6 +486,7 @@ try
|
||||
registerFunctions();
|
||||
registerAggregateFunctions();
|
||||
registerTableFunctions();
|
||||
registerDatabases();
|
||||
registerStorages();
|
||||
registerDictionaries();
|
||||
registerDisks(/* global_skip_access_check= */ true);
|
||||
|
@ -97,9 +97,42 @@ void cckMetadataPathForOrdinary(const ASTCreateQuery & create, const String & me
|
||||
|
||||
}
|
||||
|
||||
/// validate validates the database engine that's specified in the create query for
|
||||
/// engine arguments, settings and table overrides.
|
||||
void validate(const ASTCreateQuery & create_query)
|
||||
|
||||
{
|
||||
auto * storage = create_query.storage;
|
||||
|
||||
/// Check engine may have arguments
|
||||
static const std::unordered_set<std::string_view> engines_with_arguments{"MySQL", "MaterializeMySQL", "MaterializedMySQL",
|
||||
"Lazy", "Replicated", "PostgreSQL", "MaterializedPostgreSQL", "SQLite", "Filesystem", "S3", "HDFS"};
|
||||
|
||||
const String & engine_name = storage->engine->name;
|
||||
bool engine_may_have_arguments = engines_with_arguments.contains(engine_name);
|
||||
|
||||
if (storage->engine->arguments && !engine_may_have_arguments)
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Database engine `{}` cannot have arguments", engine_name);
|
||||
|
||||
/// Check engine may have settings
|
||||
bool may_have_settings = endsWith(engine_name, "MySQL") || engine_name == "Replicated" || engine_name == "MaterializedPostgreSQL";
|
||||
bool has_unexpected_element = storage->engine->parameters || storage->partition_by ||
|
||||
storage->primary_key || storage->order_by ||
|
||||
storage->sample_by;
|
||||
if (has_unexpected_element || (!may_have_settings && storage->settings))
|
||||
throw Exception(ErrorCodes::UNKNOWN_ELEMENT_IN_AST,
|
||||
"Database engine `{}` cannot have parameters, primary_key, order_by, sample_by, settings", engine_name);
|
||||
|
||||
/// Check engine with table overrides
|
||||
static const std::unordered_set<std::string_view> engines_with_table_overrides{"MaterializeMySQL", "MaterializedMySQL", "MaterializedPostgreSQL"};
|
||||
if (create_query.table_overrides && !engines_with_table_overrides.contains(engine_name))
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Database engine `{}` cannot have table overrides", engine_name);
|
||||
}
|
||||
|
||||
DatabasePtr DatabaseFactory::get(const ASTCreateQuery & create, const String & metadata_path, ContextPtr context)
|
||||
{
|
||||
cckMetadataPathForOrdinary(create, metadata_path);
|
||||
validate(create);
|
||||
|
||||
DatabasePtr impl = getImpl(create, metadata_path, context);
|
||||
|
||||
@ -115,7 +148,7 @@ DatabasePtr DatabaseFactory::get(const ASTCreateQuery & create, const String & m
|
||||
|
||||
void DatabaseFactory::registerDatabase(const std::string & name, CreatorFn creator_fn)
|
||||
{
|
||||
if (!database_engines.emplace(name, Creator{std::move(creator_fn)}).second)
|
||||
if (!database_engines.emplace(name, std::move(creator_fn)).second)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "DatabaseFactory: the database engine name '{}' is not unique", name);
|
||||
}
|
||||
|
||||
@ -125,16 +158,15 @@ DatabaseFactory & DatabaseFactory::instance()
|
||||
return db_fact;
|
||||
}
|
||||
|
||||
|
||||
DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String & metadata_path, ContextPtr context)
|
||||
{
|
||||
auto * storage = create.storage;
|
||||
const String & database_name = create.getDatabase();
|
||||
const String & engine_name = storage->engine->name;
|
||||
const UUID & uuid = create.uuid;
|
||||
const ASTFunction & engine_def = *storage->engine;
|
||||
|
||||
bool has_engine_args = false;
|
||||
if (engine_def.arguments)
|
||||
if (storage->engine->arguments)
|
||||
has_engine_args = true;
|
||||
|
||||
if (!database_engines.contains(engine_name))
|
||||
@ -147,40 +179,15 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String
|
||||
.create_query = create,
|
||||
.database_name = database_name,
|
||||
.metadata_path = metadata_path,
|
||||
.uuid = uuid,
|
||||
.uuid = create.uuid,
|
||||
.context = context};
|
||||
|
||||
assert(arguments.getContext() == arguments.getContext()->getGlobalContext());
|
||||
|
||||
auto res = database_engines.at(engine_name).creator_fn(arguments);
|
||||
// creator_fn creates and returns a DatabasePtr with the supplied arguments
|
||||
auto creator_fn = database_engines.at(engine_name);
|
||||
|
||||
return res;
|
||||
|
||||
/// TODO: move validation to respective engines
|
||||
// static const std::unordered_set<std::string_view> database_engines{"Ordinary", "Atomic", "Memory",
|
||||
// "Dictionary", "Lazy", "Replicated", "MySQL", "MaterializeMySQL", "MaterializedMySQL",
|
||||
// "PostgreSQL", "MaterializedPostgreSQL", "SQLite", "Filesystem", "S3", "HDFS"};
|
||||
|
||||
// static const std::unordered_set<std::string_view> engines_with_arguments{"MySQL", "MaterializeMySQL", "MaterializedMySQL",
|
||||
// "Lazy", "Replicated", "PostgreSQL", "MaterializedPostgreSQL", "SQLite", "Filesystem", "S3", "HDFS"};
|
||||
//
|
||||
// static const std::unordered_set<std::string_view> engines_with_table_overrides{"MaterializeMySQL", "MaterializedMySQL", "MaterializedPostgreSQL"};
|
||||
// bool engine_may_have_arguments = engines_with_arguments.contains(engine_name);
|
||||
//
|
||||
// if (engine_define->engine->arguments && !engine_may_have_arguments)
|
||||
// throw Exception(ErrorCodes::BAD_ARGUMENTS, "Database engine `{}` cannot have arguments", engine_name);
|
||||
//
|
||||
// bool has_unexpected_element = engine_define->engine->parameters || engine_define->partition_by ||
|
||||
// engine_define->primary_key || engine_define->order_by ||
|
||||
// engine_define->sample_by;
|
||||
// bool may_have_settings = endsWith(engine_name, "MySQL") || engine_name == "Replicated" || engine_name == "MaterializedPostgreSQL";
|
||||
//
|
||||
// if (has_unexpected_element || (!may_have_settings && engine_define->settings))
|
||||
// throw Exception(ErrorCodes::UNKNOWN_ELEMENT_IN_AST,
|
||||
// "Database engine `{}` cannot have parameters, primary_key, order_by, sample_by, settings", engine_name);
|
||||
//
|
||||
// if (create.table_overrides && !engines_with_table_overrides.contains(engine_name))
|
||||
// throw Exception(ErrorCodes::BAD_ARGUMENTS, "Database engine `{}` cannot have table overrides", engine_name);
|
||||
return creator_fn(arguments);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -34,7 +34,7 @@ public:
|
||||
{
|
||||
const String & engine_name;
|
||||
ASTs & engine_args;
|
||||
ASTStorage * storage_def;
|
||||
ASTStorage * storage;
|
||||
const ASTCreateQuery & create_query;
|
||||
const String & database_name;
|
||||
const String & metadata_path;
|
||||
@ -48,16 +48,10 @@ public:
|
||||
|
||||
using CreatorFn = std::function<DatabasePtr(const Arguments & arguments)>;
|
||||
|
||||
struct Creator
|
||||
{
|
||||
CreatorFn creator_fn;
|
||||
};
|
||||
using DatabaseEngines = std::unordered_map<std::string, Creator>;
|
||||
using DatabaseEngines = std::unordered_map<std::string, CreatorFn>;
|
||||
|
||||
void registerDatabase(const std::string & name, CreatorFn creator_fn);
|
||||
|
||||
const DatabaseEngines & getAllDatabases() const { return database_engines; }
|
||||
|
||||
private:
|
||||
DatabaseEngines database_engines;
|
||||
};
|
||||
|
@ -6,7 +6,6 @@
|
||||
# include <Common/parseRemoteDescription.h>
|
||||
# include <Databases/MySQL/DatabaseMaterializedMySQL.h>
|
||||
|
||||
# include <Interpreters/Context.h>
|
||||
# include <Interpreters/evaluateConstantExpression.h>
|
||||
# include <Databases/DatabaseFactory.h>
|
||||
# include <Databases/MySQL/DatabaseMaterializedTablesIterator.h>
|
||||
@ -16,7 +15,6 @@
|
||||
# include <Parsers/queryToString.h>
|
||||
# include <Storages/StorageMySQL.h>
|
||||
# include <Storages/StorageMaterializedMySQL.h>
|
||||
# include <Storages/MySQL/MySQLHelpers.h>
|
||||
# include <Storages/NamedCollectionsHelpers.h>
|
||||
# include <Common/setThreadName.h>
|
||||
# include <Common/PoolId.h>
|
||||
@ -265,7 +263,8 @@ void registerDatabaseMaterializedMySQL(DatabaseFactory & factory)
|
||||
std::move(client),
|
||||
std::move(materialize_mode_settings));
|
||||
};
|
||||
factory.registerDatabase("MySQL", create_fn);
|
||||
factory.registerDatabase("MaterializeMySQL", create_fn);
|
||||
factory.registerDatabase("MaterializedMySQL", create_fn);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -22,14 +22,11 @@
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ParserCreateQuery.h>
|
||||
#include <Parsers/parseQuery.h>
|
||||
#include <Parsers/queryToString.h>
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
#include <Interpreters/InterpreterAlterQuery.h>
|
||||
#include <Common/escapeForFileName.h>
|
||||
#include <Poco/DirectoryIterator.h>
|
||||
#include <Poco/File.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -10,7 +10,6 @@
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/ASTColumnDeclaration.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Storages/StorageSQLite.h>
|
||||
#include <Databases/SQLite/SQLiteUtils.h>
|
||||
|
||||
|
@ -2,6 +2,7 @@
|
||||
#include <Interpreters/Context.h>
|
||||
#include "Processors/Executors/PullingPipelineExecutor.h"
|
||||
|
||||
#include <Functions/registerDatabases.h>
|
||||
#include <Functions/registerFunctions.h>
|
||||
#include <AggregateFunctions/registerAggregateFunctions.h>
|
||||
#include <TableFunctions/registerTableFunctions.h>
|
||||
@ -31,6 +32,7 @@ extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size)
|
||||
registerFunctions();
|
||||
registerAggregateFunctions();
|
||||
registerTableFunctions();
|
||||
registerDatabases();
|
||||
registerStorages();
|
||||
registerDictionaries();
|
||||
registerDisks(/* global_skip_access_check= */ true);
|
||||
|
Loading…
Reference in New Issue
Block a user