mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 08:02:02 +00:00
Fix
This commit is contained in:
parent
8a1e23dcc4
commit
b40a412086
@ -225,10 +225,9 @@ InterpreterSelectQuery::InterpreterSelectQuery(
|
||||
const ContextPtr & context_,
|
||||
const StoragePtr & storage_,
|
||||
const StorageMetadataPtr & metadata_snapshot_,
|
||||
const SelectQueryOptions & options_,
|
||||
PreparedSetsPtr prepared_sets_)
|
||||
const SelectQueryOptions & options_)
|
||||
: InterpreterSelectQuery(
|
||||
query_ptr_, context_, std::nullopt, storage_, options_.copy().noSubquery(), {}, metadata_snapshot_, prepared_sets_)
|
||||
query_ptr_, context_, std::nullopt, storage_, options_.copy().noSubquery(), {}, metadata_snapshot_)
|
||||
{
|
||||
}
|
||||
|
||||
@ -2337,8 +2336,7 @@ void collectFiltersForAnalysis(
|
||||
const ContextPtr & query_context,
|
||||
const StorageSnapshotPtr & storage_snapshot,
|
||||
const SelectQueryOptions & options,
|
||||
SelectQueryInfo & query_info,
|
||||
PreparedSetsPtr prepared_sets)
|
||||
SelectQueryInfo & query_info)
|
||||
{
|
||||
const auto & storage = storage_snapshot->storage;
|
||||
bool collect_filters = typeid_cast<const StorageMerge *>(&storage);
|
||||
@ -2352,8 +2350,7 @@ void collectFiltersForAnalysis(
|
||||
storage_snapshot->storage.getStorageID(), ColumnsDescription(storage_snapshot->getColumns(get_column_options)), storage_snapshot);
|
||||
|
||||
QueryPlan query_plan;
|
||||
InterpreterSelectQuery(query_ptr, query_context, dummy, dummy->getInMemoryMetadataPtr(), options, prepared_sets)
|
||||
.buildQueryPlan(query_plan);
|
||||
InterpreterSelectQuery(query_ptr, query_context, dummy, dummy->getInMemoryMetadataPtr(), options).buildQueryPlan(query_plan);
|
||||
|
||||
auto optimization_settings = QueryPlanOptimizationSettings::fromContext(query_context);
|
||||
query_plan.optimize(optimization_settings);
|
||||
@ -2502,7 +2499,7 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc
|
||||
}
|
||||
else if (storage)
|
||||
{
|
||||
collectFiltersForAnalysis(query_ptr, context, storage_snapshot, options, query_info, prepared_sets);
|
||||
collectFiltersForAnalysis(query_ptr, context, storage_snapshot, options, query_info);
|
||||
|
||||
/// Table.
|
||||
if (max_streams == 0)
|
||||
|
@ -81,8 +81,7 @@ public:
|
||||
const ContextPtr & context_,
|
||||
const StoragePtr & storage_,
|
||||
const StorageMetadataPtr & metadata_snapshot_ = nullptr,
|
||||
const SelectQueryOptions & = {},
|
||||
PreparedSetsPtr prepared_sets_ = nullptr);
|
||||
const SelectQueryOptions & = {});
|
||||
|
||||
/// Reuse existing prepared_sets for another pass of analysis. It's used for projection.
|
||||
/// TODO: Find a general way of sharing sets among different interpreters, such as subqueries.
|
||||
|
@ -15,7 +15,7 @@ struct StorageInMemoryMetadata;
|
||||
using StorageMetadataPtr = std::shared_ptr<const StorageInMemoryMetadata>;
|
||||
|
||||
/// Optimizer that tries to replace columns to equal columns (according to constraints)
|
||||
/// with lower size (accorsing to compressed and uncomressed size).
|
||||
/// with lower size (according to compressed and uncomressed size).
|
||||
class SubstituteColumnOptimizer
|
||||
{
|
||||
public:
|
||||
|
@ -72,16 +72,6 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes &)
|
||||
if (!optimize_move_to_prewhere)
|
||||
return;
|
||||
|
||||
ColumnsWithTypeAndName required_columns_after_filter;
|
||||
if (source_step_with_filter->isQueryWithSampling())
|
||||
{
|
||||
const auto & sampling_key = storage_snapshot->getMetadataForQuery()->getSamplingKey();
|
||||
const auto & sampling_source_columns = sampling_key.expression->getRequiredColumnsWithTypes();
|
||||
for (const auto & column : sampling_source_columns)
|
||||
required_columns_after_filter.push_back(ColumnWithTypeAndName(column.type, column.name));
|
||||
const auto & sampling_result_columns = sampling_key.sample_block.getColumnsWithTypeAndName();
|
||||
required_columns_after_filter.insert(required_columns_after_filter.end(), sampling_result_columns.begin(), sampling_result_columns.end());
|
||||
}
|
||||
|
||||
const auto & storage_metadata = storage_snapshot->metadata;
|
||||
auto column_sizes = storage.getColumnSizes();
|
||||
|
@ -1766,6 +1766,18 @@ ReadFromMergeTree::AnalysisResult ReadFromMergeTree::getAnalysisResult() const
|
||||
return *result_ptr;
|
||||
}
|
||||
|
||||
bool ReadFromMergeTree::isQueryWithSampling() const
|
||||
{
|
||||
if (context->getSettingsRef().parallel_replicas_count > 1 && data.supportsSampling())
|
||||
return true;
|
||||
|
||||
const auto & select = query_info.query->as<ASTSelectQuery &>();
|
||||
if (query_info.table_expression_modifiers)
|
||||
return query_info.table_expression_modifiers->getSampleSizeRatio() != std::nullopt;
|
||||
else
|
||||
return select.sampleSize() != nullptr;
|
||||
}
|
||||
|
||||
Pipe ReadFromMergeTree::spreadMarkRanges(
|
||||
RangesInDataParts && parts_with_ranges, size_t num_streams, AnalysisResult & result, ActionsDAGPtr & result_projection)
|
||||
{
|
||||
|
@ -180,6 +180,7 @@ public:
|
||||
bool readsInOrder() const;
|
||||
|
||||
void updatePrewhereInfo(const PrewhereInfoPtr & prewhere_info_value) override;
|
||||
bool isQueryWithSampling() const;
|
||||
|
||||
/// Returns true if the optimization is applicable (and applies it then).
|
||||
bool requestOutputEachPartitionThroughSeparatePort();
|
||||
|
@ -71,18 +71,6 @@ Block SourceStepWithFilter::applyPrewhereActions(Block block, const PrewhereInfo
|
||||
return block;
|
||||
}
|
||||
|
||||
bool SourceStepWithFilter::isQueryWithSampling() const
|
||||
{
|
||||
if (context->getSettingsRef().parallel_replicas_count > 1 && storage_snapshot->storage.supportsSampling())
|
||||
return true;
|
||||
|
||||
const auto & select = query_info.query->as<ASTSelectQuery &>();
|
||||
if (query_info.table_expression_modifiers)
|
||||
return query_info.table_expression_modifiers->getSampleSizeRatio() != std::nullopt;
|
||||
else
|
||||
return select.sampleSize() != nullptr;
|
||||
}
|
||||
|
||||
void SourceStepWithFilter::updatePrewhereInfo(const PrewhereInfoPtr & prewhere_info_value)
|
||||
{
|
||||
query_info.prewhere_info = prewhere_info_value;
|
||||
|
@ -41,7 +41,6 @@ public:
|
||||
const StorageSnapshotPtr & getStorageSnapshot() const { return storage_snapshot; }
|
||||
|
||||
bool isQueryWithFinal() const { return query_info.isFinal(); }
|
||||
bool isQueryWithSampling() const;
|
||||
|
||||
const Names & requiredSourceColumns() const { return required_source_columns; }
|
||||
|
||||
|
@ -1,6 +1,9 @@
|
||||
DROP TABLE IF EXISTS constraint_test_assumption;
|
||||
DROP TABLE IF EXISTS constraint_test_transitivity;
|
||||
DROP TABLE IF EXISTS constraint_test_transitivity2;
|
||||
DROP TABLE IF EXISTS constraint_test_transitivity3;
|
||||
DROP TABLE IF EXISTS constraint_test_constants_repl;
|
||||
DROP TABLE IF EXISTS constraint_test_constants;
|
||||
|
||||
SET convert_query_to_cnf = 1;
|
||||
SET optimize_using_constraints = 1;
|
||||
@ -37,6 +40,7 @@ SELECT count() FROM constraint_test_transitivity WHERE a = d; ---> assumption ->
|
||||
|
||||
DROP TABLE constraint_test_transitivity;
|
||||
|
||||
DROP TABLE IF EXISTS constraint_test_transitivity;
|
||||
|
||||
CREATE TABLE constraint_test_strong_connectivity (a String, b String, c String, d String, CONSTRAINT c1 ASSUME a <= b AND b <= c AND c <= d AND d <= a) ENGINE = TinyLog;
|
||||
|
||||
@ -71,7 +75,6 @@ SELECT count() FROM constraint_test_transitivity3 WHERE b >= a; ---> assumption
|
||||
|
||||
DROP TABLE constraint_test_transitivity3;
|
||||
|
||||
|
||||
CREATE TABLE constraint_test_constants_repl (a Int64, b Int64, c Int64, d Int64, CONSTRAINT c1 ASSUME a - b = 10 AND c + d = 20) ENGINE = TinyLog;
|
||||
|
||||
INSERT INTO constraint_test_constants_repl (a, b, c, d) VALUES (1, 2, 3, 4);
|
||||
|
Loading…
Reference in New Issue
Block a user