mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 16:12:01 +00:00
Fixing build.
This commit is contained in:
parent
fe3a954f00
commit
d50f073dbe
@ -124,21 +124,21 @@ static void correctNullabilityInplace(ColumnWithTypeAndName & column, bool nulla
|
||||
}
|
||||
}
|
||||
|
||||
static void filterBlock(Block & block, const IColumn::Filter & filter)
|
||||
{
|
||||
for (auto & elem : block)
|
||||
{
|
||||
if (elem.column->size() != filter.size())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Size of column {} doesn't match size of filter {}",
|
||||
elem.column->size(), filter.size());
|
||||
// static void filterBlock(Block & block, const IColumn::Filter & filter)
|
||||
// {
|
||||
// for (auto & elem : block)
|
||||
// {
|
||||
// if (elem.column->size() != filter.size())
|
||||
// throw Exception(ErrorCodes::LOGICAL_ERROR, "Size of column {} doesn't match size of filter {}",
|
||||
// elem.column->size(), filter.size());
|
||||
|
||||
if (elem.column->empty())
|
||||
{
|
||||
block.clear();
|
||||
return;
|
||||
}
|
||||
}
|
||||
}
|
||||
// if (elem.column->empty())
|
||||
// {
|
||||
// block.clear();
|
||||
// return;
|
||||
// }
|
||||
// }
|
||||
// }
|
||||
|
||||
HashJoin::HashJoin(
|
||||
std::shared_ptr<TableJoin> table_join_,
|
||||
@ -577,12 +577,12 @@ bool HashJoin::addBlockToJoin(ScatteredBlock & source_block, bool check_limits)
|
||||
if (shrink_blocks)
|
||||
block_to_save = block_to_save.shrinkToFit();
|
||||
|
||||
Block right_key_columns_for_filter;
|
||||
ScatteredBlock right_key_columns_for_filter;
|
||||
if (save_right_key_columns_for_filter)
|
||||
{
|
||||
right_key_columns_for_filter = prepareRightBlock(source_block, right_table_keys);
|
||||
right_key_columns_for_filter = filterColumnsPresentInSampleBlock(source_block, right_table_keys);
|
||||
if (shrink_blocks)
|
||||
right_key_columns_for_filter = right_key_columns_for_filter.shrinkToFit();
|
||||
right_key_columns_for_filter.shrinkToFit();
|
||||
|
||||
data->right_key_columns_for_filter.resize(table_join->getClauses().size());
|
||||
}
|
||||
@ -656,15 +656,17 @@ bool HashJoin::addBlockToJoin(ScatteredBlock & source_block, bool check_limits)
|
||||
|
||||
if (save_right_key_columns_for_filter)
|
||||
{
|
||||
if (null_map)
|
||||
right_key_columns_for_filter.filter(*null_map);
|
||||
|
||||
right_key_columns_for_filter.filterBySelector();
|
||||
|
||||
const auto & required_names = right_keys_for_fiter_per_clause[onexpr_idx];
|
||||
|
||||
Block right_keys_for_clause;
|
||||
for (const auto & name : required_names)
|
||||
right_keys_for_clause.insert(right_key_columns_for_filter.getByName(name));
|
||||
|
||||
if (null_map)
|
||||
filterBlock(right_keys_for_clause, *null_map);
|
||||
|
||||
data->right_key_columns_for_filter[onexpr_idx].emplace_back(right_keys_for_clause);
|
||||
}
|
||||
|
||||
|
@ -102,7 +102,7 @@ void DynamicJoinFilters::filterDynamicPartsByFilledJoin(const IJoin & join)
|
||||
const auto & clause = clauses[i];
|
||||
auto squashed = squashBlocks(clause.keys, blocks[i]);
|
||||
|
||||
std::cerr << "Right join data rows " << squashed.front().column->size() << std::endl;
|
||||
// std::cerr << "Right join data rows " << squashed.front().column->size() << std::endl;
|
||||
|
||||
auto set = std::make_shared<FutureSetFromTuple>(squashed, settings);
|
||||
clause.set->setData(std::move(set));
|
||||
|
@ -127,7 +127,7 @@ size_t MergeTreeDataSelectExecutor::getApproximateTotalRowsToRead(
|
||||
MarkRanges exact_ranges;
|
||||
for (const auto & part : parts)
|
||||
{
|
||||
MarkRanges part_ranges = markRangesFromPKRange(part, 0, part->index_granularity.getMarksCount(), metadata_snapshot, key_condition, {}, &exact_ranges, settings, log);
|
||||
MarkRanges part_ranges = markRangesFromPKRange(part, 0, part->index_granularity->getMarksCount(), metadata_snapshot, key_condition, {}, &exact_ranges, settings, log);
|
||||
for (const auto & range : part_ranges)
|
||||
rows_count += part->index_granularity->getRowsCountInRange(range);
|
||||
}
|
||||
@ -695,7 +695,7 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd
|
||||
CurrentMetrics::Increment metric(CurrentMetrics::FilteringMarksWithPrimaryKey);
|
||||
ranges.ranges = markRangesFromPKRange(
|
||||
part,
|
||||
0, part->index_granularity.getMarksCount(),
|
||||
0, part->index_granularity->getMarksCount(),
|
||||
metadata_snapshot,
|
||||
key_condition,
|
||||
part_offset_condition,
|
||||
|
Loading…
Reference in New Issue
Block a user