mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-15 20:24:07 +00:00
Merge pull request #68574 from ClickHouse/validate-create-query-after-alter
Validate create query after altering tables
This commit is contained in:
commit
2b87bff734
@ -150,7 +150,7 @@ void DatabaseMemory::alterTable(ContextPtr local_context, const StorageID & tabl
|
||||
if (it == create_queries.end() || !it->second)
|
||||
throw Exception(ErrorCodes::UNKNOWN_TABLE, "Cannot alter: There is no metadata of table {}", table_id.getNameForLogs());
|
||||
|
||||
applyMetadataChangesToCreateQuery(it->second, metadata);
|
||||
applyMetadataChangesToCreateQuery(it->second, metadata, local_context);
|
||||
|
||||
/// The create query of the table has been just changed, we need to update dependencies too.
|
||||
auto ref_dependencies = getDependenciesFromCreateQuery(local_context->getGlobalContext(), table_id.getQualifiedName(), it->second, local_context->getCurrentDatabase());
|
||||
|
@ -123,7 +123,7 @@ std::pair<String, StoragePtr> createTableFromAST(
|
||||
else
|
||||
{
|
||||
columns = InterpreterCreateQuery::getColumnsDescription(*ast_create_query.columns_list->columns, context, mode);
|
||||
constraints = InterpreterCreateQuery::getConstraintsDescription(ast_create_query.columns_list->constraints);
|
||||
constraints = InterpreterCreateQuery::getConstraintsDescription(ast_create_query.columns_list->constraints, columns, context);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -567,7 +567,7 @@ void DatabaseOrdinary::alterTable(ContextPtr local_context, const StorageID & ta
|
||||
local_context->getSettingsRef()[Setting::max_parser_depth],
|
||||
local_context->getSettingsRef()[Setting::max_parser_backtracks]);
|
||||
|
||||
applyMetadataChangesToCreateQuery(ast, metadata);
|
||||
applyMetadataChangesToCreateQuery(ast, metadata, local_context);
|
||||
|
||||
statement = getObjectDefinitionFromCreateQuery(ast);
|
||||
{
|
||||
|
@ -87,7 +87,7 @@ public:
|
||||
|
||||
void shutdown() override;
|
||||
|
||||
std::vector<std::pair<ASTPtr, StoragePtr>> getTablesForBackup(const FilterByNameFunction & filter, const ContextPtr & local_context) const override;
|
||||
std::vector<std::pair<ASTPtr, StoragePtr>> getTablesForBackup(const FilterByNameFunction & filter, const ContextPtr &) const override;
|
||||
void createTableRestoredFromBackup(const ASTPtr & create_table_query, ContextMutablePtr local_context, std::shared_ptr<IRestoreCoordination> restore_coordination, UInt64 timeout_ms) override;
|
||||
|
||||
bool shouldReplicateQuery(const ContextPtr & query_context, const ASTPtr & query_ptr) const override;
|
||||
|
@ -2,15 +2,22 @@
|
||||
|
||||
#include <Backups/BackupEntriesCollector.h>
|
||||
#include <Backups/RestorerFromBackup.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/DatabaseCatalog.h>
|
||||
#include <Interpreters/InterpreterCreateQuery.h>
|
||||
#include <Interpreters/TreeRewriter.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
#include <Parsers/ASTSelectWithUnionQuery.h>
|
||||
#include <Parsers/ParserCreateQuery.h>
|
||||
#include <Parsers/formatAST.h>
|
||||
#include <Parsers/parseQuery.h>
|
||||
#include <Storages/ColumnsDescription.h>
|
||||
#include <Storages/KeyDescription.h>
|
||||
#include <Storages/StorageDictionary.h>
|
||||
#include <Storages/StorageFactory.h>
|
||||
#include <Storages/TTLDescription.h>
|
||||
#include <Storages/Utils.h>
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
#include <Common/CurrentMetrics.h>
|
||||
@ -18,10 +25,14 @@
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace Setting
|
||||
{
|
||||
extern const SettingsUInt64 max_parser_backtracks;
|
||||
extern const SettingsUInt64 max_parser_depth;
|
||||
}
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int TABLE_ALREADY_EXISTS;
|
||||
@ -31,8 +42,87 @@ namespace ErrorCodes
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int CANNOT_GET_CREATE_TABLE_QUERY;
|
||||
}
|
||||
namespace
|
||||
{
|
||||
void validateCreateQuery(const ASTCreateQuery & query, ContextPtr context)
|
||||
{
|
||||
/// First validate that the query can be parsed
|
||||
const auto serialized_query = serializeAST(query);
|
||||
ParserCreateQuery parser;
|
||||
ASTPtr new_query_raw = parseQuery(
|
||||
parser,
|
||||
serialized_query.data(),
|
||||
serialized_query.data() + serialized_query.size(),
|
||||
"after altering table ",
|
||||
0,
|
||||
context->getSettingsRef()[Setting::max_parser_backtracks],
|
||||
context->getSettingsRef()[Setting::max_parser_depth]);
|
||||
const auto & new_query = new_query_raw->as<const ASTCreateQuery &>();
|
||||
/// If there are no columns, then there is nothing much we can do
|
||||
if (!new_query.columns_list || !new_query.columns_list->columns)
|
||||
return;
|
||||
|
||||
void applyMetadataChangesToCreateQuery(const ASTPtr & query, const StorageInMemoryMetadata & metadata)
|
||||
const auto & columns = *new_query.columns_list;
|
||||
/// Do some basic sanity checks. We cannot do the same strict checks as on create, because context might not have the same settings if it is not called directly from an alter query.
|
||||
/// SECONDARY_CREATE should check most of the important things.
|
||||
const auto columns_desc
|
||||
= InterpreterCreateQuery::getColumnsDescription(*columns.columns, context, LoadingStrictnessLevel::SECONDARY_CREATE, false);
|
||||
|
||||
/// Default expressions are only validated in level CREATE, so let's check them now
|
||||
DefaultExpressionsInfo default_expr_info{std::make_shared<ASTExpressionList>()};
|
||||
|
||||
for (const auto & ast : columns.columns->children)
|
||||
{
|
||||
const auto & col_decl = ast->as<ASTColumnDeclaration &>();
|
||||
/// There might be some special columns for which `columns_desc.get` would throw, e.g. Nested column when flatten_nested is enabled.
|
||||
/// At the time of writing I am not aware of anything else, but my knowledge is limited and new types might be added, so let's be safe.
|
||||
if (!col_decl.default_expression)
|
||||
continue;
|
||||
|
||||
/// If no column description for the name, let's skip the validation of default expressions, but let's log the fact that something went wrong
|
||||
if (const auto * maybe_column_desc = columns_desc.tryGet(col_decl.name); maybe_column_desc)
|
||||
getDefaultExpressionInfoInto(col_decl, maybe_column_desc->type, default_expr_info);
|
||||
else
|
||||
LOG_WARNING(getLogger("validateCreateQuery"), "Couldn't get column description for column {}", col_decl.name);
|
||||
}
|
||||
|
||||
if (default_expr_info.expr_list)
|
||||
validateColumnsDefaultsAndGetSampleBlock(default_expr_info.expr_list, columns_desc.getAll(), context);
|
||||
|
||||
if (columns.indices)
|
||||
{
|
||||
for (const auto & child : columns.indices->children)
|
||||
IndexDescription::getIndexFromAST(child, columns_desc, context);
|
||||
}
|
||||
if (columns.constraints)
|
||||
{
|
||||
InterpreterCreateQuery::getConstraintsDescription(columns.constraints, columns_desc, context);
|
||||
}
|
||||
if (columns.projections)
|
||||
{
|
||||
for (const auto & child : columns.projections->children)
|
||||
ProjectionDescription::getProjectionFromAST(child, columns_desc, context);
|
||||
}
|
||||
if (!new_query.storage)
|
||||
return;
|
||||
const auto & storage = *new_query.storage;
|
||||
|
||||
std::optional<KeyDescription> primary_key;
|
||||
/// First get the key description from order by, so if there is no primary key we will use that
|
||||
if (storage.order_by)
|
||||
primary_key = KeyDescription::getKeyFromAST(storage.order_by->ptr(), columns_desc, context);
|
||||
if (storage.primary_key)
|
||||
primary_key = KeyDescription::getKeyFromAST(storage.primary_key->ptr(), columns_desc, context);
|
||||
if (storage.partition_by)
|
||||
KeyDescription::getKeyFromAST(storage.partition_by->ptr(), columns_desc, context);
|
||||
if (storage.sample_by)
|
||||
KeyDescription::getKeyFromAST(storage.sample_by->ptr(), columns_desc, context);
|
||||
if (storage.ttl_table && primary_key.has_value())
|
||||
TTLTableDescription::getTTLForTableFromAST(storage.ttl_table->ptr(), columns_desc, context, *primary_key, true);
|
||||
}
|
||||
}
|
||||
|
||||
void applyMetadataChangesToCreateQuery(const ASTPtr & query, const StorageInMemoryMetadata & metadata, ContextPtr context)
|
||||
{
|
||||
auto & ast_create_query = query->as<ASTCreateQuery &>();
|
||||
|
||||
@ -115,6 +205,8 @@ void applyMetadataChangesToCreateQuery(const ASTPtr & query, const StorageInMemo
|
||||
ast_create_query.reset(ast_create_query.comment);
|
||||
else
|
||||
ast_create_query.set(ast_create_query.comment, std::make_shared<ASTLiteral>(metadata.comment));
|
||||
|
||||
validateCreateQuery(ast_create_query, context);
|
||||
}
|
||||
|
||||
|
||||
|
@ -1,10 +1,9 @@
|
||||
#pragma once
|
||||
|
||||
#include <base/types.h>
|
||||
#include <Databases/IDatabase.h>
|
||||
#include <Parsers/IAST_fwd.h>
|
||||
#include <Storages/IStorage_fwd.h>
|
||||
#include <Databases/IDatabase.h>
|
||||
#include <mutex>
|
||||
#include <base/types.h>
|
||||
|
||||
|
||||
/// General functionality for several different database engines.
|
||||
@ -12,15 +11,13 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
void applyMetadataChangesToCreateQuery(const ASTPtr & query, const StorageInMemoryMetadata & metadata);
|
||||
void applyMetadataChangesToCreateQuery(const ASTPtr & query, const StorageInMemoryMetadata & metadata, ContextPtr context);
|
||||
ASTPtr getCreateQueryFromStorage(const StoragePtr & storage, const ASTPtr & ast_storage, bool only_ordinary,
|
||||
uint32_t max_parser_depth, uint32_t max_parser_backtracks, bool throw_on_error);
|
||||
|
||||
/// Cleans a CREATE QUERY from temporary flags like "IF NOT EXISTS", "OR REPLACE", "AS SELECT" (for non-views), etc.
|
||||
void cleanupObjectDefinitionFromTemporaryFlags(ASTCreateQuery & query);
|
||||
|
||||
class Context;
|
||||
|
||||
/// A base class for databases that manage their own list of tables.
|
||||
class DatabaseWithOwnTablesBase : public IDatabase, protected WithContext
|
||||
{
|
||||
|
@ -575,6 +575,47 @@ ASTPtr InterpreterCreateQuery::formatProjections(const ProjectionsDescription &
|
||||
return res;
|
||||
}
|
||||
|
||||
DataTypePtr InterpreterCreateQuery::getColumnType(
|
||||
const ASTColumnDeclaration & col_decl, const LoadingStrictnessLevel mode, const bool make_columns_nullable)
|
||||
{
|
||||
if (!col_decl.type)
|
||||
{
|
||||
/// we're creating dummy DataTypeUInt8 in order to prevent the NullPointerException in ExpressionActions
|
||||
return std::make_shared<DataTypeUInt8>();
|
||||
}
|
||||
|
||||
DataTypePtr column_type = DataTypeFactory::instance().get(col_decl.type);
|
||||
|
||||
if (LoadingStrictnessLevel::ATTACH <= mode)
|
||||
setVersionToAggregateFunctions(column_type, true);
|
||||
|
||||
if (col_decl.null_modifier)
|
||||
{
|
||||
if (column_type->isNullable())
|
||||
throw Exception(ErrorCodes::ILLEGAL_SYNTAX_FOR_DATA_TYPE, "Can't use [NOT] NULL modifier with Nullable type");
|
||||
if (*col_decl.null_modifier)
|
||||
column_type = makeNullable(column_type);
|
||||
}
|
||||
else if (make_columns_nullable)
|
||||
{
|
||||
column_type = makeNullable(column_type);
|
||||
}
|
||||
else if (
|
||||
!hasNullable(column_type) && col_decl.default_specifier == "DEFAULT" && col_decl.default_expression
|
||||
&& col_decl.default_expression->as<ASTLiteral>() && col_decl.default_expression->as<ASTLiteral>()->value.isNull())
|
||||
{
|
||||
if (column_type->lowCardinality())
|
||||
{
|
||||
const auto * low_cardinality_type = typeid_cast<const DataTypeLowCardinality *>(column_type.get());
|
||||
assert(low_cardinality_type);
|
||||
column_type = std::make_shared<DataTypeLowCardinality>(makeNullable(low_cardinality_type->getDictionaryType()));
|
||||
}
|
||||
else
|
||||
column_type = makeNullable(column_type);
|
||||
}
|
||||
return column_type;
|
||||
}
|
||||
|
||||
ColumnsDescription InterpreterCreateQuery::getColumnsDescription(
|
||||
const ASTExpressionList & columns_ast, ContextPtr context_, LoadingStrictnessLevel mode, bool is_restore_from_backup)
|
||||
{
|
||||
@ -583,11 +624,10 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription(
|
||||
/** all default_expressions as a single expression list,
|
||||
* mixed with conversion-columns for each explicitly specified type */
|
||||
|
||||
ASTPtr default_expr_list = std::make_shared<ASTExpressionList>();
|
||||
DefaultExpressionsInfo default_expr_info{std::make_shared<ASTExpressionList>()};
|
||||
NamesAndTypesList column_names_and_types;
|
||||
bool make_columns_nullable = mode <= LoadingStrictnessLevel::SECONDARY_CREATE && !is_restore_from_backup
|
||||
&& context_->getSettingsRef()[Setting::data_type_default_nullable];
|
||||
bool has_columns_with_default_without_type = false;
|
||||
|
||||
for (const auto & ast : columns_ast.children)
|
||||
{
|
||||
@ -595,78 +635,15 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription(
|
||||
|
||||
if (col_decl.collation && !context_->getSettingsRef()[Setting::compatibility_ignore_collation_in_create_table])
|
||||
{
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot support collation, please set compatibility_ignore_collation_in_create_table=true");
|
||||
throw Exception(
|
||||
ErrorCodes::NOT_IMPLEMENTED, "Cannot support collation, please set compatibility_ignore_collation_in_create_table=true");
|
||||
}
|
||||
|
||||
DataTypePtr column_type = nullptr;
|
||||
if (col_decl.type)
|
||||
{
|
||||
column_type = DataTypeFactory::instance().get(col_decl.type);
|
||||
|
||||
if (LoadingStrictnessLevel::ATTACH <= mode)
|
||||
setVersionToAggregateFunctions(column_type, true);
|
||||
|
||||
if (col_decl.null_modifier)
|
||||
{
|
||||
if (column_type->isNullable())
|
||||
throw Exception(ErrorCodes::ILLEGAL_SYNTAX_FOR_DATA_TYPE, "Can't use [NOT] NULL modifier with Nullable type");
|
||||
if (*col_decl.null_modifier)
|
||||
column_type = makeNullable(column_type);
|
||||
}
|
||||
else if (make_columns_nullable)
|
||||
{
|
||||
column_type = makeNullable(column_type);
|
||||
}
|
||||
else if (!hasNullable(column_type) &&
|
||||
col_decl.default_specifier == "DEFAULT" &&
|
||||
col_decl.default_expression &&
|
||||
col_decl.default_expression->as<ASTLiteral>() &&
|
||||
col_decl.default_expression->as<ASTLiteral>()->value.isNull())
|
||||
{
|
||||
if (column_type->lowCardinality())
|
||||
{
|
||||
const auto * low_cardinality_type = typeid_cast<const DataTypeLowCardinality *>(column_type.get());
|
||||
assert(low_cardinality_type);
|
||||
column_type = std::make_shared<DataTypeLowCardinality>(makeNullable(low_cardinality_type->getDictionaryType()));
|
||||
}
|
||||
else
|
||||
column_type = makeNullable(column_type);
|
||||
}
|
||||
|
||||
column_names_and_types.emplace_back(col_decl.name, column_type);
|
||||
}
|
||||
else
|
||||
{
|
||||
/// we're creating dummy DataTypeUInt8 in order to prevent the NullPointerException in ExpressionActions
|
||||
column_names_and_types.emplace_back(col_decl.name, std::make_shared<DataTypeUInt8>());
|
||||
}
|
||||
column_names_and_types.emplace_back(col_decl.name, getColumnType(col_decl, mode, make_columns_nullable));
|
||||
|
||||
/// add column to postprocessing if there is a default_expression specified
|
||||
if (col_decl.default_expression)
|
||||
{
|
||||
/** For columns with explicitly-specified type create two expressions:
|
||||
* 1. default_expression aliased as column name with _tmp suffix
|
||||
* 2. conversion of expression (1) to explicitly-specified type alias as column name
|
||||
*/
|
||||
if (col_decl.type)
|
||||
{
|
||||
const auto & final_column_name = col_decl.name;
|
||||
const auto tmp_column_name = final_column_name + "_tmp_alter" + toString(randomSeed());
|
||||
const auto * data_type_ptr = column_names_and_types.back().type.get();
|
||||
|
||||
default_expr_list->children.emplace_back(
|
||||
setAlias(addTypeConversionToAST(std::make_shared<ASTIdentifier>(tmp_column_name), data_type_ptr->getName()),
|
||||
final_column_name));
|
||||
|
||||
default_expr_list->children.emplace_back(
|
||||
setAlias(col_decl.default_expression->clone(), tmp_column_name));
|
||||
}
|
||||
else
|
||||
{
|
||||
has_columns_with_default_without_type = true;
|
||||
default_expr_list->children.emplace_back(setAlias(col_decl.default_expression->clone(), col_decl.name));
|
||||
}
|
||||
}
|
||||
getDefaultExpressionInfoInto(col_decl, column_names_and_types.back().type, default_expr_info);
|
||||
}
|
||||
|
||||
Block defaults_sample_block;
|
||||
@ -674,9 +651,10 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription(
|
||||
/// We try to avoid that validation while restoring from a backup because it might be slow or troublesome
|
||||
/// (for example, a default expression can contain dictGet() and that dictionary can access remote servers or
|
||||
/// require different users to authenticate).
|
||||
if (!default_expr_list->children.empty() && (has_columns_with_default_without_type || (mode <= LoadingStrictnessLevel::CREATE)))
|
||||
if (!default_expr_info.expr_list->children.empty()
|
||||
&& (default_expr_info.has_columns_with_default_without_type || (mode <= LoadingStrictnessLevel::CREATE)))
|
||||
{
|
||||
defaults_sample_block = validateColumnsDefaultsAndGetSampleBlock(default_expr_list, column_names_and_types, context_);
|
||||
defaults_sample_block = validateColumnsDefaultsAndGetSampleBlock(default_expr_info.expr_list, column_names_and_types, context_);
|
||||
}
|
||||
|
||||
bool skip_checks = LoadingStrictnessLevel::SECONDARY_CREATE <= mode;
|
||||
@ -778,13 +756,18 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription(
|
||||
}
|
||||
|
||||
|
||||
ConstraintsDescription InterpreterCreateQuery::getConstraintsDescription(const ASTExpressionList * constraints)
|
||||
ConstraintsDescription InterpreterCreateQuery::getConstraintsDescription(
|
||||
const ASTExpressionList * constraints, const ColumnsDescription & columns, ContextPtr local_context)
|
||||
{
|
||||
ASTs constraints_data;
|
||||
const auto column_names_and_types = columns.getAllPhysical();
|
||||
if (constraints)
|
||||
for (const auto & constraint : constraints->children)
|
||||
{
|
||||
auto clone = constraint->clone();
|
||||
TreeRewriter(local_context).analyze(clone, column_names_and_types);
|
||||
constraints_data.push_back(constraint->clone());
|
||||
|
||||
}
|
||||
return ConstraintsDescription{constraints_data};
|
||||
}
|
||||
|
||||
@ -847,7 +830,7 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti
|
||||
properties.projections.add(std::move(projection));
|
||||
}
|
||||
|
||||
properties.constraints = getConstraintsDescription(create.columns_list->constraints);
|
||||
properties.constraints = getConstraintsDescription(create.columns_list->constraints, properties.columns, getContext());
|
||||
}
|
||||
else if (!create.as_table.empty())
|
||||
{
|
||||
|
@ -14,6 +14,7 @@ namespace DB
|
||||
{
|
||||
|
||||
class ASTCreateQuery;
|
||||
class ASTColumnDeclaration;
|
||||
class ASTExpressionList;
|
||||
class ASTConstraintDeclaration;
|
||||
class ASTStorage;
|
||||
@ -71,10 +72,13 @@ public:
|
||||
is_restore_from_backup = is_restore_from_backup_;
|
||||
}
|
||||
|
||||
static DataTypePtr getColumnType(const ASTColumnDeclaration & col_decl, LoadingStrictnessLevel mode, bool make_columns_nullable);
|
||||
|
||||
/// 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, LoadingStrictnessLevel mode, bool is_restore_from_backup = false);
|
||||
static ConstraintsDescription getConstraintsDescription(const ASTExpressionList * constraints);
|
||||
static ConstraintsDescription
|
||||
getConstraintsDescription(const ASTExpressionList * constraints, const ColumnsDescription & columns, ContextPtr local_context);
|
||||
|
||||
static void prepareOnClusterQuery(ASTCreateQuery & create, ContextPtr context, const String & cluster_name);
|
||||
|
||||
|
@ -889,7 +889,7 @@ StoragePtr InterpreterSystemQuery::tryRestartReplica(const StorageID & replica,
|
||||
create.attach = true;
|
||||
|
||||
auto columns = InterpreterCreateQuery::getColumnsDescription(*create.columns_list->columns, system_context, LoadingStrictnessLevel::ATTACH);
|
||||
auto constraints = InterpreterCreateQuery::getConstraintsDescription(create.columns_list->constraints);
|
||||
auto constraints = InterpreterCreateQuery::getConstraintsDescription(create.columns_list->constraints, columns, system_context);
|
||||
auto data_path = database->getTableDataPath(create);
|
||||
|
||||
table = StorageFactory::instance().get(create,
|
||||
|
@ -1,36 +1,42 @@
|
||||
#include <Storages/ColumnsDescription.h>
|
||||
|
||||
#include <memory>
|
||||
#include <Compression/CompressionFactory.h>
|
||||
#include <Core/Defines.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeFactory.h>
|
||||
#include <DataTypes/DataTypeNested.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <DataTypes/NestedUtils.h>
|
||||
#include <IO/ReadBuffer.h>
|
||||
#include <IO/ReadBufferFromString.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteBuffer.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/ExpressionActions.h>
|
||||
#include <Interpreters/ExpressionAnalyzer.h>
|
||||
#include <Interpreters/FunctionNameNormalizer.h>
|
||||
#include <Interpreters/TreeRewriter.h>
|
||||
#include <Interpreters/addTypeConversionToAST.h>
|
||||
#include <Parsers/ASTColumnDeclaration.h>
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
#include <Parsers/ASTSelectWithUnionQuery.h>
|
||||
#include <Parsers/ASTSetQuery.h>
|
||||
#include <Parsers/ASTSubquery.h>
|
||||
#include <Parsers/ExpressionElementParsers.h>
|
||||
#include <Parsers/ExpressionListParsers.h>
|
||||
#include <Parsers/ParserCreateQuery.h>
|
||||
#include <Parsers/parseQuery.h>
|
||||
#include <Parsers/queryToString.h>
|
||||
#include <Parsers/ASTSubquery.h>
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
#include <Parsers/ASTSelectWithUnionQuery.h>
|
||||
#include <Parsers/ASTSetQuery.h>
|
||||
#include <IO/WriteBuffer.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/ReadBuffer.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <IO/ReadBufferFromString.h>
|
||||
#include <DataTypes/DataTypeFactory.h>
|
||||
#include <DataTypes/NestedUtils.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <DataTypes/DataTypeNested.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Storages/IStorage.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/randomSeed.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Core/Defines.h>
|
||||
#include <Compression/CompressionFactory.h>
|
||||
#include <Interpreters/ExpressionAnalyzer.h>
|
||||
#include <Interpreters/TreeRewriter.h>
|
||||
#include <Interpreters/ExpressionActions.h>
|
||||
#include <Interpreters/FunctionNameNormalizer.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -970,7 +976,36 @@ std::vector<String> ColumnsDescription::getAllRegisteredNames() const
|
||||
return names;
|
||||
}
|
||||
|
||||
Block validateColumnsDefaultsAndGetSampleBlock(ASTPtr default_expr_list, const NamesAndTypesList & all_columns, ContextPtr context)
|
||||
void getDefaultExpressionInfoInto(const ASTColumnDeclaration & col_decl, const DataTypePtr & data_type, DefaultExpressionsInfo & info)
|
||||
{
|
||||
if (!col_decl.default_expression)
|
||||
return;
|
||||
|
||||
/** For columns with explicitly-specified type create two expressions:
|
||||
* 1. default_expression aliased as column name with _tmp suffix
|
||||
* 2. conversion of expression (1) to explicitly-specified type alias as column name
|
||||
*/
|
||||
if (col_decl.type)
|
||||
{
|
||||
const auto & final_column_name = col_decl.name;
|
||||
const auto tmp_column_name = final_column_name + "_tmp_alter" + toString(randomSeed());
|
||||
const auto * data_type_ptr = data_type.get();
|
||||
|
||||
info.expr_list->children.emplace_back(setAlias(
|
||||
addTypeConversionToAST(std::make_shared<ASTIdentifier>(tmp_column_name), data_type_ptr->getName()), final_column_name));
|
||||
|
||||
info.expr_list->children.emplace_back(setAlias(col_decl.default_expression->clone(), tmp_column_name));
|
||||
}
|
||||
else
|
||||
{
|
||||
info.has_columns_with_default_without_type = true;
|
||||
info.expr_list->children.emplace_back(setAlias(col_decl.default_expression->clone(), col_decl.name));
|
||||
}
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
std::optional<Block> validateColumnsDefaultsAndGetSampleBlockImpl(ASTPtr default_expr_list, const NamesAndTypesList & all_columns, ContextPtr context, bool get_sample_block)
|
||||
{
|
||||
for (const auto & child : default_expr_list->children)
|
||||
if (child->as<ASTSelectQuery>() || child->as<ASTSelectWithUnionQuery>() || child->as<ASTSubquery>())
|
||||
@ -984,6 +1019,9 @@ Block validateColumnsDefaultsAndGetSampleBlock(ASTPtr default_expr_list, const N
|
||||
if (action.node->type == ActionsDAG::ActionType::ARRAY_JOIN)
|
||||
throw Exception(ErrorCodes::THERE_IS_NO_DEFAULT_VALUE, "Unsupported default value that requires ARRAY JOIN action");
|
||||
|
||||
if (!get_sample_block)
|
||||
return {};
|
||||
|
||||
return actions->getSampleBlock();
|
||||
}
|
||||
catch (Exception & ex)
|
||||
@ -992,5 +1030,19 @@ Block validateColumnsDefaultsAndGetSampleBlock(ASTPtr default_expr_list, const N
|
||||
throw;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
void validateColumnsDefaults(ASTPtr default_expr_list, const NamesAndTypesList & all_columns, ContextPtr context)
|
||||
{
|
||||
/// Do not execute the default expressions as they might be heavy, e.g.: access remote servers, etc.
|
||||
validateColumnsDefaultsAndGetSampleBlockImpl(default_expr_list, all_columns, context, /*get_sample_block=*/false);
|
||||
}
|
||||
|
||||
Block validateColumnsDefaultsAndGetSampleBlock(ASTPtr default_expr_list, const NamesAndTypesList & all_columns, ContextPtr context)
|
||||
{
|
||||
auto result = validateColumnsDefaultsAndGetSampleBlockImpl(default_expr_list, all_columns, context, /*get_sample_block=*/true);
|
||||
chassert(result.has_value());
|
||||
return std::move(*result);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -268,10 +268,21 @@ private:
|
||||
void removeSubcolumns(const String & name_in_storage);
|
||||
};
|
||||
|
||||
class ASTColumnDeclaration;
|
||||
|
||||
struct DefaultExpressionsInfo
|
||||
{
|
||||
ASTPtr expr_list = nullptr;
|
||||
bool has_columns_with_default_without_type = false;
|
||||
};
|
||||
|
||||
void getDefaultExpressionInfoInto(const ASTColumnDeclaration & col_decl, const DataTypePtr & data_type, DefaultExpressionsInfo & info);
|
||||
|
||||
/// Validate default expressions and corresponding types compatibility, i.e.
|
||||
/// default expression result can be casted to column_type. Also checks, that we
|
||||
/// don't have strange constructions in default expression like SELECT query or
|
||||
/// arrayJoin function.
|
||||
void validateColumnsDefaults(ASTPtr default_expr_list, const NamesAndTypesList & all_columns, ContextPtr context);
|
||||
Block validateColumnsDefaultsAndGetSampleBlock(ASTPtr default_expr_list, const NamesAndTypesList & all_columns, ContextPtr context);
|
||||
|
||||
}
|
||||
|
@ -378,11 +378,13 @@ void StorageMergeTree::alter(
|
||||
if (commands.isSettingsAlter())
|
||||
{
|
||||
changeSettings(new_metadata.settings_changes, table_lock_holder);
|
||||
/// It is safe to ignore exceptions here as only settings are changed, which is not validated in `alterTable`
|
||||
DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(local_context, table_id, new_metadata);
|
||||
}
|
||||
else if (commands.isCommentAlter())
|
||||
{
|
||||
setInMemoryMetadata(new_metadata);
|
||||
/// It is safe to ignore exceptions here as only the comment changed, which is not validated in `alterTable`
|
||||
DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(local_context, table_id, new_metadata);
|
||||
}
|
||||
else
|
||||
@ -413,7 +415,17 @@ void StorageMergeTree::alter(
|
||||
/// Reinitialize primary key because primary key column types might have changed.
|
||||
setProperties(new_metadata, old_metadata, false, local_context);
|
||||
|
||||
DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(local_context, table_id, new_metadata);
|
||||
try
|
||||
{
|
||||
DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(local_context, table_id, new_metadata);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
LOG_ERROR(log, "Failed to alter table in database, reverting changes");
|
||||
changeSettings(old_metadata.settings_changes, table_lock_holder);
|
||||
setProperties(old_metadata, new_metadata, false, local_context);
|
||||
throw;
|
||||
}
|
||||
|
||||
{
|
||||
/// Reset Object columns, because column of type
|
||||
|
@ -1564,7 +1564,16 @@ void StorageReplicatedMergeTree::setTableStructure(const StorageID & table_id, c
|
||||
checkTTLExpressions(new_metadata, old_metadata);
|
||||
setProperties(new_metadata, old_metadata);
|
||||
|
||||
DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(local_context, table_id, new_metadata);
|
||||
try
|
||||
{
|
||||
DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(local_context, table_id, new_metadata);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
LOG_ERROR(log, "Failed to set table structure, reverting changes");
|
||||
setProperties(old_metadata, new_metadata);
|
||||
throw;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@ -6160,40 +6169,42 @@ void StorageReplicatedMergeTree::alter(
|
||||
auto table_id = getStorageID();
|
||||
const auto & query_settings = query_context->getSettingsRef();
|
||||
|
||||
StorageInMemoryMetadata future_metadata = getInMemoryMetadata();
|
||||
commands.apply(future_metadata, query_context);
|
||||
|
||||
if (commands.isSettingsAlter())
|
||||
{
|
||||
/// We don't replicate storage_settings_ptr ALTER. It's local operation.
|
||||
/// Also we don't upgrade alter lock to table structure lock.
|
||||
StorageInMemoryMetadata future_metadata = getInMemoryMetadata();
|
||||
commands.apply(future_metadata, query_context);
|
||||
|
||||
merge_strategy_picker.refreshState();
|
||||
|
||||
changeSettings(future_metadata.settings_changes, table_lock_holder);
|
||||
|
||||
/// It is safe to ignore exceptions here as only settings are changed, which is not validated in `alterTable`
|
||||
DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(query_context, table_id, future_metadata);
|
||||
return;
|
||||
}
|
||||
|
||||
if (commands.isCommentAlter())
|
||||
{
|
||||
StorageInMemoryMetadata future_metadata = getInMemoryMetadata();
|
||||
commands.apply(future_metadata, query_context);
|
||||
|
||||
setInMemoryMetadata(future_metadata);
|
||||
|
||||
/// It is safe to ignore exceptions here as only the comment is changed, which is not validated in `alterTable`
|
||||
DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(query_context, table_id, future_metadata);
|
||||
return;
|
||||
}
|
||||
|
||||
if (!query_settings[Setting::allow_suspicious_primary_key])
|
||||
{
|
||||
StorageInMemoryMetadata future_metadata = getInMemoryMetadata();
|
||||
commands.apply(future_metadata, query_context);
|
||||
|
||||
MergeTreeData::verifySortingKey(future_metadata.sorting_key);
|
||||
}
|
||||
|
||||
{
|
||||
/// Call applyMetadataChangesToCreateQuery to validate the resulting CREATE query
|
||||
auto ast = DatabaseCatalog::instance().getDatabase(table_id.database_name)->getCreateTableQuery(table_id.table_name, query_context);
|
||||
applyMetadataChangesToCreateQuery(ast, future_metadata, query_context);
|
||||
}
|
||||
|
||||
auto ast_to_str = [](ASTPtr query) -> String
|
||||
{
|
||||
if (!query)
|
||||
@ -6239,9 +6250,6 @@ void StorageReplicatedMergeTree::alter(
|
||||
|
||||
auto current_metadata = getInMemoryMetadataPtr();
|
||||
|
||||
StorageInMemoryMetadata future_metadata = *current_metadata;
|
||||
commands.apply(future_metadata, query_context);
|
||||
|
||||
ReplicatedMergeTreeTableMetadata future_metadata_in_zk(*this, current_metadata);
|
||||
if (ast_to_str(future_metadata.sorting_key.definition_ast) != ast_to_str(current_metadata->sorting_key.definition_ast))
|
||||
{
|
||||
@ -6309,6 +6317,8 @@ void StorageReplicatedMergeTree::alter(
|
||||
setInMemoryMetadata(metadata_copy);
|
||||
}
|
||||
|
||||
/// Only the comment and/or settings changed here, so it is okay to assume alterTable won't throw as neither
|
||||
/// of them are validated in alterTable.
|
||||
DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(query_context, table_id, metadata_copy);
|
||||
}
|
||||
|
||||
@ -6378,7 +6388,7 @@ void StorageReplicatedMergeTree::alter(
|
||||
/// so we have to update metadata of DatabaseReplicated here.
|
||||
String metadata_zk_path = fs::path(txn->getDatabaseZooKeeperPath()) / "metadata" / escapeForFileName(table_id.table_name);
|
||||
auto ast = DatabaseCatalog::instance().getDatabase(table_id.database_name)->getCreateTableQuery(table_id.table_name, query_context);
|
||||
applyMetadataChangesToCreateQuery(ast, future_metadata);
|
||||
applyMetadataChangesToCreateQuery(ast, future_metadata, query_context);
|
||||
ops.emplace_back(zkutil::makeSetRequest(metadata_zk_path, getObjectDefinitionFromCreateQuery(ast), -1));
|
||||
}
|
||||
|
||||
@ -10592,7 +10602,7 @@ void StorageReplicatedMergeTree::adjustCreateQueryForBackup(ASTPtr & create_quer
|
||||
auto current_metadata = getInMemoryMetadataPtr();
|
||||
auto metadata_diff = ReplicatedMergeTreeTableMetadata(*this, current_metadata).checkAndFindDiff(metadata_from_entry, current_metadata->getColumns(), getContext());
|
||||
auto adjusted_metadata = metadata_diff.getNewMetadata(columns_from_entry, getContext(), *current_metadata);
|
||||
applyMetadataChangesToCreateQuery(create_query, adjusted_metadata);
|
||||
applyMetadataChangesToCreateQuery(create_query, adjusted_metadata, getContext());
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
|
@ -40,7 +40,7 @@ SELECT val FROM tab WHERE nowInBlock(tz) != toDateTime('2000-01-01 00:00:00') OR
|
||||
SELECT val FROM tab WHERE nowInBlock(tz) != toDateTime('2000-01-01 00:00:00') ORDER BY val SETTINGS allow_nonconst_timezone_arguments = 1;
|
||||
|
||||
SELECT val FROM tab WHERE toTimeZone(toDateTime(val), tz) != toDateTime('2023-06-11 14:14:14') ORDER BY val SETTINGS allow_nonconst_timezone_arguments = 0; -- { serverError ILLEGAL_COLUMN }
|
||||
SELECT val FROM tab WHERE toTimeZone(toDateTime(val), tz) != toDateTime('2023-06-11 14:14:14') ORDER BY val SETTINGS allow_nonconst_timezone_arguments = 1;
|
||||
SELECT val FROM tab WHERE toTimeZone(toDateTime(val), tz) != toDateTime('2023-06-11 14:14:14') ORDER BY val SETTINGS allow_nonconst_timezone_arguments = 1;
|
||||
|
||||
SELECT val FROM tab WHERE fromUnixTimestamp64Milli(val, tz) != toDateTime64('2023-06-11 14:14:14', 6) ORDER BY val SETTINGS allow_nonconst_timezone_arguments = 0; -- { serverError ILLEGAL_COLUMN }
|
||||
SELECT val FROM tab WHERE fromUnixTimestamp64Milli(val, tz) != toDateTime64('2023-06-11 14:14:14', 6) ORDER BY val SETTINGS allow_nonconst_timezone_arguments = 1;
|
||||
@ -72,5 +72,5 @@ SET allow_nonconst_timezone_arguments = 0;
|
||||
ATTACH TABLE tab;
|
||||
DROP TABLE tab;
|
||||
|
||||
-- CREATE TABLE does check the default expessions, so the following is expected to fail:
|
||||
-- CREATE TABLE does check the default expressions, so the following is expected to fail:
|
||||
CREATE TABLE tab (`country` LowCardinality(FixedString(7)) DEFAULT 'unknown', `city` LowCardinality(String) DEFAULT 'unknown', `region` LowCardinality(String) DEFAULT 'unknown', `continent` LowCardinality(FixedString(7)) DEFAULT 'unknown', `is_eu_country` Bool, `date` DateTime CODEC(DoubleDelta, LZ4), `viewer_date` DateTime ALIAS toTimezone(date, timezone), `device_browser` LowCardinality(String) DEFAULT 'unknown', `metro_code` LowCardinality(String) DEFAULT 'unknown', `domain` String DEFAULT 'unknown', `device_platform` LowCardinality(String) DEFAULT 'unknown', `device_type` LowCardinality(String) DEFAULT 'unknown', `device_vendor` LowCardinality(String) DEFAULT 'unknown', `ip` FixedString(39) DEFAULT 'unknown', `lat` Decimal(8, 6) CODEC(T64), `lng` Decimal(9, 6) CODEC(T64), `asset_id` String DEFAULT 'unknown', `is_personalized` Bool, `metric` String, `origin` String DEFAULT 'unknown', `product_id` UInt64 CODEC(T64), `referer` String DEFAULT 'unknown', `server_side` Int8 CODEC(T64), `third_party_id` String DEFAULT 'unknown', `partner_slug` LowCardinality(FixedString(10)) DEFAULT 'unknown', `user_agent` String DEFAULT 'unknown', `user_id` UUID, `zip` FixedString(10) DEFAULT 'unknown', `timezone` LowCardinality(String), `as_organization` LowCardinality(String) DEFAULT 'unknown', `content_cat` Array(String), `playback_method` LowCardinality(String) DEFAULT 'unknown', `store_id` LowCardinality(String) DEFAULT 'unknown', `store_url` String DEFAULT 'unknown', `timestamp` Nullable(DateTime), `ad_count` Int8 CODEC(T64), `ad_type` LowCardinality(FixedString(10)) DEFAULT 'unknown', `ad_categories` Array(FixedString(8)), `blocked_ad_categories` Array(FixedString(8)), `break_max_ad_length` Int8 CODEC(T64), `break_max_ads` Int8 CODEC(T64), `break_max_duration` Int8 CODEC(T64), `break_min_ad_length` Int8 CODEC(T64), `break_position` LowCardinality(FixedString(18)) DEFAULT 'unknown', `media_playhead` String DEFAULT 'unknown', `placement_type` Int8 CODEC(T64), `transaction_id` String, `universal_ad_id` Array(String), `client_ua` LowCardinality(String) DEFAULT 'unknown', `device_ip` FixedString(39) DEFAULT 'unknown', `device_ua` LowCardinality(String) DEFAULT 'unknown', `ifa` String, `ifa_type` LowCardinality(String) DEFAULT 'unknown', `vast_lat` Decimal(8, 6) CODEC(T64), `vast_long` Decimal(9, 6) CODEC(T64), `server_ua` String DEFAULT 'unknown', `app_bundle` String DEFAULT 'unknown', `page_url` String DEFAULT 'unknown', `api_framework` Array(UInt8), `click_type` LowCardinality(String), `extensions` Array(String), `media_mime` Array(String), `om_id_partner` LowCardinality(String) DEFAULT 'unknown', `player_capabilities` Array(FixedString(12)), `vast_versions` Array(UInt8), `verification_vendors` Array(String), `ad_play_head` String DEFAULT 'unknown', `ad_serving_id` String DEFAULT 'unknown', `asset_uri` String DEFAULT 'unknown', `content_id` String DEFAULT 'unknown', `content_uri` String DEFAULT 'unknown', `inventory_state` Array(FixedString(14)), `player_size` Array(UInt8), `player_state` Array(FixedString(12)), `pod_sequence` Int8 CODEC(T64), `click_position` Array(UInt32), `error_code` Int16 CODEC(T64), `error_reason` Int8 CODEC(T64), `gdpr_consent` String DEFAULT 'unknown', `limited_tracking` Bool, `regulations` String DEFAULT 'unknown', `content_category` Array(String), PROJECTION projection_TPAG_VAST_date (SELECT * ORDER BY toYYYYMMDD(date), metric, product_id, asset_id)) ENGINE = MergeTree ORDER BY (product_id, metric, asset_id, toYYYYMMDD(date)); -- { serverError ILLEGAL_COLUMN }
|
||||
|
8
tests/queries/0_stateless/03224_invalid_alter.reference
Normal file
8
tests/queries/0_stateless/03224_invalid_alter.reference
Normal file
@ -0,0 +1,8 @@
|
||||
test testa testa testc
|
||||
test2 test2a test2a test2c
|
||||
localhost 9000 0 0 0
|
||||
localhost 9000 0 0 0
|
||||
localhost 9000 0 0 0
|
||||
localhost 9000 0 0 0
|
||||
test3 test3a test3a test3c
|
||||
test4 test4a test4a test4c
|
84
tests/queries/0_stateless/03224_invalid_alter.sql
Normal file
84
tests/queries/0_stateless/03224_invalid_alter.sql
Normal file
@ -0,0 +1,84 @@
|
||||
-- Tags: no-replicated-database
|
||||
-- no-replicated-database: It messes up the output and this test explicitly checks the replicated database
|
||||
|
||||
CREATE TABLE test
|
||||
(
|
||||
str String,
|
||||
column_with_codec String CODEC(ZSTD),
|
||||
column_with_alias String MATERIALIZED concat(str, 'a' AS a),
|
||||
)
|
||||
ENGINE = MergeTree()
|
||||
ORDER BY tuple();
|
||||
|
||||
-- Cannot have a different expression with the same alias
|
||||
ALTER TABLE test ADD COLUMN invalid_column String MATERIALIZED concat(str, 'b' AS a); -- { serverError MULTIPLE_EXPRESSIONS_FOR_ALIAS }
|
||||
ALTER TABLE test ADD COLUMN invalid_column String DEFAULT concat(str, 'b' AS a); -- { serverError MULTIPLE_EXPRESSIONS_FOR_ALIAS }
|
||||
-- Cannot specify codec for column type ALIAS
|
||||
ALTER TABLE test MODIFY COLUMN column_with_codec String ALIAS str; -- { serverError BAD_ARGUMENTS }
|
||||
-- alias is defined exactly the same
|
||||
ALTER TABLE test ADD COLUMN valid_column_1 String DEFAULT concat(str, 'a' AS a);
|
||||
-- different alias
|
||||
ALTER TABLE test ADD COLUMN valid_column_2 String MATERIALIZED concat(str, 'c' AS c);
|
||||
-- do one insert to make sure we can insert into the table
|
||||
INSERT INTO test(str, column_with_codec) VALUES ('test', 'test2');
|
||||
SELECT str, column_with_alias, valid_column_1, valid_column_2 FROM test;
|
||||
DROP TABLE test;
|
||||
|
||||
CREATE TABLE test2
|
||||
(
|
||||
str String,
|
||||
column_with_codec String CODEC(ZSTD),
|
||||
column_with_alias String MATERIALIZED concat(str, 'a' AS a),
|
||||
)
|
||||
ENGINE = ReplicatedMergeTree('/clickhouse/03224_invalid_alter/{database}/{table}', 'r1')
|
||||
ORDER BY tuple();
|
||||
|
||||
ALTER TABLE test2 ADD COLUMN invalid_column String MATERIALIZED concat(str, 'b' AS a); -- { serverError MULTIPLE_EXPRESSIONS_FOR_ALIAS }
|
||||
ALTER TABLE test2 ADD COLUMN invalid_column String DEFAULT concat(str, 'b' AS a); -- { serverError MULTIPLE_EXPRESSIONS_FOR_ALIAS }
|
||||
ALTER TABLE test2 MODIFY COLUMN column_with_codec String ALIAS str; -- { serverError BAD_ARGUMENTS }
|
||||
ALTER TABLE test2 ADD COLUMN valid_column_1 String DEFAULT concat(str, 'a' AS a);
|
||||
ALTER TABLE test2 ADD COLUMN valid_column_2 String MATERIALIZED concat(str, 'c' AS c);
|
||||
INSERT INTO test2(str, column_with_codec) VALUES ('test2', 'test22');
|
||||
SELECT str, column_with_alias, valid_column_1, valid_column_2 FROM test2;
|
||||
|
||||
DROP DATABASE {CLICKHOUSE_DATABASE:Identifier};
|
||||
|
||||
CREATE DATABASE {CLICKHOUSE_DATABASE:Identifier} ON CLUSTER test_shard_localhost ENGINE = Atomic;
|
||||
|
||||
CREATE TABLE test3 ON CLUSTER test_shard_localhost
|
||||
(
|
||||
str String,
|
||||
column_with_codec String CODEC(ZSTD),
|
||||
column_with_alias String MATERIALIZED concat(str, 'a' AS a),
|
||||
)
|
||||
ENGINE = ReplicatedMergeTree('/clickhouse/03224_invalid_alter/{database}_atomic/{table}', 'r1')
|
||||
ORDER BY tuple();
|
||||
|
||||
ALTER TABLE test3 ON CLUSTER test_shard_localhost ADD COLUMN invalid_column String MATERIALIZED concat(str, 'b' AS a) FORMAT Null SETTINGS distributed_ddl_output_mode='throw'; -- { serverError MULTIPLE_EXPRESSIONS_FOR_ALIAS }
|
||||
ALTER TABLE test3 ON CLUSTER test_shard_localhost ADD COLUMN invalid_column String DEFAULT concat(str, 'b' AS a) FORMAT Null SETTINGS distributed_ddl_output_mode='throw'; -- { serverError MULTIPLE_EXPRESSIONS_FOR_ALIAS }
|
||||
ALTER TABLE test3 ON CLUSTER test_shard_localhost MODIFY COLUMN column_with_codec String ALIAS str FORMAT Null SETTINGS distributed_ddl_output_mode='throw'; -- { serverError BAD_ARGUMENTS }
|
||||
ALTER TABLE test3 ON CLUSTER test_shard_localhost ADD COLUMN valid_column_1 String DEFAULT concat(str, 'a' AS a);
|
||||
ALTER TABLE test3 ON CLUSTER test_shard_localhost ADD COLUMN valid_column_2 String MATERIALIZED concat(str, 'c' AS c);
|
||||
INSERT INTO test3(str, column_with_codec) VALUES ('test3', 'test32');
|
||||
SELECT str, column_with_alias, valid_column_1, valid_column_2 FROM test3;
|
||||
|
||||
DROP DATABASE {CLICKHOUSE_DATABASE:Identifier};
|
||||
CREATE DATABASE {CLICKHOUSE_DATABASE:Identifier} ENGINE = Replicated('/clickhouse/03224_invalid_alter/{database}_replicated', 'shard1', 'replica1') FORMAT Null;
|
||||
|
||||
CREATE TABLE test4
|
||||
(
|
||||
str String,
|
||||
column_with_codec String CODEC(ZSTD),
|
||||
column_with_alias String MATERIALIZED concat(str, 'a' AS a),
|
||||
)
|
||||
ENGINE = ReplicatedMergeTree()
|
||||
ORDER BY tuple()
|
||||
FORMAT Null;
|
||||
|
||||
ALTER TABLE test4 ADD COLUMN invalid_column String MATERIALIZED concat(str, 'b' AS a) FORMAT Null SETTINGS distributed_ddl_output_mode='throw'; -- { serverError MULTIPLE_EXPRESSIONS_FOR_ALIAS }
|
||||
ALTER TABLE test4 ADD COLUMN invalid_column String DEFAULT concat(str, 'b' AS a) FORMAT Null SETTINGS distributed_ddl_output_mode='throw'; -- { serverError MULTIPLE_EXPRESSIONS_FOR_ALIAS }
|
||||
ALTER TABLE test4 MODIFY COLUMN column_with_codec String ALIAS str FORMAT Null SETTINGS distributed_ddl_output_mode='throw'; -- { serverError BAD_ARGUMENTS }
|
||||
ALTER TABLE test4 ADD COLUMN valid_column_1 String DEFAULT concat(str, 'a' AS a) FORMAT Null SETTINGS distributed_ddl_output_mode='throw';
|
||||
ALTER TABLE test4 ADD COLUMN valid_column_2 String MATERIALIZED concat(str, 'c' AS c) FORMAT Null SETTINGS distributed_ddl_output_mode='throw';
|
||||
INSERT INTO test4(str, column_with_codec) VALUES ('test4', 'test42');
|
||||
SELECT str, column_with_alias, valid_column_1, valid_column_2 FROM test4;
|
Loading…
Reference in New Issue
Block a user