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

Added method "getHeader" in IBlockOutputStream: development [#CLICKHOUSE-2]

上级 4add285d
......@@ -6,6 +6,8 @@
#include <IO/Operators.h>
#include <Common/typeid_cast.h>
#include <Columns/ColumnConst.h>
#include <iterator>
#include <memory>
......@@ -18,6 +20,7 @@ namespace ErrorCodes
extern const int POSITION_OUT_OF_BOUND;
extern const int NOT_FOUND_COLUMN_IN_BLOCK;
extern const int SIZES_OF_COLUMNS_DOESNT_MATCH;
extern const int BLOCKS_HAVE_DIFFERENT_STRUCTURE;
}
......@@ -276,13 +279,7 @@ std::string Block::dumpStructure() const
{
if (it != data.begin())
out << ", ";
out << it->name << ' ' << it->type->getName();
if (it->column)
out << ' ' << it->column->dumpStructure();
else
out << " nullptr";
it->dumpStructure(out);
}
return out.str();
}
......@@ -379,22 +376,58 @@ Names Block::getNames() const
}
bool blocksHaveEqualStructure(const Block & lhs, const Block & rhs)
template <typename ReturnType>
static ReturnType checkBlockStructure(const Block & lhs, const Block & rhs, const std::string & context_description)
{
size_t columns = lhs.columns();
if (rhs.columns() != columns)
return false;
auto on_error = [](const std::string & message [[maybe_unused]], int code [[maybe_unused]])
{
if constexpr (std::is_same_v<ReturnType, void>)
throw Exception(message, code);
else
return false;
};
size_t columns = rhs.columns();
if (lhs.columns() != columns)
return on_error("Block structure mismatch in " + context_description + " stream: different number of columns:\n"
+ lhs.dumpStructure() + "\n" + rhs.dumpStructure(), ErrorCodes::BLOCKS_HAVE_DIFFERENT_STRUCTURE);
for (size_t i = 0; i < columns; ++i)
{
const IDataType & lhs_type = *lhs.safeGetByPosition(i).type;
const IDataType & rhs_type = *rhs.safeGetByPosition(i).type;
const auto & expected = rhs.getByPosition(i);
const auto & actual = lhs.getByPosition(i);
if (!lhs_type.equals(rhs_type))
return false;
if (actual.name != expected.name)
return on_error("Block structure mismatch in " + context_description + " stream: different names of columns:\n"
+ lhs.dumpStructure() + "\n" + rhs.dumpStructure(), ErrorCodes::BLOCKS_HAVE_DIFFERENT_STRUCTURE);
if (!actual.type->equals(*expected.type))
return on_error("Block structure mismatch in " + context_description + " stream: different types:\n"
+ lhs.dumpStructure() + "\n" + rhs.dumpStructure(), ErrorCodes::BLOCKS_HAVE_DIFFERENT_STRUCTURE);
if (actual.column->getName() != expected.column->getName())
return on_error("Block structure mismatch in " + context_description + " stream: different columns:\n"
+ lhs.dumpStructure() + "\n" + rhs.dumpStructure(), ErrorCodes::BLOCKS_HAVE_DIFFERENT_STRUCTURE);
if (actual.column->isColumnConst() && expected.column->isColumnConst()
&& static_cast<const ColumnConst &>(*actual.column).getField() != static_cast<const ColumnConst &>(*expected.column).getField())
return on_error("Block structure mismatch in " + context_description + " stream: different values of constants",
ErrorCodes::BLOCKS_HAVE_DIFFERENT_STRUCTURE);
}
return true;
return ReturnType(true);
}
bool blocksHaveEqualStructure(const Block & lhs, const Block & rhs)
{
return checkBlockStructure<bool>(lhs, rhs, {});
}
void assertBlocksHaveEqualStructure(const Block & lhs, const Block & rhs, const std::string & context_description)
{
checkBlockStructure<void>(lhs, rhs, context_description);
}
......@@ -453,12 +486,12 @@ void getBlocksDifference(const Block & lhs, const Block & rhs, std::string & out
for (auto it = left_columns.rbegin(); it != left_columns.rend(); ++it)
{
lhs_diff_writer << it->prettyPrint();
lhs_diff_writer << it->dumpStructure();
lhs_diff_writer << ", position: " << lhs.getPositionByName(it->name) << '\n';
}
for (auto it = right_columns.rbegin(); it != right_columns.rend(); ++it)
{
rhs_diff_writer << it->prettyPrint();
rhs_diff_writer << it->dumpStructure();
rhs_diff_writer << ", position: " << rhs.getPositionByName(it->name) << '\n';
}
}
......
......@@ -137,10 +137,13 @@ using Blocks = std::vector<Block>;
using BlocksList = std::list<Block>;
/// Compare column types for blocks. The order of the columns matters. Names do not matter.
/// Compare number of columns, data types, column types, column names, and values of constant columns.
bool blocksHaveEqualStructure(const Block & lhs, const Block & rhs);
/// Calculate difference in structure of blocks and write description into output strings.
/// Throw exception when blocks are different.
void assertBlocksHaveEqualStructure(const Block & lhs, const Block & rhs, const std::string & context_description);
/// Calculate difference in structure of blocks and write description into output strings. NOTE It doesn't compare values of constant columns.
void getBlocksDifference(const Block & lhs, const Block & rhs, std::string & out_lhs_diff, std::string & out_rhs_diff);
......
#include <Core/ColumnsWithTypeAndName.h>
#include <IO/WriteBufferFromString.h>
#include <IO/WriteHelpers.h>
#include <IO/Operators.h>
namespace DB
......@@ -19,7 +20,7 @@ ColumnWithTypeAndName ColumnWithTypeAndName::cloneEmpty() const
}
bool ColumnWithTypeAndName::operator== (const ColumnWithTypeAndName & other) const
bool ColumnWithTypeAndName::operator==(const ColumnWithTypeAndName & other) const
{
return name == other.name
&& ((!type && !other.type) || (type && other.type && type->equals(*other.type)))
......@@ -27,20 +28,25 @@ bool ColumnWithTypeAndName::operator== (const ColumnWithTypeAndName & other) con
}
String ColumnWithTypeAndName::prettyPrint() const
void ColumnWithTypeAndName::dumpStructure(WriteBuffer & out) const
{
WriteBufferFromOwnString out;
writeString(name, out);
out << name;
if (type)
{
writeChar(' ', out);
writeString(type->getName(), out);
}
out << ' ' << type->getName();
else
out << " nullptr";
if (column)
{
writeChar(' ', out);
writeString(column->getName(), out);
}
out << ' ' << column->dumpStructure();
else
out << " nullptr";
}
String ColumnWithTypeAndName::dumpStructure() const
{
WriteBufferFromOwnString out;
dumpStructure(out);
return out.str();
}
......
......@@ -7,6 +7,9 @@
namespace DB
{
class WriteBuffer;
/** Column data along with its data type and name.
* Column data could be nullptr - to represent just 'header' of column.
* Name could be either name from a table or some temporary generated name during expression evaluation.
......@@ -28,7 +31,9 @@ struct ColumnWithTypeAndName
ColumnWithTypeAndName cloneEmpty() const;
bool operator==(const ColumnWithTypeAndName & other) const;
String prettyPrint() const;
void dumpStructure(WriteBuffer & out) const;
String dumpStructure() const;
};
}
......@@ -11,7 +11,6 @@ namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int NUMBER_OF_COLUMNS_DOESNT_MATCH;
extern const int BLOCKS_HAVE_DIFFERENT_STRUCTURE;
}
......@@ -92,19 +91,7 @@ void MergingSortedBlockInputStream::init(Block & header, MutableColumns & merged
if (!*shared_block_ptr)
continue;
size_t src_columns = shared_block_ptr->columns();
size_t dst_columns = header.columns();
if (src_columns != dst_columns)
throw Exception("Merging blocks have different number of columns ("
+ toString(src_columns) + " and " + toString(dst_columns) + ")",
ErrorCodes::NUMBER_OF_COLUMNS_DOESNT_MATCH);
for (size_t i = 0; i < src_columns; ++i)
if (!blocksHaveEqualStructure(*shared_block_ptr, header))
throw Exception("Merging blocks have different names or types of columns:\n"
+ shared_block_ptr->dumpStructure() + "\nand\n" + header.dumpStructure(),
ErrorCodes::BLOCKS_HAVE_DIFFERENT_STRUCTURE);
assertBlocksHaveEqualStructure(*shared_block_ptr, header, getName());
}
merged_columns.resize(num_columns);
......
......@@ -46,14 +46,7 @@ RemoteBlockOutputStream::RemoteBlockOutputStream(Connection & connection_, const
void RemoteBlockOutputStream::write(const Block & block)
{
if (!blocksHaveEqualStructure(block, header))
{
std::stringstream message;
message << "Block structure is different from table structure.\n"
<< "\nTable structure:\n(" << header.dumpStructure() << ")\nBlock structure:\n(" << block.dumpStructure() << ")\n";
throw Exception(message.str());
}
assertBlocksHaveEqualStructure(block, header, "RemoteBlockOutputStream");
connection.sendData(block);
}
......
......@@ -82,6 +82,14 @@ public:
children = inputs;
if (additional_input_at_end)
children.push_back(additional_input_at_end);
size_t num_children = children.size();
if (num_children > 1)
{
Block header = children.at(0)->getHeader();
for (size_t i = 1; i < num_children; ++i)
assertBlocksHaveEqualStructure(children[i]->getHeader(), header, "UNION");
}
}
String getName() const override { return "Union"; }
......
......@@ -92,49 +92,6 @@ InterpreterCheckQuery::InterpreterCheckQuery(const ASTPtr & query_ptr_, const Co
{
}
Block InterpreterCheckQuery::getSampleBlock() const
{
Block block;
ColumnWithTypeAndName col;
col.name = "status";
col.type = std::make_shared<DataTypeUInt8>();
col.column = col.type->createColumn();
block.insert(col);
col.name = "host_name";
col.type = std::make_shared<DataTypeString>();
col.column = col.type->createColumn();
block.insert(col);
col.name = "host_address";
col.type = std::make_shared<DataTypeString>();
col.column = col.type->createColumn();
block.insert(col);
col.name = "port";
col.type = std::make_shared<DataTypeUInt16>();
col.column = col.type->createColumn();
block.insert(col);
col.name = "user";
col.type = std::make_shared<DataTypeString>();
col.column = col.type->createColumn();
block.insert(col);
col.name = "structure_class";
col.type = std::make_shared<DataTypeUInt32>();
col.column = col.type->createColumn();
block.insert(col);
col.name = "structure";
col.type = std::make_shared<DataTypeString>();
col.column = col.type->createColumn();
block.insert(col);
return block;
}
BlockIO InterpreterCheckQuery::execute()
{
......
......@@ -16,9 +16,6 @@ public:
BlockIO execute() override;
private:
Block getSampleBlock() const;
private:
ASTPtr query_ptr;
......
......@@ -213,20 +213,6 @@ void InterpreterSelectQuery::basicInit(const BlockInputStreamPtr & input)
if (input)
streams.push_back(input);
if (is_first_select_inside_union_all)
{
/// We check that the results of all SELECT queries are compatible.
Block first = getSampleBlock();
for (auto p = next_select_in_union_all.get(); p != nullptr; p = p->next_select_in_union_all.get())
{
Block current = p->getSampleBlock();
if (!blocksHaveEqualStructure(first, current))
throw Exception("Result structures mismatch in the SELECT queries of the UNION ALL chain. Found result structure:\n\n" + current.dumpStructure()
+ "\n\nwhile expecting:\n\n" + first.dumpStructure() + "\n\ninstead",
ErrorCodes::UNION_ALL_RESULT_STRUCTURES_MISMATCH);
}
}
}
void InterpreterSelectQuery::initQueryAnalyzer()
......
......@@ -44,7 +44,6 @@ namespace DB
namespace ErrorCodes
{
extern const int INFINITE_LOOP;
extern const int BLOCKS_HAVE_DIFFERENT_STRUCTURE;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
}
......@@ -150,9 +149,7 @@ static void appendBlock(const Block & from, Block & to)
if (!to)
throw Exception("Cannot append to empty block", ErrorCodes::LOGICAL_ERROR);
if (!blocksHaveEqualStructure(from, to))
throw Exception("Cannot append block to buffer: block has different structure. "
"Block: " + from.dumpStructure() + ", Buffer: " + to.dumpStructure(), ErrorCodes::BLOCKS_HAVE_DIFFERENT_STRUCTURE);
assertBlocksHaveEqualStructure(from, to, "Buffer");
from.checkNumberOfRows();
to.checkNumberOfRows();
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册