dbms: removed useless code [#METR-16739].

This commit is contained in:
Alexey Milovidov 2015-06-08 23:22:02 +03:00
parent 0b3c071eb9
commit c94bd2fc09
41 changed files with 43 additions and 53 deletions

View File

@ -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
{

View File

@ -35,7 +35,7 @@ public:
{
}
String getName() const override { return "AddingDefaultBlockInputStream"; }
String getName() const override { return "AddingDefault"; }
String getID() const override
{

View File

@ -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
{

View File

@ -26,7 +26,7 @@ public:
{
}
String getName() const override { return "AggregatingSortedBlockInputStream"; }
String getName() const override { return "AggregatingSorted"; }
String getID() const override
{

View File

@ -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
{

View File

@ -23,7 +23,7 @@ public:
~CollapsingFinalBlockInputStream();
String getName() const override { return "CollapsingFinalBlockInputStream"; }
String getName() const override { return "CollapsingFinal"; }
String getID() const override
{

View File

@ -30,7 +30,7 @@ public:
{
}
String getName() const override { return "CollapsingSortedBlockInputStream"; }
String getName() const override { return "CollapsingSorted"; }
String getID() const override
{

View File

@ -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
{

View File

@ -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
{

View File

@ -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
{

View File

@ -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
{

View File

@ -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
{

View File

@ -51,7 +51,6 @@ public:
/** Для вывода дерева преобразований потока данных (плана выполнения запроса).
*/
virtual String getName() const = 0;
virtual String getShortName() const; /// То же самое, но без BlockInputStream на конце.
/** Уникальный идентификатор части конвейера выполнения запроса.
* Источники с одинаковым идентификатором считаются идентичными

View File

@ -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
{

View File

@ -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
{

View File

@ -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
{

View File

@ -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
{

View File

@ -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
{

View File

@ -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
{

View File

@ -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
{

View File

@ -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
{

View File

@ -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
{

View File

@ -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
{

View File

@ -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
{

View File

@ -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
{

View File

@ -61,7 +61,7 @@ public:
}
String getName() const override { return "RemoteBlockInputStream"; }
String getName() const override { return "Remote"; }
String getID() const override

View File

@ -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
{

View File

@ -30,7 +30,7 @@ public:
{
}
String getName() const override { return "SummingSortedBlockInputStream"; }
String getName() const override { return "SummingSorted"; }
String getID() const override
{

View File

@ -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
{

View File

@ -36,7 +36,7 @@ public:
children = inputs;
}
String getName() const override { return "UnionBlockInputStream"; }
String getName() const override { return "Union"; }
String getID() const override
{

View File

@ -90,7 +90,7 @@ public:
}
}
String getName() const override { return "MySQLBlockInputStream"; }
String getName() const override { return "MySQL"; }
String getID() const override
{

View File

@ -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() + ")"; }

View File

@ -100,7 +100,7 @@ public:
setTotalRowsApprox(total_rows);
}
String getName() const override { return "MergeTreeBlockInputStream"; }
String getName() const override { return "MergeTree"; }
String getID() const override
{

View File

@ -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;

View File

@ -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)

View File

@ -723,7 +723,7 @@ public:
{
}
String getName() const override { return "NonJoinedBlockInputStream"; }
String getName() const override { return "NonJoined"; }
String getID() const override
{

View File

@ -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
{

View File

@ -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
{

View File

@ -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
{

View File

@ -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:

View File

@ -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;