skip more sanity checks for secondary create queries

This commit is contained in:
Alexander Tokmakov 2024-03-23 00:03:31 +01:00
parent 41d9b19a4a
commit ace983f3d5
5 changed files with 33 additions and 30 deletions

View File

@ -75,7 +75,7 @@ std::pair<String, StoragePtr> createTableFromAST(
auto table_function = factory.get(table_function_ast, context);
ColumnsDescription columns;
if (ast_create_query.columns_list && ast_create_query.columns_list->columns)
columns = InterpreterCreateQuery::getColumnsDescription(*ast_create_query.columns_list->columns, context, true, false);
columns = InterpreterCreateQuery::getColumnsDescription(*ast_create_query.columns_list->columns, context, LoadingStrictnessLevel::ATTACH);
StoragePtr storage = table_function->execute(table_function_ast, context, ast_create_query.getTable(), std::move(columns));
storage->renameInMemory(ast_create_query);
return {ast_create_query.getTable(), storage};
@ -107,7 +107,7 @@ std::pair<String, StoragePtr> createTableFromAST(
}
else
{
columns = InterpreterCreateQuery::getColumnsDescription(*ast_create_query.columns_list->columns, context, true, false);
columns = InterpreterCreateQuery::getColumnsDescription(*ast_create_query.columns_list->columns, context, LoadingStrictnessLevel::ATTACH);
constraints = InterpreterCreateQuery::getConstraintsDescription(ast_create_query.columns_list->constraints);
}
}

View File

@ -513,7 +513,7 @@ ASTPtr InterpreterCreateQuery::formatProjections(const ProjectionsDescription &
}
ColumnsDescription InterpreterCreateQuery::getColumnsDescription(
const ASTExpressionList & columns_ast, ContextPtr context_, bool attach, bool is_restore_from_backup)
const ASTExpressionList & columns_ast, ContextPtr context_, LoadingStrictnessLevel mode)
{
/// First, deduce implicit types.
@ -522,7 +522,7 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription(
ASTPtr default_expr_list = std::make_shared<ASTExpressionList>();
NamesAndTypesList column_names_and_types;
bool make_columns_nullable = !attach && !is_restore_from_backup && context_->getSettingsRef().data_type_default_nullable;
bool make_columns_nullable = mode <= LoadingStrictnessLevel::CREATE && context_->getSettingsRef().data_type_default_nullable;
for (const auto & ast : columns_ast.children)
{
@ -538,7 +538,7 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription(
{
column_type = DataTypeFactory::instance().get(col_decl.type);
if (attach)
if (LoadingStrictnessLevel::ATTACH <= mode)
setVersionToAggregateFunctions(column_type, true);
if (col_decl.null_modifier)
@ -606,10 +606,11 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription(
if (!default_expr_list->children.empty())
defaults_sample_block = validateColumnsDefaultsAndGetSampleBlock(default_expr_list, column_names_and_types, context_);
bool sanity_check_compression_codecs = !attach && !context_->getSettingsRef().allow_suspicious_codecs;
bool allow_experimental_codecs = attach || context_->getSettingsRef().allow_experimental_codecs;
bool enable_deflate_qpl_codec = attach || context_->getSettingsRef().enable_deflate_qpl_codec;
bool enable_zstd_qat_codec = attach || context_->getSettingsRef().enable_zstd_qat_codec;
bool skip_checks = LoadingStrictnessLevel::SECONDARY_CREATE <= mode;
bool sanity_check_compression_codecs = !skip_checks && !context_->getSettingsRef().allow_suspicious_codecs;
bool allow_experimental_codecs = skip_checks || context_->getSettingsRef().allow_experimental_codecs;
bool enable_deflate_qpl_codec = skip_checks || context_->getSettingsRef().enable_deflate_qpl_codec;
bool enable_zstd_qat_codec = skip_checks || context_->getSettingsRef().enable_zstd_qat_codec;
ColumnsDescription res;
auto name_type_it = column_names_and_types.begin();
@ -675,7 +676,7 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription(
if (col_decl.stat_type)
{
if (!attach && !context_->getSettingsRef().allow_experimental_statistic)
if (!skip_checks && !context_->getSettingsRef().allow_experimental_statistic)
throw Exception(ErrorCodes::INCORRECT_QUERY, "Create table with statistic is now disabled. Turn on allow_experimental_statistic");
column.stat = StatisticDescription::getStatisticFromColumnDeclaration(col_decl);
}
@ -692,7 +693,7 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription(
res.add(std::move(column));
}
if (!attach && !is_restore_from_backup && context_->getSettingsRef().flatten_nested)
if (mode <= LoadingStrictnessLevel::CREATE && context_->getSettingsRef().flatten_nested)
res.flattenNested();
@ -714,7 +715,8 @@ ConstraintsDescription InterpreterCreateQuery::getConstraintsDescription(const A
}
InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTablePropertiesAndNormalizeCreateQuery(ASTCreateQuery & create) const
InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTablePropertiesAndNormalizeCreateQuery(
ASTCreateQuery & create, LoadingStrictnessLevel mode) const
{
/// Set the table engine if it was not specified explicitly.
setEngine(create);
@ -740,7 +742,7 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti
if (create.columns_list->columns)
{
properties.columns = getColumnsDescription(*create.columns_list->columns, getContext(), create.attach, is_restore_from_backup);
properties.columns = getColumnsDescription(*create.columns_list->columns, getContext(), mode);
}
if (create.columns_list->indices)
@ -1203,11 +1205,14 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create)
if (!UserDefinedSQLFunctionFactory::instance().empty())
UserDefinedSQLFunctionVisitor::visit(query_ptr);
bool is_secondary_query = getContext()->getZooKeeperMetadataTransaction() && !getContext()->getZooKeeperMetadataTransaction()->isInitialQuery();
auto mode = getLoadingStrictnessLevel(create.attach, /*force_attach*/ false, /*has_force_restore_data_flag*/ false, is_secondary_query || is_restore_from_backup);
/// Set and retrieve list of columns, indices and constraints. Set table engine if needed. Rewrite query in canonical way.
TableProperties properties = getTablePropertiesAndNormalizeCreateQuery(create);
TableProperties properties = getTablePropertiesAndNormalizeCreateQuery(create, mode);
/// Check type compatible for materialized dest table and select columns
if (create.select && create.is_materialized_view && create.to_table_id && !create.attach && !is_restore_from_backup)
if (create.select && create.is_materialized_view && create.to_table_id && mode <= LoadingStrictnessLevel::CREATE)
{
if (StoragePtr to_table = DatabaseCatalog::instance().tryGetTable(
{create.to_table_id.database_name, create.to_table_id.table_name, create.to_table_id.uuid},
@ -1291,11 +1296,11 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create)
if (create.replace_table)
{
chassert(!ddl_guard);
return doCreateOrReplaceTable(create, properties);
return doCreateOrReplaceTable(create, properties, mode);
}
/// Actually creates table
bool created = doCreateTable(create, properties, ddl_guard);
bool created = doCreateTable(create, properties, ddl_guard, mode);
ddl_guard.reset();
if (!created) /// Table already exists
@ -1312,11 +1317,8 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create)
bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create,
const InterpreterCreateQuery::TableProperties & properties,
DDLGuardPtr & ddl_guard)
DDLGuardPtr & ddl_guard, LoadingStrictnessLevel mode)
{
bool is_secondary_query = getContext()->getZooKeeperMetadataTransaction() && !getContext()->getZooKeeperMetadataTransaction()->isInitialQuery();
auto mode = getLoadingStrictnessLevel(create.attach, /*force_attach*/ false, /*has_force_restore_data_flag*/ false, is_secondary_query);
if (create.temporary)
{
if (create.if_not_exists && getContext()->tryResolveStorageID({"", create.getTable()}, Context::ResolveExternal))
@ -1552,7 +1554,7 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create,
BlockIO InterpreterCreateQuery::doCreateOrReplaceTable(ASTCreateQuery & create,
const InterpreterCreateQuery::TableProperties & properties)
const InterpreterCreateQuery::TableProperties & properties, LoadingStrictnessLevel mode)
{
/// Replicated database requires separate contexts for each DDL query
ContextPtr current_context = getContext();
@ -1602,7 +1604,7 @@ BlockIO InterpreterCreateQuery::doCreateOrReplaceTable(ASTCreateQuery & create,
{
/// Create temporary table (random name will be generated)
DDLGuardPtr ddl_guard;
[[maybe_unused]] bool done = InterpreterCreateQuery(query_ptr, create_context).doCreateTable(create, properties, ddl_guard);
[[maybe_unused]] bool done = InterpreterCreateQuery(query_ptr, create_context).doCreateTable(create, properties, ddl_guard, mode);
ddl_guard.reset();
assert(done);
created = true;

View File

@ -3,6 +3,7 @@
#include <Core/NamesAndAliases.h>
#include <Core/SettingsEnums.h>
#include <Access/Common/AccessRightsElement.h>
#include <Databases/LoadingStrictnessLevel.h>
#include <Interpreters/IInterpreter.h>
#include <Storages/ColumnsDescription.h>
#include <Storages/ConstraintsDescription.h>
@ -73,7 +74,7 @@ public:
/// Obtain information about columns, their types, default values and column comments,
/// for case when columns in CREATE query is specified explicitly.
static ColumnsDescription getColumnsDescription(const ASTExpressionList & columns, ContextPtr context, bool attach, bool is_restore_from_backup);
static ColumnsDescription getColumnsDescription(const ASTExpressionList & columns, ContextPtr context, LoadingStrictnessLevel mode);
static ConstraintsDescription getConstraintsDescription(const ASTExpressionList * constraints);
static void prepareOnClusterQuery(ASTCreateQuery & create, ContextPtr context, const String & cluster_name);
@ -96,14 +97,14 @@ private:
BlockIO createTable(ASTCreateQuery & create);
/// Calculate list of columns, constraints, indices, etc... of table. Rewrite query in canonical way.
TableProperties getTablePropertiesAndNormalizeCreateQuery(ASTCreateQuery & create) const;
TableProperties getTablePropertiesAndNormalizeCreateQuery(ASTCreateQuery & create, LoadingStrictnessLevel mode) const;
void validateTableStructure(const ASTCreateQuery & create, const TableProperties & properties) const;
void setEngine(ASTCreateQuery & create) const;
AccessRightsElements getRequiredAccess() const;
/// Create IStorage and add it to database. If table already exists and IF NOT EXISTS specified, do nothing and return false.
bool doCreateTable(ASTCreateQuery & create, const TableProperties & properties, DDLGuardPtr & ddl_guard);
BlockIO doCreateOrReplaceTable(ASTCreateQuery & create, const InterpreterCreateQuery::TableProperties & properties);
bool doCreateTable(ASTCreateQuery & create, const TableProperties & properties, DDLGuardPtr & ddl_guard, LoadingStrictnessLevel mode);
BlockIO doCreateOrReplaceTable(ASTCreateQuery & create, const InterpreterCreateQuery::TableProperties & properties, LoadingStrictnessLevel mode);
/// Inserts data in created table if it's CREATE ... SELECT
BlockIO fillTableIfNeeded(const ASTCreateQuery & create);

View File

@ -853,7 +853,7 @@ StoragePtr InterpreterSystemQuery::tryRestartReplica(const StorageID & replica,
auto & create = create_ast->as<ASTCreateQuery &>();
create.attach = true;
auto columns = InterpreterCreateQuery::getColumnsDescription(*create.columns_list->columns, system_context, true, false);
auto columns = InterpreterCreateQuery::getColumnsDescription(*create.columns_list->columns, system_context, LoadingStrictnessLevel::ATTACH);
auto constraints = InterpreterCreateQuery::getConstraintsDescription(create.columns_list->constraints);
auto data_path = database->getTableDataPath(create);

View File

@ -121,7 +121,7 @@ ColumnsDescription parseColumnsListFromString(const std::string & structure, con
if (!columns_list)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not cast AST to ASTExpressionList");
auto columns = InterpreterCreateQuery::getColumnsDescription(*columns_list, context, false, false);
auto columns = InterpreterCreateQuery::getColumnsDescription(*columns_list, context, LoadingStrictnessLevel::CREATE);
auto validation_settings = DataTypeValidationSettings(context->getSettingsRef());
for (const auto & [name, type] : columns.getAll())
validateDataType(type, validation_settings);
@ -149,7 +149,7 @@ bool tryParseColumnsListFromString(const std::string & structure, ColumnsDescrip
try
{
columns = InterpreterCreateQuery::getColumnsDescription(*columns_list, context, false, false);
columns = InterpreterCreateQuery::getColumnsDescription(*columns_list, context, LoadingStrictnessLevel::CREATE);
auto validation_settings = DataTypeValidationSettings(context->getSettingsRef());
for (const auto & [name, type] : columns.getAll())
validateDataType(type, validation_settings);