提交 9cd4c749 编写于 作者: V Vitaly Baranov

Implement an utility class for reading protobuf messages.

上级 74cc6f28
......@@ -410,7 +410,7 @@ namespace ErrorCodes
extern const int ILLEGAL_CODEC_PARAMETER = 433;
extern const int CANNOT_PARSE_PROTOBUF_SCHEMA = 434;
extern const int NO_DATA_FOR_REQUIRED_PROTOBUF_FIELD = 435;
extern const int CANNOT_CONVERT_TO_PROTOBUF_TYPE = 436;
extern const int PROTOBUF_BAD_CAST = 436;
extern const int PROTOBUF_FIELD_NOT_REPEATED = 437;
extern const int DATA_TYPE_CANNOT_BE_PROMOTED = 438;
extern const int CANNOT_SCHEDULE_TASK = 439;
......@@ -418,6 +418,7 @@ namespace ErrorCodes
extern const int CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING = 441;
extern const int BAD_DATABASE_FOR_TEMPORARY_TABLE = 442;
extern const int NO_COMMON_COLUMNS_WITH_PROTOBUF_SCHEMA = 443;
extern const int UNKNOWN_PROTOBUF_FORMAT = 444;
extern const int KEEPER_EXCEPTION = 999;
extern const int POCO_EXCEPTION = 1000;
......
此差异已折叠。
#pragma once
#include <common/DayNum.h>
#include <Common/PODArray.h>
#include <Common/UInt128.h>
#include <Core/UUID.h>
#include <Common/config.h>
#if USE_PROTOBUF
#include <boost/noncopyable.hpp>
#include <Formats/ProtobufColumnMatcher.h>
#include <memory>
namespace google
{
namespace protobuf
{
class Descriptor;
}
}
namespace DB
{
class Arena;
class IAggregateFunction;
class ReadBuffer;
using AggregateDataPtr = char *;
using AggregateFunctionPtr = std::shared_ptr<IAggregateFunction>;
/** Deserializes a protobuf, tries to cast data types if necessarily.
*/
class ProtobufReader : private boost::noncopyable
{
public:
ProtobufReader(ReadBuffer & in_, const google::protobuf::Descriptor * message_type, const std::vector<String> & column_names);
~ProtobufReader();
/// Should be called when we start reading a new message.
bool startMessage();
/// Ends reading a message.
void endMessage();
/// Reads the column index.
/// The function returns false if there are no more columns to read (call endMessage() in this case).
bool readColumnIndex(size_t & column_index);
/// Reads a value which should be put to column at index received with readColumnIndex().
/// The function returns false if there are no more values to read now (call readColumnIndex() in this case).
bool readNumber(Int8 & value) { return current_converter->readInt8(value); }
bool readNumber(UInt8 & value) { return current_converter->readUInt8(value); }
bool readNumber(Int16 & value) { return current_converter->readInt16(value); }
bool readNumber(UInt16 & value) { return current_converter->readUInt16(value); }
bool readNumber(Int32 & value) { return current_converter->readInt32(value); }
bool readNumber(UInt32 & value) { return current_converter->readUInt32(value); }
bool readNumber(Int64 & value) { return current_converter->readInt64(value); }
bool readNumber(UInt64 & value) { return current_converter->readUInt64(value); }
bool readNumber(UInt128 & value) { return current_converter->readUInt128(value); }
bool readNumber(Float32 & value) { return current_converter->readFloat32(value); }
bool readNumber(Float64 & value) { return current_converter->readFloat64(value); }
bool readStringInto(PaddedPODArray<UInt8> & str) { return current_converter->readStringInto(str); }
void prepareEnumMapping(const std::vector<std::pair<std::string, Int8>> & name_value_pairs) { current_converter->prepareEnumMapping8(name_value_pairs); }
void prepareEnumMapping(const std::vector<std::pair<std::string, Int16>> & name_value_pairs) { current_converter->prepareEnumMapping16(name_value_pairs); }
bool readEnum(Int8 & value) { return current_converter->readEnum8(value); }
bool readEnum(Int16 & value) { return current_converter->readEnum16(value); }
bool readUUID(UUID & uuid) { return current_converter->readUUID(uuid); }
bool readDate(DayNum & date) { return current_converter->readDate(date); }
bool readDateTime(time_t & tm) { return current_converter->readDateTime(tm); }
bool readDecimal(Decimal32 & decimal, UInt32 precision, UInt32 scale) { return current_converter->readDecimal32(decimal, precision, scale); }
bool readDecimal(Decimal64 & decimal, UInt32 precision, UInt32 scale) { return current_converter->readDecimal64(decimal, precision, scale); }
bool readDecimal(Decimal128 & decimal, UInt32 precision, UInt32 scale) { return current_converter->readDecimal128(decimal, precision, scale); }
bool readAggregateFunction(const AggregateFunctionPtr & function, AggregateDataPtr place, Arena & arena) { return current_converter->readAggregateFunction(function, place, arena); }
/// When it returns false there is no more values left and from now on all the read<Type>() functions will return false
/// until readColumnIndex() is called. When it returns true it's unclear.
bool maybeCanReadValue() const { return simple_reader.maybeCanReadValue(); }
private:
class SimpleReader
{
public:
SimpleReader(ReadBuffer & in_);
bool startMessage();
void endMessage();
void endRootMessage();
bool readFieldNumber(UInt32 & field_number);
bool readInt(Int64 & value);
bool readSInt(Int64 & value);
bool readUInt(UInt64 & value);
template<typename T> bool readFixed(T & value);
bool readStringInto(PaddedPODArray<UInt8> & str);
bool maybeCanReadValue() const { return field_end != REACHED_END; }
private:
void readBinary(void* data, size_t size);
void ignore(UInt64 num_bytes);
void moveCursorBackward(UInt64 num_bytes);
UInt64 readVarint();
void ignoreVarint();
void ignoreGroup();
static constexpr UInt64 REACHED_END = 0;
ReadBuffer & in;
UInt64 cursor;
std::vector<UInt64> parent_message_ends;
UInt64 current_message_end;
UInt64 field_end;
};
class IConverter
{
public:
virtual ~IConverter() = default;
virtual bool readStringInto(PaddedPODArray<UInt8> &) = 0;
virtual bool readInt8(Int8&) = 0;
virtual bool readUInt8(UInt8 &) = 0;
virtual bool readInt16(Int16 &) = 0;
virtual bool readUInt16(UInt16 &) = 0;
virtual bool readInt32(Int32 &) = 0;
virtual bool readUInt32(UInt32 &) = 0;
virtual bool readInt64(Int64 &) = 0;
virtual bool readUInt64(UInt64 &) = 0;
virtual bool readUInt128(UInt128 &) = 0;
virtual bool readFloat32(Float32 &) = 0;
virtual bool readFloat64(Float64 &) = 0;
virtual void prepareEnumMapping8(const std::vector<std::pair<std::string, Int8>> &) = 0;
virtual void prepareEnumMapping16(const std::vector<std::pair<std::string, Int16>> &) = 0;
virtual bool readEnum8(Int8 &) = 0;
virtual bool readEnum16(Int16 &) = 0;
virtual bool readUUID(UUID &) = 0;
virtual bool readDate(DayNum &) = 0;
virtual bool readDateTime(time_t &) = 0;
virtual bool readDecimal32(Decimal32 &, UInt32, UInt32) = 0;
virtual bool readDecimal64(Decimal64 &, UInt32, UInt32) = 0;
virtual bool readDecimal128(Decimal128 &, UInt32, UInt32) = 0;
virtual bool readAggregateFunction(const AggregateFunctionPtr &, AggregateDataPtr, Arena &) = 0;
};
class ConverterBaseImpl;
template <int type_id> class ConverterImpl;
class ConverterFromString;
template<int field_type_id, typename FromType> class ConverterFromNumber;
struct ColumnMatcherTraits
{
struct FieldData
{
std::unique_ptr<IConverter> converter;
};
struct MessageData
{
std::unordered_map<UInt32, const ProtobufColumnMatcher::Field<ColumnMatcherTraits>*> field_number_to_field_map;
};
};
using Message = ProtobufColumnMatcher::Message<ColumnMatcherTraits>;
using Field = ProtobufColumnMatcher::Field<ColumnMatcherTraits>;
void setTraitsDataAfterMatchingColumns(Message * message);
SimpleReader simple_reader;
std::unique_ptr<Message> root_message;
Message* current_message = nullptr;
size_t current_field_index = 0;
IConverter* current_converter = nullptr;
};
}
#else
namespace DB
{
class Arena;
class IAggregateFunction;
class ReadBuffer;
using AggregateDataPtr = char *;
using AggregateFunctionPtr = std::shared_ptr<IAggregateFunction>;
class ProtobufReader
{
public:
bool startMessage() { return false; }
void endMessage() {}
bool readColumnIndex(size_t & column_index) { return false; }
bool readNumber(Int8 & value) { return false; }
bool readNumber(UInt8 & value) { return false; }
bool readNumber(Int16 & value) { return false; }
bool readNumber(UInt16 & value) { return false; }
bool readNumber(Int32 & value) { return false; }
bool readNumber(UInt32 & value) { return false; }
bool readNumber(Int64 & value) { return false; }
bool readNumber(UInt64 & value) { return false; }
bool readNumber(UInt128 & value) { return false; }
bool readNumber(Float32 & value) { return false; }
bool readNumber(Float64 & value) { return false; }
bool readStringInto(PaddedPODArray<UInt8> & str) { return false; }
void prepareEnumMapping(const std::vector<std::pair<std::string, Int8>> & name_value_pairs) {}
void prepareEnumMapping(const std::vector<std::pair<std::string, Int16>> & name_value_pairs) {}
bool readEnum(Int8 & value) { return false; }
bool readEnum(Int16 & value) { return false; }
bool readUUID(UUID & uuid) { return false; }
bool readDate(DayNum & date) { return false; }
bool readDateTime(time_t & tm) { return false; }
bool readDecimal(Decimal32 & decimal, UInt32 precision, UInt32 scale) { return false; }
bool readDecimal(Decimal64 & decimal, UInt32 precision, UInt32 scale) { return false; }
bool readDecimal(Decimal128 & decimal, UInt32 precision, UInt32 scale) { return false; }
bool readAggregateFunction(const AggregateFunctionPtr & function, AggregateDataPtr place, Arena & arena) { return false; }
bool maybeCanReadValue() const { return false; }
};
}
#endif
......@@ -18,7 +18,7 @@ namespace ErrorCodes
{
extern const int NOT_IMPLEMENTED;
extern const int NO_DATA_FOR_REQUIRED_PROTOBUF_FIELD;
extern const int CANNOT_CONVERT_TO_PROTOBUF_TYPE;
extern const int PROTOBUF_BAD_CAST;
extern const int PROTOBUF_FIELD_NOT_REPEATED;
}
......@@ -67,14 +67,14 @@ protected:
{
throw Exception(
"Could not convert data type '" + type_name + "' to protobuf type '" + field->type_name() + "' (field: " + field->name() + ")",
ErrorCodes::CANNOT_CONVERT_TO_PROTOBUF_TYPE);
ErrorCodes::PROTOBUF_BAD_CAST);
}
void cannotConvertValue(const String & value)
{
throw Exception(
"Could not convert value '" + value + "' to protobuf type '" + field->type_name() + "' (field: " + field->name() + ")",
ErrorCodes::CANNOT_CONVERT_TO_PROTOBUF_TYPE);
ErrorCodes::PROTOBUF_BAD_CAST);
}
template <typename To, typename From>
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册