#include #include #include #include #include #include #include #include #include #include #include #include namespace DB { namespace ErrorCodes { extern const int TABLE_IS_DROPPED; extern const int NOT_IMPLEMENTED; extern const int DEADLOCK_AVOIDED; } bool IStorage::isVirtualColumn(const String & column_name, const StorageMetadataPtr & metadata_snapshot) const { /// Virtual column maybe overridden by real column return !metadata_snapshot->getColumns().has(column_name) && getVirtuals().contains(column_name); } RWLockImpl::LockHolder IStorage::tryLockTimed( const RWLock & rwlock, RWLockImpl::Type type, const String & query_id, const std::chrono::milliseconds & acquire_timeout) const { auto lock_holder = rwlock->getLock(type, query_id, acquire_timeout); if (!lock_holder) { const String type_str = type == RWLockImpl::Type::Read ? "READ" : "WRITE"; throw Exception( type_str + " locking attempt on \"" + getStorageID().getFullTableName() + "\" has timed out! (" + std::to_string(acquire_timeout.count()) + "ms) " "Possible deadlock avoided. Client should retry.", ErrorCodes::DEADLOCK_AVOIDED); } return lock_holder; } TableLockHolder IStorage::lockForShare(const String & query_id, const std::chrono::milliseconds & acquire_timeout) { TableLockHolder result = tryLockTimed(drop_lock, RWLockImpl::Read, query_id, acquire_timeout); if (is_dropped) throw Exception("Table is dropped", ErrorCodes::TABLE_IS_DROPPED); return result; } TableLockHolder IStorage::lockForAlter(const String & query_id, const std::chrono::milliseconds & acquire_timeout) { TableLockHolder result = tryLockTimed(alter_lock, RWLockImpl::Write, query_id, acquire_timeout); if (is_dropped) throw Exception("Table is dropped", ErrorCodes::TABLE_IS_DROPPED); return result; } TableExclusiveLockHolder IStorage::lockExclusively(const String & query_id, const std::chrono::milliseconds & acquire_timeout) { 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); result.drop_lock = tryLockTimed(drop_lock, RWLockImpl::Write, query_id, acquire_timeout); return result; } Pipe IStorage::read( const Names & /*column_names*/, const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & /*query_info*/, const Context & /*context*/, QueryProcessingStage::Enum /*processed_stage*/, size_t /*max_block_size*/, unsigned /*num_streams*/) { throw Exception("Method read is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED); } void IStorage::read( QueryPlan & query_plan, TableLockHolder table_lock, StorageMetadataPtr metadata_snapshot, StreamLocalLimits & limits, SizeLimits & leaf_limits, std::shared_ptr quota, const Names & column_names, const SelectQueryInfo & query_info, std::shared_ptr context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, unsigned num_streams) { auto read_step = std::make_unique( std::move(table_lock), std::move(metadata_snapshot), limits, leaf_limits, std::move(quota), shared_from_this(), column_names, query_info, std::move(context), processed_stage, max_block_size, num_streams); read_step->setStepDescription("Read from " + getName()); query_plan.addStep(std::move(read_step)); } Pipe IStorage::alterPartition( const ASTPtr & /* query */, const StorageMetadataPtr & /* metadata_snapshot */, const PartitionCommands & /* commands */, const Context & /* context */) { throw Exception("Partition operations are not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED); } void IStorage::alter( const AlterCommands & params, const Context & context, TableLockHolder &) { auto table_id = getStorageID(); 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); } void IStorage::checkAlterIsPossible(const AlterCommands & commands, const Settings & /* settings */) const { for (const auto & command : commands) { if (!command.isCommentAlter()) throw Exception( "Alter of type '" + alterTypeToString(command.type) + "' is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED); } } void IStorage::checkAlterPartitionIsPossible(const PartitionCommands & /*commands*/, const StorageMetadataPtr & /*metadata_snapshot*/, const Settings & /*settings*/) const { throw Exception("Table engine " + getName() + " doesn't support partitioning", ErrorCodes::NOT_IMPLEMENTED); } StorageID IStorage::getStorageID() const { std::lock_guard lock(id_mutex); return storage_id; } void IStorage::renameInMemory(const StorageID & new_table_id) { std::lock_guard lock(id_mutex); storage_id = new_table_id; } NamesAndTypesList IStorage::getVirtuals() const { return {}; } }