Do not change the snapshot after at reading from MT

This commit is contained in:
Nikolai Kochetov 2023-08-22 13:50:51 +00:00
parent 4222ac528d
commit 5ee9b56040
6 changed files with 19 additions and 18 deletions

View File

@ -189,7 +189,8 @@ void collectFiltersForAnalysis(const QueryTreeNodePtr & query_tree, const Planne
const auto & query_context = planner_context->getQueryContext();
Planner planner(updated_query_tree, {});
SelectQueryOptions select_query_options;
Planner planner(updated_query_tree, select_query_options);
planner.buildQueryPlanIfNeeded();
auto & result_query_plan = planner.getQueryPlan();

View File

@ -1304,6 +1304,10 @@ QueryPlanStepPtr MergeTreeDataSelectExecutor::readFromParts(
selectColumnNames(column_names_to_return, data, real_column_names, virt_column_names, sample_factor_column_queried);
/// Do not keep data parts in snapshot.
/// They are stored separately, and some could be released after PK analysis.
auto storage_snapshot_copy = storage_snapshot->clone(std::make_unique<MergeTreeData::SnapshotData>());
return std::make_unique<ReadFromMergeTree>(
std::move(parts),
std::move(alter_conversions),
@ -1311,7 +1315,7 @@ QueryPlanStepPtr MergeTreeDataSelectExecutor::readFromParts(
virt_column_names,
data,
query_info,
storage_snapshot,
storage_snapshot_copy,
context,
max_block_size,
num_streams,

View File

@ -255,13 +255,6 @@ void StorageMergeTree::read(
processed_stage, nullptr, enable_parallel_reading))
query_plan = std::move(*plan);
}
/// Now, copy of parts that is required for the query, stored in the processors,
/// while snapshot_data.parts includes all parts, even one that had been filtered out with partition pruning,
/// reset them to avoid holding them.
auto & snapshot_data = assert_cast<MergeTreeData::SnapshotData &>(*storage_snapshot->data);
snapshot_data.parts = {};
snapshot_data.alter_conversions = {};
}
std::optional<UInt64> StorageMergeTree::totalRows(const Settings &) const

View File

@ -5103,15 +5103,6 @@ void StorageReplicatedMergeTree::read(
const size_t max_block_size,
const size_t num_streams)
{
SCOPE_EXIT({
/// Now, copy of parts that is required for the query, stored in the processors,
/// while snapshot_data.parts includes all parts, even one that had been filtered out with partition pruning,
/// reset them to avoid holding them.
auto & snapshot_data = assert_cast<MergeTreeData::SnapshotData &>(*storage_snapshot->data);
snapshot_data.parts = {};
snapshot_data.alter_conversions = {};
});
const auto & settings = local_context->getSettingsRef();
/// The `select_sequential_consistency` setting has two meanings:

View File

@ -17,6 +17,16 @@ namespace ErrorCodes
extern const int COLUMN_QUERIED_MORE_THAN_ONCE;
}
std::shared_ptr<StorageSnapshot> StorageSnapshot::clone(DataPtr data_) const
{
auto res = std::make_shared<StorageSnapshot>(storage, metadata, object_columns);
res->projection = projection;
res->data = std::move(data_);
return res;
}
void StorageSnapshot::init()
{
for (const auto & [name, type] : storage.getVirtuals())

View File

@ -60,6 +60,8 @@ struct StorageSnapshot
init();
}
std::shared_ptr<StorageSnapshot> clone(DataPtr data_) const;
/// Get all available columns with types according to options.
NamesAndTypesList getColumns(const GetColumnsOptions & options) const;