Merge pull request #36799 from azat/cleanup

Tiny cleanup
This commit is contained in:
Alexey Milovidov 2022-05-01 14:23:13 +03:00 committed by GitHub
commit b034146ba4
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
8 changed files with 6 additions and 36 deletions

View File

@ -8,10 +8,9 @@ namespace ErrorCodes
extern const int SIZES_OF_COLUMNS_DOESNT_MATCH;
}
SquashingTransform::SquashingTransform(size_t min_block_size_rows_, size_t min_block_size_bytes_, bool reserve_memory_)
SquashingTransform::SquashingTransform(size_t min_block_size_rows_, size_t min_block_size_bytes_)
: min_block_size_rows(min_block_size_rows_)
, min_block_size_bytes(min_block_size_bytes_)
, reserve_memory(reserve_memory_)
{
}
@ -95,13 +94,7 @@ void SquashingTransform::append(ReferenceType input_block)
const auto source_column = input_block.getByPosition(i).column;
auto mutable_column = IColumn::mutate(std::move(accumulated_block.getByPosition(i).column));
if (reserve_memory)
{
mutable_column->reserve(min_block_size_bytes);
}
mutable_column->insertRangeFrom(*source_column, 0, source_column->size());
accumulated_block.getByPosition(i).column = std::move(mutable_column);
}
}

View File

@ -23,7 +23,7 @@ class SquashingTransform
{
public:
/// Conditions on rows and bytes are OR-ed. If one of them is zero, then corresponding condition is ignored.
SquashingTransform(size_t min_block_size_rows_, size_t min_block_size_bytes_, bool reserve_memory_ = false);
SquashingTransform(size_t min_block_size_rows_, size_t min_block_size_bytes_);
/** Add next block and possibly returns squashed block.
* At end, you need to pass empty block. As the result for last (empty) block, you will get last Result with ready = true.
@ -34,7 +34,6 @@ public:
private:
size_t min_block_size_rows;
size_t min_block_size_bytes;
bool reserve_memory;
Block accumulated_block;

View File

@ -401,7 +401,6 @@ void ThreadStatus::detachQuery(bool exit_if_already_detached, bool thread_exits)
performance_counters.setParent(&ProfileEvents::global_counters);
memory_tracker.reset();
/// Must reset pointer to thread_group's memory_tracker, because it will be destroyed two lines below (will reset to its parent).
memory_tracker.setParent(thread_group->memory_tracker.getParent());
query_id.clear();

View File

@ -8,15 +8,6 @@
namespace DB
{
class AggregatedArenasChunkInfo : public ChunkInfo
{
public:
Arenas arenas;
explicit AggregatedArenasChunkInfo(Arenas arenas_)
: arenas(std::move(arenas_))
{}
};
class AggregatedChunkInfo : public ChunkInfo
{
public:

View File

@ -5,9 +5,9 @@ namespace DB
{
SquashingChunksTransform::SquashingChunksTransform(
const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes, bool reserve_memory)
const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes)
: ExceptionKeepingTransform(header, header, false)
, squashing(min_block_size_rows, min_block_size_bytes, reserve_memory)
, squashing(min_block_size_rows, min_block_size_bytes)
{
}

View File

@ -10,7 +10,7 @@ class SquashingChunksTransform : public ExceptionKeepingTransform
{
public:
explicit SquashingChunksTransform(
const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes, bool reserve_memory = false);
const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes);
String getName() const override { return "SquashingTransform"; }

View File

@ -426,19 +426,7 @@ static QueryPipeline process(Block block, ViewRuntimeData & view, const ViewsDat
std::move(block),
views_data.source_storage->getVirtuals()));
/// We need keep InterpreterSelectQuery, until the processing will be finished, since:
///
/// - We copy Context inside InterpreterSelectQuery to support
/// modification of context (Settings) for subqueries
/// - InterpreterSelectQuery lives shorter than query pipeline.
/// It's used just to build the query pipeline and no longer needed
/// - ExpressionAnalyzer and then, Functions, that created in InterpreterSelectQuery,
/// **can** take a reference to Context from InterpreterSelectQuery
/// (the problem raises only when function uses context from the
/// execute*() method, like FunctionDictGet do)
/// - These objects live inside query pipeline (DataStreams) and the reference become dangling.
InterpreterSelectQuery select(view.query, local_context, SelectQueryOptions());
auto pipeline = select.buildQueryPipeline();
pipeline.resize(1);

View File

@ -284,7 +284,7 @@ Block ProjectionDescription::calculate(const Block & block, ContextPtr context)
: QueryProcessingStage::WithMergeableState})
.buildQueryPipeline();
builder.resize(1);
builder.addTransform(std::make_shared<SquashingChunksTransform>(builder.getHeader(), block.rows(), 0));
builder.addTransform(std::make_shared<SquashingChunksTransform>(builder.getHeader(), block.rows(), block.bytes()));
auto pipeline = QueryPipelineBuilder::getPipeline(std::move(builder));
PullingPipelineExecutor executor(pipeline);