Removed unused code [#CLICKHOUSE-2]

This commit is contained in:
Alexey Milovidov 2018-02-18 06:46:18 +03:00
parent 754a4296df
commit afe09d3138
86 changed files with 3 additions and 717 deletions

View File

@ -18,7 +18,7 @@ namespace DB
std::ostream & operator<<(std::ostream & stream, const IBlockInputStream & what) std::ostream & operator<<(std::ostream & stream, const IBlockInputStream & what)
{ {
stream << "IBlockInputStream(id = " << what.getID() << ", name = " << what.getName() << ")"; stream << "IBlockInputStream(name = " << what.getName() << ")";
//what.dumpTree(stream); // todo: set const //what.dumpTree(stream); // todo: set const
return stream; return stream;
} }

View File

@ -24,13 +24,6 @@ public:
String getName() const override { return "AddingConstColumn"; } String getName() const override { return "AddingConstColumn"; }
String getID() const override
{
std::stringstream res;
res << "AddingConstColumn(" << children.back()->getID() << ")";
return res.str();
}
Block getHeader() const override Block getHeader() const override
{ {
Block res = children.back()->getHeader(); Block res = children.back()->getHeader();

View File

@ -30,13 +30,6 @@ public:
String getName() const override { return "Aggregating"; } String getName() const override { return "Aggregating"; }
String getID() const override
{
std::stringstream res;
res << "Aggregating(" << children.back()->getID() << ", " << aggregator.getID() << ")";
return res.str();
}
Block getHeader() const override; Block getHeader() const override;
protected: protected:

View File

@ -28,23 +28,6 @@ public:
String getName() const override { return "AggregatingSorted"; } String getName() const override { return "AggregatingSorted"; }
String getID() const override
{
std::stringstream res;
res << "AggregatingSorted(inputs";
for (size_t i = 0; i < children.size(); ++i)
res << ", " << children[i]->getID();
res << ", description";
for (size_t i = 0; i < description.size(); ++i)
res << ", " << description[i].getID();
res << ")";
return res.str();
}
bool isGroupedOutput() const override { return true; } bool isGroupedOutput() const override { return true; }
bool isSortedOutput() const override { return true; } bool isSortedOutput() const override { return true; }

View File

@ -35,13 +35,6 @@ public:
String getName() const override { return "Asynchronous"; } String getName() const override { return "Asynchronous"; }
String getID() const override
{
std::stringstream res;
res << "Asynchronous(" << children.back()->getID() << ")";
return res.str();
}
void readPrefix() override void readPrefix() override
{ {
/// Do not call `readPrefix` on the child, so that the corresponding actions are performed in a separate thread. /// Do not call `readPrefix` on the child, so that the corresponding actions are performed in a separate thread.

View File

@ -24,13 +24,6 @@ public:
String getName() const override { return "BlockExtraInfoInput"; } String getName() const override { return "BlockExtraInfoInput"; }
String getID() const override
{
std::stringstream res;
res << "BlockExtraInfoInput(" << children.back()->getID() << ")";
return res.str();
}
Block getHeader() const override { return children.back()->getHeader(); } Block getHeader() const override { return children.back()->getHeader(); }
protected: protected:

View File

@ -29,13 +29,6 @@ public:
String getName() const override { return "BlockInputStreamFromRowInputStream"; } String getName() const override { return "BlockInputStreamFromRowInputStream"; }
String getID() const override
{
std::stringstream res;
res << this;
return res.str();
}
RowInputStreamPtr & getRowInput() { return row_input; } RowInputStreamPtr & getRowInput() { return row_input; }
Block getHeader() const override { return sample; } Block getHeader() const override { return sample; }

View File

@ -22,13 +22,6 @@ public:
String getName() const override { return "BlocksList"; } String getName() const override { return "BlocksList"; }
String getID() const override
{
std::stringstream res;
res << this;
return res.str();
}
protected: protected:
Block readImpl() override Block readImpl() override
{ {

View File

@ -20,11 +20,6 @@ String CastTypeBlockInputStream::getName() const
return "CastType"; return "CastType";
} }
String CastTypeBlockInputStream::getID() const
{
return "CastType(" + children.back()->getID() + ")";
}
Block CastTypeBlockInputStream::readImpl() Block CastTypeBlockInputStream::readImpl()
{ {
Block block = children.back()->read(); Block block = children.back()->read();

View File

@ -17,8 +17,6 @@ public:
String getName() const override; String getName() const override;
String getID() const override;
Block getHeader() const override { return ref_definition; } Block getHeader() const override { return ref_definition; }
protected: protected:

View File

@ -28,23 +28,6 @@ public:
String getName() const override { return "CollapsingFinal"; } String getName() const override { return "CollapsingFinal"; }
String getID() const override
{
std::stringstream res;
res << "CollapsingFinal(inputs";
for (size_t i = 0; i < children.size(); ++i)
res << ", " << children[i]->getID();
res << ", description";
for (size_t i = 0; i < description.size(); ++i)
res << ", " << description[i].getID();
res << ", sign_column, " << sign_column_name << ")";
return res.str();
}
bool isSortedOutput() const override { return true; } bool isSortedOutput() const override { return true; }
const SortDescription & getSortDescription() const override { return description; } const SortDescription & getSortDescription() const override { return description; }

View File

@ -33,23 +33,6 @@ public:
String getName() const override { return "CollapsingSorted"; } String getName() const override { return "CollapsingSorted"; }
String getID() const override
{
std::stringstream res;
res << "CollapsingSorted(inputs";
for (size_t i = 0; i < children.size(); ++i)
res << ", " << children[i]->getID();
res << ", description";
for (size_t i = 0; i < description.size(); ++i)
res << ", " << description[i].getID();
res << ", sign_column, " << sign_column << ")";
return res.str();
}
protected: protected:
/// Can return 1 more records than max_block_size. /// Can return 1 more records than max_block_size.
Block readImpl() override; Block readImpl() override;

View File

@ -30,19 +30,6 @@ ColumnGathererStream::ColumnGathererStream(
} }
String ColumnGathererStream::getID() const
{
std::stringstream res;
res << getName() << "(";
for (size_t i = 0; i < children.size(); ++i)
res << (i == 0 ? "" : ", " ) << children[i]->getID();
res << ")";
return res.str();
}
void ColumnGathererStream::init() void ColumnGathererStream::init()
{ {
sources.reserve(children.size()); sources.reserve(children.size());
@ -107,13 +94,13 @@ void ColumnGathererStream::fetchNewBlock(Source & source, size_t source_num)
} }
catch (Exception & e) catch (Exception & e)
{ {
e.addMessage("Cannot fetch required block. Stream " + children[source_num]->getID() + ", part " + toString(source_num)); e.addMessage("Cannot fetch required block. Stream " + children[source_num]->getName() + ", part " + toString(source_num));
throw; throw;
} }
if (0 == source.size) if (0 == source.size)
{ {
throw Exception("Fetched block is empty. Stream " + children[source_num]->getID() + ", part " + toString(source_num), throw Exception("Fetched block is empty. Stream " + children[source_num]->getName() + ", part " + toString(source_num),
ErrorCodes::RECEIVED_EMPTY_DATA); ErrorCodes::RECEIVED_EMPTY_DATA);
} }
} }

View File

@ -61,8 +61,6 @@ public:
String getName() const override { return "ColumnGatherer"; } String getName() const override { return "ColumnGatherer"; }
String getID() const override;
Block readImpl() override; Block readImpl() override;
void readSuffixImpl() override; void readSuffixImpl() override;

View File

@ -22,25 +22,6 @@ public:
String getName() const override { return "Concat"; } String getName() const override { return "Concat"; }
String getID() const override
{
std::stringstream res;
res << "Concat(";
Strings children_ids(children.size());
for (size_t i = 0; i < children.size(); ++i)
children_ids[i] = children[i]->getID();
/// Let's assume that the order of concatenation of blocks does not matter.
std::sort(children_ids.begin(), children_ids.end());
for (size_t i = 0; i < children_ids.size(); ++i)
res << (i == 0 ? "" : ", ") << children_ids[i];
res << ")";
return res.str();
}
Block getHeader() const override { return children.at(0)->getHeader(); } Block getHeader() const override { return children.at(0)->getHeader(); }
protected: protected:

View File

@ -35,25 +35,6 @@ public:
String getName() const override { return "CreatingSets"; } String getName() const override { return "CreatingSets"; }
String getID() const override
{
std::stringstream res;
res << "CreatingSets(";
Strings children_ids(children.size());
for (size_t i = 0; i < children.size(); ++i)
children_ids[i] = children[i]->getID();
/// Let's assume that the order of creating sets does not matter.
std::sort(children_ids.begin(), children_ids.end() - 1);
for (size_t i = 0; i < children_ids.size(); ++i)
res << (i == 0 ? "" : ", ") << children_ids[i];
res << ")";
return res.str();
}
Block getHeader() const override { return children.back()->getHeader(); } Block getHeader() const override { return children.back()->getHeader(); }
/// Takes `totals` only from the main source, not from subquery sources. /// Takes `totals` only from the main source, not from subquery sources.

View File

@ -18,13 +18,6 @@ DistinctBlockInputStream::DistinctBlockInputStream(const BlockInputStreamPtr & i
children.push_back(input); children.push_back(input);
} }
String DistinctBlockInputStream::getID() const
{
std::stringstream res;
res << "Distinct(" << children.back()->getID() << ")";
return res.str();
}
Block DistinctBlockInputStream::readImpl() Block DistinctBlockInputStream::readImpl()
{ {
/// Execute until end of stream or until /// Execute until end of stream or until

View File

@ -22,8 +22,6 @@ public:
String getName() const override { return "Distinct"; } String getName() const override { return "Distinct"; }
String getID() const override;
Block getHeader() const override { return children.at(0)->getHeader(); } Block getHeader() const override { return children.at(0)->getHeader(); }
protected: protected:

View File

@ -19,13 +19,6 @@ DistinctSortedBlockInputStream::DistinctSortedBlockInputStream(const BlockInputS
children.push_back(input); children.push_back(input);
} }
String DistinctSortedBlockInputStream::getID() const
{
std::stringstream res;
res << "DistinctSorted(" << children.back()->getID() << ")";
return res.str();
}
Block DistinctSortedBlockInputStream::readImpl() Block DistinctSortedBlockInputStream::readImpl()
{ {
/// Execute until end of stream or until /// Execute until end of stream or until

View File

@ -25,8 +25,6 @@ public:
String getName() const override { return "DistinctSorted"; } String getName() const override { return "DistinctSorted"; }
String getID() const override;
Block getHeader() const override { return children.at(0)->getHeader(); } Block getHeader() const override { return children.at(0)->getHeader(); }
protected: protected:

View File

@ -13,13 +13,6 @@ ExpressionBlockInputStream::ExpressionBlockInputStream(const BlockInputStreamPtr
String ExpressionBlockInputStream::getName() const { return "Expression"; } String ExpressionBlockInputStream::getName() const { return "Expression"; }
String ExpressionBlockInputStream::getID() const
{
std::stringstream res;
res << "Expression(" << children.back()->getID() << ", " << expression->getID() << ")";
return res.str();
}
const Block & ExpressionBlockInputStream::getTotals() const Block & ExpressionBlockInputStream::getTotals()
{ {
if (IProfilingBlockInputStream * child = dynamic_cast<IProfilingBlockInputStream *>(&*children.back())) if (IProfilingBlockInputStream * child = dynamic_cast<IProfilingBlockInputStream *>(&*children.back()))

View File

@ -22,7 +22,6 @@ public:
ExpressionBlockInputStream(const BlockInputStreamPtr & input, const ExpressionActionsPtr & expression_); ExpressionBlockInputStream(const BlockInputStreamPtr & input, const ExpressionActionsPtr & expression_);
String getName() const override; String getName() const override;
String getID() const override;
const Block & getTotals() override; const Block & getTotals() override;
Block getHeader() const override; Block getHeader() const override;

View File

@ -57,14 +57,6 @@ FilterBlockInputStream::FilterBlockInputStream(const BlockInputStreamPtr & input
String FilterBlockInputStream::getName() const { return "Filter"; } String FilterBlockInputStream::getName() const { return "Filter"; }
String FilterBlockInputStream::getID() const
{
std::stringstream res;
res << "Filter(" << children.back()->getID() << ", " << expression->getID() << ", " << filter_column << ")";
return res.str();
}
const Block & FilterBlockInputStream::getTotals() const Block & FilterBlockInputStream::getTotals()
{ {
if (IProfilingBlockInputStream * child = dynamic_cast<IProfilingBlockInputStream *>(&*children.back())) if (IProfilingBlockInputStream * child = dynamic_cast<IProfilingBlockInputStream *>(&*children.back()))

View File

@ -25,7 +25,6 @@ public:
FilterBlockInputStream(const BlockInputStreamPtr & input, const ExpressionActionsPtr & expression_, const String & filter_column_name_); FilterBlockInputStream(const BlockInputStreamPtr & input, const ExpressionActionsPtr & expression_, const String & filter_column_name_);
String getName() const override; String getName() const override;
String getID() const override;
const Block & getTotals() override; const Block & getTotals() override;
Block getHeader() const override; Block getHeader() const override;

View File

@ -3,18 +3,6 @@
namespace DB namespace DB
{ {
String FilterColumnsBlockInputStream::getID() const
{
std::stringstream res;
res << "FilterColumnsBlockInputStream(" << children.back()->getID();
for (const auto & it : columns_to_save)
res << ", " << it;
res << ")";
return res.str();
}
Block FilterColumnsBlockInputStream::getHeader() const Block FilterColumnsBlockInputStream::getHeader() const
{ {
Block block = children.back()->getHeader(); Block block = children.back()->getHeader();

View File

@ -24,8 +24,6 @@ public:
return "FilterColumnsBlockInputStream"; return "FilterColumnsBlockInputStream";
} }
String getID() const override;
Block getHeader() const override; Block getHeader() const override;
protected: protected:

View File

@ -135,23 +135,6 @@ public:
String getName() const override { return "GraphiteRollupSorted"; } String getName() const override { return "GraphiteRollupSorted"; }
String getID() const override
{
std::stringstream res;
res << "GraphiteRollupSorted(inputs";
for (size_t i = 0; i < children.size(); ++i)
res << ", " << children[i]->getID();
res << ", description";
for (size_t i = 0; i < description.size(); ++i)
res << ", " << description[i].getID();
res << ")";
return res.str();
}
~GraphiteRollupSortedBlockInputStream() ~GraphiteRollupSortedBlockInputStream()
{ {
if (aggregate_state_created) if (aggregate_state_created)

View File

@ -126,13 +126,5 @@ void IBlockInputStream::getLeavesImpl(BlockInputStreams & res, const BlockInputS
(*it)->getLeavesImpl(res, *it); (*it)->getLeavesImpl(res, *it);
} }
/// By default all instances is different streams
String IBlockInputStream::getID() const
{
std::stringstream res;
res << getName() << "(" << this << ")";
return res.str();
};
} }

View File

@ -82,14 +82,6 @@ public:
*/ */
virtual String getName() const = 0; virtual String getName() const = 0;
/** The unique identifier of the pipeline part of the query execution.
* Sources with the same identifier are considered identical
* (producing the same data), and can be replaced by one source
* if several queries are executed simultaneously.
* If the source can not be glued together with any other - return the object's address as an identifier.
*/
virtual String getID() const;
/// If this stream generates data in grouped by some keys, return true. /// If this stream generates data in grouped by some keys, return true.
virtual bool isGroupedOutput() const { return false; } virtual bool isGroupedOutput() const { return false; }
/// If this stream generates data in order by some keys, return true. /// If this stream generates data in order by some keys, return true.

View File

@ -26,7 +26,6 @@ public:
void readSuffixImpl() override { return res_stream->readSuffix(); } void readSuffixImpl() override { return res_stream->readSuffix(); }
String getName() const override { return "InputStreamFromASTInsertQuery"; } String getName() const override { return "InputStreamFromASTInsertQuery"; }
String getID() const override { return "InputStreamFromASTInsertQuery(" + toString(std::intptr_t(this)) + ")"; }
Block getHeader() const override { return res_stream->getHeader(); } Block getHeader() const override { return res_stream->getHeader(); }

View File

@ -21,13 +21,6 @@ public:
String getName() const override { return "Limit"; } String getName() const override { return "Limit"; }
String getID() const override
{
std::stringstream res;
res << "Limit(" << children.back()->getID() << ", " << limit << ", " << offset << ")";
return res.str();
}
Block getHeader() const override { return children.at(0)->getHeader(); } Block getHeader() const override { return children.at(0)->getHeader(); }
protected: protected:

View File

@ -15,13 +15,6 @@ String MaterializingBlockInputStream::getName() const
return "Materializing"; return "Materializing";
} }
String MaterializingBlockInputStream::getID() const
{
std::stringstream res;
res << "Materializing(" << children.back()->getID() << ")";
return res.str();
}
Block MaterializingBlockInputStream::getHeader() const Block MaterializingBlockInputStream::getHeader() const
{ {
return materializeBlock(children.back()->getHeader()); return materializeBlock(children.back()->getHeader());

View File

@ -12,7 +12,6 @@ class MaterializingBlockInputStream : public IProfilingBlockInputStream
public: public:
MaterializingBlockInputStream(const BlockInputStreamPtr & input); MaterializingBlockInputStream(const BlockInputStreamPtr & input);
String getName() const override; String getName() const override;
String getID() const override;
Block getHeader() const override; Block getHeader() const override;
protected: protected:

View File

@ -33,7 +33,6 @@ public:
size_t max_merged_block_size_, size_t limit_ = 0); size_t max_merged_block_size_, size_t limit_ = 0);
String getName() const override { return "MergeSortingBlocks"; } String getName() const override { return "MergeSortingBlocks"; }
String getID() const override { return getName(); }
bool isGroupedOutput() const override { return true; } bool isGroupedOutput() const override { return true; }
bool isSortedOutput() const override { return true; } bool isSortedOutput() const override { return true; }
@ -82,18 +81,6 @@ public:
String getName() const override { return "MergeSorting"; } String getName() const override { return "MergeSorting"; }
String getID() const override
{
std::stringstream res;
res << "MergeSorting(" << children.back()->getID();
for (size_t i = 0; i < description.size(); ++i)
res << ", " << description[i].getID();
res << ")";
return res.str();
}
bool isGroupedOutput() const override { return true; } bool isGroupedOutput() const override { return true; }
bool isSortedOutput() const override { return true; } bool isSortedOutput() const override { return true; }
const SortDescription & getSortDescription() const override { return description; } const SortDescription & getSortDescription() const override { return description; }

View File

@ -22,13 +22,6 @@ public:
String getName() const override { return "MergingAggregated"; } String getName() const override { return "MergingAggregated"; }
String getID() const override
{
std::stringstream res;
res << "MergingAggregated(" << children.back()->getID() << ", " << aggregator.getID() << ")";
return res.str();
}
Block getHeader() const override; Block getHeader() const override;
protected: protected:

View File

@ -96,17 +96,6 @@ Block MergingAggregatedMemoryEfficientBlockInputStream::getHeader() const
} }
String MergingAggregatedMemoryEfficientBlockInputStream::getID() const
{
std::stringstream res;
res << "MergingAggregatedMemoryEfficient(" << aggregator.getID();
for (size_t i = 0, size = children.size(); i < size; ++i)
res << ", " << children.back()->getID();
res << ")";
return res.str();
}
void MergingAggregatedMemoryEfficientBlockInputStream::readPrefix() void MergingAggregatedMemoryEfficientBlockInputStream::readPrefix()
{ {
start(); start();

View File

@ -67,8 +67,6 @@ public:
String getName() const override { return "MergingAggregatedMemoryEfficient"; } String getName() const override { return "MergingAggregatedMemoryEfficient"; }
String getID() const override;
/// Sends the request (initiates calculations) earlier than `read`. /// Sends the request (initiates calculations) earlier than `read`.
void readPrefix() override; void readPrefix() override;

View File

@ -24,28 +24,6 @@ MergingSortedBlockInputStream::MergingSortedBlockInputStream(
children.insert(children.end(), inputs_.begin(), inputs_.end()); children.insert(children.end(), inputs_.begin(), inputs_.end());
} }
String MergingSortedBlockInputStream::getID() const
{
std::stringstream res;
res << "MergingSorted(";
Strings children_ids(children.size());
for (size_t i = 0; i < children.size(); ++i)
children_ids[i] = children[i]->getID();
/// The order does not matter.
std::sort(children_ids.begin(), children_ids.end());
for (size_t i = 0; i < children_ids.size(); ++i)
res << (i == 0 ? "" : ", ") << children_ids[i];
for (size_t i = 0; i < description.size(); ++i)
res << ", " << description[i].getID();
res << ")";
return res.str();
}
void MergingSortedBlockInputStream::init(Block & header, MutableColumns & merged_columns) void MergingSortedBlockInputStream::init(Block & header, MutableColumns & merged_columns)
{ {
/// Read the first blocks, initialize the queue. /// Read the first blocks, initialize the queue.

View File

@ -70,8 +70,6 @@ public:
String getName() const override { return "MergingSorted"; } String getName() const override { return "MergingSorted"; }
String getID() const override;
bool isGroupedOutput() const override { return true; } bool isGroupedOutput() const override { return true; }
bool isSortedOutput() const override { return true; } bool isSortedOutput() const override { return true; }
const SortDescription & getSortDescription() const override { return description; } const SortDescription & getSortDescription() const override { return description; }

View File

@ -74,13 +74,6 @@ public:
String getName() const override { return "Native"; } String getName() const override { return "Native"; }
String getID() const override
{
std::stringstream res;
res << this;
return res.str();
}
static void readData(const IDataType & type, IColumn & column, ReadBuffer & istr, size_t rows, double avg_value_size_hint); static void readData(const IDataType & type, IColumn & column, ReadBuffer & istr, size_t rows, double avg_value_size_hint);
Block getHeader() const override; Block getHeader() const override;

View File

@ -28,13 +28,6 @@ public:
String getName() const override { return "NullAndDoCopy"; } String getName() const override { return "NullAndDoCopy"; }
String getID() const override
{
std::stringstream res;
res << "copy from " << input->getID();
return res.str();
}
Block getHeader() const override { return {}; } Block getHeader() const override { return {}; }
protected: protected:

View File

@ -23,13 +23,6 @@ NullableAdapterBlockInputStream::NullableAdapterBlockInputStream(
children.push_back(input); children.push_back(input);
} }
String NullableAdapterBlockInputStream::getID() const
{
std::stringstream res;
res << "NullableAdapterBlockInputStream(" << children.back()->getID() << ")";
return res.str();
}
Block NullableAdapterBlockInputStream::readImpl() Block NullableAdapterBlockInputStream::readImpl()
{ {
Block block = children.back()->read(); Block block = children.back()->read();

View File

@ -22,8 +22,6 @@ public:
String getName() const override { return "NullableAdapterBlockInputStream"; } String getName() const override { return "NullableAdapterBlockInputStream"; }
String getID() const override;
Block getHeader() const override { return header; } Block getHeader() const override { return header; }
protected: protected:

View File

@ -27,8 +27,6 @@ private:
String getName() const override { return "Owning"; } String getName() const override { return "Owning"; }
String getID() const override { return "Owning(" + stream->getID() + ")"; }
protected: protected:
BlockInputStreamPtr stream; BlockInputStreamPtr stream;
std::unique_ptr<OwnType> own; std::unique_ptr<OwnType> own;

View File

@ -35,26 +35,6 @@ Block ParallelAggregatingBlockInputStream::getHeader() const
} }
String ParallelAggregatingBlockInputStream::getID() const
{
std::stringstream res;
res << "ParallelAggregating(";
Strings children_ids(children.size());
for (size_t i = 0; i < children.size(); ++i)
children_ids[i] = children[i]->getID();
/// Order does not matter.
std::sort(children_ids.begin(), children_ids.end());
for (size_t i = 0; i < children_ids.size(); ++i)
res << (i == 0 ? "" : ", ") << children_ids[i];
res << ", " << aggregator.getID() << ")";
return res.str();
}
void ParallelAggregatingBlockInputStream::cancel() void ParallelAggregatingBlockInputStream::cancel()
{ {
bool old_val = false; bool old_val = false;

View File

@ -27,8 +27,6 @@ public:
String getName() const override { return "ParallelAggregating"; } String getName() const override { return "ParallelAggregating"; }
String getID() const override;
void cancel() override; void cancel() override;
Block getHeader() const override; Block getHeader() const override;

View File

@ -23,18 +23,6 @@ public:
String getName() const override { return "PartialSorting"; } String getName() const override { return "PartialSorting"; }
String getID() const override
{
std::stringstream res;
res << "PartialSorting(" << children.back()->getID();
for (size_t i = 0; i < description.size(); ++i)
res << ", " << description[i].getID();
res << ")";
return res.str();
}
bool isGroupedOutput() const override { return true; } bool isGroupedOutput() const override { return true; }
bool isSortedOutput() const override { return true; } bool isSortedOutput() const override { return true; }
const SortDescription & getSortDescription() const override { return description; } const SortDescription & getSortDescription() const override { return description; }

View File

@ -66,13 +66,6 @@ public:
String getName() const override { return "Remote"; } String getName() const override { return "Remote"; }
String getID() const override
{
std::stringstream res;
res << this;
return res.str();
}
BlockExtraInfo getBlockExtraInfo() const override BlockExtraInfo getBlockExtraInfo() const override
{ {
return multiplexed_connections->getBlockExtraInfo(); return multiplexed_connections->getBlockExtraInfo();

View File

@ -22,18 +22,6 @@ public:
String getName() const override { return "RemoveColumns"; } String getName() const override { return "RemoveColumns"; }
String getID() const override
{
std::stringstream res;
res << "RemoveColumns(" << children.back()->getID();
for (const auto & it : columns_to_remove)
res << ", " << it;
res << ")";
return res.str();
}
Block getHeader() const override Block getHeader() const override
{ {
Block res = children.back()->getHeader(); Block res = children.back()->getHeader();

View File

@ -24,23 +24,6 @@ public:
String getName() const override { return "ReplacingSorted"; } String getName() const override { return "ReplacingSorted"; }
String getID() const override
{
std::stringstream res;
res << "ReplacingSorted(inputs";
for (size_t i = 0; i < children.size(); ++i)
res << ", " << children[i]->getID();
res << ", description";
for (size_t i = 0; i < description.size(); ++i)
res << ", " << description[i].getID();
res << ", version_column, " << version_column << ")";
return res.str();
}
protected: protected:
/// Can return 1 more records than max_block_size. /// Can return 1 more records than max_block_size.
Block readImpl() override; Block readImpl() override;

View File

@ -16,13 +16,6 @@ public:
String getName() const override { return "Squashing"; } String getName() const override { return "Squashing"; }
String getID() const override
{
std::stringstream res;
res << "Squashing(" << children.at(0)->getID() << ")";
return res.str();
}
Block getHeader() const override { return children.at(0)->getHeader(); } Block getHeader() const override { return children.at(0)->getHeader(); }
protected: protected:

View File

@ -23,24 +23,6 @@ namespace ErrorCodes
} }
String SummingSortedBlockInputStream::getID() const
{
std::stringstream res;
res << "SummingSorted(inputs";
for (size_t i = 0; i < children.size(); ++i)
res << ", " << children[i]->getID();
res << ", description";
for (size_t i = 0; i < description.size(); ++i)
res << ", " << description[i].getID();
res << ")";
return res.str();
}
void SummingSortedBlockInputStream::insertCurrentRowIfNeeded(MutableColumns & merged_columns, bool force_insertion) void SummingSortedBlockInputStream::insertCurrentRowIfNeeded(MutableColumns & merged_columns, bool force_insertion)
{ {
for (auto & desc : columns_to_aggregate) for (auto & desc : columns_to_aggregate)

View File

@ -35,8 +35,6 @@ public:
String getName() const override { return "SummingSorted"; } String getName() const override { return "SummingSorted"; }
String getID() const override;
protected: protected:
/// Can return 1 more records than max_block_size. /// Can return 1 more records than max_block_size.
Block readImpl() override; Block readImpl() override;

View File

@ -30,15 +30,6 @@ TotalsHavingBlockInputStream::TotalsHavingBlockInputStream(
} }
String TotalsHavingBlockInputStream::getID() const
{
std::stringstream res;
res << "TotalsHavingBlockInputStream(" << children.back()->getID()
<< "," << filter_column_name << ")";
return res.str();
}
static void finalize(Block & block) static void finalize(Block & block)
{ {
for (size_t i = 0; i < block.columns(); ++i) for (size_t i = 0; i < block.columns(); ++i)

View File

@ -27,8 +27,6 @@ public:
String getName() const override { return "TotalsHaving"; } String getName() const override { return "TotalsHaving"; }
String getID() const override;
const Block & getTotals() override; const Block & getTotals() override;
Block getHeader() const override; Block getHeader() const override;

View File

@ -86,26 +86,6 @@ public:
String getName() const override { return "Union"; } String getName() const override { return "Union"; }
String getID() const override
{
std::stringstream res;
res << "Union(";
Strings children_ids(children.size());
for (size_t i = 0; i < children.size(); ++i)
children_ids[i] = children[i]->getID();
/// Order does not matter.
std::sort(children_ids.begin(), children_ids.end());
for (size_t i = 0; i < children_ids.size(); ++i)
res << (i == 0 ? "" : ", ") << children_ids[i];
res << ")";
return res.str();
}
~UnionBlockInputStream() override ~UnionBlockInputStream() override
{ {
try try

View File

@ -185,24 +185,6 @@ public:
String getName() const override { return "VersionedCollapsingSorted"; } String getName() const override { return "VersionedCollapsingSorted"; }
String getID() const override
{
std::stringstream res;
res << "VersionedCollapsingSortedBlockInputStream(inputs";
for (const auto & child : children)
res << ", " << child->getID();
res << ", description";
for (const auto & descr : description)
res << ", " << descr.getID();
res << ", sign_column, " << sign_column;
res << ", version_column, " << sign_column << ")";
return res.str();
}
protected: protected:
/// Can return 1 more records than max_block_size. /// Can return 1 more records than max_block_size.
Block readImpl() override; Block readImpl() override;

View File

@ -8,13 +8,6 @@ DictionaryBlockInputStreamBase::DictionaryBlockInputStreamBase(size_t rows_count
{ {
} }
String DictionaryBlockInputStreamBase::getID() const
{
std::stringstream ss;
ss << static_cast<const void*>(this);
return ss.str();
}
Block DictionaryBlockInputStreamBase::readImpl() Block DictionaryBlockInputStreamBase::readImpl()
{ {
if (next_row == rows_count) if (next_row == rows_count)

View File

@ -11,8 +11,6 @@ protected:
DictionaryBlockInputStreamBase(size_t rows_count, size_t max_block_size); DictionaryBlockInputStreamBase(size_t rows_count, size_t max_block_size);
String getID() const override;
virtual Block getBlock(size_t start, size_t length) const = 0; virtual Block getBlock(size_t start, size_t length) const = 0;
Block getHeader() const override; Block getHeader() const override;

View File

@ -120,7 +120,6 @@ private:
} }
String getName() const override { return "WithBackgroundThread"; } String getName() const override { return "WithBackgroundThread"; }
String getID() const override { return "WithBackgroundThread(" + stream->getID() + ")"; }
BlockInputStreamPtr stream; BlockInputStreamPtr stream;
std::unique_ptr<ShellCommand> command; std::unique_ptr<ShellCommand> command;

View File

@ -38,14 +38,6 @@ MongoDBBlockInputStream::MongoDBBlockInputStream(
MongoDBBlockInputStream::~MongoDBBlockInputStream() = default; MongoDBBlockInputStream::~MongoDBBlockInputStream() = default;
String MongoDBBlockInputStream::getID() const
{
std::ostringstream stream;
stream << cursor.get();
return "MongoDB(@" + stream.str() + ")";
}
namespace namespace
{ {
using ValueType = ExternalResultDescription::ValueType; using ValueType = ExternalResultDescription::ValueType;

View File

@ -32,8 +32,6 @@ public:
String getName() const override { return "MongoDB"; } String getName() const override { return "MongoDB"; }
String getID() const override;
Block getHeader() const override { return description.sample_block; }; Block getHeader() const override { return description.sample_block; };
private: private:

View File

@ -33,12 +33,6 @@ MySQLBlockInputStream::MySQLBlockInputStream(
} }
String MySQLBlockInputStream::getID() const
{
return "MySQL(" + query.str() + ")";
}
namespace namespace
{ {
using ValueType = ExternalResultDescription::ValueType; using ValueType = ExternalResultDescription::ValueType;

View File

@ -21,8 +21,6 @@ public:
String getName() const override { return "MySQL"; } String getName() const override { return "MySQL"; }
String getID() const override;
Block getHeader() const override { return description.sample_block; }; Block getHeader() const override { return description.sample_block; };
private: private:

View File

@ -38,12 +38,6 @@ ODBCBlockInputStream::ODBCBlockInputStream(
} }
String ODBCBlockInputStream::getID() const
{
return "ODBC(" + statement.toString() + ")";
}
namespace namespace
{ {
using ValueType = ExternalResultDescription::ValueType; using ValueType = ExternalResultDescription::ValueType;

View File

@ -27,8 +27,6 @@ public:
String getName() const override { return "ODBC"; } String getName() const override { return "ODBC"; }
String getID() const override;
Block getHeader() const override { return description.sample_block; }; Block getHeader() const override { return description.sample_block; };
private: private:

View File

@ -1588,13 +1588,6 @@ public:
String getName() const override { return "MergingAndConverting"; } String getName() const override { return "MergingAndConverting"; }
String getID() const override
{
std::stringstream res;
res << this;
return res.str();
}
Block getHeader() const override { return aggregator.getHeader(final); } Block getHeader() const override { return aggregator.getHeader(final); }
~MergingAndConvertingBlockInputStream() ~MergingAndConvertingBlockInputStream()
@ -2406,21 +2399,6 @@ void Aggregator::destroyAllAggregateStates(AggregatedDataVariants & result)
} }
String Aggregator::getID() const
{
std::stringstream res;
res << "keys";
for (size_t i = 0; i < params.keys.size(); ++i)
res << ", " << params.keys[i];
res << ", aggregates";
for (size_t i = 0; i < params.aggregates_size; ++i)
res << ", " << params.aggregates[i].column_name;
return res.str();
}
void Aggregator::setCancellationHook(const CancellationHook cancellation_hook) void Aggregator::setCancellationHook(const CancellationHook cancellation_hook)
{ {
isCancelled = cancellation_hook; isCancelled = cancellation_hook;

View File

@ -1090,9 +1090,6 @@ public:
*/ */
void setCancellationHook(const CancellationHook cancellation_hook); void setCancellationHook(const CancellationHook cancellation_hook);
/// For IBlockInputStream.
String getID() const;
/// For external aggregation. /// For external aggregation.
void writeToTemporaryFile(AggregatedDataVariants & data_variants); void writeToTemporaryFile(AggregatedDataVariants & data_variants);

View File

@ -964,11 +964,6 @@ public:
return "DDLQueryStatusInputSream"; return "DDLQueryStatusInputSream";
} }
String getID() const override
{
return "DDLQueryStatusInputSream(" + node_path + ")";
}
Block getHeader() const override { return sample; }; Block getHeader() const override { return sample; };
Block readImpl() override Block readImpl() override

View File

@ -910,45 +910,6 @@ void ExpressionActions::finalize(const Names & output_columns)
} }
std::string ExpressionActions::getID() const
{
std::stringstream ss;
for (size_t i = 0; i < actions.size(); ++i)
{
if (i)
ss << ", ";
if (actions[i].type == ExpressionAction::APPLY_FUNCTION)
ss << actions[i].result_name;
if (actions[i].type == ExpressionAction::ARRAY_JOIN)
{
ss << (actions[i].array_join_is_left ? "LEFT ARRAY JOIN" : "ARRAY JOIN") << "{";
for (NameSet::const_iterator it = actions[i].array_joined_columns.begin();
it != actions[i].array_joined_columns.end(); ++it)
{
if (it != actions[i].array_joined_columns.begin())
ss << ", ";
ss << *it;
}
ss << "}";
}
/// TODO JOIN
}
ss << ": {";
NamesAndTypesList output_columns = sample_block.getNamesAndTypesList();
for (NamesAndTypesList::const_iterator it = output_columns.begin(); it != output_columns.end(); ++it)
{
if (it != output_columns.begin())
ss << ", ";
ss << it->name;
}
ss << "}";
return ss.str();
}
std::string ExpressionActions::dumpActions() const std::string ExpressionActions::dumpActions() const
{ {
std::stringstream ss; std::stringstream ss;

View File

@ -194,8 +194,6 @@ public:
/// Obtain a sample block that contains the names and types of result columns. /// Obtain a sample block that contains the names and types of result columns.
const Block & getSampleBlock() const { return sample_block; } const Block & getSampleBlock() const { return sample_block; }
std::string getID() const;
std::string dumpActions() const; std::string dumpActions() const;
static std::string getSmallestColumn(const NamesAndTypesList & columns); static std::string getSmallestColumn(const NamesAndTypesList & columns);

View File

@ -118,11 +118,6 @@ public:
return "SynchronousQueryKiller"; return "SynchronousQueryKiller";
} }
String getID() const override
{
return "SynchronousQueryKiller_" + toString(intptr_t(this));
}
Block getHeader() const override { return res_sample_block; }; Block getHeader() const override { return res_sample_block; };
Block readImpl() override Block readImpl() override

View File

@ -1038,13 +1038,6 @@ public:
String getName() const override { return "NonJoined"; } String getName() const override { return "NonJoined"; }
String getID() const override
{
std::stringstream res;
res << "NonJoined(" << &parent << ")";
return res.str();
}
Block getHeader() const override { return result_sample_block; }; Block getHeader() const override { return result_sample_block; };

View File

@ -69,26 +69,6 @@ Block MergeTreeBlockInputStream::getHeader() const
} }
String MergeTreeBlockInputStream::getID() const
{
std::stringstream res;
res << "MergeTree(" << path << ", columns";
for (const NameAndTypePair & column : columns)
res << ", " << column.name;
if (prewhere_actions)
res << ", prewhere, " << prewhere_actions->getID();
res << ", marks";
for (size_t i = 0; i < all_mark_ranges.size(); ++i)
res << ", " << all_mark_ranges[i].begin << ", " << all_mark_ranges[i].end;
res << ")";
return res.str();
}
bool MergeTreeBlockInputStream::getNewTask() bool MergeTreeBlockInputStream::getNewTask()
try try
{ {

View File

@ -38,8 +38,6 @@ public:
String getName() const override { return "MergeTree"; } String getName() const override { return "MergeTree"; }
String getID() const override;
Block getHeader() const override; Block getHeader() const override;
/// Closes readers and unlock part locks /// Closes readers and unlock part locks

View File

@ -44,15 +44,6 @@ Block MergeTreeThreadBlockInputStream::getHeader() const
}; };
String MergeTreeThreadBlockInputStream::getID() const
{
std::stringstream res;
/// @todo print some meaningful information
res << static_cast<const void *>(this);
return res.str();
}
/// Requests read task from MergeTreeReadPool and signals whether it got one /// Requests read task from MergeTreeReadPool and signals whether it got one
bool MergeTreeThreadBlockInputStream::getNewTask() bool MergeTreeThreadBlockInputStream::getNewTask()
{ {

View File

@ -30,8 +30,6 @@ public:
String getName() const override { return "MergeTreeThread"; } String getName() const override { return "MergeTreeThread"; }
String getID() const override;
~MergeTreeThreadBlockInputStream() override; ~MergeTreeThreadBlockInputStream() override;
Block getHeader() const override; Block getHeader() const override;

View File

@ -76,18 +76,6 @@ public:
String getName() const override { return "Buffer"; } String getName() const override { return "Buffer"; }
String getID() const override
{
std::stringstream res;
res << "Buffer(" << &buffer;
for (const auto & name : column_names)
res << ", " << name;
res << ")";
return res.str();
}
Block getHeader() const override { return storage.getSampleBlockForColumns(column_names); }; Block getHeader() const override { return storage.getSampleBlockForColumns(column_names); };
protected: protected:

View File

@ -39,11 +39,6 @@ public:
return "CatBoostDatasetBlockInputStream"; return "CatBoostDatasetBlockInputStream";
} }
String getID() const override
{
return "CatBoostDataset(" + format_name + ", " + file_name + ")";
}
Block readImpl() override Block readImpl() override
{ {
return reader->read(); return reader->read();

View File

@ -144,18 +144,6 @@ public:
return storage.getName(); return storage.getName();
} }
String getID() const override
{
std::stringstream res_stream;
res_stream << "File(" << storage.format_name << ", ";
if (!storage.path.empty())
res_stream << storage.path;
else
res_stream << storage.table_fd;
res_stream << ")";
return res_stream.str();
}
Block readImpl() override Block readImpl() override
{ {
return reader->read(); return reader->read();

View File

@ -161,13 +161,6 @@ public:
return storage.getName(); return storage.getName();
} }
String getID() const override
{
std::stringstream res_stream;
res_stream << "Kafka(" << storage.topics.size() << ", " << storage.format_name << ")";
return res_stream.str();
}
Block readImpl() override Block readImpl() override
{ {
if (isCancelled()) if (isCancelled())

View File

@ -62,18 +62,6 @@ public:
String getName() const override { return "Log"; } String getName() const override { return "Log"; }
String getID() const override
{
std::stringstream res;
res << "Log(" << storage.getTableName() << ", " << &storage << ", " << mark_number << ", " << rows_limit;
for (const auto & name_type : columns)
res << ", " << name_type.name;
res << ")";
return res.str();
}
Block getHeader() const override Block getHeader() const override
{ {
Block res; Block res;

View File

@ -25,18 +25,6 @@ public:
String getName() const override { return "Memory"; } String getName() const override { return "Memory"; }
String getID() const override
{
std::stringstream res;
res << "Memory(" << &*begin << ", " << &*end;
for (const auto & name : column_names)
res << ", " << name;
res << ")";
return res.str();
}
Block getHeader() const override { return storage.getSampleBlockForColumns(column_names); } Block getHeader() const override { return storage.getSampleBlockForColumns(column_names); }
protected: protected:

View File

@ -59,13 +59,6 @@ public:
String getName() const override { return "StripeLog"; } String getName() const override { return "StripeLog"; }
String getID() const override
{
std::stringstream s;
s << this;
return s.str();
}
Block getHeader() const override Block getHeader() const override
{ {
if (index_begin == index_end) if (index_begin == index_end)

View File

@ -60,8 +60,6 @@ public:
String getName() const override { return "TinyLog"; } String getName() const override { return "TinyLog"; }
String getID() const override;
Block getHeader() const override Block getHeader() const override
{ {
Block res; Block res;
@ -154,19 +152,6 @@ private:
}; };
String TinyLogBlockInputStream::getID() const
{
std::stringstream res;
res << "TinyLog(" << storage.getTableName() << ", " << &storage;
for (const auto & name_type : columns)
res << ", " << name_type.name;
res << ")";
return res.str();
}
Block TinyLogBlockInputStream::readImpl() Block TinyLogBlockInputStream::readImpl()
{ {
Block res; Block res;

View File

@ -16,7 +16,6 @@ public:
: block_size(block_size_), next(offset_), step(step_) {} : block_size(block_size_), next(offset_), step(step_) {}
String getName() const override { return "Numbers"; } String getName() const override { return "Numbers"; }
String getID() const override { return "Numbers"; }
Block getHeader() const override Block getHeader() const override
{ {