2017-04-01 09:19:00 +00:00
|
|
|
#include <Storages/IStorage.h>
|
2019-05-17 14:34:25 +00:00
|
|
|
|
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>
|
2020-08-03 11:33:11 +00:00
|
|
|
#include <Processors/Pipe.h>
|
2020-09-14 14:13:58 +00:00
|
|
|
#include <Processors/QueryPlan/ReadFromStorageStep.h>
|
2020-02-10 15:50:12 +00:00
|
|
|
#include <Interpreters/Context.h>
|
2019-11-19 12:46:07 +00:00
|
|
|
#include <Common/StringUtils/StringUtils.h>
|
2019-10-08 18:42:22 +00:00
|
|
|
#include <Common/quoteString.h>
|
2020-05-20 12:16:55 +00:00
|
|
|
#include <Interpreters/ExpressionActions.h>
|
2017-01-21 04:24:28 +00:00
|
|
|
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
2019-05-17 14:34:25 +00:00
|
|
|
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;
|
2020-04-06 23:45:51 +00:00
|
|
|
extern const int DEADLOCK_AVOIDED;
|
2019-05-17 14:34:25 +00:00
|
|
|
}
|
|
|
|
|
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-08-08 00:47:03 +00:00
|
|
|
/// Virtual column maybe overridden 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
|
|
|
}
|
|
|
|
|
2020-04-09 18:10:27 +00:00
|
|
|
RWLockImpl::LockHolder IStorage::tryLockTimed(
|
2020-07-16 13:38:58 +00:00
|
|
|
const RWLock & rwlock, RWLockImpl::Type type, const String & query_id, const std::chrono::milliseconds & acquire_timeout) const
|
2020-04-06 23:45:51 +00:00
|
|
|
{
|
2020-07-16 13:38:58 +00:00
|
|
|
auto lock_holder = rwlock->getLock(type, query_id, acquire_timeout);
|
2020-04-06 23:45:51 +00:00
|
|
|
if (!lock_holder)
|
2020-04-07 11:34:35 +00:00
|
|
|
{
|
|
|
|
const String type_str = type == RWLockImpl::Type::Read ? "READ" : "WRITE";
|
2020-04-06 23:45:51 +00:00
|
|
|
throw Exception(
|
2020-04-07 11:34:35 +00:00
|
|
|
type_str + " locking attempt on \"" + getStorageID().getFullTableName() +
|
2020-07-16 13:38:58 +00:00
|
|
|
"\" has timed out! (" + std::to_string(acquire_timeout.count()) + "ms) "
|
2020-04-07 11:34:35 +00:00
|
|
|
"Possible deadlock avoided. Client should retry.",
|
2020-04-06 23:45:51 +00:00
|
|
|
ErrorCodes::DEADLOCK_AVOIDED);
|
2020-04-07 11:34:35 +00:00
|
|
|
}
|
2020-04-07 07:15:59 +00:00
|
|
|
return lock_holder;
|
2020-04-06 23:45:51 +00:00
|
|
|
}
|
|
|
|
|
2020-07-16 13:38:58 +00:00
|
|
|
TableLockHolder IStorage::lockForShare(const String & query_id, const std::chrono::milliseconds & acquire_timeout)
|
2019-05-17 14:48:03 +00:00
|
|
|
{
|
2020-06-18 16:10:47 +00:00
|
|
|
TableLockHolder result = tryLockTimed(drop_lock, RWLockImpl::Read, query_id, acquire_timeout);
|
2019-05-17 14:48:03 +00:00
|
|
|
|
|
|
|
if (is_dropped)
|
|
|
|
throw Exception("Table is dropped", ErrorCodes::TABLE_IS_DROPPED);
|
2020-06-18 16:10:47 +00:00
|
|
|
|
2019-05-17 14:48:03 +00:00
|
|
|
return result;
|
|
|
|
}
|
|
|
|
|
2020-07-16 13:38:58 +00:00
|
|
|
TableLockHolder IStorage::lockForAlter(const String & query_id, const std::chrono::milliseconds & acquire_timeout)
|
2019-05-17 14:48:03 +00:00
|
|
|
{
|
2020-06-18 16:10:47 +00:00
|
|
|
TableLockHolder result = tryLockTimed(alter_lock, RWLockImpl::Write, query_id, acquire_timeout);
|
2019-05-17 14:48:03 +00:00
|
|
|
|
|
|
|
if (is_dropped)
|
|
|
|
throw Exception("Table is dropped", ErrorCodes::TABLE_IS_DROPPED);
|
2020-06-18 16:10:47 +00:00
|
|
|
|
2019-05-17 14:48:03 +00:00
|
|
|
return result;
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2020-07-16 13:38:58 +00:00
|
|
|
TableExclusiveLockHolder IStorage::lockExclusively(const String & query_id, const std::chrono::milliseconds & acquire_timeout)
|
2019-05-17 14:48:03 +00:00
|
|
|
{
|
2020-06-18 16:10:47 +00:00
|
|
|
TableExclusiveLockHolder result;
|
|
|
|
result.alter_lock = tryLockTimed(alter_lock, RWLockImpl::Write, query_id, acquire_timeout);
|
2019-05-17 14:48:03 +00:00
|
|
|
|
|
|
|
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);
|
2019-05-17 14:48:03 +00:00
|
|
|
|
|
|
|
return result;
|
|
|
|
}
|
|
|
|
|
2020-08-03 11:33:11 +00:00
|
|
|
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);
|
|
|
|
}
|
|
|
|
|
2020-09-14 14:13:58 +00:00
|
|
|
void IStorage::read(
|
|
|
|
QueryPlan & query_plan,
|
|
|
|
const Names & column_names,
|
2020-09-18 11:39:07 +00:00
|
|
|
const StorageMetadataPtr & metadata_snapshot,
|
2020-09-14 14:13:58 +00:00
|
|
|
const SelectQueryInfo & query_info,
|
2020-09-18 11:39:07 +00:00
|
|
|
const Context & context,
|
2020-09-14 14:13:58 +00:00
|
|
|
QueryProcessingStage::Enum processed_stage,
|
|
|
|
size_t max_block_size,
|
|
|
|
unsigned num_streams)
|
|
|
|
{
|
|
|
|
auto read_step = std::make_unique<ReadFromStorageStep>(
|
2020-09-18 11:39:07 +00:00
|
|
|
shared_from_this(), column_names, metadata_snapshot, query_info, context, processed_stage, max_block_size, num_streams);
|
2020-09-14 14:13:58 +00:00
|
|
|
|
|
|
|
read_step->setStepDescription("Read from " + getName());
|
|
|
|
query_plan.addStep(std::move(read_step));
|
|
|
|
}
|
|
|
|
|
2020-08-03 11:33:11 +00:00
|
|
|
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);
|
|
|
|
}
|
|
|
|
|
2019-05-17 14:34:25 +00:00
|
|
|
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
|
|
|
{
|
2019-12-10 20:47:05 +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);
|
2020-06-15 16:55:33 +00:00
|
|
|
setInMemoryMetadata(new_metadata);
|
2019-12-26 18:17:05 +00:00
|
|
|
}
|
2019-08-27 09:34:53 +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-08-27 09:34:53 +00:00
|
|
|
{
|
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
|
|
|
}
|
|
|
|
}
|
2019-05-17 14:48:03 +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
|
|
|
{
|
2020-01-15 16:17:04 +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
|
|
|
{
|
2020-01-15 16:17:04 +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
|
|
|
}
|
|
|
|
|
2020-04-28 10:38:57 +00:00
|
|
|
NamesAndTypesList IStorage::getVirtuals() const
|
2020-04-27 13:55:30 +00:00
|
|
|
{
|
2020-04-28 10:38:57 +00:00
|
|
|
return {};
|
2020-04-27 13:55:30 +00:00
|
|
|
}
|
|
|
|
|
2017-01-21 04:24:28 +00:00
|
|
|
}
|