Change interface

This commit is contained in:
Nikita Mikhaylov 2023-08-29 16:26:48 +02:00
parent 342c78bfe6
commit 96cabcae3a
17 changed files with 27 additions and 19 deletions

View File

@ -145,8 +145,11 @@ void SerializationUUID::serializeBinaryBulk(const IColumn & column, WriteBuffer
if constexpr (std::endian::native == std::endian::big)
{
#pragma clang diagnostic push
#pragma clang diagnostic ignored "-Wunreachable-code"
std::ranges::for_each(
x | std::views::drop(offset) | std::views::take(limit), [&ostr](const auto & uuid) { writeBinaryLittleEndian(uuid, ostr); });
#pragma clang diagnostic pop
}
else
ostr.write(reinterpret_cast<const char *>(&x[offset]), sizeof(UUID) * limit);
@ -160,8 +163,11 @@ void SerializationUUID::deserializeBinaryBulk(IColumn & column, ReadBuffer & ist
const size_t size = istr.readBig(reinterpret_cast<char *>(&x[initial_size]), sizeof(UUID) * limit);
x.resize(initial_size + size / sizeof(UUID));
#pragma clang diagnostic push
#pragma clang diagnostic ignored "-Wunreachable-code"
if constexpr (std::endian::native == std::endian::big)
std::ranges::for_each(
x | std::views::drop(initial_size), [](auto & uuid) { transformEndianness<std::endian::big, std::endian::little>(uuid); });
#pragma clang diagnostic pop
}
}

View File

@ -208,7 +208,7 @@ BlockIO InterpreterDropQuery::executeToTableImpl(ContextPtr context_, ASTDropQue
if (table->isStaticStorage())
throw Exception(ErrorCodes::TABLE_IS_READ_ONLY, "Table is read-only");
table->checkTableCanBeDropped();
table->checkTableCanBeDropped(context_);
TableExclusiveLockHolder table_excl_lock;
/// We don't need any lock for ReplicatedMergeTree and for simple MergeTree
@ -228,10 +228,10 @@ BlockIO InterpreterDropQuery::executeToTableImpl(ContextPtr context_, ASTDropQue
{
/// If DROP DICTIONARY query is not used, check if Dictionary can be dropped with DROP TABLE query
if (!query.is_dictionary)
table->checkTableCanBeDropped();
table->checkTableCanBeDropped(context_);
}
else
table->checkTableCanBeDropped();
table->checkTableCanBeDropped(context_);
/// Check dependencies before shutting table down
bool check_ref_deps = getContext()->getSettingsRef().check_referential_table_dependencies;

View File

@ -602,7 +602,7 @@ public:
/// Checks that table could be dropped right now
/// Otherwise - throws an exception with detailed information.
/// We do not use mutex because it is not very important that the size could change during the operation.
virtual void checkTableCanBeDropped() const {}
virtual void checkTableCanBeDropped([[ maybe_unused ]] ContextPtr query_context) const {}
/// Similar to above but checks for DETACH. It's only used for DICTIONARIES.
virtual void checkTableCanBeDetached() const {}

View File

@ -263,7 +263,7 @@ NamesAndTypesList StorageLiveView::getVirtuals() const
};
}
void StorageLiveView::checkTableCanBeDropped() const
void StorageLiveView::checkTableCanBeDropped([[ maybe_unused ]] ContextPtr query_context) const
{
auto table_id = getStorageID();
auto view_ids = DatabaseCatalog::instance().getDependentViews(table_id);

View File

@ -75,7 +75,7 @@ public:
NamesAndTypesList getVirtuals() const override;
void checkTableCanBeDropped() const override;
void checkTableCanBeDropped([[ maybe_unused ]] ContextPtr query_context) const override;
void drop() override;

View File

@ -38,7 +38,7 @@ public:
/// actions require an open connection. Therefore there needs to be a way inside shutdown() method to know whether it is called
/// because of drop query. And drop() method is not suitable at all, because it will not only require to reopen connection, but also
/// it can be called considerable time after table is dropped (for example, in case of Atomic database), which is not appropriate for the case.
void checkTableCanBeDropped() const override { drop_table = true; }
void checkTableCanBeDropped([[ maybe_unused ]] ContextPtr query_context) const override { drop_table = true; }
/// Always return virtual columns in addition to required columns
void read(

View File

@ -41,7 +41,7 @@ public:
/// actions require an open connection. Therefore there needs to be a way inside shutdown() method to know whether it is called
/// because of drop query. And drop() method is not suitable at all, because it will not only require to reopen connection, but also
/// it can be called considerable time after table is dropped (for example, in case of Atomic database), which is not appropriate for the case.
void checkTableCanBeDropped() const override { drop_table = true; }
void checkTableCanBeDropped([[ maybe_unused ]] ContextPtr query_context) const override { drop_table = true; }
/// Always return virtual columns in addition to required columns
void read(

View File

@ -145,7 +145,7 @@ StorageDictionary::~StorageDictionary()
removeDictionaryConfigurationFromRepository();
}
void StorageDictionary::checkTableCanBeDropped() const
void StorageDictionary::checkTableCanBeDropped([[ maybe_unused ]] ContextPtr query_context) const
{
if (location == Location::SameDatabaseAndNameAsDictionary)
throw Exception(ErrorCodes::CANNOT_DETACH_DICTIONARY_AS_TABLE,
@ -159,7 +159,9 @@ void StorageDictionary::checkTableCanBeDropped() const
void StorageDictionary::checkTableCanBeDetached() const
{
checkTableCanBeDropped();
/// Actually query context (from DETACH query) should be passed here.
/// But we don't use it for this type of storage
checkTableCanBeDropped(getContext());
}
Pipe StorageDictionary::read(

View File

@ -62,7 +62,7 @@ public:
~StorageDictionary() override;
void checkTableCanBeDropped() const override;
void checkTableCanBeDropped([[ maybe_unused ]] ContextPtr query_context) const override;
void checkTableCanBeDetached() const override;
Pipe read(

View File

@ -288,7 +288,7 @@ StorageMergeTree::write(const ASTPtr & /*query*/, const StorageMetadataPtr & met
*this, metadata_snapshot, settings.max_partitions_per_insert_block, local_context);
}
void StorageMergeTree::checkTableCanBeDropped() const
void StorageMergeTree::checkTableCanBeDropped([[ maybe_unused ]] ContextPtr query_context) const
{
if (!supportsReplication() && isStaticStorage())
return;

View File

@ -102,7 +102,7 @@ public:
void alter(const AlterCommands & commands, ContextPtr context, AlterLockHolder & table_lock_holder) override;
void checkTableCanBeDropped() const override;
void checkTableCanBeDropped([[ maybe_unused ]] ContextPtr query_context) const override;
ActionLock getActionLock(StorageActionBlockType action_type) override;

View File

@ -150,7 +150,7 @@ public:
}
CheckResults checkData(const ASTPtr & query, ContextPtr context) override { return getNested()->checkData(query, context); }
void checkTableCanBeDropped() const override { getNested()->checkTableCanBeDropped(); }
void checkTableCanBeDropped([[ maybe_unused ]] ContextPtr query_context) const override { getNested()->checkTableCanBeDropped(query_context); }
bool storesDataOnDisk() const override { return getNested()->storesDataOnDisk(); }
Strings getDataPaths() const override { return getNested()->getDataPaths(); }
StoragePolicyPtr getStoragePolicy() const override { return getNested()->getStoragePolicy(); }

View File

@ -6164,7 +6164,7 @@ PartitionCommandsResultInfo StorageReplicatedMergeTree::attachPartition(
}
void StorageReplicatedMergeTree::checkTableCanBeDropped() const
void StorageReplicatedMergeTree::checkTableCanBeDropped([[ maybe_unused ]] ContextPtr query_context) const
{
auto table_id = getStorageID();
getContext()->checkTableCanBeDropped(table_id.database_name, table_id.table_name, getTotalActiveSizeInBytes());

View File

@ -201,7 +201,7 @@ public:
bool supportsIndexForIn() const override { return true; }
void checkTableCanBeDropped() const override;
void checkTableCanBeDropped([[ maybe_unused ]] ContextPtr query_context) const override;
ActionLock getActionLock(StorageActionBlockType action_type) override;

View File

@ -153,7 +153,7 @@ public:
}
bool isView() const override { return false; }
void checkTableCanBeDropped() const override {}
void checkTableCanBeDropped([[ maybe_unused ]] ContextPtr query_context) const override {}
private:
mutable std::recursive_mutex nested_mutex;

View File

@ -1599,7 +1599,7 @@ void StorageWindowView::shutdown()
DatabaseCatalog::instance().removeViewDependency(select_table_id, table_id);
}
void StorageWindowView::checkTableCanBeDropped() const
void StorageWindowView::checkTableCanBeDropped([[ maybe_unused ]] ContextPtr query_context) const
{
auto table_id = getStorageID();
auto view_ids = DatabaseCatalog::instance().getDependentViews(table_id);

View File

@ -119,7 +119,7 @@ public:
bool supportsSampling() const override { return true; }
bool supportsFinal() const override { return true; }
void checkTableCanBeDropped() const override;
void checkTableCanBeDropped([[ maybe_unused ]] ContextPtr query_context) const override;
void dropInnerTableIfAny(bool sync, ContextPtr context) override;