Merge pull request #35152 from rschu1ze/protobuf-batch-write

ProtobufList
This commit is contained in:
Kruglov Pavel 2022-03-18 13:24:34 +01:00 committed by GitHub
commit aa3c05e9d4
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
23 changed files with 1609 additions and 68 deletions

View File

@ -51,6 +51,7 @@ The supported formats are:
| [PrettySpace](#prettyspace) | ✗ | ✔ |
| [Protobuf](#protobuf) | ✔ | ✔ |
| [ProtobufSingle](#protobufsingle) | ✔ | ✔ |
| [ProtobufList](#protobuflist) | ✔ | ✔ |
| [Avro](#data-format-avro) | ✔ | ✔ |
| [AvroConfluent](#data-format-avro-confluent) | ✔ | ✗ |
| [Parquet](#data-format-parquet) | ✔ | ✔ |
@ -1230,7 +1231,38 @@ See also [how to read/write length-delimited protobuf messages in popular langua
## ProtobufSingle {#protobufsingle}
Same as [Protobuf](#protobuf) but for storing/parsing single Protobuf message without length delimiters.
Same as [Protobuf](#protobuf) but for storing/parsing a single Protobuf message without length delimiter.
As a result, only a single table row can be written/read.
## ProtobufList {#protobuflist}
Similar to Protobuf but rows are represented as a sequence of sub-messages contained in a message with fixed name "Envelope".
Usage example:
``` sql
SELECT * FROM test.table FORMAT ProtobufList SETTINGS format_schema = 'schemafile:MessageType'
```
``` bash
cat protobuflist_messages.bin | clickhouse-client --query "INSERT INTO test.table FORMAT ProtobufList SETTINGS format_schema='schemafile:MessageType'"
```
where the file `schemafile.proto` looks like this:
``` capnp
syntax = "proto3";
message Envelope {
message MessageType {
string name = 1;
string surname = 2;
uint32 birthDate = 3;
repeated string phoneNumbers = 4;
};
MessageType row = 1;
};
```
## Avro {#data-format-avro}

View File

@ -24,7 +24,9 @@ ProtobufSchemas & ProtobufSchemas::instance()
class ProtobufSchemas::ImporterWithSourceTree : public google::protobuf::compiler::MultiFileErrorCollector
{
public:
explicit ImporterWithSourceTree(const String & schema_directory) : importer(&disk_source_tree, this)
explicit ImporterWithSourceTree(const String & schema_directory, WithEnvelope with_envelope_)
: importer(&disk_source_tree, this)
, with_envelope(with_envelope_)
{
disk_source_tree.MapPath("", schema_directory);
}
@ -39,16 +41,33 @@ public:
return descriptor;
const auto * file_descriptor = importer.Import(schema_path);
// If there are parsing errors AddError() throws an exception and in this case the following line
// If there are parsing errors, AddError() throws an exception and in this case the following line
// isn't executed.
assert(file_descriptor);
descriptor = file_descriptor->FindMessageTypeByName(message_name);
if (!descriptor)
if (with_envelope == WithEnvelope::No)
{
const auto * message_descriptor = file_descriptor->FindMessageTypeByName(message_name);
if (!message_descriptor)
throw Exception(
"Not found a message named '" + message_name + "' in the schema file '" + schema_path + "'", ErrorCodes::BAD_ARGUMENTS);
"Could not find a message named '" + message_name + "' in the schema file '" + schema_path + "'", ErrorCodes::BAD_ARGUMENTS);
return descriptor;
return message_descriptor;
}
else
{
const auto * envelope_descriptor = file_descriptor->FindMessageTypeByName("Envelope");
if (!envelope_descriptor)
throw Exception(
"Could not find a message named 'Envelope' in the schema file '" + schema_path + "'", ErrorCodes::BAD_ARGUMENTS);
const auto * message_descriptor = envelope_descriptor->FindNestedTypeByName(message_name); // silly protobuf API disallows a restricting the field type to messages
if (!message_descriptor)
throw Exception(
"Could not find a message named '" + message_name + "' in the schema file '" + schema_path + "'", ErrorCodes::BAD_ARGUMENTS);
return message_descriptor;
}
}
private:
@ -63,18 +82,16 @@ private:
google::protobuf::compiler::DiskSourceTree disk_source_tree;
google::protobuf::compiler::Importer importer;
const WithEnvelope with_envelope;
};
ProtobufSchemas::ProtobufSchemas() = default;
ProtobufSchemas::~ProtobufSchemas() = default;
const google::protobuf::Descriptor * ProtobufSchemas::getMessageTypeForFormatSchema(const FormatSchemaInfo & info)
const google::protobuf::Descriptor * ProtobufSchemas::getMessageTypeForFormatSchema(const FormatSchemaInfo & info, WithEnvelope with_envelope)
{
std::lock_guard lock(mutex);
auto it = importers.find(info.schemaDirectory());
if (it == importers.end())
it = importers.emplace(info.schemaDirectory(), std::make_unique<ImporterWithSourceTree>(info.schemaDirectory())).first;
it = importers.emplace(info.schemaDirectory(), std::make_unique<ImporterWithSourceTree>(info.schemaDirectory(), with_envelope)).first;
auto * importer = it->second.get();
return importer->import(info.schemaPath(), info.messageName());
}

View File

@ -28,14 +28,36 @@ class FormatSchemaInfo;
class ProtobufSchemas : private boost::noncopyable
{
public:
static ProtobufSchemas & instance();
enum class WithEnvelope
{
// Return descriptor for a top-level message with a user-provided name.
// Example: In protobuf schema
// message MessageType {
// string colA = 1;
// int32 colB = 2;
// }
// message_name = "MessageType" returns a descriptor. Used by IO
// formats Protobuf and ProtobufSingle.
No,
// Return descriptor for a message with a user-provided name one level
// below a top-level message with the hardcoded name "Envelope".
// Example: In protobuf schema
// message Envelope {
// message MessageType {
// string colA = 1;
// int32 colB = 2;
// }
// }
// message_name = "MessageType" returns a descriptor. Used by IO format
// ProtobufList.
Yes
};
ProtobufSchemas();
~ProtobufSchemas();
static ProtobufSchemas & instance();
/// Parses the format schema, then parses the corresponding proto file, and returns the descriptor of the message type.
/// The function never returns nullptr, it throws an exception if it cannot load or parse the file.
const google::protobuf::Descriptor * getMessageTypeForFormatSchema(const FormatSchemaInfo & info);
const google::protobuf::Descriptor * getMessageTypeForFormatSchema(const FormatSchemaInfo & info, WithEnvelope with_envelope);
private:
class ImporterWithSourceTree;

View File

@ -2171,6 +2171,11 @@ namespace
field_index_by_field_tag.emplace(field_infos[i].field_tag, i);
}
void setHasEnvelopeAsParent()
{
has_envelope_as_parent = true;
}
void setColumns(const ColumnPtr * columns_, size_t num_columns_) override
{
if (!num_columns_)
@ -2217,7 +2222,7 @@ namespace
void writeRow(size_t row_num) override
{
if (parent_field_descriptor)
if (parent_field_descriptor || has_envelope_as_parent)
writer->startNestedMessage();
else
writer->startMessage();
@ -2236,13 +2241,17 @@ namespace
bool is_group = (parent_field_descriptor->type() == FieldTypeId::TYPE_GROUP);
writer->endNestedMessage(parent_field_descriptor->number(), is_group, should_skip_if_empty);
}
else if (has_envelope_as_parent)
{
writer->endNestedMessage(1, false, should_skip_if_empty);
}
else
writer->endMessage(with_length_delimiter);
}
void readRow(size_t row_num) override
{
if (parent_field_descriptor)
if (parent_field_descriptor || has_envelope_as_parent)
reader->startNestedMessage();
else
reader->startMessage(with_length_delimiter);
@ -2285,7 +2294,7 @@ namespace
}
}
if (parent_field_descriptor)
if (parent_field_descriptor || has_envelope_as_parent)
reader->endNestedMessage();
else
reader->endMessage(false);
@ -2375,6 +2384,7 @@ namespace
};
const FieldDescriptor * const parent_field_descriptor;
bool has_envelope_as_parent = false;
const bool with_length_delimiter;
const std::unique_ptr<RowInputMissingColumnsFiller> missing_columns_filler;
const bool should_skip_if_empty;
@ -2388,6 +2398,86 @@ namespace
size_t last_field_index = static_cast<size_t>(-1);
};
/// Serializes a top-level envelope message in the protobuf schema.
/// "Envelope" means that the contained subtree of serializers is enclosed in a message just once,
/// i.e. only when the first and the last row read/write trigger a read/write of the msg header.
class ProtobufSerializerEnvelope : public ProtobufSerializer
{
public:
ProtobufSerializerEnvelope(
std::unique_ptr<ProtobufSerializerMessage>&& serializer_,
const ProtobufReaderOrWriter & reader_or_writer_)
: serializer(std::move(serializer_))
, reader(reader_or_writer_.reader)
, writer(reader_or_writer_.writer)
{
// The inner serializer has a backreference of type protobuf::FieldDescriptor * to it's parent
// serializer. If it is unset, it considers itself the top-level message, otherwise a nested
// message and accordingly it makes start/endMessage() vs. startEndNestedMessage() calls into
// Protobuf(Writer|Reader). There is no field descriptor because Envelopes merely forward calls
// but don't contain data to be serialized. We must still force the inner serializer to act
// as nested message.
serializer->setHasEnvelopeAsParent();
}
void setColumns(const ColumnPtr * columns_, size_t num_columns_) override
{
serializer->setColumns(columns_, num_columns_);
}
void setColumns(const MutableColumnPtr * columns_, size_t num_columns_) override
{
serializer->setColumns(columns_, num_columns_);
}
void writeRow(size_t row_num) override
{
if (first_call_of_write_row)
{
writer->startMessage();
first_call_of_write_row = false;
}
serializer->writeRow(row_num);
}
void finalizeWrite() override
{
writer->endMessage(/*with_length_delimiter = */ true);
}
void readRow(size_t row_num) override
{
if (first_call_of_read_row)
{
reader->startMessage(/*with_length_delimiter = */ true);
first_call_of_read_row = false;
}
int field_tag;
[[maybe_unused]] bool ret = reader->readFieldNumber(field_tag);
assert(ret);
serializer->readRow(row_num);
}
void insertDefaults(size_t row_num) override
{
serializer->insertDefaults(row_num);
}
void describeTree(WriteBuffer & out, size_t indent) const override
{
writeIndent(out, indent) << "ProtobufSerializerEnvelope ->\n";
serializer->describeTree(out, indent + 1);
}
std::unique_ptr<ProtobufSerializerMessage> serializer;
ProtobufReader * const reader;
ProtobufWriter * const writer;
bool first_call_of_write_row = true;
bool first_call_of_read_row = true;
};
/// Serializes a tuple with explicit names as a nested message.
class ProtobufSerializerTupleAsNestedMessage : public ProtobufSerializer
@ -2610,7 +2700,8 @@ namespace
const DataTypes & data_types,
std::vector<size_t> & missing_column_indices,
const MessageDescriptor & message_descriptor,
bool with_length_delimiter)
bool with_length_delimiter,
bool with_envelope)
{
root_serializer_ptr = std::make_shared<ProtobufSerializer *>();
get_root_desc_function = [root_serializer_ptr = root_serializer_ptr](size_t indent) -> String
@ -2648,14 +2739,24 @@ namespace
boost::range::set_difference(collections::range(column_names.size()), used_column_indices_sorted,
std::back_inserter(missing_column_indices));
if (!with_envelope)
{
*root_serializer_ptr = message_serializer.get();
#if 0
LOG_INFO(&Poco::Logger::get("ProtobufSerializer"), "Serialization tree:\n{}", get_root_desc_function(0));
#endif
return message_serializer;
}
else
{
auto envelope_serializer = std::make_unique<ProtobufSerializerEnvelope>(std::move(message_serializer), reader_or_writer);
*root_serializer_ptr = envelope_serializer.get();
#if 0
LOG_INFO(&Poco::Logger::get("ProtobufSerializer"), "Serialization tree:\n{}", get_root_desc_function(0));
#endif
return envelope_serializer;
}
}
private:
/// Collects all field names from the message (used only to format error messages).
@ -3337,9 +3438,10 @@ std::unique_ptr<ProtobufSerializer> ProtobufSerializer::create(
std::vector<size_t> & missing_column_indices,
const google::protobuf::Descriptor & message_descriptor,
bool with_length_delimiter,
bool with_envelope,
ProtobufReader & reader)
{
return ProtobufSerializerBuilder(reader).buildMessageSerializer(column_names, data_types, missing_column_indices, message_descriptor, with_length_delimiter);
return ProtobufSerializerBuilder(reader).buildMessageSerializer(column_names, data_types, missing_column_indices, message_descriptor, with_length_delimiter, with_envelope);
}
std::unique_ptr<ProtobufSerializer> ProtobufSerializer::create(
@ -3347,10 +3449,11 @@ std::unique_ptr<ProtobufSerializer> ProtobufSerializer::create(
const DataTypes & data_types,
const google::protobuf::Descriptor & message_descriptor,
bool with_length_delimiter,
bool with_envelope,
ProtobufWriter & writer)
{
std::vector<size_t> missing_column_indices;
return ProtobufSerializerBuilder(writer).buildMessageSerializer(column_names, data_types, missing_column_indices, message_descriptor, with_length_delimiter);
return ProtobufSerializerBuilder(writer).buildMessageSerializer(column_names, data_types, missing_column_indices, message_descriptor, with_length_delimiter, with_envelope);
}
NamesAndTypesList protobufSchemaToCHSchema(const google::protobuf::Descriptor * message_descriptor)

View File

@ -26,6 +26,7 @@ public:
virtual void setColumns(const ColumnPtr * columns, size_t num_columns) = 0;
virtual void writeRow(size_t row_num) = 0;
virtual void finalizeWrite() {}
virtual void setColumns(const MutableColumnPtr * columns, size_t num_columns) = 0;
virtual void readRow(size_t row_num) = 0;
@ -39,6 +40,7 @@ public:
std::vector<size_t> & missing_column_indices,
const google::protobuf::Descriptor & message_descriptor,
bool with_length_delimiter,
bool with_envelope,
ProtobufReader & reader);
static std::unique_ptr<ProtobufSerializer> create(
@ -46,6 +48,7 @@ public:
const DataTypes & data_types,
const google::protobuf::Descriptor & message_descriptor,
bool with_length_delimiter,
bool with_envelope,
ProtobufWriter & writer);
};

View File

@ -36,6 +36,8 @@ void registerInputFormatJSONCompactEachRow(FormatFactory & factory);
void registerOutputFormatJSONCompactEachRow(FormatFactory & factory);
void registerInputFormatProtobuf(FormatFactory & factory);
void registerOutputFormatProtobuf(FormatFactory & factory);
void registerInputFormatProtobufList(FormatFactory & factory);
void registerOutputFormatProtobufList(FormatFactory & factory);
void registerInputFormatTemplate(FormatFactory & factory);
void registerOutputFormatTemplate(FormatFactory & factory);
void registerInputFormatMsgPack(FormatFactory & factory);
@ -98,6 +100,7 @@ void registerNativeSchemaReader(FormatFactory & factory);
void registerRowBinaryWithNamesAndTypesSchemaReader(FormatFactory & factory);
void registerAvroSchemaReader(FormatFactory & factory);
void registerProtobufSchemaReader(FormatFactory & factory);
void registerProtobufListSchemaReader(FormatFactory & factory);
void registerLineAsStringSchemaReader(FormatFactory & factory);
void registerJSONAsStringSchemaReader(FormatFactory & factory);
void registerRawBLOBSchemaReader(FormatFactory & factory);
@ -140,6 +143,8 @@ void registerFormats()
registerInputFormatJSONCompactEachRow(factory);
registerOutputFormatJSONCompactEachRow(factory);
registerInputFormatProtobuf(factory);
registerOutputFormatProtobufList(factory);
registerInputFormatProtobufList(factory);
registerOutputFormatProtobuf(factory);
registerInputFormatTemplate(factory);
registerOutputFormatTemplate(factory);
@ -199,6 +204,7 @@ void registerFormats()
registerRowBinaryWithNamesAndTypesSchemaReader(factory);
registerAvroSchemaReader(factory);
registerProtobufSchemaReader(factory);
registerProtobufListSchemaReader(factory);
registerLineAsStringSchemaReader(factory);
registerJSONAsStringSchemaReader(factory);
registerRawBLOBSchemaReader(factory);

View File

@ -0,0 +1,97 @@
#include "ProtobufListInputFormat.h"
#if USE_PROTOBUF
# include <Core/Block.h>
# include <Formats/FormatFactory.h>
# include <Formats/ProtobufReader.h>
# include <Formats/ProtobufSchemas.h>
# include <Formats/ProtobufSerializer.h>
namespace DB
{
ProtobufListInputFormat::ProtobufListInputFormat(ReadBuffer & in_, const Block & header_, const Params & params_, const FormatSchemaInfo & schema_info_)
: IRowInputFormat(header_, in_, params_)
, reader(std::make_unique<ProtobufReader>(in_))
, serializer(ProtobufSerializer::create(
header_.getNames(),
header_.getDataTypes(),
missing_column_indices,
*ProtobufSchemas::instance().getMessageTypeForFormatSchema(schema_info_, ProtobufSchemas::WithEnvelope::Yes),
/* with_length_delimiter = */ true,
/* with_envelope = */ true,
*reader))
{
}
bool ProtobufListInputFormat::readRow(MutableColumns & columns, RowReadExtension & row_read_extension)
{
if (reader->eof())
{
reader->endMessage(/*ignore_errors =*/ false);
return false;
}
size_t row_num = columns.empty() ? 0 : columns[0]->size();
if (!row_num)
serializer->setColumns(columns.data(), columns.size());
serializer->readRow(row_num);
row_read_extension.read_columns.clear();
row_read_extension.read_columns.resize(columns.size(), true);
for (size_t column_idx : missing_column_indices)
row_read_extension.read_columns[column_idx] = false;
return true;
}
ProtobufListSchemaReader::ProtobufListSchemaReader(const FormatSettings & format_settings)
: schema_info(
format_settings.schema.format_schema,
"Protobuf",
true,
format_settings.schema.is_server,
format_settings.schema.format_schema_path)
{
}
NamesAndTypesList ProtobufListSchemaReader::readSchema()
{
const auto * message_descriptor = ProtobufSchemas::instance().getMessageTypeForFormatSchema(schema_info, ProtobufSchemas::WithEnvelope::Yes);
return protobufSchemaToCHSchema(message_descriptor);
}
void registerInputFormatProtobufList(FormatFactory & factory)
{
factory.registerInputFormat(
"ProtobufList",
[](ReadBuffer &buf,
const Block & sample,
RowInputFormatParams params,
const FormatSettings & settings)
{
return std::make_shared<ProtobufListInputFormat>(buf, sample, std::move(params), FormatSchemaInfo(settings, "Protobuf", true));
});
factory.markFormatAsColumnOriented("ProtobufList");
}
void registerProtobufListSchemaReader(FormatFactory & factory)
{
factory.registerExternalSchemaReader("ProtobufList", [](const FormatSettings & settings)
{
return std::make_shared<ProtobufListSchemaReader>(settings);
});
}
}
#else
namespace DB
{
class FormatFactory;
void registerInputFormatProtobufList(FormatFactory &) {}
void registerProtobufListSchemaReader(FormatFactory &) {}
}
#endif

View File

@ -0,0 +1,52 @@
#pragma once
#include "config_formats.h"
#if USE_PROTOBUF
# include <Formats/FormatSchemaInfo.h>
# include <Processors/Formats/IRowInputFormat.h>
# include <Processors/Formats/ISchemaReader.h>
namespace DB
{
class Block;
class ProtobufReader;
class ProtobufSerializer;
class ReadBuffer;
/** Stream designed to deserialize data from the google protobuf format.
* One nested Protobuf message is parsed as one row of data.
*
* Parsing of the protobuf format requires the 'format_schema' setting to be set, e.g.
* INSERT INTO table FORMAT Protobuf SETTINGS format_schema = 'schema:Message'
* where schema is the name of "schema.proto" file specifying protobuf schema.
*/
class ProtobufListInputFormat final : public IRowInputFormat
{
public:
ProtobufListInputFormat(ReadBuffer & in_, const Block & header_, const Params & params_, const FormatSchemaInfo & schema_info_);
String getName() const override { return "ProtobufListInputFormat"; }
private:
bool readRow(MutableColumns & columns, RowReadExtension & row_read_extension) override;
std::unique_ptr<ProtobufReader> reader;
std::vector<size_t> missing_column_indices;
std::unique_ptr<ProtobufSerializer> serializer;
};
class ProtobufListSchemaReader : public IExternalSchemaReader
{
public:
explicit ProtobufListSchemaReader(const FormatSettings & format_settings);
NamesAndTypesList readSchema() override;
private:
const FormatSchemaInfo schema_info;
};
}
#endif

View File

@ -0,0 +1,68 @@
#include "ProtobufListOutputFormat.h"
#if USE_PROTOBUF
# include <Formats/FormatFactory.h>
# include <Formats/FormatSchemaInfo.h>
# include <Formats/ProtobufWriter.h>
# include <Formats/ProtobufSerializer.h>
# include <Formats/ProtobufSchemas.h>
namespace DB
{
ProtobufListOutputFormat::ProtobufListOutputFormat(
WriteBuffer & out_,
const Block & header_,
const RowOutputFormatParams & params_,
const FormatSchemaInfo & schema_info_)
: IRowOutputFormat(header_, out_, params_)
, writer(std::make_unique<ProtobufWriter>(out))
, serializer(ProtobufSerializer::create(
header_.getNames(),
header_.getDataTypes(),
*ProtobufSchemas::instance().getMessageTypeForFormatSchema(schema_info_, ProtobufSchemas::WithEnvelope::Yes),
/* with_length_delimiter = */ true,
/* with_envelope = */ true,
*writer))
{
}
void ProtobufListOutputFormat::write(const Columns & columns, size_t row_num)
{
if (row_num == 0)
serializer->setColumns(columns.data(), columns.size());
serializer->writeRow(row_num);
}
void ProtobufListOutputFormat::finalizeImpl()
{
serializer->finalizeWrite();
}
void registerOutputFormatProtobufList(FormatFactory & factory)
{
factory.registerOutputFormat(
"ProtobufList",
[](WriteBuffer & buf,
const Block & header,
const RowOutputFormatParams & params,
const FormatSettings & settings)
{
return std::make_shared<ProtobufListOutputFormat>(
buf, header, params,
FormatSchemaInfo(settings, "Protobuf", true));
});
}
}
#else
namespace DB
{
class FormatFactory;
void registerOutputFormatProtobufList(FormatFactory &) {}
}
#endif

View File

@ -0,0 +1,48 @@
#pragma once
#include "config_formats.h"
#if USE_PROTOBUF
# include <Processors/Formats/IRowOutputFormat.h>
namespace DB
{
class FormatSchemaInfo;
class ProtobufWriter;
class ProtobufSerializer;
/** Stream designed to serialize data in the google protobuf format.
* Each row is written as a separated nested message, and all rows are enclosed by a single
* top-level, envelope message
*
* Serializing in the protobuf format requires the 'format_schema' setting to be set, e.g.
* SELECT * from table FORMAT Protobuf SETTINGS format_schema = 'schema:Message'
* where schema is the name of "schema.proto" file specifying protobuf schema.
*/
// class ProtobufListOutputFormat final : public IOutputFormat
class ProtobufListOutputFormat final : public IRowOutputFormat
{
public:
ProtobufListOutputFormat(
WriteBuffer & out_,
const Block & header_,
const RowOutputFormatParams & params_,
const FormatSchemaInfo & schema_info_);
String getName() const override { return "ProtobufListOutputFormat"; }
String getContentType() const override { return "application/octet-stream"; }
private:
void write(const Columns & columns, size_t row_num) override;
void writeField(const IColumn &, const ISerialization &, size_t) override {}
void finalizeImpl() override;
std::unique_ptr<ProtobufWriter> writer;
std::unique_ptr<ProtobufSerializer> serializer;
};
}
#endif

View File

@ -3,16 +3,13 @@
#if USE_PROTOBUF
# include <Core/Block.h>
# include <Formats/FormatFactory.h>
# include <Formats/FormatSchemaInfo.h>
# include <Formats/ProtobufReader.h>
# include <Formats/ProtobufSchemas.h>
# include <Formats/ProtobufSerializer.h>
# include <Interpreters/Context.h>
# include <base/range.h>
namespace DB
{
ProtobufRowInputFormat::ProtobufRowInputFormat(ReadBuffer & in_, const Block & header_, const Params & params_, const FormatSchemaInfo & schema_info_, bool with_length_delimiter_)
: IRowInputFormat(header_, in_, params_)
, reader(std::make_unique<ProtobufReader>(in_))
@ -20,14 +17,13 @@ ProtobufRowInputFormat::ProtobufRowInputFormat(ReadBuffer & in_, const Block & h
header_.getNames(),
header_.getDataTypes(),
missing_column_indices,
*ProtobufSchemas::instance().getMessageTypeForFormatSchema(schema_info_),
*ProtobufSchemas::instance().getMessageTypeForFormatSchema(schema_info_, ProtobufSchemas::WithEnvelope::No),
with_length_delimiter_,
/* with_envelope = */ false,
*reader))
{
}
ProtobufRowInputFormat::~ProtobufRowInputFormat() = default;
bool ProtobufRowInputFormat::readRow(MutableColumns & columns, RowReadExtension & row_read_extension)
{
if (reader->eof())
@ -85,7 +81,7 @@ ProtobufSchemaReader::ProtobufSchemaReader(const FormatSettings & format_setting
NamesAndTypesList ProtobufSchemaReader::readSchema()
{
const auto * message_descriptor = ProtobufSchemas::instance().getMessageTypeForFormatSchema(schema_info);
const auto * message_descriptor = ProtobufSchemas::instance().getMessageTypeForFormatSchema(schema_info, ProtobufSchemas::WithEnvelope::No);
return protobufSchemaToCHSchema(message_descriptor);
}
@ -111,7 +107,6 @@ namespace DB
{
class FormatFactory;
void registerInputFormatProtobuf(FormatFactory &) {}
void registerProtobufSchemaReader(FormatFactory &) {}
}

View File

@ -3,17 +3,16 @@
#include "config_formats.h"
#if USE_PROTOBUF
# include <Formats/FormatSchemaInfo.h>
# include <Processors/Formats/IRowInputFormat.h>
# include <Processors/Formats/ISchemaReader.h>
# include <Formats/FormatSchemaInfo.h>
namespace DB
{
class Block;
class FormatSchemaInfo;
class ProtobufReader;
class ProtobufSerializer;
class ReadBuffer;
/** Stream designed to deserialize data from the google protobuf format.
* One Protobuf message is parsed as one row of data.
@ -30,12 +29,11 @@ class ProtobufRowInputFormat final : public IRowInputFormat
{
public:
ProtobufRowInputFormat(ReadBuffer & in_, const Block & header_, const Params & params_, const FormatSchemaInfo & schema_info_, bool with_length_delimiter_);
~ProtobufRowInputFormat() override;
String getName() const override { return "ProtobufRowInputFormat"; }
private:
bool readRow(MutableColumns & columns, RowReadExtension &) override;
bool readRow(MutableColumns & columns, RowReadExtension & row_read_extension) override;
bool allowSyncAfterError() const override;
void syncAfterError() override;
@ -52,7 +50,7 @@ public:
NamesAndTypesList readSchema() override;
private:
FormatSchemaInfo schema_info;
const FormatSchemaInfo schema_info;
};
}

View File

@ -4,12 +4,12 @@
# include <Formats/FormatFactory.h>
# include <Core/Block.h>
# include <Formats/FormatSchemaInfo.h>
# include <Formats/FormatSettings.h>
# include <Formats/ProtobufSchemas.h>
# include <Formats/ProtobufSerializer.h>
# include <Formats/ProtobufWriter.h>
# include <google/protobuf/descriptor.h>
namespace DB
{
namespace ErrorCodes
@ -17,7 +17,6 @@ namespace ErrorCodes
extern const int NO_ROW_DELIMITER;
}
ProtobufRowOutputFormat::ProtobufRowOutputFormat(
WriteBuffer & out_,
const Block & header_,
@ -30,8 +29,9 @@ ProtobufRowOutputFormat::ProtobufRowOutputFormat(
, serializer(ProtobufSerializer::create(
header_.getNames(),
header_.getDataTypes(),
*ProtobufSchemas::instance().getMessageTypeForFormatSchema(schema_info_),
*ProtobufSchemas::instance().getMessageTypeForFormatSchema(schema_info_, ProtobufSchemas::WithEnvelope::No),
with_length_delimiter_,
/* with_envelope = */ false,
*writer))
, allow_multiple_rows(with_length_delimiter_ || settings_.protobuf.allow_multiple_rows_without_delimiter)
{
@ -44,13 +44,12 @@ void ProtobufRowOutputFormat::write(const Columns & columns, size_t row_num)
"The ProtobufSingle format can't be used to write multiple rows because this format doesn't have any row delimiter.",
ErrorCodes::NO_ROW_DELIMITER);
if (!row_num)
if (row_num == 0)
serializer->setColumns(columns.data(), columns.size());
serializer->writeRow(row_num);
}
void registerOutputFormatProtobuf(FormatFactory & factory)
{
for (bool with_length_delimiter : {false, true})

View File

@ -3,17 +3,15 @@
#include "config_formats.h"
#if USE_PROTOBUF
# include <Core/Block.h>
# include <Formats/FormatSchemaInfo.h>
# include <Formats/FormatSettings.h>
# include <Processors/Formats/IRowOutputFormat.h>
namespace DB
{
class ProtobufWriter;
class ProtobufSerializer;
class DB;
class FormatSchemaInfo;
class ProtobufSerializer;
class ProtobufWriter;
class WriteBuffer;
struct FormatSettings;
/** Stream designed to serialize data in the google protobuf format.

View File

@ -1,6 +1,10 @@
#!/usr/bin/env bash
# Tags: no-fasttest
# End-to-end test of serialization/deserialization of a table with different
# data types to/from Protobuf format.
# Cf. 02240_protobuflist_format_persons.sh
# To generate reference file for this test use the following commands:
# ninja ProtobufDelimitedMessagesSerializer
# build/utils/test-data-generator/ProtobufDelimitedMessagesSerializer

View File

@ -28,7 +28,7 @@ def run_test(data_format, gen_data_template, settings):
exit(1)
formats = client.query("SELECT name FROM system.formats WHERE is_input AND is_output \
AND name NOT IN ('CapnProto', 'RawBLOB', 'Template', 'ProtobufSingle', 'LineAsString', 'Protobuf') ORDER BY name").strip().split('\n')
AND name NOT IN ('CapnProto', 'RawBLOB', 'Template', 'ProtobufSingle', 'LineAsString', 'Protobuf', 'ProtobufList') ORDER BY name").strip().split('\n')
# Generic formats
client.query("DROP TABLE IF EXISTS t_async_insert")

View File

@ -0,0 +1,569 @@
a7522158-3d41-4b77-ad69-6c598ee55c49 Ivan Petrov male 1980-12-29 png +74951234567\0 1 2019-01-05 18:45:00 38 capricorn ['Yesterday','Flowers'] [255,0,0] Moscow [55.753215,37.622504] 3.14 214.1 0.1 5.8 17060000000 ['meter','centimeter','kilometer'] [1,0.01,1000] 500 [501,502]
c694ad8a-f714-4ea3-907d-fd54fb25d9b5 Natalia Sokolova female 1992-03-08 jpg \N 0 \N 26 pisces [] [100,200,50] Plymouth [50.403724,-4.142123] 3.14159 \N 0.007 5.4 -20000000000000 [] [] \N []
a7da1aa6-f425-4789-8947-b034786ed374 Vasily Sidorov male 1995-07-28 bmp +442012345678 1 2018-12-30 00:00:00 23 leo ['Sunny'] [250,244,10] Murmansk [68.970682,33.074981] 3.14159265358979 100000000000 800 -3.2 154400000 ['pound'] [16] 503 []
Schema 02240_protobuflist1_format_persons:Person
Binary representation:
00000000 ba 04 0a f4 01 0a 24 61 37 35 32 32 31 35 38 2d |......$a7522158-|
00000010 33 64 34 31 2d 34 62 37 37 2d 61 64 36 39 2d 36 |3d41-4b77-ad69-6|
00000020 63 35 39 38 65 65 35 35 63 34 39 12 04 49 76 61 |c598ee55c49..Iva|
00000030 6e 1a 06 50 65 74 72 6f 76 20 01 28 af 1f 32 03 |n..Petrov .(..2.|
00000040 70 6e 67 3a 0d 2b 37 34 39 35 31 32 33 34 35 36 |png:.+7495123456|
00000050 37 00 40 01 4d fc d0 30 5c 50 26 58 09 62 09 59 |7.@.M..0\P&X.b.Y|
00000060 65 73 74 65 72 64 61 79 62 07 46 6c 6f 77 65 72 |esterdayb.Flower|
00000070 73 6a 04 ff 01 00 00 72 06 4d 6f 73 63 6f 77 7a |sj.....r.Moscowz|
00000080 08 4b 03 5f 42 72 7d 16 42 81 01 1f 85 eb 51 b8 |.K._Br}.B.....Q.|
00000090 1e 09 40 89 01 33 33 33 33 33 c3 6a 40 95 01 cd |..@..33333.j@...|
000000a0 cc cc 3d 9d 01 9a 99 b9 40 a0 01 80 c4 d7 8d 7f |..=.....@.......|
000000b0 aa 01 0c 0a 05 6d 65 74 65 72 15 00 00 80 3f aa |.....meter....?.|
000000c0 01 11 0a 0a 63 65 6e 74 69 6d 65 74 65 72 15 0a |....centimeter..|
000000d0 d7 23 3c aa 01 10 0a 09 6b 69 6c 6f 6d 65 74 65 |.#<.....kilomete|
000000e0 72 15 00 00 7a 44 b2 01 10 0a 0e a2 06 0b 0a 09 |r...zD..........|
000000f0 08 f4 03 12 04 f5 03 f6 03 0a 7e 0a 24 63 36 39 |..........~.$c69|
00000100 34 61 64 38 61 2d 66 37 31 34 2d 34 65 61 33 2d |4ad8a-f714-4ea3-|
00000110 39 30 37 64 2d 66 64 35 34 66 62 32 35 64 39 62 |907d-fd54fb25d9b|
00000120 35 12 07 4e 61 74 61 6c 69 61 1a 08 53 6f 6b 6f |5..Natalia..Soko|
00000130 6c 6f 76 61 28 a6 3f 32 03 6a 70 67 50 1a 58 0b |lova(.?2.jpgP.X.|
00000140 6a 04 64 c8 01 32 72 08 50 6c 79 6d 6f 75 74 68 |j.d..2r.Plymouth|
00000150 7a 08 6a 9d 49 42 46 8c 84 c0 81 01 6e 86 1b f0 |z.j.IBF.....n...|
00000160 f9 21 09 40 95 01 42 60 e5 3b 9d 01 cd cc ac 40 |.!.@..B`.;.....@|
00000170 a0 01 ff ff a9 ce 93 8c 09 0a c0 01 0a 24 61 37 |.............$a7|
00000180 64 61 31 61 61 36 2d 66 34 32 35 2d 34 37 38 39 |da1aa6-f425-4789|
00000190 2d 38 39 34 37 2d 62 30 33 34 37 38 36 65 64 33 |-8947-b034786ed3|
000001a0 37 34 12 06 56 61 73 69 6c 79 1a 07 53 69 64 6f |74..Vasily..Sido|
000001b0 72 6f 76 20 01 28 fb 48 32 03 62 6d 70 3a 0d 2b |rov .(.H2.bmp:.+|
000001c0 34 34 32 30 31 32 33 34 35 36 37 38 40 01 4d 50 |442012345678@.MP|
000001d0 e0 27 5c 50 17 58 04 62 05 53 75 6e 6e 79 6a 05 |.'\P.X.b.Sunnyj.|
000001e0 fa 01 f4 01 0a 72 08 4d 75 72 6d 61 6e 73 6b 7a |.....r.Murmanskz|
000001f0 08 fd f0 89 42 c8 4c 04 42 81 01 11 2d 44 54 fb |....B.L.B...-DT.|
00000200 21 09 40 89 01 00 00 00 e8 76 48 37 42 95 01 00 |!.@......vH7B...|
00000210 00 48 44 9d 01 cd cc 4c c0 a0 01 80 d4 9f 93 01 |.HD....L........|
00000220 aa 01 0c 0a 05 70 6f 75 6e 64 15 00 00 80 41 b2 |.....pound....A.|
00000230 01 0a 0a 08 a2 06 05 0a 03 08 f7 03 |............|
0000023c
MESSAGE #1 AT 0x00000005
uuid: "a7522158-3d41-4b77-ad69-6c598ee55c49"
name: "Ivan"
surname: "Petrov"
gender: male
birthDate: 4015
photo: "png"
phoneNumber: "+74951234567\000"
isOnline: true
visitTime: 1546703100
age: 38
zodiacSign: capricorn
songs: "Yesterday"
songs: "Flowers"
color: 255
color: 0
color: 0
hometown: "Moscow"
location: 55.7532158
location: 37.6225052
pi: 3.14
lotteryWin: 214.1
someRatio: 0.1
temperature: 5.8
randomBigNumber: 17060000000
measureUnits {
unit: "meter"
coef: 1
}
measureUnits {
unit: "centimeter"
coef: 0.01
}
measureUnits {
unit: "kilometer"
coef: 1000
}
nestiness {
a {
b {
c {
d: 500
e: 501
e: 502
}
}
}
}
MESSAGE #2 AT 0x000000FB
uuid: "c694ad8a-f714-4ea3-907d-fd54fb25d9b5"
name: "Natalia"
surname: "Sokolova"
birthDate: 8102
photo: "jpg"
age: 26
zodiacSign: pisces
color: 100
color: 200
color: 50
hometown: "Plymouth"
location: 50.4037247
location: -4.14212322
pi: 3.14159
someRatio: 0.007
temperature: 5.4
randomBigNumber: -20000000000000
MESSAGE #3 AT 0x0000017C
uuid: "a7da1aa6-f425-4789-8947-b034786ed374"
name: "Vasily"
surname: "Sidorov"
gender: male
birthDate: 9339
photo: "bmp"
phoneNumber: "+442012345678"
isOnline: true
visitTime: 1546117200
age: 23
zodiacSign: leo
songs: "Sunny"
color: 250
color: 244
color: 10
hometown: "Murmansk"
location: 68.9706802
location: 33.0749817
pi: 3.14159265358979
lotteryWin: 100000000000
someRatio: 800
temperature: -3.2
randomBigNumber: 154400000
measureUnits {
unit: "pound"
coef: 16
}
nestiness {
a {
b {
c {
d: 503
}
}
}
}
Binary representation is as expected
Roundtrip:
a7522158-3d41-4b77-ad69-6c598ee55c49 Ivan Petrov male 1980-12-29 png +74951234567\0 1 2019-01-05 18:45:00 38 capricorn ['Yesterday','Flowers'] [255,0,0] Moscow [55.753216,37.622504] 3.14 214.1 0.1 5.8 17060000000 ['meter','centimeter','kilometer'] [1,0.01,1000] 500 [501,502]
c694ad8a-f714-4ea3-907d-fd54fb25d9b5 Natalia Sokolova female 1992-03-08 jpg \N 0 \N 26 pisces [] [100,200,50] Plymouth [50.403724,-4.142123] 3.14159 \N 0.007 5.4 -20000000000000 [] [] \N []
a7da1aa6-f425-4789-8947-b034786ed374 Vasily Sidorov male 1995-07-28 bmp +442012345678 1 2018-12-30 00:00:00 23 leo ['Sunny'] [250,244,10] Murmansk [68.97068,33.074982] 3.14159265358979 100000000000 800 -3.2 154400000 ['pound'] [16] 503 []
Schema 02240_protobuflist2_format_persons:AltPerson
Binary representation:
00000000 f4 03 0a c4 01 08 01 12 04 49 76 61 6e 28 87 a8 |.........Ivan(..|
00000010 c4 9b 97 02 52 06 50 65 74 72 6f 76 72 0c 00 00 |....R.Petrovr...|
00000020 7f 43 00 00 00 00 00 00 00 00 79 fc d0 30 5c 00 |.C........y..0\.|
00000030 00 00 00 c8 02 0a c2 05 0c 00 00 80 3f 0a d7 23 |............?..#|
00000040 3c 00 00 7a 44 9a 06 05 6d 65 74 65 72 9a 06 0a |<..zD...meter...|
00000050 63 65 6e 74 69 6d 65 74 65 72 9a 06 09 6b 69 6c |centimeter...kil|
00000060 6f 6d 65 74 65 72 a1 06 00 00 00 a0 99 99 b9 3f |ometer.........?|
00000070 a8 06 37 a8 06 25 bd 06 c3 f5 48 40 fa 06 02 33 |..7..%....H@...3|
00000080 38 90 08 c6 09 e1 08 00 f1 da f8 03 00 00 00 b0 |8...............|
00000090 09 af 1f d0 0c d6 01 e2 12 24 61 37 35 32 32 31 |.........$a75221|
000000a0 35 38 2d 33 64 34 31 2d 34 62 37 37 2d 61 64 36 |58-3d41-4b77-ad6|
000000b0 39 2d 36 63 35 39 38 65 65 35 35 63 34 39 a0 38 |9-6c598ee55c49.8|
000000c0 f4 03 aa 38 04 f5 03 f6 03 0a 84 01 12 07 4e 61 |...8..........Na|
000000d0 74 61 6c 69 61 52 08 53 6f 6b 6f 6c 6f 76 61 72 |taliaR.Sokolovar|
000000e0 0c 00 00 c8 42 00 00 48 43 00 00 48 42 c8 02 0a |....B..HC..HB...|
000000f0 a1 06 00 00 00 40 08 ac 7c 3f a8 06 32 a8 06 fc |.....@..|?..2...|
00000100 ff ff ff ff ff ff ff ff 01 b0 06 01 bd 06 d0 0f |................|
00000110 49 40 fa 06 02 32 36 90 08 db 01 e1 08 00 c0 1a |I@...26.........|
00000120 63 cf ed ff ff b0 09 a6 3f e2 12 24 63 36 39 34 |c.......?..$c694|
00000130 61 64 38 61 2d 66 37 31 34 2d 34 65 61 33 2d 39 |ad8a-f714-4ea3-9|
00000140 30 37 64 2d 66 64 35 34 66 62 32 35 64 39 62 35 |07d-fd54fb25d9b5|
00000150 0a a3 01 08 01 12 06 56 61 73 69 6c 79 28 ce ca |.......Vasily(..|
00000160 f4 cf ee 0c 52 07 53 69 64 6f 72 6f 76 72 0c 00 |....R.Sidorovr..|
00000170 00 7a 43 00 00 74 43 00 00 20 41 79 50 e0 27 5c |.zC..tC.. AyP.'\|
00000180 00 00 00 00 c8 02 05 c2 05 04 00 00 80 41 9a 06 |.............A..|
00000190 05 70 6f 75 6e 64 a1 06 00 00 00 00 00 00 89 40 |.pound.........@|
000001a0 a8 06 44 a8 06 21 bd 06 db 0f 49 40 fa 06 02 32 |..D..!....I@...2|
000001b0 33 90 08 d3 05 e1 08 00 f5 33 09 00 00 00 00 b0 |3........3......|
000001c0 09 fb 48 d0 0c 80 d0 db c3 f4 02 e2 12 24 61 37 |..H..........$a7|
000001d0 64 61 31 61 61 36 2d 66 34 32 35 2d 34 37 38 39 |da1aa6-f425-4789|
000001e0 2d 38 39 34 37 2d 62 30 33 34 37 38 36 65 64 33 |-8947-b034786ed3|
000001f0 37 34 a0 38 f7 03 |74.8..|
000001f6
MESSAGE #1 AT 0x00000005
isOnline: online
name: "Ivan"
phoneNumber: 74951234567
surname: "Petrov"
color: 255
color: 0
color: 0
visitTime: 1546703100
temperature: 5
measureUnits_coef: 1
measureUnits_coef: 0.01
measureUnits_coef: 1000
measureUnits_unit: "meter"
measureUnits_unit: "centimeter"
measureUnits_unit: "kilometer"
someRatio: 0.10000000149011612
location: 55
location: 37
pi: 3.14
age: "38"
zodiacSign: 1222
randomBigNumber: 17060000000
birthDate: 4015
lotteryWin: 214
uuid: "a7522158-3d41-4b77-ad69-6c598ee55c49"
nestiness_a_b_c_d: 500
nestiness_a_b_c_e: 501
nestiness_a_b_c_e: 502
MESSAGE #2 AT 0x000000CC
name: "Natalia"
surname: "Sokolova"
color: 100
color: 200
color: 50
temperature: 5
someRatio: 0.0070000002160668373
location: 50
location: -4
gender: female
pi: 3.14159
age: "26"
zodiacSign: 219
randomBigNumber: -20000000000000
birthDate: 8102
uuid: "c694ad8a-f714-4ea3-907d-fd54fb25d9b5"
MESSAGE #3 AT 0x00000153
isOnline: online
name: "Vasily"
phoneNumber: 442012345678
surname: "Sidorov"
color: 250
color: 244
color: 10
visitTime: 1546117200
temperature: -3
measureUnits_coef: 16
measureUnits_unit: "pound"
someRatio: 800
location: 68
location: 33
pi: 3.14159274
age: "23"
zodiacSign: 723
randomBigNumber: 154400000
birthDate: 9339
lotteryWin: 100000000000
uuid: "a7da1aa6-f425-4789-8947-b034786ed374"
nestiness_a_b_c_d: 503
Binary representation is as expected
Roundtrip:
a7522158-3d41-4b77-ad69-6c598ee55c49 Ivan Petrov male 1980-12-29 \N 74951234567\0\0 1 2019-01-05 18:45:00 38 capricorn [] [255,0,0] [55,37] 3.140000104904175 214 0.1 5 17060000000 ['meter','centimeter','kilometer'] [1,0.01,1000] 500 [501,502]
c694ad8a-f714-4ea3-907d-fd54fb25d9b5 Natalia Sokolova female 1992-03-08 \N \N 0 \N 26 pisces [] [100,200,50] [50,-4] 3.141590118408203 \N 0.007 5 -20000000000000 [] [] \N []
a7da1aa6-f425-4789-8947-b034786ed374 Vasily Sidorov male 1995-07-28 \N 442012345678\0 1 2018-12-30 00:00:00 23 leo [] [250,244,10] [68,33] 3.1415927410125732 100000000000 800 -3 154400000 ['pound'] [16] 503 []
Schema 02240_protobuflist3_format_persons:StrPerson as ProtobufList
Binary representation:
00000000 e4 05 0a a6 02 0a 24 61 37 35 32 32 31 35 38 2d |......$a7522158-|
00000010 33 64 34 31 2d 34 62 37 37 2d 61 64 36 39 2d 36 |3d41-4b77-ad69-6|
00000020 63 35 39 38 65 65 35 35 63 34 39 12 04 49 76 61 |c598ee55c49..Iva|
00000030 6e 1a 06 50 65 74 72 6f 76 22 04 6d 61 6c 65 2a |n..Petrov".male*|
00000040 0a 31 39 38 30 2d 31 32 2d 32 39 3a 0d 2b 37 34 |.1980-12-29:.+74|
00000050 39 35 31 32 33 34 35 36 37 00 42 01 31 4a 13 32 |951234567.B.1J.2|
00000060 30 31 39 2d 30 31 2d 30 35 20 31 38 3a 34 35 3a |019-01-05 18:45:|
00000070 30 30 52 02 33 38 5a 09 63 61 70 72 69 63 6f 72 |00R.38Z.capricor|
00000080 6e 62 09 59 65 73 74 65 72 64 61 79 62 07 46 6c |nb.Yesterdayb.Fl|
00000090 6f 77 65 72 73 6a 03 32 35 35 6a 01 30 6a 01 30 |owersj.255j.0j.0|
000000a0 72 06 4d 6f 73 63 6f 77 7a 09 35 35 2e 37 35 33 |r.Moscowz.55.753|
000000b0 32 31 35 7a 09 33 37 2e 36 32 32 35 30 34 82 01 |215z.37.622504..|
000000c0 04 33 2e 31 34 8a 01 05 32 31 34 2e 31 92 01 03 |.3.14...214.1...|
000000d0 30 2e 31 9a 01 03 35 2e 38 a2 01 0b 31 37 30 36 |0.1...5.8...1706|
000000e0 30 30 30 30 30 30 30 aa 01 2d 0a 05 6d 65 74 65 |0000000..-..mete|
000000f0 72 0a 0a 63 65 6e 74 69 6d 65 74 65 72 0a 09 6b |r..centimeter..k|
00000100 69 6c 6f 6d 65 74 65 72 12 01 31 12 04 30 2e 30 |ilometer..1..0.0|
00000110 31 12 04 31 30 30 30 b2 01 11 0a 0f 0a 03 35 30 |1..1000.......50|
00000120 30 12 03 35 30 31 12 03 35 30 32 0a b4 01 0a 24 |0..501..502....$|
00000130 63 36 39 34 61 64 38 61 2d 66 37 31 34 2d 34 65 |c694ad8a-f714-4e|
00000140 61 33 2d 39 30 37 64 2d 66 64 35 34 66 62 32 35 |a3-907d-fd54fb25|
00000150 64 39 62 35 12 07 4e 61 74 61 6c 69 61 1a 08 53 |d9b5..Natalia..S|
00000160 6f 6b 6f 6c 6f 76 61 22 06 66 65 6d 61 6c 65 2a |okolova".female*|
00000170 0a 31 39 39 32 2d 30 33 2d 30 38 42 01 30 52 02 |.1992-03-08B.0R.|
00000180 32 36 5a 06 70 69 73 63 65 73 6a 03 31 30 30 6a |26Z.piscesj.100j|
00000190 03 32 30 30 6a 02 35 30 72 08 50 6c 79 6d 6f 75 |.200j.50r.Plymou|
000001a0 74 68 7a 09 35 30 2e 34 30 33 37 32 34 7a 09 2d |thz.50.403724z.-|
000001b0 34 2e 31 34 32 31 32 33 82 01 07 33 2e 31 34 31 |4.142123...3.141|
000001c0 35 39 92 01 05 30 2e 30 30 37 9a 01 03 35 2e 34 |59...0.007...5.4|
000001d0 a2 01 0f 2d 32 30 30 30 30 30 30 30 30 30 30 30 |...-200000000000|
000001e0 30 30 0a 81 02 0a 24 61 37 64 61 31 61 61 36 2d |00....$a7da1aa6-|
000001f0 66 34 32 35 2d 34 37 38 39 2d 38 39 34 37 2d 62 |f425-4789-8947-b|
00000200 30 33 34 37 38 36 65 64 33 37 34 12 06 56 61 73 |034786ed374..Vas|
00000210 69 6c 79 1a 07 53 69 64 6f 72 6f 76 22 04 6d 61 |ily..Sidorov".ma|
00000220 6c 65 2a 0a 31 39 39 35 2d 30 37 2d 32 38 3a 0d |le*.1995-07-28:.|
00000230 2b 34 34 32 30 31 32 33 34 35 36 37 38 42 01 31 |+442012345678B.1|
00000240 4a 13 32 30 31 38 2d 31 32 2d 33 30 20 30 30 3a |J.2018-12-30 00:|
00000250 30 30 3a 30 30 52 02 32 33 5a 03 6c 65 6f 62 05 |00:00R.23Z.leob.|
00000260 53 75 6e 6e 79 6a 03 32 35 30 6a 03 32 34 34 6a |Sunnyj.250j.244j|
00000270 02 31 30 72 08 4d 75 72 6d 61 6e 73 6b 7a 09 36 |.10r.Murmanskz.6|
00000280 38 2e 39 37 30 36 38 32 7a 09 33 33 2e 30 37 34 |8.970682z.33.074|
00000290 39 38 31 82 01 10 33 2e 31 34 31 35 39 32 36 35 |981...3.14159265|
000002a0 33 35 38 39 37 39 8a 01 0c 31 30 30 30 30 30 30 |358979...1000000|
000002b0 30 30 30 30 30 92 01 03 38 30 30 9a 01 04 2d 33 |00000...800...-3|
000002c0 2e 32 a2 01 09 31 35 34 34 30 30 30 30 30 aa 01 |.2...154400000..|
000002d0 0b 0a 05 70 6f 75 6e 64 12 02 31 36 b2 01 07 0a |...pound..16....|
000002e0 05 0a 03 35 30 33 |...503|
000002e6
MESSAGE #1 AT 0x00000005
uuid: "a7522158-3d41-4b77-ad69-6c598ee55c49"
name: "Ivan"
surname: "Petrov"
gender: "male"
birthDate: "1980-12-29"
phoneNumber: "+74951234567\000"
isOnline: "1"
visitTime: "2019-01-05 18:45:00"
age: "38"
zodiacSign: "capricorn"
songs: "Yesterday"
songs: "Flowers"
color: "255"
color: "0"
color: "0"
hometown: "Moscow"
location: "55.753215"
location: "37.622504"
pi: "3.14"
lotteryWin: "214.1"
someRatio: "0.1"
temperature: "5.8"
randomBigNumber: "17060000000"
measureUnits {
unit: "meter"
unit: "centimeter"
unit: "kilometer"
coef: "1"
coef: "0.01"
coef: "1000"
}
nestiness_a {
b_c {
d: "500"
e: "501"
e: "502"
}
}
MESSAGE #2 AT 0x0000012E
uuid: "c694ad8a-f714-4ea3-907d-fd54fb25d9b5"
name: "Natalia"
surname: "Sokolova"
gender: "female"
birthDate: "1992-03-08"
isOnline: "0"
age: "26"
zodiacSign: "pisces"
color: "100"
color: "200"
color: "50"
hometown: "Plymouth"
location: "50.403724"
location: "-4.142123"
pi: "3.14159"
someRatio: "0.007"
temperature: "5.4"
randomBigNumber: "-20000000000000"
MESSAGE #3 AT 0x000001E5
uuid: "a7da1aa6-f425-4789-8947-b034786ed374"
name: "Vasily"
surname: "Sidorov"
gender: "male"
birthDate: "1995-07-28"
phoneNumber: "+442012345678"
isOnline: "1"
visitTime: "2018-12-30 00:00:00"
age: "23"
zodiacSign: "leo"
songs: "Sunny"
color: "250"
color: "244"
color: "10"
hometown: "Murmansk"
location: "68.970682"
location: "33.074981"
pi: "3.14159265358979"
lotteryWin: "100000000000"
someRatio: "800"
temperature: "-3.2"
randomBigNumber: "154400000"
measureUnits {
unit: "pound"
coef: "16"
}
nestiness_a {
b_c {
d: "503"
}
}
Binary representation is as expected
Roundtrip:
a7522158-3d41-4b77-ad69-6c598ee55c49 Ivan Petrov male 1980-12-29 \N +74951234567\0 1 2019-01-05 18:45:00 38 capricorn ['Yesterday','Flowers'] [255,0,0] Moscow [55.753215,37.622504] 3.14 214.1 0.1 5.8 17060000000 ['meter','centimeter','kilometer'] [1,0.01,1000] 500 [501,502]
c694ad8a-f714-4ea3-907d-fd54fb25d9b5 Natalia Sokolova female 1992-03-08 \N \N 0 \N 26 pisces [] [100,200,50] Plymouth [50.403724,-4.142123] 3.14159 \N 0.007 5.4 -20000000000000 [] [] \N []
a7da1aa6-f425-4789-8947-b034786ed374 Vasily Sidorov male 1995-07-28 \N +442012345678 1 2018-12-30 00:00:00 23 leo ['Sunny'] [250,244,10] Murmansk [68.970682,33.074981] 3.14159265358979 100000000000 800 -3.2 154400000 ['pound'] [16] 503 []
Schema 02240_protobuf_format_syntax2:Syntax2Person
Binary representation:
00000000 c0 04 0a f1 01 0a 24 61 37 35 32 32 31 35 38 2d |......$a7522158-|
00000010 33 64 34 31 2d 34 62 37 37 2d 61 64 36 39 2d 36 |3d41-4b77-ad69-6|
00000020 63 35 39 38 65 65 35 35 63 34 39 12 04 49 76 61 |c598ee55c49..Iva|
00000030 6e 1a 06 50 65 74 72 6f 76 20 01 28 af 1f 32 03 |n..Petrov .(..2.|
00000040 70 6e 67 3a 0d 2b 37 34 39 35 31 32 33 34 35 36 |png:.+7495123456|
00000050 37 00 40 01 4d fc d0 30 5c 50 26 58 09 62 09 59 |7.@.M..0\P&X.b.Y|
00000060 65 73 74 65 72 64 61 79 62 07 46 6c 6f 77 65 72 |esterdayb.Flower|
00000070 73 68 ff 01 68 00 68 00 72 06 4d 6f 73 63 6f 77 |sh..h.h.r.Moscow|
00000080 7a 08 4b 03 5f 42 72 7d 16 42 81 01 1f 85 eb 51 |z.K._Br}.B.....Q|
00000090 b8 1e 09 40 89 01 33 33 33 33 33 c3 6a 40 95 01 |...@..33333.j@..|
000000a0 cd cc cc 3d 9d 01 9a 99 b9 40 a0 01 80 c4 d7 8d |...=.....@......|
000000b0 7f ab 01 0d 00 00 80 3f 0d 0a d7 23 3c 0d 00 00 |.......?...#<...|
000000c0 7a 44 12 05 6d 65 74 65 72 12 0a 63 65 6e 74 69 |zD..meter..centi|
000000d0 6d 65 74 65 72 12 09 6b 69 6c 6f 6d 65 74 65 72 |meter..kilometer|
000000e0 ac 01 b3 01 0b a2 06 0b 0b 08 f4 03 10 f5 03 10 |................|
000000f0 f6 03 0c 0c b4 01 0a 83 01 0a 24 63 36 39 34 61 |..........$c694a|
00000100 64 38 61 2d 66 37 31 34 2d 34 65 61 33 2d 39 30 |d8a-f714-4ea3-90|
00000110 37 64 2d 66 64 35 34 66 62 32 35 64 39 62 35 12 |7d-fd54fb25d9b5.|
00000120 07 4e 61 74 61 6c 69 61 1a 08 53 6f 6b 6f 6c 6f |.Natalia..Sokolo|
00000130 76 61 20 00 28 a6 3f 32 03 6a 70 67 40 00 50 1a |va .(.?2.jpg@.P.|
00000140 58 0b 68 64 68 c8 01 68 32 72 08 50 6c 79 6d 6f |X.hdh..h2r.Plymo|
00000150 75 74 68 7a 08 6a 9d 49 42 46 8c 84 c0 81 01 6e |uthz.j.IBF.....n|
00000160 86 1b f0 f9 21 09 40 95 01 42 60 e5 3b 9d 01 cd |....!.@..B`.;...|
00000170 cc ac 40 a0 01 ff ff a9 ce 93 8c 09 0a c3 01 0a |..@.............|
00000180 24 61 37 64 61 31 61 61 36 2d 66 34 32 35 2d 34 |$a7da1aa6-f425-4|
00000190 37 38 39 2d 38 39 34 37 2d 62 30 33 34 37 38 36 |789-8947-b034786|
000001a0 65 64 33 37 34 12 06 56 61 73 69 6c 79 1a 07 53 |ed374..Vasily..S|
000001b0 69 64 6f 72 6f 76 20 01 28 fb 48 32 03 62 6d 70 |idorov .(.H2.bmp|
000001c0 3a 0d 2b 34 34 32 30 31 32 33 34 35 36 37 38 40 |:.+442012345678@|
000001d0 01 4d 50 e0 27 5c 50 17 58 04 62 05 53 75 6e 6e |.MP.'\P.X.b.Sunn|
000001e0 79 68 fa 01 68 f4 01 68 0a 72 08 4d 75 72 6d 61 |yh..h..h.r.Murma|
000001f0 6e 73 6b 7a 08 fd f0 89 42 c8 4c 04 42 81 01 11 |nskz....B.L.B...|
00000200 2d 44 54 fb 21 09 40 89 01 00 00 00 e8 76 48 37 |-DT.!.@......vH7|
00000210 42 95 01 00 00 48 44 9d 01 cd cc 4c c0 a0 01 80 |B....HD....L....|
00000220 d4 9f 93 01 ab 01 0d 00 00 80 41 12 05 70 6f 75 |..........A..pou|
00000230 6e 64 ac 01 b3 01 0b a2 06 05 0b 08 f7 03 0c 0c |nd..............|
00000240 b4 01 |..|
00000242
MESSAGE #1 AT 0x00000005
uuid: "a7522158-3d41-4b77-ad69-6c598ee55c49"
name: "Ivan"
surname: "Petrov"
gender: male
birthDate: 4015
photo: "png"
phoneNumber: "+74951234567\000"
isOnline: true
visitTime: 1546703100
age: 38
zodiacSign: capricorn
songs: "Yesterday"
songs: "Flowers"
color: 255
color: 0
color: 0
hometown: "Moscow"
location: 55.7532158
location: 37.6225052
pi: 3.14
lotteryWin: 214.1
someRatio: 0.1
temperature: 5.8
randomBigNumber: 17060000000
MeasureUnits {
coef: 1
coef: 0.01
coef: 1000
unit: "meter"
unit: "centimeter"
unit: "kilometer"
}
Nestiness {
A {
b {
C {
d: 500
e: 501
e: 502
}
}
}
}
MESSAGE #2 AT 0x000000F9
uuid: "c694ad8a-f714-4ea3-907d-fd54fb25d9b5"
name: "Natalia"
surname: "Sokolova"
gender: female
birthDate: 8102
photo: "jpg"
isOnline: false
age: 26
zodiacSign: pisces
color: 100
color: 200
color: 50
hometown: "Plymouth"
location: 50.4037247
location: -4.14212322
pi: 3.14159
someRatio: 0.007
temperature: 5.4
randomBigNumber: -20000000000000
MESSAGE #3 AT 0x0000017F
uuid: "a7da1aa6-f425-4789-8947-b034786ed374"
name: "Vasily"
surname: "Sidorov"
gender: male
birthDate: 9339
photo: "bmp"
phoneNumber: "+442012345678"
isOnline: true
visitTime: 1546117200
age: 23
zodiacSign: leo
songs: "Sunny"
color: 250
color: 244
color: 10
hometown: "Murmansk"
location: 68.9706802
location: 33.0749817
pi: 3.14159265358979
lotteryWin: 100000000000
someRatio: 800
temperature: -3.2
randomBigNumber: 154400000
MeasureUnits {
coef: 16
unit: "pound"
}
Nestiness {
A {
b {
C {
d: 503
}
}
}
}
Binary representation is as expected
Roundtrip:
a7522158-3d41-4b77-ad69-6c598ee55c49 Ivan Petrov male 1980-12-29 png +74951234567\0 1 2019-01-05 18:45:00 38 capricorn ['Yesterday','Flowers'] [255,0,0] Moscow [55.753216,37.622504] 3.14 214.1 0.1 5.8 17060000000 ['meter','centimeter','kilometer'] [1,0.01,1000] 500 [501,502]
c694ad8a-f714-4ea3-907d-fd54fb25d9b5 Natalia Sokolova female 1992-03-08 jpg \N 0 \N 26 pisces [] [100,200,50] Plymouth [50.403724,-4.142123] 3.14159 \N 0.007 5.4 -20000000000000 [] [] \N []
a7da1aa6-f425-4789-8947-b034786ed374 Vasily Sidorov male 1995-07-28 bmp +442012345678 1 2018-12-30 00:00:00 23 leo ['Sunny'] [250,244,10] Murmansk [68.97068,33.074982] 3.14159265358979 100000000000 800 -3.2 154400000 ['pound'] [16] 503 []

View File

@ -0,0 +1,127 @@
#!/usr/bin/env bash
# Tags: no-fasttest
# End-to-end test of serialization/deserialization of a table with different
# data types to/from ProtobufList format.
# Cf. 00825_protobuf_format_persons.sh
# To generate reference file for this test use the following commands:
# ninja ProtobufDelimitedMessagesSerializer
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
SCHEMADIR=$CURDIR/format_schemas
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
set -eo pipefail
# Run the client.
$CLICKHOUSE_CLIENT --multiquery <<EOF
DROP TABLE IF EXISTS persons_02240;
DROP TABLE IF EXISTS roundtrip_persons_02240;
DROP TABLE IF EXISTS alt_persons_02240;
DROP TABLE IF EXISTS str_persons_02240;
DROP TABLE IF EXISTS syntax2_persons_02240;
CREATE TABLE persons_02240 (uuid UUID,
name String,
surname String,
gender Enum8('male'=1, 'female'=0),
birthDate Date,
photo Nullable(String),
phoneNumber Nullable(FixedString(13)),
isOnline UInt8,
visitTime Nullable(DateTime('Europe/Moscow')),
age UInt8,
zodiacSign Enum16('aries'=321, 'taurus'=420, 'gemini'=521, 'cancer'=621, 'leo'=723, 'virgo'=823,
'libra'=923, 'scorpius'=1023, 'sagittarius'=1122, 'capricorn'=1222, 'aquarius'=120,
'pisces'=219),
songs Array(String),
color Array(UInt8),
hometown LowCardinality(String),
location Array(Decimal32(6)),
pi Nullable(Float64),
lotteryWin Nullable(Decimal64(2)),
someRatio Float32,
temperature Decimal32(1),
randomBigNumber Int64,
measureUnits Nested(unit String, coef Float32),
nestiness_a_b_c_d Nullable(UInt32),
"nestiness_a_B.c_E" Array(UInt32)
) ENGINE = MergeTree ORDER BY tuple();
INSERT INTO persons_02240 VALUES (toUUID('a7522158-3d41-4b77-ad69-6c598ee55c49'), 'Ivan', 'Petrov', 'male', toDate('1980-12-29'), 'png', '+74951234567', 1, toDateTime('2019-01-05 18:45:00', 'Europe/Moscow'), 38, 'capricorn', ['Yesterday', 'Flowers'], [255, 0, 0], 'Moscow', [55.753215, 37.622504], 3.14, 214.10, 0.1, 5.8, 17060000000, ['meter', 'centimeter', 'kilometer'], [1, 0.01, 1000], 500, [501, 502]);
INSERT INTO persons_02240 VALUES (toUUID('c694ad8a-f714-4ea3-907d-fd54fb25d9b5'), 'Natalia', 'Sokolova', 'female', toDate('1992-03-08'), 'jpg', NULL, 0, NULL, 26, 'pisces', [], [100, 200, 50], 'Plymouth', [50.403724, -4.142123], 3.14159, NULL, 0.007, 5.4, -20000000000000, [], [], NULL, []);
INSERT INTO persons_02240 VALUES (toUUID('a7da1aa6-f425-4789-8947-b034786ed374'), 'Vasily', 'Sidorov', 'male', toDate('1995-07-28'), 'bmp', '+442012345678', 1, toDateTime('2018-12-30 00:00:00', 'Europe/Moscow'), 23, 'leo', ['Sunny'], [250, 244, 10], 'Murmansk', [68.970682, 33.074981], 3.14159265358979, 100000000000, 800, -3.2, 154400000, ['pound'], [16], 503, []);
SELECT * FROM persons_02240 ORDER BY name;
EOF
# Note: if you actually look into below used schemafiles, you find that the message payload was duplicated. This is a workaround caused by Google protoc
# not being able to decode or reference nested elements, only top-level elements. In theory, we could make protoc read the top-level Envelope message but even
# that is not possible if it is length-delimited (and it is). Protobuf_length_delimited_encoder.py with '--format "protobuflist"' takes care to remove
# the top level Envelope message manually so that the remaining (binary) nested message looks to protoc like instances of the duplicated messages. Not pretty
# but does the job ...
# Use schema 02240_protobuflist1_format_persons:Person
echo
echo "Schema 02240_protobuflist1_format_persons:Person"
BINARY_FILE_PATH=$(mktemp "$CURDIR/02240_protobuflist1_format_persons.XXXXXX.binary")
$CLICKHOUSE_CLIENT --query "SELECT * FROM persons_02240 ORDER BY name FORMAT ProtobufList SETTINGS format_schema = '$SCHEMADIR/02240_protobuflist1_format_persons:Person'" > $BINARY_FILE_PATH
echo
$CURDIR/helpers/protobuf_length_delimited_encoder.py --decode_and_check --format_schema "$SCHEMADIR/02240_protobuflist1_format_persons:Person" --input "$BINARY_FILE_PATH" --format "protobuflist"
echo
echo "Roundtrip:"
$CLICKHOUSE_CLIENT --query "CREATE TABLE roundtrip_persons_02240 AS persons_02240"
$CLICKHOUSE_CLIENT --query "INSERT INTO roundtrip_persons_02240 FORMAT ProtobufList SETTINGS format_schema='$SCHEMADIR/02240_protobuflist1_format_persons:Person'" < "$BINARY_FILE_PATH"
$CLICKHOUSE_CLIENT --query "SELECT * FROM roundtrip_persons_02240 ORDER BY name"
rm "$BINARY_FILE_PATH"
# Use schema 02240_protobuflist2_format_persons:AltPerson
echo
echo "Schema 02240_protobuflist2_format_persons:AltPerson"
BINARY_FILE_PATH=$(mktemp "$CURDIR/02240_protobuflist2_format_persons.XXXXXX.binary")
$CLICKHOUSE_CLIENT --query "SELECT * FROM persons_02240 ORDER BY name FORMAT ProtobufList SETTINGS format_schema = '$SCHEMADIR/02240_protobuflist2_format_persons:AltPerson'" > $BINARY_FILE_PATH
echo
$CURDIR/helpers/protobuf_length_delimited_encoder.py --decode_and_check --format_schema "$SCHEMADIR/02240_protobuflist2_format_persons:AltPerson" --input "$BINARY_FILE_PATH" --format="protobuflist"
echo
echo "Roundtrip:"
$CLICKHOUSE_CLIENT --query "CREATE TABLE alt_persons_02240 AS persons_02240"
$CLICKHOUSE_CLIENT --query "INSERT INTO alt_persons_02240 FORMAT ProtobufList SETTINGS format_schema='$SCHEMADIR/02240_protobuflist2_format_persons:AltPerson'" < "$BINARY_FILE_PATH"
$CLICKHOUSE_CLIENT --query "SELECT * FROM alt_persons_02240 ORDER BY name"
rm "$BINARY_FILE_PATH"
# Use schema 02240_protobuflist3_format_persons:StrPerson
echo
echo "Schema 02240_protobuflist3_format_persons:StrPerson as ProtobufList"
BINARY_FILE_PATH=$(mktemp "$CURDIR/02240_protobuflist3_format_persons.XXXXXX.binary")
$CLICKHOUSE_CLIENT --query "SELECT * FROM persons_02240 ORDER BY name FORMAT ProtobufList SETTINGS format_schema = '$SCHEMADIR/02240_protobuflist3_format_persons:StrPerson'" > $BINARY_FILE_PATH
echo
$CURDIR/helpers/protobuf_length_delimited_encoder.py --decode_and_check --format_schema "$SCHEMADIR/02240_protobuflist3_format_persons:StrPerson" --input "$BINARY_FILE_PATH" --format="protobuflist"
# echo
echo "Roundtrip:"
$CLICKHOUSE_CLIENT --query "CREATE TABLE str_persons_02240 AS persons_02240"
$CLICKHOUSE_CLIENT --query "INSERT INTO str_persons_02240 FORMAT ProtobufList SETTINGS format_schema='$SCHEMADIR/02240_protobuflist3_format_persons:StrPerson'" < "$BINARY_FILE_PATH"
$CLICKHOUSE_CLIENT --query "SELECT * FROM str_persons_02240 ORDER BY name"
rm "$BINARY_FILE_PATH"
# Use schema 02240_protobuflist_format_syntax2:Syntax2Person
echo
echo "Schema 02240_protobuf_format_syntax2:Syntax2Person"
BINARY_FILE_PATH=$(mktemp "$CURDIR/02240_protobuflist_format_persons.XXXXXX.binary")
$CLICKHOUSE_CLIENT --query "SELECT * FROM persons_02240 ORDER BY name FORMAT ProtobufList SETTINGS format_schema = '$SCHEMADIR/02240_protobuflist_format_persons_syntax2:Syntax2Person'" > $BINARY_FILE_PATH
echo
$CURDIR/helpers/protobuf_length_delimited_encoder.py --decode_and_check --format_schema "$SCHEMADIR/02240_protobuflist_format_persons_syntax2:Syntax2Person" --input "$BINARY_FILE_PATH" --format="protobuflist"
echo
echo "Roundtrip:"
$CLICKHOUSE_CLIENT --query "CREATE TABLE syntax2_persons_02240 AS persons_02240"
$CLICKHOUSE_CLIENT --query "INSERT INTO syntax2_persons_02240 FORMAT ProtobufList SETTINGS format_schema='$SCHEMADIR/02240_protobuflist_format_persons_syntax2:Syntax2Person'" < "$BINARY_FILE_PATH"
$CLICKHOUSE_CLIENT --query "SELECT * FROM syntax2_persons_02240 ORDER BY name"
rm "$BINARY_FILE_PATH"
$CLICKHOUSE_CLIENT --multiquery <<EOF
DROP TABLE persons_02240;
DROP TABLE roundtrip_persons_02240;
DROP TABLE alt_persons_02240;
DROP TABLE str_persons_02240;
DROP TABLE syntax2_persons_02240;
EOF

View File

@ -0,0 +1,100 @@
syntax = "proto3";
enum Gender {
female = 0;
male = 1;
};
enum ZodiacSign {
aries = 0;
taurus = 1;
gemini = 2;
cancer = 3;
leo = 4;
virgo = 5;
libra = 6;
scorpius = 7;
sagittarius = 8;
capricorn = 9;
aquarius = 10;
pisces = 11;
};
message Nestiness
{
message SubA {
message SubB {
message SubC {
uint32 d = 1;
repeated uint32 e = 2;
};
SubC c = 1;
};
SubB b = 100;
};
SubA a = 1;
};
message Envelope {
message Person {
message MeasureUnit
{
string unit = 1;
float coef = 2;
};
string uuid = 1;
string name = 2;
string surname = 3;
Gender gender = 4;
uint32 birthDate = 5;
bytes photo = 6;
string phoneNumber = 7;
bool isOnline = 8;
fixed32 visitTime = 9;
uint32 age = 10;
ZodiacSign zodiacSign = 11;
repeated string songs = 12;
repeated uint32 color = 13;
string hometown = 14;
repeated float location = 15;
double pi = 16;
double lotteryWin = 17;
float someRatio = 18;
float temperature = 19;
sint64 randomBigNumber = 20;
repeated MeasureUnit measureUnits = 21;
Nestiness nestiness = 22;
};
repeated Person person = 1;
};
// same as wrapped in Envelope, used for verification
message Person {
message MeasureUnit
{
string unit = 1;
float coef = 2;
};
string uuid = 1;
string name = 2;
string surname = 3;
Gender gender = 4;
uint32 birthDate = 5;
bytes photo = 6;
string phoneNumber = 7;
bool isOnline = 8;
fixed32 visitTime = 9;
uint32 age = 10;
ZodiacSign zodiacSign = 11;
repeated string songs = 12;
repeated uint32 color = 13;
string hometown = 14;
repeated float location = 15;
double pi = 16;
double lotteryWin = 17;
float someRatio = 18;
float temperature = 19;
sint64 randomBigNumber = 20;
repeated MeasureUnit measureUnits = 21;
Nestiness nestiness = 22;
};

View File

@ -0,0 +1,83 @@
syntax = "proto3";
enum OnlineStatus {
offline = 0;
online = 1;
};
message Envelope {
message AltPerson {
enum Gender {
male = 0;
female = 1;
};
message Dummy {
message Empty {};
repeated Empty empty = 1;
float z = 2;
};
repeated int32 location = 101 [packed=false];
float pi = 103;
bytes uuid = 300;
bool newFieldBool = 299;
string name = 2;
Gender gender = 102;
int32 zodiacSign = 130;
int64 birthDate = 150;
bytes age = 111;
OnlineStatus isOnline = 1;
double someRatio = 100;
fixed64 visitTime = 15;
Dummy newMessage = 1000;
sfixed64 randomBigNumber = 140;
repeated int32 newFieldInt = 104;
repeated float color = 14;
uint64 lotteryWin = 202;
bytes surname = 10;
uint64 phoneNumber = 5;
sint32 temperature = 41;
string newFieldStr = 21;
repeated string measureUnits_unit = 99;
repeated float measureUnits_coef = 88;
uint32 nestiness_a_b_c_d = 900;
repeated uint32 nestiness_a_b_c_e = 901;
};
};
// same as wrapped in Envelope, used for verification
message AltPerson {
enum Gender {
male = 0;
female = 1;
};
message Dummy {
message Empty {};
repeated Empty empty = 1;
float z = 2;
};
repeated int32 location = 101 [packed=false];
float pi = 103;
bytes uuid = 300;
bool newFieldBool = 299;
string name = 2;
Gender gender = 102;
int32 zodiacSign = 130;
int64 birthDate = 150;
bytes age = 111;
OnlineStatus isOnline = 1;
double someRatio = 100;
fixed64 visitTime = 15;
Dummy newMessage = 1000;
sfixed64 randomBigNumber = 140;
repeated int32 newFieldInt = 104;
repeated float color = 14;
uint64 lotteryWin = 202;
bytes surname = 10;
uint64 phoneNumber = 5;
sint32 temperature = 41;
string newFieldStr = 21;
repeated string measureUnits_unit = 99;
repeated float measureUnits_coef = 88;
uint32 nestiness_a_b_c_d = 900;
repeated uint32 nestiness_a_b_c_e = 901;
};

View File

@ -0,0 +1,78 @@
syntax = "proto3";
message Envelope {
message StrPerson {
message MeasureUnits
{
repeated string unit = 1;
repeated string coef = 2;
};
message NestinessA
{
message SubBC {
string d = 1;
repeated string e = 2;
};
SubBC b_c = 1;
};
string uuid = 1;
string name = 2;
string surname = 3;
string gender = 4;
string birthDate = 5;
string phoneNumber = 7;
string isOnline = 8;
string visitTime = 9;
string age = 10;
string zodiacSign = 11;
repeated string songs = 12;
repeated string color = 13;
string hometown = 14;
repeated string location = 15;
string pi = 16;
string lotteryWin = 17;
string someRatio = 18;
string temperature = 19;
string randomBigNumber = 20;
MeasureUnits measureUnits = 21;
NestinessA nestiness_a = 22;
};
};
// same as wrapped in Envelope, used for verification
message StrPerson {
message MeasureUnits
{
repeated string unit = 1;
repeated string coef = 2;
};
message NestinessA
{
message SubBC {
string d = 1;
repeated string e = 2;
};
SubBC b_c = 1;
};
string uuid = 1;
string name = 2;
string surname = 3;
string gender = 4;
string birthDate = 5;
string phoneNumber = 7;
string isOnline = 8;
string visitTime = 9;
string age = 10;
string zodiacSign = 11;
repeated string songs = 12;
repeated string color = 13;
string hometown = 14;
repeated string location = 15;
string pi = 16;
string lotteryWin = 17;
string someRatio = 18;
string temperature = 19;
string randomBigNumber = 20;
MeasureUnits measureUnits = 21;
NestinessA nestiness_a = 22;
};

View File

@ -0,0 +1,128 @@
syntax = "proto2";
message Envelope {
message Syntax2Person {
enum Gender {
female = 0;
male = 1;
};
enum ZodiacSign {
aries = 0;
taurus = 1;
gemini = 2;
cancer = 3;
leo = 4;
virgo = 5;
libra = 6;
scorpius = 7;
sagittarius = 8;
capricorn = 9;
aquarius = 10;
pisces = 11;
};
required string uuid = 1;
required string name = 2;
required string surname = 3;
required Gender gender = 4;
required uint32 birthDate = 5;
optional bytes photo = 6;
optional string phoneNumber = 7;
optional bool isOnline = 8;
optional fixed32 visitTime = 9;
optional uint32 age = 10;
optional ZodiacSign zodiacSign = 11;
repeated string songs = 12;
repeated uint32 color = 13;
optional string hometown = 14 [default='Moscow'];
repeated float location = 15 [packed=true];
optional double pi = 16;
optional double lotteryWin = 17;
optional float someRatio = 18;
optional float temperature = 19;
optional sint64 randomBigNumber = 20;
optional group MeasureUnits = 21 {
repeated float coef = 1;
repeated string unit = 2;
};
optional group Nestiness = 22
{
optional group A = 1 {
message SubB {
optional group C = 1 {
optional uint32 d = 1;
repeated uint32 e = 2;
};
};
optional SubB b = 100;
};
};
optional string newFieldStr = 23 [default='abc'];
optional int32 newFieldInt = 24 [default=-11];
optional bool newBool = 25 [default=true];
};
};
// same as wrapped in Envelope, used for verification
message Syntax2Person {
enum Gender {
female = 0;
male = 1;
};
enum ZodiacSign {
aries = 0;
taurus = 1;
gemini = 2;
cancer = 3;
leo = 4;
virgo = 5;
libra = 6;
scorpius = 7;
sagittarius = 8;
capricorn = 9;
aquarius = 10;
pisces = 11;
};
required string uuid = 1;
required string name = 2;
required string surname = 3;
required Gender gender = 4;
required uint32 birthDate = 5;
optional bytes photo = 6;
optional string phoneNumber = 7;
optional bool isOnline = 8;
optional fixed32 visitTime = 9;
optional uint32 age = 10;
optional ZodiacSign zodiacSign = 11;
repeated string songs = 12;
repeated uint32 color = 13;
optional string hometown = 14 [default='Moscow'];
repeated float location = 15 [packed=true];
optional double pi = 16;
optional double lotteryWin = 17;
optional float someRatio = 18;
optional float temperature = 19;
optional sint64 randomBigNumber = 20;
optional group MeasureUnits = 21 {
repeated float coef = 1;
repeated string unit = 2;
};
optional group Nestiness = 22
{
optional group A = 1 {
message SubB {
optional group C = 1 {
optional uint32 d = 1;
repeated uint32 e = 2;
};
};
optional SubB b = 100;
};
};
optional string newFieldStr = 23 [default='abc'];
optional int32 newFieldInt = 24 [default=-11];
optional bool newBool = 25 [default=true];
};

View File

@ -5,6 +5,7 @@
import argparse
import os.path
import io
import struct
import subprocess
import sys
@ -56,11 +57,15 @@ class FormatSchemaSplitted:
self.schemaname = self.schemaname + ".proto"
self.message_type = splitted[1]
def decode(input, output, format_schema):
def decode(input, output, format_schema, format):
if not type(format_schema) is FormatSchemaSplitted:
format_schema = FormatSchemaSplitted(format_schema)
msgindex = 1
if format == 'protobuflist':
read_varint(input) # envelope msg size
while True:
if format == 'protobuflist':
read_varint(input) # wiretype and field id of nested msg
sz = read_varint(input)
if sz is None:
break
@ -82,11 +87,12 @@ def decode(input, output, format_schema):
output.flush()
msgindex = msgindex + 1
def encode(input, output, format_schema):
def encode(input, output, format_schema, format):
if not type(format_schema) is FormatSchemaSplitted:
format_schema = FormatSchemaSplitted(format_schema)
line_offset = input.tell()
line = input.readline()
buf = io.BytesIO()
while True:
if len(line) == 0:
break
@ -109,11 +115,18 @@ def encode(input, output, format_schema):
msgbin = proc.communicate(msg)[0]
if proc.returncode != 0:
raise RuntimeError("protoc returned code " + str(proc.returncode))
write_varint(output, len(msgbin))
output.write(msgbin)
if format == 'protobuflist':
field_number = 1
wire_type = 2 # length-delimited
write_varint(buf, (field_number << 3) | wire_type)
write_varint(buf, len(msgbin))
buf.write(msgbin)
if format == 'protobuflist':
write_varint(output, len(buf.getvalue()))
output.write(buf.getvalue())
output.flush()
def decode_and_check(input, output, format_schema):
def decode_and_check(input, output, format_schema, format):
input_data = input.read()
output.write(b"Binary representation:\n")
output.flush()
@ -125,13 +138,13 @@ def decode_and_check(input, output, format_schema):
tmp_input.write(input_data)
tmp_input.flush()
tmp_input.seek(0)
decode(tmp_input, tmp_decoded, format_schema)
decode(tmp_input, tmp_decoded, format_schema, format)
tmp_decoded.seek(0)
decoded_text = tmp_decoded.read()
output.write(decoded_text)
output.flush()
tmp_decoded.seek(0)
encode(tmp_decoded, tmp_encoded, format_schema)
encode(tmp_decoded, tmp_encoded, format_schema, format)
tmp_encoded.seek(0)
encoded_data = tmp_encoded.read()
@ -149,6 +162,7 @@ if __name__ == "__main__":
parser.add_argument('--input', help='The input file, the standard input will be used if not specified.')
parser.add_argument('--output', help='The output file, the standard output will be used if not specified')
parser.add_argument('--format_schema', required=True, help='Format schema in the format "schemafile:MessageType"')
parser.add_argument('--format', choices=['protobuf', 'protobuflist'], default='protobuf', help='The input/output format, "protobuf" if not specified')
group = parser.add_mutually_exclusive_group(required=True)
group.add_argument('--encode', action='store_true', help='Specify to encode length-delimited messages.'
'The utility will read text-format messages of the given type from the input and write it in binary to the output.')
@ -169,11 +183,11 @@ if __name__ == "__main__":
output = custom_output_file if custom_output_file else sys.stdout.buffer
if args.encode:
encode(input, output, args.format_schema)
encode(input, output, args.format_schema, args.format)
elif args.decode:
decode(input, output, args.format_schema)
decode(input, output, args.format_schema, args.format)
elif args.decode_and_check:
decode_and_check(input, output, args.format_schema)
decode_and_check(input, output, args.format_schema, args.format)
finally:
if custom_input_file: