Remove some of the garbage from code

This commit is contained in:
Alexey Milovidov 2020-03-28 05:26:48 +03:00
parent 1ea236c91d
commit 4ec30b01b1
9 changed files with 17 additions and 17 deletions

View File

@ -358,7 +358,7 @@ void DatabaseMySQL::cleanOutdatedTables()
++iterator;
else
{
const auto table_lock = (*iterator)->lockAlterIntention(RWLockImpl::NO_QUERY);
const auto table_lock = (*iterator)->lockAlterIntention();
(*iterator)->shutdown();
(*iterator)->is_dropped = true;

View File

@ -101,7 +101,7 @@ BlockIO InterpreterAlterQuery::execute()
switch (command.type)
{
case LiveViewCommand::REFRESH:
live_view->refresh(context);
live_view->refresh();
break;
}
}
@ -109,7 +109,7 @@ BlockIO InterpreterAlterQuery::execute()
if (!alter_commands.empty())
{
auto table_lock_holder = table->lockAlterIntention(context.getCurrentQueryId());
auto table_lock_holder = table->lockAlterIntention();
StorageInMemoryMetadata metadata = table->getInMemoryMetadata();
alter_commands.validate(metadata, context);
alter_commands.prepare(metadata);

View File

@ -324,10 +324,10 @@ TableStructureReadLockHolder IStorage::lockStructureForShare(const String & quer
return result;
}
TableStructureWriteLockHolder IStorage::lockAlterIntention(const String & query_id)
TableStructureWriteLockHolder IStorage::lockAlterIntention()
{
TableStructureWriteLockHolder result;
result.alter_intention_lock = alter_intention_lock->getLock(RWLockImpl::Write, query_id);
result.alter_lock = std::unique_lock(alter_lock);
if (is_dropped)
throw Exception("Table is dropped", ErrorCodes::TABLE_IS_DROPPED);
@ -336,7 +336,7 @@ TableStructureWriteLockHolder IStorage::lockAlterIntention(const String & query_
void IStorage::lockStructureExclusively(TableStructureWriteLockHolder & lock_holder, const String & query_id)
{
if (!lock_holder.alter_intention_lock)
if (!lock_holder.alter_lock)
throw Exception("Alter intention lock for table " + getStorageID().getNameForLogs() + " was not taken. This is a bug.", ErrorCodes::LOGICAL_ERROR);
lock_holder.structure_lock = structure_lock->getLock(RWLockImpl::Write, query_id);
@ -345,7 +345,7 @@ void IStorage::lockStructureExclusively(TableStructureWriteLockHolder & lock_hol
TableStructureWriteLockHolder IStorage::lockExclusively(const String & query_id)
{
TableStructureWriteLockHolder result;
result.alter_intention_lock = alter_intention_lock->getLock(RWLockImpl::Write, query_id);
result.alter_lock = std::unique_lock(alter_lock);
if (is_dropped)
throw Exception("Table is dropped", ErrorCodes::TABLE_IS_DROPPED);

View File

@ -203,7 +203,7 @@ public:
/// Acquire this lock at the start of ALTER to lock out other ALTERs and make sure that only you
/// can modify the table structure. It can later be upgraded to the exclusive lock.
TableStructureWriteLockHolder lockAlterIntention(const String & query_id);
TableStructureWriteLockHolder lockAlterIntention();
/// Upgrade alter intention lock to the full exclusive structure lock. This is done by ALTER queries
/// to ensure that no other query uses the table structure and it can be safely changed.
@ -472,7 +472,7 @@ private:
/// If you hold this lock exclusively, you can be sure that no other structure modifying queries
/// (e.g. ALTER, DROP) are concurrently executing. But queries that only read table structure
/// (e.g. SELECT, INSERT) can continue to execute.
mutable RWLock alter_intention_lock = RWLockImpl::create();
mutable std::mutex alter_lock;
/// Lock for the table column structure (names, types, etc.) and data path.
/// It is taken in exclusive mode by queries that modify them (e.g. RENAME, ALTER and DROP)

View File

@ -517,9 +517,9 @@ void StorageLiveView::drop(TableStructureWriteLockHolder &)
condition.notify_all();
}
void StorageLiveView::refresh(const Context & context)
void StorageLiveView::refresh()
{
auto alter_lock = lockAlterIntention(context.getCurrentQueryId());
auto alter_lock = lockAlterIntention();
{
std::lock_guard lock(mutex);
if (getNewBlocks())

View File

@ -123,7 +123,7 @@ public:
void startup() override;
void shutdown() override;
void refresh(const Context & context);
void refresh();
Pipes read(
const Names & column_names,

View File

@ -236,7 +236,7 @@ void StorageMergeTree::alter(
DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(context, table_id.table_name, metadata);
/// We release all locks except alter_intention_lock which allows
/// We release all locks except alter_lock which allows
/// to execute alter queries sequentially
table_lock_holder.releaseAllExceptAlterIntention();

View File

@ -3210,7 +3210,7 @@ void StorageReplicatedMergeTree::alter(
alter_entry.emplace();
mutation_znode.reset();
/// We can safely read structure, because we guarded with alter_intention_lock
/// We can safely read structure, because we guarded with alter_lock
if (is_readonly)
throw Exception("Can't ALTER readonly table", ErrorCodes::TABLE_IS_READ_ONLY);

View File

@ -12,7 +12,7 @@ struct TableStructureWriteLockHolder
{
void release()
{
*this = TableStructureWriteLockHolder();
*this = {};
}
void releaseAllExceptAlterIntention()
@ -24,7 +24,7 @@ private:
friend class IStorage;
/// Order is important.
RWLockImpl::LockHolder alter_intention_lock;
std::unique_lock<std::mutex> alter_lock;
RWLockImpl::LockHolder structure_lock;
};
@ -32,7 +32,7 @@ struct TableStructureReadLockHolder
{
void release()
{
*this = TableStructureReadLockHolder();
*this = {};
}
private: