mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Added one more check
This commit is contained in:
parent
de5f559304
commit
8255547656
@ -136,6 +136,7 @@ Checks: '-*,
|
||||
clang-analyzer-core.uninitialized.CapturedBlockVariable,
|
||||
clang-analyzer-core.uninitialized.UndefReturn,
|
||||
clang-analyzer-cplusplus.InnerPointer,
|
||||
clang-analyzer-cplusplus.NewDelete,
|
||||
clang-analyzer-cplusplus.NewDeleteLeaks,
|
||||
clang-analyzer-cplusplus.PlacementNewChecker,
|
||||
clang-analyzer-cplusplus.SelfAssignment,
|
||||
@ -161,6 +162,3 @@ Checks: '-*,
|
||||
boost-use-to-string,
|
||||
'
|
||||
WarningsAsErrors: '*'
|
||||
|
||||
# TODO:
|
||||
# clang-analyzer-cplusplus.NewDelete,
|
||||
|
@ -10,17 +10,7 @@ namespace DB
|
||||
|
||||
CollapsingFinalBlockInputStream::~CollapsingFinalBlockInputStream()
|
||||
{
|
||||
/// You must cancel all `MergingBlockPtr` so that they do not try to put blocks in `output_blocks`.
|
||||
previous.block.cancel();
|
||||
last_positive.block.cancel();
|
||||
|
||||
while (!queue.empty())
|
||||
{
|
||||
Cursor c = queue.top();
|
||||
queue.pop();
|
||||
c.block.cancel();
|
||||
}
|
||||
|
||||
queue = {};
|
||||
for (auto & block : output_blocks)
|
||||
delete block;
|
||||
}
|
||||
|
@ -1,4 +1,5 @@
|
||||
#pragma once
|
||||
|
||||
#include <common/logger_useful.h>
|
||||
#include <DataStreams/IBlockInputStream.h>
|
||||
#include <Core/SortDescription.h>
|
||||
@ -6,6 +7,7 @@
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <queue>
|
||||
#include <cassert>
|
||||
#include <boost/intrusive_ptr.hpp>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -15,6 +17,7 @@ namespace ErrorCodes
|
||||
extern const int BAD_TYPE_OF_FIELD;
|
||||
}
|
||||
|
||||
|
||||
/// Collapses the same rows with the opposite sign roughly like CollapsingSortedBlockInputStream.
|
||||
/// Outputs the rows in random order (the input streams must still be ordered).
|
||||
/// Outputs only rows with a positive sign.
|
||||
@ -39,10 +42,7 @@ public:
|
||||
|
||||
Block getHeader() const override { return children.at(0)->getHeader(); }
|
||||
|
||||
protected:
|
||||
Block readImpl() override;
|
||||
|
||||
private:
|
||||
struct MergingBlock;
|
||||
using BlockPlainPtrs = std::vector<MergingBlock*>;
|
||||
|
||||
@ -97,76 +97,11 @@ private:
|
||||
BlockPlainPtrs * output_blocks;
|
||||
};
|
||||
|
||||
private:
|
||||
Block readImpl() override;
|
||||
|
||||
/// When deleting the last block reference, adds a block to `output_blocks`.
|
||||
class MergingBlockPtr
|
||||
{
|
||||
public:
|
||||
MergingBlockPtr() : ptr() {}
|
||||
|
||||
explicit MergingBlockPtr(MergingBlock * ptr_) : ptr(ptr_)
|
||||
{
|
||||
if (ptr)
|
||||
++ptr->refcount;
|
||||
}
|
||||
|
||||
MergingBlockPtr(const MergingBlockPtr & rhs) : ptr(rhs.ptr)
|
||||
{
|
||||
if (ptr)
|
||||
++ptr->refcount;
|
||||
}
|
||||
|
||||
MergingBlockPtr & operator=(const MergingBlockPtr & rhs)
|
||||
{
|
||||
assert(ptr != rhs.ptr);
|
||||
|
||||
destroy();
|
||||
ptr = rhs.ptr;
|
||||
if (ptr)
|
||||
++ptr->refcount;
|
||||
return *this;
|
||||
}
|
||||
|
||||
~MergingBlockPtr()
|
||||
{
|
||||
destroy();
|
||||
}
|
||||
|
||||
/// Zero the pointer and do not add a block to output_blocks.
|
||||
void cancel()
|
||||
{
|
||||
if (ptr)
|
||||
{
|
||||
--ptr->refcount;
|
||||
if (!ptr->refcount)
|
||||
delete ptr;
|
||||
ptr = nullptr;
|
||||
}
|
||||
}
|
||||
|
||||
MergingBlock & operator*() const { return *ptr; }
|
||||
MergingBlock * operator->() const { return ptr; }
|
||||
operator bool() const { return !!ptr; }
|
||||
bool operator!() const { return !ptr; }
|
||||
|
||||
private:
|
||||
MergingBlock * ptr;
|
||||
|
||||
void destroy()
|
||||
{
|
||||
if (ptr)
|
||||
{
|
||||
--ptr->refcount;
|
||||
if (!ptr->refcount)
|
||||
{
|
||||
if (std::uncaught_exceptions())
|
||||
delete ptr;
|
||||
else
|
||||
ptr->output_blocks->push_back(ptr);
|
||||
}
|
||||
ptr = nullptr;
|
||||
}
|
||||
}
|
||||
};
|
||||
using MergingBlockPtr = boost::intrusive_ptr<MergingBlock>;
|
||||
|
||||
struct Cursor
|
||||
{
|
||||
@ -261,4 +196,16 @@ private:
|
||||
void reportBadSign(Int8 sign);
|
||||
};
|
||||
|
||||
|
||||
inline void intrusive_ptr_add_ref(CollapsingFinalBlockInputStream::MergingBlock * ptr)
|
||||
{
|
||||
++ptr->refcount;
|
||||
}
|
||||
|
||||
inline void intrusive_ptr_release(CollapsingFinalBlockInputStream::MergingBlock * ptr)
|
||||
{
|
||||
if (0 == --ptr->refcount)
|
||||
ptr->output_blocks->push_back(ptr);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -2,6 +2,8 @@
|
||||
#include <iomanip>
|
||||
#include <sstream>
|
||||
|
||||
#include <common/logger_useful.h>
|
||||
|
||||
#include <DataStreams/MergingSortedBlockInputStream.h>
|
||||
#include <DataStreams/ColumnGathererStream.h>
|
||||
|
||||
@ -21,6 +23,7 @@ MergingSortedBlockInputStream::MergingSortedBlockInputStream(
|
||||
: description(description_), max_block_size(max_block_size_), limit(limit_), quiet(quiet_)
|
||||
, average_block_sizes(average_block_sizes_), source_blocks(inputs_.size())
|
||||
, cursors(inputs_.size()), out_row_sources_buf(out_row_sources_buf_)
|
||||
, log(&Logger::get("MergingSortedBlockInputStream"))
|
||||
{
|
||||
children.insert(children.end(), inputs_.begin(), inputs_.end());
|
||||
header = children.at(0)->getHeader();
|
||||
|
@ -1,8 +1,5 @@
|
||||
#pragma once
|
||||
|
||||
#include <boost/smart_ptr/intrusive_ptr.hpp>
|
||||
|
||||
#include <common/logger_useful.h>
|
||||
#include <Common/SharedBlockRowRef.h>
|
||||
|
||||
#include <Core/Row.h>
|
||||
@ -14,6 +11,9 @@
|
||||
#include <DataStreams/IBlockInputStream.h>
|
||||
|
||||
|
||||
namespace Poco { class Logger; }
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
@ -176,7 +176,7 @@ private:
|
||||
template <typename TSortingHeap>
|
||||
void merge(MutableColumns & merged_columns, TSortingHeap & queue);
|
||||
|
||||
Logger * log = &Logger::get("MergingSortedBlockInputStream");
|
||||
Poco::Logger * log;
|
||||
|
||||
/// Read is finished.
|
||||
bool finished = false;
|
||||
|
@ -129,7 +129,8 @@ SummingSortedBlockInputStream::SummingSortedBlockInputStream(
|
||||
/// List of columns to be summed. If empty, all numeric columns that are not in the description are taken.
|
||||
const Names & column_names_to_sum,
|
||||
size_t max_block_size_)
|
||||
: MergingSortedBlockInputStream(inputs_, description_, max_block_size_)
|
||||
: MergingSortedBlockInputStream(inputs_, description_, max_block_size_),
|
||||
log(&Logger::get("SummingSortedBlockInputStream"))
|
||||
{
|
||||
current_row.resize(num_columns);
|
||||
|
||||
|
@ -10,6 +10,8 @@
|
||||
#include <AggregateFunctions/AggregateFunctionFactory.h>
|
||||
|
||||
|
||||
namespace Poco { class Logger; }
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
@ -47,7 +49,7 @@ protected:
|
||||
Block readImpl() override;
|
||||
|
||||
private:
|
||||
Logger * log = &Logger::get("SummingSortedBlockInputStream");
|
||||
Poco::Logger * log;
|
||||
|
||||
/// Read up to the end.
|
||||
bool finished = false;
|
||||
|
@ -1,3 +1,5 @@
|
||||
#ifndef __clang_analyzer__ // It's too hard to analyze.
|
||||
|
||||
#include "GatherUtils.h"
|
||||
#include "Selectors.h"
|
||||
#include "Algorithms.h"
|
||||
@ -46,6 +48,9 @@ void concat(const std::vector<std::unique_ptr<IArraySource>> & sources, IArraySi
|
||||
throw Exception("Concat function should get at least 1 ArraySource", ErrorCodes::LOGICAL_ERROR);
|
||||
return ArrayConcat::select(*sources.front(), sink, sources);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
#endif
|
||||
|
@ -1,3 +1,5 @@
|
||||
#ifndef __clang_analyzer__ // It's too hard to analyze.
|
||||
|
||||
#include "GatherUtils.h"
|
||||
#include "Selectors.h"
|
||||
#include "Algorithms.h"
|
||||
@ -21,3 +23,5 @@ void resizeConstantSize(IArraySource & array_source, IValueSource & value_source
|
||||
ArrayResizeConstant::select(sink, array_source, value_source, size);
|
||||
}
|
||||
}
|
||||
|
||||
#endif
|
||||
|
@ -1,3 +1,5 @@
|
||||
#ifndef __clang_analyzer__ // It's too hard to analyze.
|
||||
|
||||
#include "GatherUtils.h"
|
||||
#include "Selectors.h"
|
||||
#include "Algorithms.h"
|
||||
@ -21,3 +23,5 @@ void resizeDynamicSize(IArraySource & array_source, IValueSource & value_source,
|
||||
ArrayResizeDynamic::select(sink, array_source, value_source, size_column);
|
||||
}
|
||||
}
|
||||
|
||||
#endif
|
||||
|
Loading…
Reference in New Issue
Block a user