Don't replicate mutations for KeeperMap tables

This commit is contained in:
Antonio Andelic 2023-04-03 13:56:03 +00:00
parent 2933c6b9be
commit e7d00c8f34
3 changed files with 20 additions and 4 deletions

View File

@ -34,6 +34,7 @@
#include <Parsers/parseQuery.h>
#include <Parsers/ParserCreateQuery.h>
#include <Parsers/queryToString.h>
#include <Storages/StorageKeeperMap.h>
namespace DB
{
@ -1390,6 +1391,13 @@ bool DatabaseReplicated::shouldReplicateQuery(const ContextPtr & query_context,
/// Some ALTERs are not replicated on database level
if (const auto * alter = query_ptr->as<const ASTAlterQuery>())
{
auto table_id = query_context->resolveStorageID(*alter, Context::ResolveOrdinary);
StoragePtr table = DatabaseCatalog::instance().getTable(table_id, query_context);
/// we never replicate KeeperMap operations because it doesn't make sense
if (auto * keeper_map = table->as<StorageKeeperMap>())
return false;
return !alter->isAttachAlter() && !alter->isFetchAlter() && !alter->isDropPartitionAlter();
}

View File

@ -20,6 +20,7 @@
#include <Storages/LiveView/StorageLiveView.h>
#include <Storages/MutationCommands.h>
#include <Storages/PartitionCommands.h>
#include <Storages/StorageKeeperMap.h>
#include <Common/typeid_cast.h>
#include <Functions/UserDefined/UserDefinedSQLFunctionFactory.h>
@ -39,6 +40,7 @@ namespace ErrorCodes
extern const int INCORRECT_QUERY;
extern const int NOT_IMPLEMENTED;
extern const int TABLE_IS_READ_ONLY;
extern const int BAD_ARGUMENTS;
}
@ -72,16 +74,21 @@ BlockIO InterpreterAlterQuery::executeToTable(const ASTAlterQuery & alter)
if (!UserDefinedSQLFunctionFactory::instance().empty())
UserDefinedSQLFunctionVisitor::visit(query_ptr);
auto table_id = getContext()->resolveStorageID(alter, Context::ResolveOrdinary);
query_ptr->as<ASTAlterQuery &>().setDatabase(table_id.database_name);
StoragePtr table = DatabaseCatalog::instance().getTable(table_id, getContext());
if (!alter.cluster.empty() && !maybeRemoveOnCluster(query_ptr, getContext()))
{
if (table->as<StorageKeeperMap>())
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Mutations with ON CLUSTER are not allowed for KeeperMap tables");
DDLQueryOnClusterParams params;
params.access_to_check = getRequiredAccess();
return executeDDLQueryOnCluster(query_ptr, getContext(), params);
}
getContext()->checkAccess(getRequiredAccess());
auto table_id = getContext()->resolveStorageID(alter, Context::ResolveOrdinary);
query_ptr->as<ASTAlterQuery &>().setDatabase(table_id.database_name);
DatabasePtr database = DatabaseCatalog::instance().getDatabase(table_id.database_name);
if (database->shouldReplicateQuery(getContext(), query_ptr))
@ -91,7 +98,6 @@ BlockIO InterpreterAlterQuery::executeToTable(const ASTAlterQuery & alter)
return database->tryEnqueueReplicatedDDL(query_ptr, getContext());
}
StoragePtr table = DatabaseCatalog::instance().getTable(table_id, getContext());
checkStorageSupportsTransactionsIfNeeded(table, getContext());
if (table->isStaticStorage())
throw Exception(ErrorCodes::TABLE_IS_READ_ONLY, "Table is read-only");

View File

@ -31,7 +31,7 @@ ALTER TABLE 02661_keepermap_delete_update UPDATE value = 'Another' WHERE key > 2
SELECT * FROM 02661_keepermap_delete_update ORDER BY key;
SELECT '-----------';
ALTER TABLE 02661_keepermap_delete_update UPDATE key = key * 10 WHERE 1 = 1; -- { serverError 36 }
ALTER TABLE 02661_keepermap_delete_update UPDATE key = key * 10 WHERE 1 = 1; -- { serverError BAD_ARGUMENTS }
SELECT * FROM 02661_keepermap_delete_update ORDER BY key;
SELECT '-----------';
@ -39,4 +39,6 @@ ALTER TABLE 02661_keepermap_delete_update UPDATE value2 = value2 * 10 + 2 WHERE
SELECT * FROM 02661_keepermap_delete_update ORDER BY key;
SELECT '-----------';
ALTER TABLE 02661_keepermap_delete_update ON CLUSTER test_shard_localhost UPDATE value2 = value2 * 10 + 2 WHERE value2 < 100; -- { serverError BAD_ARGUMENTS }
DROP TABLE IF EXISTS 02661_keepermap_delete_update;