提交 afe09d31 编写于 作者: A Alexey Milovidov

Removed unused code [#CLICKHOUSE-2]

上级 754a4296
......@@ -18,7 +18,7 @@ namespace DB
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
return stream;
}
......
......@@ -24,13 +24,6 @@ public:
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 res = children.back()->getHeader();
......
......@@ -30,13 +30,6 @@ public:
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;
protected:
......
......@@ -28,23 +28,6 @@ public:
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 isSortedOutput() const override { return true; }
......
......@@ -35,13 +35,6 @@ public:
String getName() const override { return "Asynchronous"; }
String getID() const override
{
std::stringstream res;
res << "Asynchronous(" << children.back()->getID() << ")";
return res.str();
}
void readPrefix() override
{
/// Do not call `readPrefix` on the child, so that the corresponding actions are performed in a separate thread.
......
......@@ -24,13 +24,6 @@ public:
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(); }
protected:
......
......@@ -29,13 +29,6 @@ public:
String getName() const override { return "BlockInputStreamFromRowInputStream"; }
String getID() const override
{
std::stringstream res;
res << this;
return res.str();
}
RowInputStreamPtr & getRowInput() { return row_input; }
Block getHeader() const override { return sample; }
......
......@@ -22,13 +22,6 @@ public:
String getName() const override { return "BlocksList"; }
String getID() const override
{
std::stringstream res;
res << this;
return res.str();
}
protected:
Block readImpl() override
{
......
......@@ -20,11 +20,6 @@ String CastTypeBlockInputStream::getName() const
return "CastType";
}
String CastTypeBlockInputStream::getID() const
{
return "CastType(" + children.back()->getID() + ")";
}
Block CastTypeBlockInputStream::readImpl()
{
Block block = children.back()->read();
......
......@@ -17,8 +17,6 @@ public:
String getName() const override;
String getID() const override;
Block getHeader() const override { return ref_definition; }
protected:
......
......@@ -28,23 +28,6 @@ public:
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; }
const SortDescription & getSortDescription() const override { return description; }
......
......@@ -33,23 +33,6 @@ public:
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:
/// Can return 1 more records than max_block_size.
Block readImpl() override;
......
......@@ -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()
{
sources.reserve(children.size());
......@@ -107,13 +94,13 @@ void ColumnGathererStream::fetchNewBlock(Source & source, size_t source_num)
}
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;
}
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);
}
}
......
......@@ -61,8 +61,6 @@ public:
String getName() const override { return "ColumnGatherer"; }
String getID() const override;
Block readImpl() override;
void readSuffixImpl() override;
......
......@@ -22,25 +22,6 @@ public:
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(); }
protected:
......
......@@ -35,25 +35,6 @@ public:
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(); }
/// Takes `totals` only from the main source, not from subquery sources.
......
......@@ -18,13 +18,6 @@ DistinctBlockInputStream::DistinctBlockInputStream(const BlockInputStreamPtr & i
children.push_back(input);
}
String DistinctBlockInputStream::getID() const
{
std::stringstream res;
res << "Distinct(" << children.back()->getID() << ")";
return res.str();
}
Block DistinctBlockInputStream::readImpl()
{
/// Execute until end of stream or until
......
......@@ -22,8 +22,6 @@ public:
String getName() const override { return "Distinct"; }
String getID() const override;
Block getHeader() const override { return children.at(0)->getHeader(); }
protected:
......
......@@ -19,13 +19,6 @@ DistinctSortedBlockInputStream::DistinctSortedBlockInputStream(const BlockInputS
children.push_back(input);
}
String DistinctSortedBlockInputStream::getID() const
{
std::stringstream res;
res << "DistinctSorted(" << children.back()->getID() << ")";
return res.str();
}
Block DistinctSortedBlockInputStream::readImpl()
{
/// Execute until end of stream or until
......
......@@ -25,8 +25,6 @@ public:
String getName() const override { return "DistinctSorted"; }
String getID() const override;
Block getHeader() const override { return children.at(0)->getHeader(); }
protected:
......
......@@ -13,13 +13,6 @@ ExpressionBlockInputStream::ExpressionBlockInputStream(const BlockInputStreamPtr
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()
{
if (IProfilingBlockInputStream * child = dynamic_cast<IProfilingBlockInputStream *>(&*children.back()))
......
......@@ -22,7 +22,6 @@ public:
ExpressionBlockInputStream(const BlockInputStreamPtr & input, const ExpressionActionsPtr & expression_);
String getName() const override;
String getID() const override;
const Block & getTotals() override;
Block getHeader() const override;
......
......@@ -57,14 +57,6 @@ FilterBlockInputStream::FilterBlockInputStream(const BlockInputStreamPtr & input
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()
{
if (IProfilingBlockInputStream * child = dynamic_cast<IProfilingBlockInputStream *>(&*children.back()))
......
......@@ -25,7 +25,6 @@ public:
FilterBlockInputStream(const BlockInputStreamPtr & input, const ExpressionActionsPtr & expression_, const String & filter_column_name_);
String getName() const override;
String getID() const override;
const Block & getTotals() override;
Block getHeader() const override;
......
......@@ -3,18 +3,6 @@
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 block = children.back()->getHeader();
......
......@@ -24,8 +24,6 @@ public:
return "FilterColumnsBlockInputStream";
}
String getID() const override;
Block getHeader() const override;
protected:
......
......@@ -135,23 +135,6 @@ public:
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()
{
if (aggregate_state_created)
......
......@@ -126,13 +126,5 @@ void IBlockInputStream::getLeavesImpl(BlockInputStreams & res, const BlockInputS
(*it)->getLeavesImpl(res, *it);
}
/// By default all instances is different streams
String IBlockInputStream::getID() const
{
std::stringstream res;
res << getName() << "(" << this << ")";
return res.str();
};
}
......@@ -82,14 +82,6 @@ public:
*/
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.
virtual bool isGroupedOutput() const { return false; }
/// If this stream generates data in order by some keys, return true.
......
......@@ -26,7 +26,6 @@ public:
void readSuffixImpl() override { return res_stream->readSuffix(); }
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(); }
......
......@@ -21,13 +21,6 @@ public:
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(); }
protected:
......
......@@ -15,13 +15,6 @@ String MaterializingBlockInputStream::getName() const
return "Materializing";
}
String MaterializingBlockInputStream::getID() const
{
std::stringstream res;
res << "Materializing(" << children.back()->getID() << ")";
return res.str();
}
Block MaterializingBlockInputStream::getHeader() const
{
return materializeBlock(children.back()->getHeader());
......
......@@ -12,7 +12,6 @@ class MaterializingBlockInputStream : public IProfilingBlockInputStream
public:
MaterializingBlockInputStream(const BlockInputStreamPtr & input);
String getName() const override;
String getID() const override;
Block getHeader() const override;
protected:
......
......@@ -33,7 +33,6 @@ public:
size_t max_merged_block_size_, size_t limit_ = 0);
String getName() const override { return "MergeSortingBlocks"; }
String getID() const override { return getName(); }
bool isGroupedOutput() const override { return true; }
bool isSortedOutput() const override { return true; }
......@@ -82,18 +81,6 @@ public:
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 isSortedOutput() const override { return true; }
const SortDescription & getSortDescription() const override { return description; }
......
......@@ -22,13 +22,6 @@ public:
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;
protected:
......
......@@ -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()
{
start();
......
......@@ -67,8 +67,6 @@ public:
String getName() const override { return "MergingAggregatedMemoryEfficient"; }
String getID() const override;
/// Sends the request (initiates calculations) earlier than `read`.
void readPrefix() override;
......
......@@ -24,28 +24,6 @@ MergingSortedBlockInputStream::MergingSortedBlockInputStream(
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)
{
/// Read the first blocks, initialize the queue.
......
......@@ -70,8 +70,6 @@ public:
String getName() const override { return "MergingSorted"; }
String getID() const override;
bool isGroupedOutput() const override { return true; }
bool isSortedOutput() const override { return true; }
const SortDescription & getSortDescription() const override { return description; }
......
......@@ -74,13 +74,6 @@ public:
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);
Block getHeader() const override;
......
......@@ -28,13 +28,6 @@ public:
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 {}; }
protected:
......
......@@ -23,13 +23,6 @@ NullableAdapterBlockInputStream::NullableAdapterBlockInputStream(
children.push_back(input);
}
String NullableAdapterBlockInputStream::getID() const
{
std::stringstream res;
res << "NullableAdapterBlockInputStream(" << children.back()->getID() << ")";
return res.str();
}
Block NullableAdapterBlockInputStream::readImpl()
{
Block block = children.back()->read();
......
......@@ -22,8 +22,6 @@ public:
String getName() const override { return "NullableAdapterBlockInputStream"; }
String getID() const override;
Block getHeader() const override { return header; }
protected:
......
......@@ -27,8 +27,6 @@ private:
String getName() const override { return "Owning"; }
String getID() const override { return "Owning(" + stream->getID() + ")"; }
protected:
BlockInputStreamPtr stream;
std::unique_ptr<OwnType> own;
......
......@@ -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()
{
bool old_val = false;
......
......@@ -27,8 +27,6 @@ public:
String getName() const override { return "ParallelAggregating"; }
String getID() const override;
void cancel() override;
Block getHeader() const override;
......
......@@ -23,18 +23,6 @@ public:
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 isSortedOutput() const override { return true; }
const SortDescription & getSortDescription() const override { return description; }
......
......@@ -66,13 +66,6 @@ public:
String getName() const override { return "Remote"; }
String getID() const override
{
std::stringstream res;
res << this;
return res.str();
}
BlockExtraInfo getBlockExtraInfo() const override
{
return multiplexed_connections->getBlockExtraInfo();
......
......@@ -22,18 +22,6 @@ public:
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 res = children.back()->getHeader();
......
......@@ -24,23 +24,6 @@ public:
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:
/// Can return 1 more records than max_block_size.
Block readImpl() override;
......
......@@ -16,13 +16,6 @@ public:
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(); }
protected:
......
......@@ -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)
{
for (auto & desc : columns_to_aggregate)
......
......@@ -35,8 +35,6 @@ public:
String getName() const override { return "SummingSorted"; }
String getID() const override;
protected:
/// Can return 1 more records than max_block_size.
Block readImpl() override;
......
......@@ -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)
{
for (size_t i = 0; i < block.columns(); ++i)
......
......@@ -27,8 +27,6 @@ public:
String getName() const override { return "TotalsHaving"; }
String getID() const override;
const Block & getTotals() override;
Block getHeader() const override;
......
......@@ -86,26 +86,6 @@ public:
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
{
try
......
......@@ -185,24 +185,6 @@ public:
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:
/// Can return 1 more records than max_block_size.
Block readImpl() override;
......
......@@ -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()
{
if (next_row == rows_count)
......
......@@ -11,8 +11,6 @@ protected:
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;
Block getHeader() const override;
......
......@@ -120,7 +120,6 @@ private:
}
String getName() const override { return "WithBackgroundThread"; }
String getID() const override { return "WithBackgroundThread(" + stream->getID() + ")"; }
BlockInputStreamPtr stream;
std::unique_ptr<ShellCommand> command;
......
......@@ -38,14 +38,6 @@ MongoDBBlockInputStream::MongoDBBlockInputStream(
MongoDBBlockInputStream::~MongoDBBlockInputStream() = default;
String MongoDBBlockInputStream::getID() const
{
std::ostringstream stream;
stream << cursor.get();
return "MongoDB(@" + stream.str() + ")";
}
namespace
{
using ValueType = ExternalResultDescription::ValueType;
......
......@@ -32,8 +32,6 @@ public:
String getName() const override { return "MongoDB"; }
String getID() const override;
Block getHeader() const override { return description.sample_block; };
private:
......
......@@ -33,12 +33,6 @@ MySQLBlockInputStream::MySQLBlockInputStream(
}
String MySQLBlockInputStream::getID() const
{
return "MySQL(" + query.str() + ")";
}
namespace
{
using ValueType = ExternalResultDescription::ValueType;
......
......@@ -21,8 +21,6 @@ public:
String getName() const override { return "MySQL"; }
String getID() const override;
Block getHeader() const override { return description.sample_block; };
private:
......
......@@ -38,12 +38,6 @@ ODBCBlockInputStream::ODBCBlockInputStream(
}
String ODBCBlockInputStream::getID() const
{
return "ODBC(" + statement.toString() + ")";
}
namespace
{
using ValueType = ExternalResultDescription::ValueType;
......
......@@ -27,8 +27,6 @@ public:
String getName() const override { return "ODBC"; }
String getID() const override;
Block getHeader() const override { return description.sample_block; };
private:
......
......@@ -1588,13 +1588,6 @@ public:
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); }
~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)
{
isCancelled = cancellation_hook;
......
......@@ -1090,9 +1090,6 @@ public:
*/
void setCancellationHook(const CancellationHook cancellation_hook);
/// For IBlockInputStream.
String getID() const;
/// For external aggregation.
void writeToTemporaryFile(AggregatedDataVariants & data_variants);
......
......@@ -964,11 +964,6 @@ public:
return "DDLQueryStatusInputSream";
}
String getID() const override
{
return "DDLQueryStatusInputSream(" + node_path + ")";
}
Block getHeader() const override { return sample; };
Block readImpl() override
......
......@@ -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::stringstream ss;
......
......@@ -194,8 +194,6 @@ public:
/// Obtain a sample block that contains the names and types of result columns.
const Block & getSampleBlock() const { return sample_block; }
std::string getID() const;
std::string dumpActions() const;
static std::string getSmallestColumn(const NamesAndTypesList & columns);
......
......@@ -118,11 +118,6 @@ public:
return "SynchronousQueryKiller";
}
String getID() const override
{
return "SynchronousQueryKiller_" + toString(intptr_t(this));
}
Block getHeader() const override { return res_sample_block; };
Block readImpl() override
......
......@@ -1038,13 +1038,6 @@ public:
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; };
......
......@@ -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()
try
{
......
......@@ -38,8 +38,6 @@ public:
String getName() const override { return "MergeTree"; }
String getID() const override;
Block getHeader() const override;
/// Closes readers and unlock part locks
......
......@@ -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
bool MergeTreeThreadBlockInputStream::getNewTask()
{
......
......@@ -30,8 +30,6 @@ public:
String getName() const override { return "MergeTreeThread"; }
String getID() const override;
~MergeTreeThreadBlockInputStream() override;
Block getHeader() const override;
......
......@@ -76,18 +76,6 @@ public:
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); };
protected:
......
......@@ -39,11 +39,6 @@ public:
return "CatBoostDatasetBlockInputStream";
}
String getID() const override
{
return "CatBoostDataset(" + format_name + ", " + file_name + ")";
}
Block readImpl() override
{
return reader->read();
......
......@@ -144,18 +144,6 @@ public:
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
{
return reader->read();
......
......@@ -161,13 +161,6 @@ public:
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
{
if (isCancelled())
......
......@@ -62,18 +62,6 @@ public:
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 res;
......
......@@ -25,18 +25,6 @@ public:
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); }
protected:
......
......@@ -59,13 +59,6 @@ public:
String getName() const override { return "StripeLog"; }
String getID() const override
{
std::stringstream s;
s << this;
return s.str();
}
Block getHeader() const override
{
if (index_begin == index_end)
......
......@@ -60,8 +60,6 @@ public:
String getName() const override { return "TinyLog"; }
String getID() const override;
Block getHeader() const override
{
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 res;
......
......@@ -16,7 +16,6 @@ public:
: block_size(block_size_), next(offset_), step(step_) {}
String getName() const override { return "Numbers"; }
String getID() const override { return "Numbers"; }
Block getHeader() const override
{
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册