mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-30 03:22:14 +00:00
Do not change the snapshot after at reading from MT
This commit is contained in:
parent
4222ac528d
commit
5ee9b56040
@ -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();
|
||||
|
@ -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,
|
||||
|
@ -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
|
||||
|
@ -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:
|
||||
|
@ -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())
|
||||
|
@ -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;
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user