ClickHouse/src/Storages/IStorage.cpp

128 lines
4.0 KiB
C++
Raw Normal View History

#include <Storages/IStorage.h>
2020-04-21 20:23:44 +00:00
#include <sparsehash/dense_hash_map>
#include <sparsehash/dense_hash_set>
2018-12-25 23:11:36 +00:00
#include <Storages/AlterCommands.h>
2019-08-07 15:21:45 +00:00
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/ASTSetQuery.h>
#include <Interpreters/Context.h>
#include <Common/StringUtils/StringUtils.h>
#include <Common/quoteString.h>
2020-05-20 12:16:55 +00:00
#include <Interpreters/ExpressionActions.h>
2019-10-21 16:26:29 +00:00
#include <Processors/Executors/TreeExecutorBlockInputStream.h>
2019-09-13 12:59:48 +00:00
namespace DB
{
namespace ErrorCodes
{
2019-08-27 23:47:30 +00:00
extern const int TABLE_IS_DROPPED;
2019-12-26 18:17:05 +00:00
extern const int NOT_IMPLEMENTED;
extern const int DEADLOCK_AVOIDED;
}
2020-06-17 14:37:21 +00:00
bool IStorage::isVirtualColumn(const String & column_name, const StorageMetadataPtr & metadata_snapshot) const
2019-05-21 11:24:32 +00:00
{
2020-04-27 13:55:30 +00:00
/// Virtual column maybe overriden by real column
2020-06-17 14:37:21 +00:00
return !metadata_snapshot->getColumns().has(column_name) && getVirtuals().contains(column_name);
2019-05-21 11:24:32 +00:00
}
RWLockImpl::LockHolder IStorage::tryLockTimed(
2020-04-22 06:22:14 +00:00
const RWLock & rwlock, RWLockImpl::Type type, const String & query_id, const SettingSeconds & acquire_timeout) const
{
auto lock_holder = rwlock->getLock(type, query_id, std::chrono::milliseconds(acquire_timeout.totalMilliseconds()));
if (!lock_holder)
2020-04-07 11:34:35 +00:00
{
const String type_str = type == RWLockImpl::Type::Read ? "READ" : "WRITE";
throw Exception(
2020-04-07 11:34:35 +00:00
type_str + " locking attempt on \"" + getStorageID().getFullTableName() +
2020-04-09 20:11:20 +00:00
"\" has timed out! (" + toString(acquire_timeout.totalMilliseconds()) + "ms) "
2020-04-07 11:34:35 +00:00
"Possible deadlock avoided. Client should retry.",
ErrorCodes::DEADLOCK_AVOIDED);
2020-04-07 11:34:35 +00:00
}
2020-04-07 07:15:59 +00:00
return lock_holder;
}
2020-06-18 16:10:47 +00:00
TableLockHolder IStorage::lockForShare(const String & query_id, const SettingSeconds & acquire_timeout)
{
2020-06-18 16:10:47 +00:00
TableLockHolder result = tryLockTimed(drop_lock, RWLockImpl::Read, query_id, acquire_timeout);
if (is_dropped)
throw Exception("Table is dropped", ErrorCodes::TABLE_IS_DROPPED);
2020-06-18 16:10:47 +00:00
return result;
}
2020-06-18 16:10:47 +00:00
TableLockHolder IStorage::lockForAlter(const String & query_id, const SettingSeconds & acquire_timeout)
{
2020-06-18 16:10:47 +00:00
TableLockHolder result = tryLockTimed(alter_lock, RWLockImpl::Write, query_id, acquire_timeout);
if (is_dropped)
throw Exception("Table is dropped", ErrorCodes::TABLE_IS_DROPPED);
2020-06-18 16:10:47 +00:00
return result;
}
2020-06-18 16:10:47 +00:00
TableExclusiveLockHolder IStorage::lockExclusively(const String & query_id, const SettingSeconds & acquire_timeout)
{
2020-06-18 16:10:47 +00:00
TableExclusiveLockHolder result;
result.alter_lock = tryLockTimed(alter_lock, RWLockImpl::Write, query_id, acquire_timeout);
if (is_dropped)
throw Exception("Table is dropped", ErrorCodes::TABLE_IS_DROPPED);
2020-06-18 16:10:47 +00:00
result.drop_lock = tryLockTimed(drop_lock, RWLockImpl::Write, query_id, acquire_timeout);
return result;
}
void IStorage::alter(
2020-06-18 16:10:47 +00:00
const AlterCommands & params, const Context & context, TableLockHolder &)
2018-12-25 23:11:36 +00:00
{
auto table_id = getStorageID();
2020-06-10 09:09:51 +00:00
StorageInMemoryMetadata new_metadata = getInMemoryMetadata();
params.apply(new_metadata, context);
DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(context, table_id, new_metadata);
setInMemoryMetadata(new_metadata);
2019-12-26 18:17:05 +00:00
}
2019-12-26 18:17:05 +00:00
2020-06-10 11:16:31 +00:00
void IStorage::checkAlterIsPossible(const AlterCommands & commands, const Settings & /* settings */) const
2019-12-26 18:17:05 +00:00
{
for (const auto & command : commands)
{
2019-12-27 14:36:59 +00:00
if (!command.isCommentAlter())
2019-12-26 18:17:05 +00:00
throw Exception(
"Alter of type '" + alterTypeToString(command.type) + "' is not supported by storage " + getName(),
ErrorCodes::NOT_IMPLEMENTED);
2018-12-25 23:11:36 +00:00
}
}
2020-07-13 17:27:52 +00:00
void IStorage::checkAlterPartitionIsPossible(const PartitionCommands & /*commands*/, const StorageMetadataPtr & /*metadata_snapshot*/, const Settings & /*settings*/) const
2020-07-13 16:19:08 +00:00
{
2020-07-13 17:27:52 +00:00
throw Exception("Table engine " + getName() + " doesn't support partitioning", ErrorCodes::NOT_IMPLEMENTED);
2020-07-13 16:19:08 +00:00
}
2019-09-13 12:59:48 +00:00
2019-12-27 19:30:22 +00:00
StorageID IStorage::getStorageID() const
2019-12-03 16:25:32 +00:00
{
std::lock_guard lock(id_mutex);
2019-12-12 12:30:31 +00:00
return storage_id;
2019-12-03 16:25:32 +00:00
}
2020-04-07 14:05:51 +00:00
void IStorage::renameInMemory(const StorageID & new_table_id)
2019-12-03 16:25:32 +00:00
{
std::lock_guard lock(id_mutex);
2020-04-07 14:05:51 +00:00
storage_id = new_table_id;
2019-12-03 16:25:32 +00:00
}
NamesAndTypesList IStorage::getVirtuals() const
2020-04-27 13:55:30 +00:00
{
return {};
2020-04-27 13:55:30 +00:00
}
}