mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
dbms: removed useless code [#METR-16739].
This commit is contained in:
parent
0b3c071eb9
commit
c94bd2fc09
@ -25,7 +25,7 @@ public:
|
||||
children.push_back(input_);
|
||||
}
|
||||
|
||||
String getName() const override { return "AddingConstColumnBlockInputStream"; }
|
||||
String getName() const override { return "AddingConstColumn"; }
|
||||
|
||||
String getID() const override
|
||||
{
|
||||
|
@ -35,7 +35,7 @@ public:
|
||||
{
|
||||
}
|
||||
|
||||
String getName() const override { return "AddingDefaultBlockInputStream"; }
|
||||
String getName() const override { return "AddingDefault"; }
|
||||
|
||||
String getID() const override
|
||||
{
|
||||
|
@ -32,7 +32,7 @@ public:
|
||||
children.push_back(input_);
|
||||
}
|
||||
|
||||
String getName() const override { return "AggregatingBlockInputStream"; }
|
||||
String getName() const override { return "Aggregating"; }
|
||||
|
||||
String getID() const override
|
||||
{
|
||||
|
@ -26,7 +26,7 @@ public:
|
||||
{
|
||||
}
|
||||
|
||||
String getName() const override { return "AggregatingSortedBlockInputStream"; }
|
||||
String getName() const override { return "AggregatingSorted"; }
|
||||
|
||||
String getID() const override
|
||||
{
|
||||
|
@ -25,7 +25,7 @@ public:
|
||||
children.push_back(in_);
|
||||
}
|
||||
|
||||
String getName() const override { return "AsynchronousBlockInputStream"; }
|
||||
String getName() const override { return "Asynchronous"; }
|
||||
|
||||
String getID() const override
|
||||
{
|
||||
|
@ -23,7 +23,7 @@ public:
|
||||
|
||||
~CollapsingFinalBlockInputStream();
|
||||
|
||||
String getName() const override { return "CollapsingFinalBlockInputStream"; }
|
||||
String getName() const override { return "CollapsingFinal"; }
|
||||
|
||||
String getID() const override
|
||||
{
|
||||
|
@ -30,7 +30,7 @@ public:
|
||||
{
|
||||
}
|
||||
|
||||
String getName() const override { return "CollapsingSortedBlockInputStream"; }
|
||||
String getName() const override { return "CollapsingSorted"; }
|
||||
|
||||
String getID() const override
|
||||
{
|
||||
|
@ -22,7 +22,7 @@ public:
|
||||
current_stream = children.begin();
|
||||
}
|
||||
|
||||
String getName() const override { return "ConcatBlockInputStream"; }
|
||||
String getName() const override { return "Concat"; }
|
||||
|
||||
String getID() const override
|
||||
{
|
||||
|
@ -28,7 +28,7 @@ public:
|
||||
children.push_back(input);
|
||||
}
|
||||
|
||||
String getName() const override { return "CreatingSetsBlockInputStream"; }
|
||||
String getName() const override { return "CreatingSets"; }
|
||||
|
||||
String getID() const override
|
||||
{
|
||||
|
@ -20,7 +20,7 @@ public:
|
||||
/// Пустой columns_ значит все столбцы.
|
||||
DistinctBlockInputStream(BlockInputStreamPtr input_, const Limits & limits, size_t limit_, Names columns_);
|
||||
|
||||
String getName() const override { return "DistinctBlockInputStream"; }
|
||||
String getName() const override { return "Distinct"; }
|
||||
|
||||
String getID() const override
|
||||
{
|
||||
|
@ -26,7 +26,7 @@ public:
|
||||
children.push_back(input_);
|
||||
}
|
||||
|
||||
String getName() const override { return "ExpressionBlockInputStream"; }
|
||||
String getName() const override { return "Expression"; }
|
||||
|
||||
String getID() const override
|
||||
{
|
||||
|
@ -22,7 +22,7 @@ public:
|
||||
FilterBlockInputStream(BlockInputStreamPtr input_, ssize_t filter_column_);
|
||||
FilterBlockInputStream(BlockInputStreamPtr input_, const String & filter_column_name_);
|
||||
|
||||
String getName() const override { return "FilterBlockInputStream"; }
|
||||
String getName() const override { return "Filter"; }
|
||||
|
||||
String getID() const override
|
||||
{
|
||||
|
@ -51,7 +51,6 @@ public:
|
||||
/** Для вывода дерева преобразований потока данных (плана выполнения запроса).
|
||||
*/
|
||||
virtual String getName() const = 0;
|
||||
virtual String getShortName() const; /// То же самое, но без BlockInputStream на конце.
|
||||
|
||||
/** Уникальный идентификатор части конвейера выполнения запроса.
|
||||
* Источники с одинаковым идентификатором считаются идентичными
|
||||
|
@ -18,7 +18,7 @@ public:
|
||||
LazyBlockInputStream(Generator generator_)
|
||||
: generator(generator_) {}
|
||||
|
||||
String getName() const override { return "LazyBlockInputStream"; }
|
||||
String getName() const override { return "Lazy"; }
|
||||
|
||||
String getID() const override
|
||||
{
|
||||
|
@ -18,7 +18,7 @@ class LimitBlockInputStream : public IProfilingBlockInputStream
|
||||
public:
|
||||
LimitBlockInputStream(BlockInputStreamPtr input_, size_t limit_, size_t offset_ = 0);
|
||||
|
||||
String getName() const override { return "LimitBlockInputStream"; }
|
||||
String getName() const override { return "Limit"; }
|
||||
|
||||
String getID() const override
|
||||
{
|
||||
|
@ -18,7 +18,7 @@ public:
|
||||
children.push_back(input_);
|
||||
}
|
||||
|
||||
String getName() const override { return "MaterializingBlockInputStream"; }
|
||||
String getName() const override { return "Materializing"; }
|
||||
|
||||
String getID() const override
|
||||
{
|
||||
|
@ -31,7 +31,7 @@ public:
|
||||
MergeSortingBlocksBlockInputStream(Blocks & blocks_, SortDescription & description_,
|
||||
size_t max_merged_block_size_, size_t limit_ = 0);
|
||||
|
||||
String getName() const override { return "MergeSortingBlocksBlockInputStream"; }
|
||||
String getName() const override { return "MergeSortingBlocks"; }
|
||||
String getID() const override { return getName(); }
|
||||
|
||||
protected:
|
||||
@ -73,7 +73,7 @@ public:
|
||||
children.push_back(input_);
|
||||
}
|
||||
|
||||
String getName() const override { return "MergeSortingBlockInputStream"; }
|
||||
String getName() const override { return "MergeSorting"; }
|
||||
|
||||
String getID() const override
|
||||
{
|
||||
|
@ -24,7 +24,7 @@ public:
|
||||
children.push_back(input_);
|
||||
}
|
||||
|
||||
String getName() const override { return "MergingAggregatedBlockInputStream"; }
|
||||
String getName() const override { return "MergingAggregated"; }
|
||||
|
||||
String getID() const override
|
||||
{
|
||||
|
@ -26,7 +26,7 @@ public:
|
||||
children.insert(children.end(), inputs_.begin(), inputs_.end());
|
||||
}
|
||||
|
||||
String getName() const override { return "MergingSortedBlockInputStream"; }
|
||||
String getName() const override { return "MergingSorted"; }
|
||||
|
||||
String getID() const override
|
||||
{
|
||||
|
@ -18,7 +18,7 @@ public:
|
||||
NativeBlockInputStream(ReadBuffer & istr_, UInt64 server_revision_ = 0)
|
||||
: istr(istr_), server_revision(server_revision_) {}
|
||||
|
||||
String getName() const override { return "NativeBlockInputStream"; }
|
||||
String getName() const override { return "Native"; }
|
||||
|
||||
String getID() const override
|
||||
{
|
||||
|
@ -22,7 +22,7 @@ public:
|
||||
children.push_back(input_);
|
||||
}
|
||||
|
||||
String getName() const override { return "NullAndDoCopyBlockInputStream"; }
|
||||
String getName() const override { return "NullAndDoCopy"; }
|
||||
|
||||
String getID() const override
|
||||
{
|
||||
|
@ -12,7 +12,7 @@ class NullBlockInputStream : public IBlockInputStream
|
||||
{
|
||||
public:
|
||||
Block read() override { return Block(); }
|
||||
String getName() const override { return "NullBlockInputStream"; }
|
||||
String getName() const override { return "Null"; }
|
||||
|
||||
String getID() const override
|
||||
{
|
||||
|
@ -18,7 +18,7 @@ class OneBlockInputStream : public IProfilingBlockInputStream
|
||||
public:
|
||||
OneBlockInputStream(const Block & block_) : block(block_) {}
|
||||
|
||||
String getName() const override { return "OneBlockInputStream"; }
|
||||
String getName() const override { return "One"; }
|
||||
|
||||
String getID() const override
|
||||
{
|
||||
|
@ -34,7 +34,7 @@ public:
|
||||
children.insert(children.end(), inputs.begin(), inputs.end());
|
||||
}
|
||||
|
||||
String getName() const override { return "ParallelAggregatingBlockInputStream"; }
|
||||
String getName() const override { return "ParallelAggregating"; }
|
||||
|
||||
String getID() const override
|
||||
{
|
||||
|
@ -21,7 +21,7 @@ public:
|
||||
children.push_back(input_);
|
||||
}
|
||||
|
||||
String getName() const override { return "PartialSortingBlockInputStream"; }
|
||||
String getName() const override { return "PartialSorting"; }
|
||||
|
||||
String getID() const override
|
||||
{
|
||||
|
@ -61,7 +61,7 @@ public:
|
||||
}
|
||||
|
||||
|
||||
String getName() const override { return "RemoteBlockInputStream"; }
|
||||
String getName() const override { return "Remote"; }
|
||||
|
||||
|
||||
String getID() const override
|
||||
|
@ -24,7 +24,7 @@ public:
|
||||
children.push_back(input_);
|
||||
}
|
||||
|
||||
String getName() const override { return "RemoveColumnsBlockInputStream"; }
|
||||
String getName() const override { return "RemoveColumns"; }
|
||||
|
||||
String getID() const override
|
||||
{
|
||||
|
@ -30,7 +30,7 @@ public:
|
||||
{
|
||||
}
|
||||
|
||||
String getName() const override { return "SummingSortedBlockInputStream"; }
|
||||
String getName() const override { return "SummingSorted"; }
|
||||
|
||||
String getID() const override
|
||||
{
|
||||
|
@ -28,7 +28,7 @@ public:
|
||||
children.push_back(input_);
|
||||
}
|
||||
|
||||
String getName() const override { return "TotalsHavingBlockInputStream"; }
|
||||
String getName() const override { return "TotalsHaving"; }
|
||||
|
||||
String getID() const override
|
||||
{
|
||||
|
@ -36,7 +36,7 @@ public:
|
||||
children = inputs;
|
||||
}
|
||||
|
||||
String getName() const override { return "UnionBlockInputStream"; }
|
||||
String getName() const override { return "Union"; }
|
||||
|
||||
String getID() const override
|
||||
{
|
||||
|
@ -90,7 +90,7 @@ public:
|
||||
}
|
||||
}
|
||||
|
||||
String getName() const override { return "MySQLBlockInputStream"; }
|
||||
String getName() const override { return "MySQL"; }
|
||||
|
||||
String getID() const override
|
||||
{
|
||||
|
@ -22,7 +22,7 @@ public:
|
||||
private:
|
||||
Block readImpl() override { return stream->read(); }
|
||||
|
||||
String getName() const override { return "OwningBufferBlockInputStream"; }
|
||||
String getName() const override { return "OwningBuffer"; }
|
||||
|
||||
String getID() const override { return "OwningBuffer(" + stream->getID() + ")"; }
|
||||
|
||||
|
@ -100,7 +100,7 @@ public:
|
||||
setTotalRowsApprox(total_rows);
|
||||
}
|
||||
|
||||
String getName() const override { return "MergeTreeBlockInputStream"; }
|
||||
String getName() const override { return "MergeTree"; }
|
||||
|
||||
String getID() const override
|
||||
{
|
||||
|
@ -57,9 +57,9 @@ size_t IBlockInputStream::checkDepthImpl(size_t max_depth, size_t level) const
|
||||
void IBlockInputStream::dumpTree(std::ostream & ostr, size_t indent, size_t multiplier)
|
||||
{
|
||||
/// Не будем отображать в дереве обёртку потока блоков в AsynchronousBlockInputStream.
|
||||
if (getShortName() != "Asynchronous")
|
||||
if (getName() != "Asynchronous")
|
||||
{
|
||||
ostr << String(indent, ' ') << getShortName();
|
||||
ostr << String(indent, ' ') << getName();
|
||||
if (multiplier > 1)
|
||||
ostr << " × " << multiplier;
|
||||
ostr << std::endl;
|
||||
@ -91,15 +91,6 @@ void IBlockInputStream::dumpTree(std::ostream & ostr, size_t indent, size_t mult
|
||||
}
|
||||
|
||||
|
||||
String IBlockInputStream::getShortName() const
|
||||
{
|
||||
String res = getName();
|
||||
if (0 == strcmp(res.c_str() + res.size() - strlen("BlockInputStream"), "BlockInputStream"))
|
||||
res = res.substr(0, res.size() - strlen("BlockInputStream"));
|
||||
return res;
|
||||
}
|
||||
|
||||
|
||||
BlockInputStreams IBlockInputStream::getLeaves()
|
||||
{
|
||||
BlockInputStreams res;
|
||||
|
@ -19,7 +19,7 @@ Block IProfilingBlockInputStream::read()
|
||||
if (!info.started)
|
||||
{
|
||||
info.total_stopwatch.start();
|
||||
info.stream_name = getShortName();
|
||||
info.stream_name = getName();
|
||||
|
||||
for (const auto & child : children)
|
||||
if (const IProfilingBlockInputStream * p_child = dynamic_cast<const IProfilingBlockInputStream *>(&*child))
|
||||
@ -43,7 +43,7 @@ Block IProfilingBlockInputStream::read()
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(mutex);
|
||||
|
||||
std::cerr << std::endl;
|
||||
std::cerr << "[ " << Poco::ThreadNumber::get() << " ]\t" << getShortName() << std::endl;
|
||||
std::cerr << "[ " << Poco::ThreadNumber::get() << " ]\t" << getName() << std::endl;
|
||||
std::cerr << "[ " << Poco::ThreadNumber::get() << " ]\t";
|
||||
|
||||
for (size_t i = 0; i < res.columns(); ++i)
|
||||
|
@ -723,7 +723,7 @@ public:
|
||||
{
|
||||
}
|
||||
|
||||
String getName() const override { return "NonJoinedBlockInputStream"; }
|
||||
String getName() const override { return "NonJoined"; }
|
||||
|
||||
String getID() const override
|
||||
{
|
||||
|
@ -43,7 +43,7 @@ public:
|
||||
BufferBlockInputStream(const Names & column_names_, StorageBuffer::Buffer & buffer_)
|
||||
: column_names(column_names_.begin(), column_names_.end()), buffer(buffer_) {}
|
||||
|
||||
String getName() const { return "BufferBlockInputStream"; }
|
||||
String getName() const { return "Buffer"; }
|
||||
|
||||
String getID() const
|
||||
{
|
||||
|
@ -46,7 +46,7 @@ public:
|
||||
: block_size(block_size_), column_names(column_names_), storage(storage_),
|
||||
mark_number(mark_number_), rows_limit(rows_limit_), current_mark(mark_number_), max_read_buffer_size(max_read_buffer_size_) {}
|
||||
|
||||
String getName() const { return "LogBlockInputStream"; }
|
||||
String getName() const { return "Log"; }
|
||||
|
||||
String getID() const
|
||||
{
|
||||
|
@ -20,7 +20,7 @@ public:
|
||||
MemoryBlockInputStream(const Names & column_names_, BlocksList::iterator begin_, BlocksList::iterator end_)
|
||||
: column_names(column_names_), begin(begin_), end(end_), it(begin) {}
|
||||
|
||||
String getName() const { return "MemoryBlockInputStream"; }
|
||||
String getName() const { return "Memory"; }
|
||||
|
||||
String getID() const
|
||||
{
|
||||
|
@ -19,7 +19,7 @@ public:
|
||||
NumbersBlockInputStream(size_t block_size_, size_t offset_, size_t step_)
|
||||
: block_size(block_size_), next(offset_), step(step_) {}
|
||||
|
||||
String getName() const { return "NumbersBlockInputStream"; }
|
||||
String getName() const { return "Numbers"; }
|
||||
String getID() const { return "Numbers"; }
|
||||
|
||||
protected:
|
||||
|
@ -42,7 +42,7 @@ public:
|
||||
TinyLogBlockInputStream(size_t block_size_, const Names & column_names_, StorageTinyLog & storage_, size_t max_read_buffer_size_)
|
||||
: block_size(block_size_), column_names(column_names_), storage(storage_), max_read_buffer_size(max_read_buffer_size_) {}
|
||||
|
||||
String getName() const { return "TinyLogBlockInputStream"; }
|
||||
String getName() const { return "TinyLog"; }
|
||||
|
||||
String getID() const;
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user