mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Merge pull request #21303 from ucasFL/forbid
Forbid to drop a column if it's referenced by materialized view
This commit is contained in:
commit
a4c00ab5dc
@ -137,7 +137,7 @@ BlockIO InterpreterAlterQuery::execute()
|
||||
StorageInMemoryMetadata metadata = table->getInMemoryMetadata();
|
||||
alter_commands.validate(metadata, context);
|
||||
alter_commands.prepare(metadata);
|
||||
table->checkAlterIsPossible(alter_commands, context.getSettingsRef());
|
||||
table->checkAlterIsPossible(alter_commands, context);
|
||||
table->alter(alter_commands, context, alter_lock);
|
||||
}
|
||||
|
||||
|
@ -89,6 +89,8 @@ public:
|
||||
|
||||
static void addEmptySourceToQueryPlan(QueryPlan & query_plan, const Block & source_header, const SelectQueryInfo & query_info);
|
||||
|
||||
Names getRequiredColumns() { return required_columns; }
|
||||
|
||||
private:
|
||||
InterpreterSelectQuery(
|
||||
const ASTPtr & query_ptr_,
|
||||
|
@ -134,7 +134,7 @@ void IStorage::alter(const AlterCommands & params, const Context & context, Tabl
|
||||
}
|
||||
|
||||
|
||||
void IStorage::checkAlterIsPossible(const AlterCommands & commands, const Settings & /* settings */) const
|
||||
void IStorage::checkAlterIsPossible(const AlterCommands & commands, const Context & /* context */) const
|
||||
{
|
||||
for (const auto & command : commands)
|
||||
{
|
||||
@ -182,6 +182,24 @@ Names IStorage::getAllRegisteredNames() const
|
||||
return result;
|
||||
}
|
||||
|
||||
NameDependencies IStorage::getDependentViewsByColumn(const Context & context) const
|
||||
{
|
||||
NameDependencies name_deps;
|
||||
auto dependencies = DatabaseCatalog::instance().getDependencies(storage_id);
|
||||
for (const auto & depend_id : dependencies)
|
||||
{
|
||||
auto depend_table = DatabaseCatalog::instance().getTable(depend_id, context);
|
||||
if (depend_table->getInMemoryMetadataPtr()->select.inner_query)
|
||||
{
|
||||
const auto & select_query = depend_table->getInMemoryMetadataPtr()->select.inner_query;
|
||||
auto required_columns = InterpreterSelectQuery(select_query, context, SelectQueryOptions{}.noModify()).getRequiredColumns();
|
||||
for (const auto & col_name : required_columns)
|
||||
name_deps[col_name].push_back(depend_id.table_name);
|
||||
}
|
||||
}
|
||||
return name_deps;
|
||||
}
|
||||
|
||||
std::string PrewhereDAGInfo::dump() const
|
||||
{
|
||||
WriteBufferFromOwnString ss;
|
||||
|
@ -57,6 +57,8 @@ struct StreamLocalLimits;
|
||||
class EnabledQuota;
|
||||
struct SelectQueryInfo;
|
||||
|
||||
using NameDependencies = std::unordered_map<String, std::vector<String>>;
|
||||
|
||||
struct ColumnSize
|
||||
{
|
||||
size_t marks = 0;
|
||||
@ -173,8 +175,10 @@ public:
|
||||
virtual NamesAndTypesList getVirtuals() const;
|
||||
|
||||
Names getAllRegisteredNames() const override;
|
||||
protected:
|
||||
|
||||
NameDependencies getDependentViewsByColumn(const Context & context) const;
|
||||
|
||||
protected:
|
||||
/// Returns whether the column is virtual - by default all columns are real.
|
||||
/// Initially reserved virtual column name may be shadowed by real column.
|
||||
bool isVirtualColumn(const String & column_name, const StorageMetadataPtr & metadata_snapshot) const;
|
||||
@ -362,7 +366,7 @@ public:
|
||||
/** Checks that alter commands can be applied to storage. For example, columns can be modified,
|
||||
* or primary key can be changes, etc.
|
||||
*/
|
||||
virtual void checkAlterIsPossible(const AlterCommands & commands, const Settings & settings) const;
|
||||
virtual void checkAlterIsPossible(const AlterCommands & commands, const Context & context) const;
|
||||
|
||||
/**
|
||||
* Checks that mutation commands can be applied to storage.
|
||||
|
@ -4,22 +4,22 @@
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <DataTypes/DataTypeEnum.h>
|
||||
#include <DataTypes/DataTypeUUID.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <DataTypes/DataTypeLowCardinality.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <DataTypes/DataTypeUUID.h>
|
||||
#include <DataTypes/NestedUtils.h>
|
||||
#include <Formats/FormatFactory.h>
|
||||
#include <Processors/Formats/InputStreamFromInputFormat.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <IO/ConcatReadBuffer.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <IO/ReadBufferFromMemory.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/ExpressionAnalyzer.h>
|
||||
#include <Interpreters/InterpreterSelectQuery.h>
|
||||
#include <Interpreters/PartLog.h>
|
||||
#include <Interpreters/TreeRewriter.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/inplaceBlockConversions.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
@ -29,10 +29,11 @@
|
||||
#include <Parsers/ExpressionListParsers.h>
|
||||
#include <Parsers/parseQuery.h>
|
||||
#include <Parsers/queryToString.h>
|
||||
#include <Processors/Formats/InputStreamFromInputFormat.h>
|
||||
#include <Storages/AlterCommands.h>
|
||||
#include <Storages/MergeTree/MergeTreeData.h>
|
||||
#include <Storages/MergeTree/MergeTreeDataPartInMemory.h>
|
||||
#include <Storages/MergeTree/MergeTreeDataPartCompact.h>
|
||||
#include <Storages/MergeTree/MergeTreeDataPartInMemory.h>
|
||||
#include <Storages/MergeTree/MergeTreeDataPartWide.h>
|
||||
#include <Storages/MergeTree/MergeTreeSequentialSource.h>
|
||||
#include <Storages/MergeTree/MergedBlockOutputStream.h>
|
||||
@ -1406,12 +1407,14 @@ void checkVersionColumnTypesConversion(const IDataType * old_type, const IDataTy
|
||||
|
||||
}
|
||||
|
||||
void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, const Settings & settings) const
|
||||
void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, const Context & context) const
|
||||
{
|
||||
/// Check that needed transformations can be applied to the list of columns without considering type conversions.
|
||||
StorageInMemoryMetadata new_metadata = getInMemoryMetadata();
|
||||
StorageInMemoryMetadata old_metadata = getInMemoryMetadata();
|
||||
|
||||
const auto & settings = context.getSettingsRef();
|
||||
|
||||
if (!settings.allow_non_metadata_alters)
|
||||
{
|
||||
|
||||
@ -1483,6 +1486,7 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, const S
|
||||
old_types.emplace(column.name, column.type.get());
|
||||
|
||||
NamesAndTypesList columns_to_check_conversion;
|
||||
auto name_deps = getDependentViewsByColumn(context);
|
||||
for (const AlterCommand & command : commands)
|
||||
{
|
||||
/// Just validate partition expression
|
||||
@ -1562,6 +1566,16 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, const S
|
||||
"Trying to ALTER DROP key " + backQuoteIfNeed(command.column_name) + " column which is a part of key expression",
|
||||
ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN);
|
||||
}
|
||||
|
||||
const auto & deps_mv = name_deps[command.column_name];
|
||||
if (!deps_mv.empty())
|
||||
{
|
||||
throw Exception(
|
||||
"Trying to ALTER DROP column " + backQuoteIfNeed(command.column_name) + " which is referenced by materialized view "
|
||||
+ toString(deps_mv),
|
||||
ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN);
|
||||
}
|
||||
|
||||
dropped_columns.emplace(command.column_name);
|
||||
}
|
||||
else if (command.isRequireMutationStage(getInMemoryMetadata()))
|
||||
|
@ -517,7 +517,7 @@ public:
|
||||
/// - all type conversions can be done.
|
||||
/// - columns corresponding to primary key, indices, sign, sampling expression and date are not affected.
|
||||
/// If something is wrong, throws an exception.
|
||||
void checkAlterIsPossible(const AlterCommands & commands, const Settings & settings) const override;
|
||||
void checkAlterIsPossible(const AlterCommands & commands, const Context & context) const override;
|
||||
|
||||
/// Checks if the Mutation can be performed.
|
||||
/// (currently no additional checks: always ok)
|
||||
|
@ -58,6 +58,7 @@ namespace ErrorCodes
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int INFINITE_LOOP;
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
extern const int ALTER_OF_COLUMN_IS_FORBIDDEN;
|
||||
}
|
||||
|
||||
|
||||
@ -926,8 +927,9 @@ void StorageBuffer::reschedule()
|
||||
flush_handle->scheduleAfter(std::min(min, max) * 1000);
|
||||
}
|
||||
|
||||
void StorageBuffer::checkAlterIsPossible(const AlterCommands & commands, const Settings & /* settings */) const
|
||||
void StorageBuffer::checkAlterIsPossible(const AlterCommands & commands, const Context & context) const
|
||||
{
|
||||
auto name_deps = getDependentViewsByColumn(context);
|
||||
for (const auto & command : commands)
|
||||
{
|
||||
if (command.type != AlterCommand::Type::ADD_COLUMN && command.type != AlterCommand::Type::MODIFY_COLUMN
|
||||
@ -935,6 +937,17 @@ void StorageBuffer::checkAlterIsPossible(const AlterCommands & commands, const S
|
||||
throw Exception(
|
||||
"Alter of type '" + alterTypeToString(command.type) + "' is not supported by storage " + getName(),
|
||||
ErrorCodes::NOT_IMPLEMENTED);
|
||||
if (command.type == AlterCommand::Type::DROP_COLUMN)
|
||||
{
|
||||
const auto & deps_mv = name_deps[command.column_name];
|
||||
if (!deps_mv.empty())
|
||||
{
|
||||
throw Exception(
|
||||
"Trying to ALTER DROP column " + backQuoteIfNeed(command.column_name) + " which is referenced by materialized view "
|
||||
+ toString(deps_mv),
|
||||
ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@ -971,7 +984,7 @@ std::optional<UInt64> StorageBuffer::totalBytes(const Settings & /*settings*/) c
|
||||
void StorageBuffer::alter(const AlterCommands & params, const Context & context, TableLockHolder &)
|
||||
{
|
||||
auto table_id = getStorageID();
|
||||
checkAlterIsPossible(params, context.getSettingsRef());
|
||||
checkAlterIsPossible(params, context);
|
||||
auto metadata_snapshot = getInMemoryMetadataPtr();
|
||||
|
||||
/// Flush all buffers to storages, so that no non-empty blocks of the old
|
||||
|
@ -99,7 +99,7 @@ public:
|
||||
|
||||
bool mayBenefitFromIndexForIn(const ASTPtr & left_in_operand, const Context & query_context, const StorageMetadataPtr & metadata_snapshot) const override;
|
||||
|
||||
void checkAlterIsPossible(const AlterCommands & commands, const Settings & /* settings */) const override;
|
||||
void checkAlterIsPossible(const AlterCommands & commands, const Context & context) const override;
|
||||
|
||||
/// The structure of the subordinate table is not checked and does not change.
|
||||
void alter(const AlterCommands & params, const Context & context, TableLockHolder & table_lock_holder) override;
|
||||
|
@ -84,6 +84,7 @@ namespace ErrorCodes
|
||||
extern const int TOO_MANY_ROWS;
|
||||
extern const int UNABLE_TO_SKIP_UNUSED_SHARDS;
|
||||
extern const int INVALID_SHARD_ID;
|
||||
extern const int ALTER_OF_COLUMN_IS_FORBIDDEN;
|
||||
}
|
||||
|
||||
namespace ActionLocks
|
||||
@ -577,8 +578,9 @@ BlockOutputStreamPtr StorageDistributed::write(const ASTPtr &, const StorageMeta
|
||||
}
|
||||
|
||||
|
||||
void StorageDistributed::checkAlterIsPossible(const AlterCommands & commands, const Settings & /* settings */) const
|
||||
void StorageDistributed::checkAlterIsPossible(const AlterCommands & commands, const Context & context) const
|
||||
{
|
||||
auto name_deps = getDependentViewsByColumn(context);
|
||||
for (const auto & command : commands)
|
||||
{
|
||||
if (command.type != AlterCommand::Type::ADD_COLUMN
|
||||
@ -589,6 +591,17 @@ void StorageDistributed::checkAlterIsPossible(const AlterCommands & commands, co
|
||||
|
||||
throw Exception("Alter of type '" + alterTypeToString(command.type) + "' is not supported by storage " + getName(),
|
||||
ErrorCodes::NOT_IMPLEMENTED);
|
||||
if (command.type == AlterCommand::DROP_COLUMN)
|
||||
{
|
||||
const auto & deps_mv = name_deps[command.column_name];
|
||||
if (!deps_mv.empty())
|
||||
{
|
||||
throw Exception(
|
||||
"Trying to ALTER DROP column " + backQuoteIfNeed(command.column_name) + " which is referenced by materialized view "
|
||||
+ toString(deps_mv),
|
||||
ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@ -596,7 +609,7 @@ void StorageDistributed::alter(const AlterCommands & params, const Context & con
|
||||
{
|
||||
auto table_id = getStorageID();
|
||||
|
||||
checkAlterIsPossible(params, context.getSettingsRef());
|
||||
checkAlterIsPossible(params, context);
|
||||
StorageInMemoryMetadata new_metadata = getInMemoryMetadata();
|
||||
params.apply(new_metadata, context);
|
||||
DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(context, table_id, new_metadata);
|
||||
|
@ -85,7 +85,7 @@ public:
|
||||
void rename(const String & new_path_to_table_data, const StorageID & new_table_id) override;
|
||||
void renameOnDisk(const String & new_path_to_table_data);
|
||||
|
||||
void checkAlterIsPossible(const AlterCommands & commands, const Settings & /* settings */) const override;
|
||||
void checkAlterIsPossible(const AlterCommands & commands, const Context & context) const override;
|
||||
|
||||
/// in the sub-tables, you need to manually add and delete columns
|
||||
/// the structure of the sub-table is not checked
|
||||
|
@ -296,8 +296,9 @@ void StorageMaterializedView::alter(
|
||||
}
|
||||
|
||||
|
||||
void StorageMaterializedView::checkAlterIsPossible(const AlterCommands & commands, const Settings & settings) const
|
||||
void StorageMaterializedView::checkAlterIsPossible(const AlterCommands & commands, const Context & context) const
|
||||
{
|
||||
const auto & settings = context.getSettingsRef();
|
||||
if (settings.allow_experimental_alter_materialized_view_structure)
|
||||
{
|
||||
for (const auto & command : commands)
|
||||
|
@ -54,7 +54,7 @@ public:
|
||||
|
||||
void checkMutationIsPossible(const MutationCommands & commands, const Settings & settings) const override;
|
||||
|
||||
void checkAlterIsPossible(const AlterCommands & commands, const Settings & settings) const override;
|
||||
void checkAlterIsPossible(const AlterCommands & commands, const Context & context) const override;
|
||||
|
||||
Pipe alterPartition(const StorageMetadataPtr & metadata_snapshot, const PartitionCommands & commands, const Context & context) override;
|
||||
|
||||
|
@ -37,6 +37,7 @@ namespace ErrorCodes
|
||||
extern const int ILLEGAL_PREWHERE;
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
extern const int SAMPLING_NOT_SUPPORTED;
|
||||
extern const int ALTER_OF_COLUMN_IS_FORBIDDEN;
|
||||
}
|
||||
|
||||
namespace
|
||||
@ -472,8 +473,9 @@ DatabaseTablesIteratorPtr StorageMerge::getDatabaseIterator(const Context & cont
|
||||
}
|
||||
|
||||
|
||||
void StorageMerge::checkAlterIsPossible(const AlterCommands & commands, const Settings & /* settings */) const
|
||||
void StorageMerge::checkAlterIsPossible(const AlterCommands & commands, const Context & context) const
|
||||
{
|
||||
auto name_deps = getDependentViewsByColumn(context);
|
||||
for (const auto & command : commands)
|
||||
{
|
||||
if (command.type != AlterCommand::Type::ADD_COLUMN && command.type != AlterCommand::Type::MODIFY_COLUMN
|
||||
@ -481,6 +483,17 @@ void StorageMerge::checkAlterIsPossible(const AlterCommands & commands, const Se
|
||||
throw Exception(
|
||||
"Alter of type '" + alterTypeToString(command.type) + "' is not supported by storage " + getName(),
|
||||
ErrorCodes::NOT_IMPLEMENTED);
|
||||
if (command.type == AlterCommand::Type::DROP_COLUMN)
|
||||
{
|
||||
const auto & deps_mv = name_deps[command.column_name];
|
||||
if (!deps_mv.empty())
|
||||
{
|
||||
throw Exception(
|
||||
"Trying to ALTER DROP column " + backQuoteIfNeed(command.column_name) + " which is referenced by materialized view "
|
||||
+ toString(deps_mv),
|
||||
ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -38,7 +38,7 @@ public:
|
||||
size_t max_block_size,
|
||||
unsigned num_streams) override;
|
||||
|
||||
void checkAlterIsPossible(const AlterCommands & commands, const Settings & /* settings */) const override;
|
||||
void checkAlterIsPossible(const AlterCommands & commands, const Context & context) const override;
|
||||
|
||||
/// you need to add and remove columns in the sub-tables manually
|
||||
/// the structure of sub-tables is not checked
|
||||
|
@ -16,6 +16,7 @@ namespace ErrorCodes
|
||||
{
|
||||
extern const int NOT_IMPLEMENTED;
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
extern const int ALTER_OF_COLUMN_IS_FORBIDDEN;
|
||||
}
|
||||
|
||||
|
||||
@ -35,8 +36,9 @@ void registerStorageNull(StorageFactory & factory)
|
||||
});
|
||||
}
|
||||
|
||||
void StorageNull::checkAlterIsPossible(const AlterCommands & commands, const Settings & /* settings */) const
|
||||
void StorageNull::checkAlterIsPossible(const AlterCommands & commands, const Context & context) const
|
||||
{
|
||||
auto name_deps = getDependentViewsByColumn(context);
|
||||
for (const auto & command : commands)
|
||||
{
|
||||
if (command.type != AlterCommand::Type::ADD_COLUMN && command.type != AlterCommand::Type::MODIFY_COLUMN
|
||||
@ -44,6 +46,17 @@ void StorageNull::checkAlterIsPossible(const AlterCommands & commands, const Set
|
||||
throw Exception(
|
||||
"Alter of type '" + alterTypeToString(command.type) + "' is not supported by storage " + getName(),
|
||||
ErrorCodes::NOT_IMPLEMENTED);
|
||||
if (command.type == AlterCommand::DROP_COLUMN)
|
||||
{
|
||||
const auto & deps_mv = name_deps[command.column_name];
|
||||
if (!deps_mv.empty())
|
||||
{
|
||||
throw Exception(
|
||||
"Trying to ALTER DROP column " + backQuoteIfNeed(command.column_name) + " which is referenced by materialized view "
|
||||
+ toString(deps_mv),
|
||||
ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -41,7 +41,7 @@ public:
|
||||
return std::make_shared<NullBlockOutputStream>(metadata_snapshot->getSampleBlock());
|
||||
}
|
||||
|
||||
void checkAlterIsPossible(const AlterCommands & commands, const Settings & /* settings */) const override;
|
||||
void checkAlterIsPossible(const AlterCommands & commands, const Context & context) const override;
|
||||
|
||||
void alter(const AlterCommands & params, const Context & context, TableLockHolder & table_lock_holder) override;
|
||||
|
||||
|
@ -97,9 +97,9 @@ public:
|
||||
IStorage::setInMemoryMetadata(getNested()->getInMemoryMetadata());
|
||||
}
|
||||
|
||||
void checkAlterIsPossible(const AlterCommands & commands, const Settings & settings) const override
|
||||
void checkAlterIsPossible(const AlterCommands & commands, const Context & context) const override
|
||||
{
|
||||
getNested()->checkAlterIsPossible(commands, settings);
|
||||
getNested()->checkAlterIsPossible(commands, context);
|
||||
}
|
||||
|
||||
Pipe alterPartition(
|
||||
|
@ -0,0 +1,172 @@
|
||||
-- MergeTree
|
||||
DROP TABLE IF EXISTS `01746_merge_tree`;
|
||||
CREATE TABLE `01746_merge_tree`
|
||||
(
|
||||
`n1` Int8,
|
||||
`n2` Int8,
|
||||
`n3` Int8,
|
||||
`n4` Int8
|
||||
)
|
||||
ENGINE = MergeTree
|
||||
ORDER BY n1;
|
||||
|
||||
DROP TABLE IF EXISTS `01746_merge_tree_mv`;
|
||||
CREATE MATERIALIZED VIEW `01746_merge_tree_mv`
|
||||
ENGINE = Memory AS
|
||||
SELECT
|
||||
n2,
|
||||
n3
|
||||
FROM `01746_merge_tree`;
|
||||
|
||||
ALTER TABLE `01746_merge_tree`
|
||||
DROP COLUMN n3; -- { serverError 524 }
|
||||
|
||||
ALTER TABLE `01746_merge_tree`
|
||||
DROP COLUMN n2; -- { serverError 524 }
|
||||
|
||||
-- ok
|
||||
ALTER TABLE `01746_merge_tree`
|
||||
DROP COLUMN n4;
|
||||
|
||||
DROP TABLE `01746_merge_tree`;
|
||||
DROP TABLE `01746_merge_tree_mv`;
|
||||
|
||||
-- Null
|
||||
DROP TABLE IF EXISTS `01746_null`;
|
||||
CREATE TABLE `01746_null`
|
||||
(
|
||||
`n1` Int8,
|
||||
`n2` Int8,
|
||||
`n3` Int8
|
||||
)
|
||||
ENGINE = Null;
|
||||
|
||||
DROP TABLE IF EXISTS `01746_null_mv`;
|
||||
CREATE MATERIALIZED VIEW `01746_null_mv`
|
||||
ENGINE = Memory AS
|
||||
SELECT
|
||||
n1,
|
||||
n2
|
||||
FROM `01746_null`;
|
||||
|
||||
ALTER TABLE `01746_null`
|
||||
DROP COLUMN n1; -- { serverError 524 }
|
||||
|
||||
ALTER TABLE `01746_null`
|
||||
DROP COLUMN n2; -- { serverError 524 }
|
||||
|
||||
-- ok
|
||||
ALTER TABLE `01746_null`
|
||||
DROP COLUMN n3;
|
||||
|
||||
DROP TABLE `01746_null`;
|
||||
DROP TABLE `01746_null_mv`;
|
||||
|
||||
-- Distributed
|
||||
|
||||
DROP TABLE IF EXISTS `01746_local`;
|
||||
CREATE TABLE `01746_local`
|
||||
(
|
||||
`n1` Int8,
|
||||
`n2` Int8,
|
||||
`n3` Int8
|
||||
)
|
||||
ENGINE = Memory;
|
||||
|
||||
DROP TABLE IF EXISTS `01746_dist`;
|
||||
CREATE TABLE `01746_dist` AS `01746_local`
|
||||
ENGINE = Distributed('test_shard_localhost', currentDatabase(), `01746_local`, rand());
|
||||
|
||||
DROP TABLE IF EXISTS `01746_dist_mv`;
|
||||
CREATE MATERIALIZED VIEW `01746_dist_mv`
|
||||
ENGINE = Memory AS
|
||||
SELECT
|
||||
n1,
|
||||
n2
|
||||
FROM `01746_dist`;
|
||||
|
||||
ALTER TABLE `01746_dist`
|
||||
DROP COLUMN n1; -- { serverError 524 }
|
||||
|
||||
ALTER TABLE `01746_dist`
|
||||
DROP COLUMN n2; -- { serverError 524 }
|
||||
|
||||
-- ok
|
||||
ALTER TABLE `01746_dist`
|
||||
DROP COLUMN n3;
|
||||
|
||||
DROP TABLE `01746_local`;
|
||||
DROP TABLE `01746_dist`;
|
||||
DROP TABLE `01746_dist_mv`;
|
||||
|
||||
-- Merge
|
||||
DROP TABLE IF EXISTS `01746_merge_t`;
|
||||
CREATE TABLE `01746_merge_t`
|
||||
(
|
||||
`n1` Int8,
|
||||
`n2` Int8,
|
||||
`n3` Int8
|
||||
)
|
||||
ENGINE = Memory;
|
||||
|
||||
DROP TABLE IF EXISTS `01746_merge`;
|
||||
CREATE TABLE `01746_merge` AS `01746_merge_t`
|
||||
ENGINE = Merge(currentDatabase(), '01746_merge_t');
|
||||
|
||||
DROP TABLE IF EXISTS `01746_merge_mv`;
|
||||
CREATE MATERIALIZED VIEW `01746_merge_mv`
|
||||
ENGINE = Memory AS
|
||||
SELECT
|
||||
n1,
|
||||
n2
|
||||
FROM `01746_merge`;
|
||||
|
||||
ALTER TABLE `01746_merge`
|
||||
DROP COLUMN n1; -- { serverError 524 }
|
||||
|
||||
ALTER TABLE `01746_merge`
|
||||
DROP COLUMN n2; -- { serverError 524 }
|
||||
|
||||
-- ok
|
||||
ALTER TABLE `01746_merge`
|
||||
DROP COLUMN n3;
|
||||
|
||||
DROP TABLE `01746_merge_t`;
|
||||
DROP TABLE `01746_merge`;
|
||||
DROP TABLE `01746_merge_mv`;
|
||||
|
||||
-- Buffer
|
||||
DROP TABLE IF EXISTS `01746_buffer_t`;
|
||||
CREATE TABLE `01746_buffer_t`
|
||||
(
|
||||
`n1` Int8,
|
||||
`n2` Int8,
|
||||
`n3` Int8
|
||||
)
|
||||
ENGINE = Memory;
|
||||
|
||||
DROP TABLE IF EXISTS `01746_buffer`;
|
||||
CREATE TABLE `01746_buffer` AS `01746_buffer_t`
|
||||
ENGINE = Buffer(currentDatabase(), `01746_buffer_t`, 16, 10, 100, 10000, 1000000, 10000000, 100000000);
|
||||
|
||||
DROP TABLE IF EXISTS `01746_buffer_mv`;
|
||||
CREATE MATERIALIZED VIEW `01746_buffer_mv`
|
||||
ENGINE = Memory AS
|
||||
SELECT
|
||||
n1,
|
||||
n2
|
||||
FROM `01746_buffer`;
|
||||
|
||||
ALTER TABLE `01746_buffer`
|
||||
DROP COLUMN n1; -- { serverError 524 }
|
||||
|
||||
ALTER TABLE `01746_buffer`
|
||||
DROP COLUMN n2; -- { serverError 524 }
|
||||
|
||||
-- ok
|
||||
ALTER TABLE `01746_buffer`
|
||||
DROP COLUMN n3;
|
||||
|
||||
DROP TABLE `01746_buffer_t`;
|
||||
DROP TABLE `01746_buffer`;
|
||||
DROP TABLE `01746_buffer_mv`;
|
Loading…
Reference in New Issue
Block a user