2017-04-01 09:19:00 +00:00
|
|
|
#include <Storages/IStorage.h>
|
2019-05-17 14:34:25 +00:00
|
|
|
|
2019-11-19 12:46:07 +00:00
|
|
|
#include <Common/StringUtils/StringUtils.h>
|
2020-12-18 17:13:28 +00:00
|
|
|
#include <IO/Operators.h>
|
|
|
|
#include <IO/WriteBufferFromString.h>
|
|
|
|
#include <Interpreters/Context.h>
|
2020-09-25 13:19:26 +00:00
|
|
|
#include <Interpreters/InterpreterSelectQuery.h>
|
2020-12-18 17:13:28 +00:00
|
|
|
#include <Parsers/ASTCreateQuery.h>
|
2022-06-14 08:53:22 +00:00
|
|
|
#include <Parsers/ASTFunction.h>
|
2020-12-18 17:13:28 +00:00
|
|
|
#include <Parsers/ASTSetQuery.h>
|
2021-10-16 14:03:50 +00:00
|
|
|
#include <QueryPipeline/Pipe.h>
|
2020-12-18 17:13:28 +00:00
|
|
|
#include <Processors/QueryPlan/ReadFromPreparedSource.h>
|
2021-09-08 18:29:38 +00:00
|
|
|
#include <Processors/QueryPlan/QueryPlan.h>
|
2020-12-18 17:13:28 +00:00
|
|
|
#include <Storages/AlterCommands.h>
|
2022-06-29 12:42:23 +00:00
|
|
|
#include <Backups/RestorerFromBackup.h>
|
|
|
|
#include <Backups/IBackup.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;
|
2022-06-29 12:42:23 +00:00
|
|
|
extern const int CANNOT_RESTORE_TABLE;
|
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(
|
2022-05-29 19:53:56 +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);
|
2022-05-29 19:53:56 +00:00
|
|
|
if (!lock_holder)
|
2020-04-07 11:34:35 +00:00
|
|
|
{
|
|
|
|
const String type_str = type == RWLockImpl::Type::Read ? "READ" : "WRITE";
|
2023-01-23 21:13:58 +00:00
|
|
|
throw Exception(ErrorCodes::DEADLOCK_AVOIDED,
|
2023-03-06 10:54:43 +00:00
|
|
|
"{} locking attempt on \"{}\" has timed out! ({}ms) Possible deadlock avoided. Client should retry",
|
2023-01-23 21:13:58 +00:00
|
|
|
type_str, getStorageID(), acquire_timeout.count());
|
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)
|
2022-09-28 16:08:10 +00:00
|
|
|
{
|
|
|
|
TableLockHolder result = tryLockTimed(drop_lock, RWLockImpl::Read, query_id, acquire_timeout);
|
|
|
|
|
2023-01-24 13:15:51 +00:00
|
|
|
if (is_dropped || is_detached)
|
2022-09-28 16:59:07 +00:00
|
|
|
{
|
|
|
|
auto table_id = getStorageID();
|
2023-01-24 13:15:51 +00:00
|
|
|
throw Exception(ErrorCodes::TABLE_IS_DROPPED, "Table {}.{} is dropped or detached", table_id.database_name, table_id.table_name);
|
2022-09-28 16:59:07 +00:00
|
|
|
}
|
2022-09-28 16:08:10 +00:00
|
|
|
return result;
|
|
|
|
}
|
|
|
|
|
|
|
|
TableLockHolder IStorage::tryLockForShare(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
|
|
|
|
2023-01-24 13:15:51 +00:00
|
|
|
if (is_dropped || is_detached)
|
2021-05-03 09:52:13 +00:00
|
|
|
{
|
2022-09-28 15:41:51 +00:00
|
|
|
// Table was dropped while acquiring the lock
|
|
|
|
result = nullptr;
|
2021-05-03 09:52:13 +00:00
|
|
|
}
|
2020-06-18 16:10:47 +00:00
|
|
|
|
2019-05-17 14:48:03 +00:00
|
|
|
return result;
|
|
|
|
}
|
|
|
|
|
2023-07-17 17:02:29 +00:00
|
|
|
std::optional<IStorage::AlterLockHolder> IStorage::tryLockForAlter(const std::chrono::milliseconds & acquire_timeout)
|
2019-05-17 14:48:03 +00:00
|
|
|
{
|
2021-10-25 17:49:49 +00:00
|
|
|
AlterLockHolder lock{alter_lock, std::defer_lock};
|
|
|
|
|
|
|
|
if (!lock.try_lock_for(acquire_timeout))
|
2023-07-17 17:15:07 +00:00
|
|
|
return {};
|
2019-05-17 14:48:03 +00:00
|
|
|
|
2023-01-24 13:15:51 +00:00
|
|
|
if (is_dropped || is_detached)
|
2023-01-23 21:13:58 +00:00
|
|
|
throw Exception(ErrorCodes::TABLE_IS_DROPPED, "Table {} is dropped or detached", getStorageID());
|
2020-06-18 16:10:47 +00:00
|
|
|
|
2021-10-25 17:49:49 +00:00
|
|
|
return lock;
|
2019-05-17 14:48:03 +00:00
|
|
|
}
|
|
|
|
|
2023-07-17 17:02:29 +00:00
|
|
|
IStorage::AlterLockHolder IStorage::lockForAlter(const std::chrono::milliseconds & acquire_timeout)
|
|
|
|
{
|
|
|
|
|
|
|
|
if (auto lock = tryLockForAlter(acquire_timeout); lock == std::nullopt)
|
|
|
|
throw Exception(ErrorCodes::DEADLOCK_AVOIDED,
|
|
|
|
"Locking attempt for ALTER on \"{}\" has timed out! ({} ms) "
|
|
|
|
"Possible deadlock avoided. Client should retry.",
|
|
|
|
getStorageID().getFullTableName(), acquire_timeout.count());
|
|
|
|
else
|
2023-07-17 17:15:07 +00:00
|
|
|
return std::move(*lock);
|
2023-07-17 17:02:29 +00:00
|
|
|
}
|
|
|
|
|
2019-05-17 14:48:03 +00:00
|
|
|
|
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;
|
2021-10-25 17:49:49 +00:00
|
|
|
result.drop_lock = tryLockTimed(drop_lock, RWLockImpl::Write, query_id, acquire_timeout);
|
2019-05-17 14:48:03 +00:00
|
|
|
|
2023-01-24 13:15:51 +00:00
|
|
|
if (is_dropped || is_detached)
|
2023-01-23 21:13:58 +00:00
|
|
|
throw Exception(ErrorCodes::TABLE_IS_DROPPED, "Table {} is dropped or detached", getStorageID());
|
2019-05-17 14:48:03 +00:00
|
|
|
|
|
|
|
return result;
|
|
|
|
}
|
|
|
|
|
2022-05-20 19:49:31 +00:00
|
|
|
Pipe IStorage::watch(
|
|
|
|
const Names & /*column_names*/,
|
|
|
|
const SelectQueryInfo & /*query_info*/,
|
|
|
|
ContextPtr /*context*/,
|
|
|
|
QueryProcessingStage::Enum & /*processed_stage*/,
|
|
|
|
size_t /*max_block_size*/,
|
2022-10-07 10:46:45 +00:00
|
|
|
size_t /*num_streams*/)
|
2022-05-20 19:49:31 +00:00
|
|
|
{
|
2023-01-23 21:13:58 +00:00
|
|
|
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method watch is not supported by storage {}", getName());
|
2022-05-20 19:49:31 +00:00
|
|
|
}
|
|
|
|
|
2020-08-03 11:33:11 +00:00
|
|
|
Pipe IStorage::read(
|
2020-12-07 09:30:47 +00:00
|
|
|
const Names & /*column_names*/,
|
2021-07-09 03:15:41 +00:00
|
|
|
const StorageSnapshotPtr & /*storage_snapshot*/,
|
2020-12-07 09:30:47 +00:00
|
|
|
SelectQueryInfo & /*query_info*/,
|
2021-04-10 23:33:54 +00:00
|
|
|
ContextPtr /*context*/,
|
2020-12-07 09:30:47 +00:00
|
|
|
QueryProcessingStage::Enum /*processed_stage*/,
|
|
|
|
size_t /*max_block_size*/,
|
2022-10-07 10:46:45 +00:00
|
|
|
size_t /*num_streams*/)
|
2020-08-03 11:33:11 +00:00
|
|
|
{
|
2023-01-23 21:13:58 +00:00
|
|
|
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method read is not supported by storage {}", getName());
|
2020-08-03 11:33:11 +00:00
|
|
|
}
|
|
|
|
|
2020-09-14 14:13:58 +00:00
|
|
|
void IStorage::read(
|
2020-12-07 09:30:47 +00:00
|
|
|
QueryPlan & query_plan,
|
|
|
|
const Names & column_names,
|
2021-07-09 03:15:41 +00:00
|
|
|
const StorageSnapshotPtr & storage_snapshot,
|
2020-12-07 09:30:47 +00:00
|
|
|
SelectQueryInfo & query_info,
|
2021-04-10 23:33:54 +00:00
|
|
|
ContextPtr context,
|
2020-12-07 09:30:47 +00:00
|
|
|
QueryProcessingStage::Enum processed_stage,
|
|
|
|
size_t max_block_size,
|
2022-10-07 10:46:45 +00:00
|
|
|
size_t num_streams)
|
2020-09-14 14:13:58 +00:00
|
|
|
{
|
2021-07-09 03:15:41 +00:00
|
|
|
auto pipe = read(column_names, storage_snapshot, query_info, context, processed_stage, max_block_size, num_streams);
|
2023-04-12 18:46:07 +00:00
|
|
|
|
|
|
|
/// parallelize processing if not yet
|
|
|
|
const size_t output_ports = pipe.numOutputPorts();
|
2023-04-24 12:22:45 +00:00
|
|
|
const bool parallelize_output = context->getSettingsRef().parallelize_output_from_storages;
|
2023-05-05 04:18:46 +00:00
|
|
|
if (parallelize_output && parallelizeOutputAfterReading(context) && output_ports > 0 && output_ports < num_streams)
|
2023-04-12 18:46:07 +00:00
|
|
|
pipe.resize(num_streams);
|
|
|
|
|
2022-05-20 19:49:31 +00:00
|
|
|
readFromPipe(query_plan, std::move(pipe), column_names, storage_snapshot, query_info, context, getName());
|
|
|
|
}
|
|
|
|
|
|
|
|
void IStorage::readFromPipe(
|
|
|
|
QueryPlan & query_plan,
|
|
|
|
Pipe pipe,
|
|
|
|
const Names & column_names,
|
|
|
|
const StorageSnapshotPtr & storage_snapshot,
|
|
|
|
SelectQueryInfo & query_info,
|
|
|
|
ContextPtr context,
|
|
|
|
std::string storage_name)
|
|
|
|
{
|
2020-09-25 13:19:26 +00:00
|
|
|
if (pipe.empty())
|
|
|
|
{
|
2021-07-09 03:15:41 +00:00
|
|
|
auto header = storage_snapshot->getSampleBlockForColumns(column_names);
|
2021-02-10 14:12:49 +00:00
|
|
|
InterpreterSelectQuery::addEmptySourceToQueryPlan(query_plan, header, query_info, context);
|
2020-09-25 13:19:26 +00:00
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
2023-10-24 19:08:17 +00:00
|
|
|
auto read_step = std::make_unique<ReadFromStorageStep>(std::move(pipe), storage_name, query_info, context);
|
2020-09-25 13:19:26 +00:00
|
|
|
query_plan.addStep(std::move(read_step));
|
|
|
|
}
|
2020-09-14 14:13:58 +00:00
|
|
|
}
|
|
|
|
|
2022-05-20 19:49:31 +00:00
|
|
|
std::optional<QueryPipeline> IStorage::distributedWrite(
|
|
|
|
const ASTInsertQuery & /*query*/,
|
|
|
|
ContextPtr /*context*/)
|
|
|
|
{
|
|
|
|
return {};
|
|
|
|
}
|
|
|
|
|
2020-08-03 11:33:11 +00:00
|
|
|
Pipe IStorage::alterPartition(
|
2021-04-10 23:33:54 +00:00
|
|
|
const StorageMetadataPtr & /* metadata_snapshot */, const PartitionCommands & /* commands */, ContextPtr /* context */)
|
2020-08-03 11:33:11 +00:00
|
|
|
{
|
2023-01-23 21:13:58 +00:00
|
|
|
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Partition operations are not supported by storage {}", getName());
|
2020-08-03 11:33:11 +00:00
|
|
|
}
|
|
|
|
|
2021-10-25 17:49:49 +00:00
|
|
|
void IStorage::alter(const AlterCommands & params, ContextPtr context, AlterLockHolder &)
|
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
|
|
|
|
2021-04-10 23:33:54 +00:00
|
|
|
void IStorage::checkAlterIsPossible(const AlterCommands & commands, ContextPtr /* context */) 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())
|
2021-09-06 14:24:03 +00:00
|
|
|
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Alter of type '{}' is not supported by storage {}",
|
|
|
|
command.type, getName());
|
2018-12-25 23:11:36 +00:00
|
|
|
}
|
|
|
|
}
|
2019-05-17 14:48:03 +00:00
|
|
|
|
2021-02-25 10:07:48 +00:00
|
|
|
void IStorage::checkMutationIsPossible(const MutationCommands & /*commands*/, const Settings & /*settings*/) const
|
|
|
|
{
|
2023-01-23 21:13:58 +00:00
|
|
|
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Table engine {} doesn't support mutations", getName());
|
2021-02-25 10:07:48 +00:00
|
|
|
}
|
|
|
|
|
2020-12-07 09:30:47 +00:00
|
|
|
void IStorage::checkAlterPartitionIsPossible(
|
2023-10-13 14:22:18 +00:00
|
|
|
const PartitionCommands & /*commands*/,
|
|
|
|
const StorageMetadataPtr & /*metadata_snapshot*/,
|
|
|
|
const Settings & /*settings*/,
|
|
|
|
ContextPtr /*context*/) const
|
2020-07-13 16:19:08 +00:00
|
|
|
{
|
2023-01-23 21:13:58 +00:00
|
|
|
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Table engine {} doesn't support partitioning", getName());
|
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
|
|
|
}
|
|
|
|
|
2020-12-07 09:30:47 +00:00
|
|
|
Names IStorage::getAllRegisteredNames() const
|
|
|
|
{
|
|
|
|
Names result;
|
|
|
|
auto getter = [](const auto & column) { return column.name; };
|
|
|
|
const NamesAndTypesList & available_columns = getInMemoryMetadata().getColumns().getAllPhysical();
|
|
|
|
std::transform(available_columns.begin(), available_columns.end(), std::back_inserter(result), getter);
|
|
|
|
return result;
|
|
|
|
}
|
|
|
|
|
2021-04-10 23:33:54 +00:00
|
|
|
NameDependencies IStorage::getDependentViewsByColumn(ContextPtr context) const
|
2021-02-28 05:24:39 +00:00
|
|
|
{
|
|
|
|
NameDependencies name_deps;
|
2022-12-02 14:05:46 +00:00
|
|
|
auto view_ids = DatabaseCatalog::instance().getDependentViews(storage_id);
|
|
|
|
for (const auto & view_id : view_ids)
|
2021-02-28 05:24:39 +00:00
|
|
|
{
|
2022-12-02 14:05:46 +00:00
|
|
|
auto view = DatabaseCatalog::instance().getTable(view_id, context);
|
|
|
|
if (view->getInMemoryMetadataPtr()->select.inner_query)
|
2021-02-28 07:42:08 +00:00
|
|
|
{
|
2022-12-02 14:05:46 +00:00
|
|
|
const auto & select_query = view->getInMemoryMetadataPtr()->select.inner_query;
|
2021-02-28 07:42:08 +00:00
|
|
|
auto required_columns = InterpreterSelectQuery(select_query, context, SelectQueryOptions{}.noModify()).getRequiredColumns();
|
|
|
|
for (const auto & col_name : required_columns)
|
2022-12-02 14:05:46 +00:00
|
|
|
name_deps[col_name].push_back(view_id.table_name);
|
2021-02-28 07:42:08 +00:00
|
|
|
}
|
2021-02-28 05:24:39 +00:00
|
|
|
}
|
|
|
|
return name_deps;
|
|
|
|
}
|
|
|
|
|
2021-09-04 09:02:07 +00:00
|
|
|
bool IStorage::isStaticStorage() const
|
2021-08-23 11:26:54 +00:00
|
|
|
{
|
|
|
|
auto storage_policy = getStoragePolicy();
|
|
|
|
if (storage_policy)
|
|
|
|
{
|
2021-08-24 07:30:33 +00:00
|
|
|
for (const auto & disk : storage_policy->getDisks())
|
2022-11-17 18:26:40 +00:00
|
|
|
if (!(disk->isReadOnly() || disk->isWriteOnce()))
|
2021-08-23 11:26:54 +00:00
|
|
|
return false;
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
|
2023-08-14 09:58:08 +00:00
|
|
|
IStorage::DataValidationTasksPtr IStorage::getCheckTaskList(const ASTPtr & /* query */, ContextPtr /* context */)
|
|
|
|
{
|
|
|
|
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Check query is not supported for {} storage", getName());
|
|
|
|
}
|
|
|
|
|
2023-10-23 10:12:30 +00:00
|
|
|
std::optional<CheckResult> IStorage::checkDataNext(DataValidationTasksPtr & /* check_task_list */)
|
2023-08-14 09:58:08 +00:00
|
|
|
{
|
|
|
|
return {};
|
|
|
|
}
|
|
|
|
|
2022-06-24 19:29:38 +00:00
|
|
|
void IStorage::adjustCreateQueryForBackup(ASTPtr &) const
|
2021-08-18 22:19:14 +00:00
|
|
|
{
|
2022-05-29 19:53:56 +00:00
|
|
|
}
|
|
|
|
|
2022-05-31 09:33:23 +00:00
|
|
|
void IStorage::backupData(BackupEntriesCollector &, const String &, const std::optional<ASTs> &)
|
2022-05-29 19:53:56 +00:00
|
|
|
{
|
2021-08-18 22:19:14 +00:00
|
|
|
}
|
|
|
|
|
2022-06-29 12:42:23 +00:00
|
|
|
void IStorage::restoreDataFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional<ASTs> &)
|
2021-08-18 22:19:14 +00:00
|
|
|
{
|
2022-06-29 12:42:23 +00:00
|
|
|
/// If an inherited class doesn't override restoreDataFromBackup() that means it doesn't backup any data.
|
|
|
|
auto filenames = restorer.getBackup()->listFiles(data_path_in_backup);
|
|
|
|
if (!filenames.empty())
|
|
|
|
throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE, "Cannot restore table {}: Folder {} in backup must be empty",
|
|
|
|
getStorageID().getFullTableName(), data_path_in_backup);
|
2021-08-18 22:19:14 +00:00
|
|
|
}
|
|
|
|
|
2021-06-25 14:49:28 +00:00
|
|
|
std::string PrewhereInfo::dump() const
|
2020-12-02 18:16:31 +00:00
|
|
|
{
|
2020-12-18 17:13:28 +00:00
|
|
|
WriteBufferFromOwnString ss;
|
2020-12-02 18:16:31 +00:00
|
|
|
ss << "PrewhereDagInfo\n";
|
|
|
|
|
2022-05-09 14:34:44 +00:00
|
|
|
if (row_level_filter)
|
|
|
|
{
|
|
|
|
ss << "row_level_filter " << row_level_filter->dumpDAG() << "\n";
|
|
|
|
}
|
|
|
|
|
2020-12-02 18:16:31 +00:00
|
|
|
if (prewhere_actions)
|
|
|
|
{
|
|
|
|
ss << "prewhere_actions " << prewhere_actions->dumpDAG() << "\n";
|
|
|
|
}
|
|
|
|
|
|
|
|
ss << "remove_prewhere_column " << remove_prewhere_column
|
|
|
|
<< ", need_filter " << need_filter << "\n";
|
|
|
|
|
|
|
|
return ss.str();
|
|
|
|
}
|
|
|
|
|
2021-02-13 22:07:13 +00:00
|
|
|
std::string FilterDAGInfo::dump() const
|
2020-12-02 18:16:31 +00:00
|
|
|
{
|
2020-12-18 17:13:28 +00:00
|
|
|
WriteBufferFromOwnString ss;
|
2021-02-13 22:07:13 +00:00
|
|
|
ss << "FilterDAGInfo for column '" << column_name <<"', do_remove_column "
|
2020-12-02 18:16:31 +00:00
|
|
|
<< do_remove_column << "\n";
|
2021-02-13 22:07:13 +00:00
|
|
|
if (actions)
|
2020-12-02 18:16:31 +00:00
|
|
|
{
|
2021-02-13 22:07:13 +00:00
|
|
|
ss << "actions " << actions->dumpDAG() << "\n";
|
2020-12-02 18:16:31 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
return ss.str();
|
2020-12-23 06:44:44 +00:00
|
|
|
}
|
2020-12-02 18:16:31 +00:00
|
|
|
|
2017-01-21 04:24:28 +00:00
|
|
|
}
|