ClickHouse/src/Storages/IStorage.cpp
roman b41421cb1c [settings]: introduce new query complexity settings for leaf-nodes
The new setting should allow to control query complexity on leaf nodes
excluding the final merging stage on the root-node. For example, distributed
query that reads 1k rows from 5 shards will breach the `max_rows_to_read=5000`,
while effectively every shard reads only 1k rows. With setting `max_rows_to_read_leaf=1500`
this limit won't be reached and query will succeed since every shard reads
not more that ~1k rows.
2020-09-17 10:37:05 +01:00

171 lines
5.7 KiB
C++

#include <Storages/IStorage.h>
#include <sparsehash/dense_hash_map>
#include <sparsehash/dense_hash_set>
#include <Storages/AlterCommands.h>
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/ASTSetQuery.h>
#include <Processors/Pipe.h>
#include <Processors/QueryPlan/ReadFromStorageStep.h>
#include <Interpreters/Context.h>
#include <Common/StringUtils/StringUtils.h>
#include <Common/quoteString.h>
#include <Interpreters/ExpressionActions.h>
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<const EnabledQuota> quota,
const Names & column_names,
const SelectQueryInfo & query_info,
std::shared_ptr<Context> context,
QueryProcessingStage::Enum processed_stage,
size_t max_block_size,
unsigned num_streams)
{
auto read_step = std::make_unique<ReadFromStorageStep>(
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 {};
}
}