Merge branch 'master' of github.com:yandex/ClickHouse

This commit is contained in:
Alexey Milovidov 2019-02-27 22:12:17 +03:00
commit 3256b816be
74 changed files with 1576 additions and 1478 deletions

View File

@ -19,6 +19,7 @@ endif()
if (NOT GTEST_SRC_DIR AND NOT GTEST_INCLUDE_DIRS AND NOT MISSING_INTERNAL_GTEST_LIBRARY)
set (USE_INTERNAL_GTEST_LIBRARY 1)
set (GTEST_MAIN_LIBRARIES gtest_main)
set (GTEST_LIBRARIES gtest)
set (GTEST_INCLUDE_DIRS ${ClickHouse_SOURCE_DIR}/contrib/googletest/googletest)
endif ()
@ -26,4 +27,4 @@ if((GTEST_INCLUDE_DIRS AND GTEST_MAIN_LIBRARIES) OR GTEST_SRC_DIR)
set(USE_GTEST 1)
endif()
message (STATUS "Using gtest=${USE_GTEST}: ${GTEST_INCLUDE_DIRS} : ${GTEST_MAIN_LIBRARIES} : ${GTEST_SRC_DIR}")
message (STATUS "Using gtest=${USE_GTEST}: ${GTEST_INCLUDE_DIRS} : ${GTEST_LIBRARIES}, ${GTEST_MAIN_LIBRARIES} : ${GTEST_SRC_DIR}")

View File

@ -7,7 +7,7 @@
# include <Common/Exception.h>
namespace DB { namespace ErrorCodes { extern const int CPUID_ERROR; }}
#elif USE_CPUINFO
# include <cpuinfo.h>
# include <cpuinfo.h> // Y_IGNORE
#endif

View File

@ -250,9 +250,12 @@ void DataTypeAggregateFunction::deserializeTextCSV(IColumn & column, ReadBuffer
}
void DataTypeAggregateFunction::serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const
void DataTypeAggregateFunction::serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf, size_t & value_index) const
{
protobuf.writeAggregateFunction(function, static_cast<const ColumnAggregateFunction &>(column).getData()[row_num]);
if (value_index)
return;
value_index = static_cast<bool>(
protobuf.writeAggregateFunction(function, static_cast<const ColumnAggregateFunction &>(column).getData()[row_num]));
}
void DataTypeAggregateFunction::deserializeProtobuf(IColumn & column, ProtobufReader & protobuf, bool allow_add_row, bool & row_added) const

View File

@ -56,7 +56,7 @@ public:
void serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override;
void serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const override;
void serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf, size_t & value_index) const override;
void deserializeProtobuf(IColumn & column, ProtobufReader & protobuf, bool allow_add_row, bool & row_added) const override;
MutableColumnPtr createColumn() const override;

View File

@ -431,15 +431,22 @@ void DataTypeArray::deserializeTextCSV(IColumn & column, ReadBuffer & istr, cons
}
void DataTypeArray::serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const
void DataTypeArray::serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf, size_t & value_index) const
{
const ColumnArray & column_array = static_cast<const ColumnArray &>(column);
const ColumnArray::Offsets & offsets = column_array.getOffsets();
size_t offset = offsets[row_num - 1];
size_t offset = offsets[row_num - 1] + value_index;
size_t next_offset = offsets[row_num];
const IColumn & nested_column = column_array.getData();
for (size_t i = offset; i < next_offset; ++i)
nested->serializeProtobuf(nested_column, i, protobuf);
size_t i;
for (i = offset; i < next_offset; ++i)
{
size_t element_stored = 0;
nested->serializeProtobuf(nested_column, i, protobuf, element_stored);
if (!element_stored)
break;
}
value_index += i - offset;
}

View File

@ -86,7 +86,8 @@ public:
void serializeProtobuf(const IColumn & column,
size_t row_num,
ProtobufWriter & protobuf) const override;
ProtobufWriter & protobuf,
size_t & value_index) const override;
void deserializeProtobuf(IColumn & column,
ProtobufReader & protobuf,
bool allow_add_row,

View File

@ -74,9 +74,11 @@ void DataTypeDate::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const
static_cast<ColumnUInt16 &>(column).getData().push_back(value.getDayNum());
}
void DataTypeDate::serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const
void DataTypeDate::serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf, size_t & value_index) const
{
protobuf.writeDate(DayNum(static_cast<const ColumnUInt16 &>(column).getData()[row_num]));
if (value_index)
return;
value_index = static_cast<bool>(protobuf.writeDate(DayNum(static_cast<const ColumnUInt16 &>(column).getData()[row_num])));
}
void DataTypeDate::deserializeProtobuf(IColumn & column, ProtobufReader & protobuf, bool allow_add_row, bool & row_added) const

View File

@ -21,7 +21,7 @@ public:
void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override;
void serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const override;
void serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf, size_t & value_index) const override;
void deserializeProtobuf(IColumn & column, ProtobufReader & protobuf, bool allow_add_row, bool & row_added) const override;
bool canBeUsedAsVersion() const override { return true; }

View File

@ -140,9 +140,11 @@ void DataTypeDateTime::deserializeTextCSV(IColumn & column, ReadBuffer & istr, c
static_cast<ColumnUInt32 &>(column).getData().push_back(x);
}
void DataTypeDateTime::serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const
void DataTypeDateTime::serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf, size_t & value_index) const
{
protobuf.writeDateTime(static_cast<const ColumnUInt32 &>(column).getData()[row_num]);
if (value_index)
return;
value_index = static_cast<bool>(protobuf.writeDateTime(static_cast<const ColumnUInt32 &>(column).getData()[row_num]));
}
void DataTypeDateTime::deserializeProtobuf(IColumn & column, ProtobufReader & protobuf, bool allow_add_row, bool & row_added) const

View File

@ -46,7 +46,7 @@ public:
void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override;
void serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const override;
void serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf, size_t & value_index) const override;
void deserializeProtobuf(IColumn & column, ProtobufReader & protobuf, bool allow_add_row, bool & row_added) const override;
bool canBeUsedAsVersion() const override { return true; }

View File

@ -225,10 +225,12 @@ void DataTypeEnum<Type>::deserializeBinaryBulk(
}
template <typename Type>
void DataTypeEnum<Type>::serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const
void DataTypeEnum<Type>::serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf, size_t & value_index) const
{
if (value_index)
return;
protobuf.prepareEnumMapping(values);
protobuf.writeEnum(static_cast<const ColumnType &>(column).getData()[row_num]);
value_index = static_cast<bool>(protobuf.writeEnum(static_cast<const ColumnType &>(column).getData()[row_num]));
}
template<typename Type>

View File

@ -105,7 +105,7 @@ public:
void serializeBinaryBulk(const IColumn & column, WriteBuffer & ostr, const size_t offset, size_t limit) const override;
void deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, const size_t limit, const double avg_value_size_hint) const override;
void serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const override;
void serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf, size_t & value_index) const override;
void deserializeProtobuf(IColumn & column, ProtobufReader & protobuf, bool allow_add_row, bool & row_added) const override;
MutableColumnPtr createColumn() const override { return ColumnType::create(); }

View File

@ -209,10 +209,12 @@ void DataTypeFixedString::deserializeTextCSV(IColumn & column, ReadBuffer & istr
}
void DataTypeFixedString::serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const
void DataTypeFixedString::serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf, size_t & value_index) const
{
if (value_index)
return;
const char * pos = reinterpret_cast<const char *>(&static_cast<const ColumnFixedString &>(column).getChars()[n * row_num]);
protobuf.writeString(StringRef(pos, n));
value_index = static_cast<bool>(protobuf.writeString(StringRef(pos, n)));
}

View File

@ -64,7 +64,7 @@ public:
void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
void serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const override;
void serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf, size_t & value_index) const override;
void deserializeProtobuf(IColumn & column, ProtobufReader & protobuf, bool allow_add_row, bool & row_added) const override;
MutableColumnPtr createColumn() const override;

View File

@ -110,9 +110,9 @@ public:
serializeImpl(column, row_num, &IDataType::serializeAsTextXML, ostr, settings);
}
void serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const override
void serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf, size_t & value_index) const override
{
serializeImpl(column, row_num, &IDataType::serializeProtobuf, protobuf);
serializeImpl(column, row_num, &IDataType::serializeProtobuf, protobuf, value_index);
}
void deserializeProtobuf(IColumn & column, ProtobufReader & protobuf, bool allow_add_row, bool & row_added) const override;

View File

@ -311,11 +311,11 @@ void DataTypeNullable::serializeTextXML(const IColumn & column, size_t row_num,
nested_data_type->serializeAsTextXML(col.getNestedColumn(), row_num, ostr, settings);
}
void DataTypeNullable::serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const
void DataTypeNullable::serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf, size_t & value_index) const
{
const ColumnNullable & col = static_cast<const ColumnNullable &>(column);
if (!col.isNullAt(row_num))
nested_data_type->serializeProtobuf(col.getNestedColumn(), row_num, protobuf);
nested_data_type->serializeProtobuf(col.getNestedColumn(), row_num, protobuf, value_index);
}
void DataTypeNullable::deserializeProtobuf(IColumn & column, ProtobufReader & protobuf, bool allow_add_row, bool & row_added) const

View File

@ -70,7 +70,7 @@ public:
void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
void serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
void serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const override;
void serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf, size_t & value_index) const override;
void deserializeProtobuf(IColumn & column, ProtobufReader & protobuf, bool allow_add_row, bool & row_added) const override;
MutableColumnPtr createColumn() const override;

View File

@ -205,9 +205,11 @@ void DataTypeNumberBase<T>::deserializeBinaryBulk(IColumn & column, ReadBuffer &
template <typename T>
void DataTypeNumberBase<T>::serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const
void DataTypeNumberBase<T>::serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf, size_t & value_index) const
{
protobuf.writeNumber(static_cast<const ColumnVector<T> &>(column).getData()[row_num]);
if (value_index)
return;
value_index = static_cast<bool>(protobuf.writeNumber(static_cast<const ColumnVector<T> &>(column).getData()[row_num]));
}

View File

@ -36,7 +36,7 @@ public:
void serializeBinaryBulk(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const override;
void deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const override;
void serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const override;
void serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf, size_t & value_index) const override;
void deserializeProtobuf(IColumn & column, ProtobufReader & protobuf, bool allow_add_row, bool & row_added) const override;
MutableColumnPtr createColumn() const override;

View File

@ -303,9 +303,11 @@ void DataTypeString::deserializeTextCSV(IColumn & column, ReadBuffer & istr, con
}
void DataTypeString::serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const
void DataTypeString::serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf, size_t & value_index) const
{
protobuf.writeString(static_cast<const ColumnString &>(column).getDataAt(row_num));
if (value_index)
return;
value_index = static_cast<bool>(protobuf.writeString(static_cast<const ColumnString &>(column).getDataAt(row_num)));
}

View File

@ -45,7 +45,7 @@ public:
void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
void serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const override;
void serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf, size_t & value_index) const override;
void deserializeProtobuf(IColumn & column, ProtobufReader & protobuf, bool allow_add_row, bool & row_added) const override;
MutableColumnPtr createColumn() const override;

View File

@ -407,10 +407,15 @@ void DataTypeTuple::deserializeBinaryBulkWithMultipleStreams(
settings.path.pop_back();
}
void DataTypeTuple::serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const
void DataTypeTuple::serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf, size_t & value_index) const
{
for (const auto i : ext::range(0, ext::size(elems)))
elems[i]->serializeProtobuf(extractElementColumn(column, i), row_num, protobuf);
for (; value_index < elems.size(); ++value_index)
{
size_t stored = 0;
elems[value_index]->serializeProtobuf(extractElementColumn(column, value_index), row_num, protobuf, stored);
if (!stored)
break;
}
}
void DataTypeTuple::deserializeProtobuf(IColumn & column, ProtobufReader & protobuf, bool allow_add_row, bool & row_added) const

View File

@ -77,7 +77,7 @@ public:
DeserializeBinaryBulkSettings & settings,
DeserializeBinaryBulkStatePtr & state) const override;
void serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const override;
void serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf, size_t & value_index) const override;
void deserializeProtobuf(IColumn & column, ProtobufReader & reader, bool allow_add_row, bool & row_added) const override;
MutableColumnPtr createColumn() const override;

View File

@ -73,9 +73,11 @@ void DataTypeUUID::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const
static_cast<ColumnUInt128 &>(column).getData().push_back(value);
}
void DataTypeUUID::serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const
void DataTypeUUID::serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf, size_t & value_index) const
{
protobuf.writeUUID(UUID(static_cast<const ColumnUInt128 &>(column).getData()[row_num]));
if (value_index)
return;
value_index = static_cast<bool>(protobuf.writeUUID(UUID(static_cast<const ColumnUInt128 &>(column).getData()[row_num])));
}
void DataTypeUUID::deserializeProtobuf(IColumn & column, ProtobufReader & protobuf, bool allow_add_row, bool & row_added) const

View File

@ -24,7 +24,7 @@ public:
void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override;
void serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const override;
void serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf, size_t & value_index) const override;
void deserializeProtobuf(IColumn & column, ProtobufReader & protobuf, bool allow_add_row, bool & row_added) const override;
bool canBeUsedInBitOperations() const override { return true; }

View File

@ -136,9 +136,11 @@ void DataTypeDecimal<T>::deserializeBinaryBulk(IColumn & column, ReadBuffer & is
template <typename T>
void DataTypeDecimal<T>::serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const
void DataTypeDecimal<T>::serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf, size_t & value_index) const
{
protobuf.writeDecimal(static_cast<const ColumnType &>(column).getData()[row_num], scale);
if (value_index)
return;
value_index = static_cast<bool>(protobuf.writeDecimal(static_cast<const ColumnType &>(column).getData()[row_num], scale));
}

View File

@ -100,7 +100,7 @@ public:
void deserializeBinary(IColumn & column, ReadBuffer & istr) const override;
void deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const override;
void serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const override;
void serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf, size_t & value_index) const override;
void deserializeProtobuf(IColumn & column, ProtobufReader & protobuf, bool allow_add_row, bool & row_added) const override;
Field getDefault() const override;

View File

@ -254,7 +254,7 @@ public:
virtual void serializeAsTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const;
/** Serialize to a protobuf. */
virtual void serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const = 0;
virtual void serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf, size_t & value_index) const = 0;
virtual void deserializeProtobuf(IColumn & column, ProtobufReader & protobuf, bool allow_add_row, bool & row_added) const = 0;
protected:

View File

@ -28,7 +28,7 @@ public:
void deserializeBinaryBulk(IColumn &, ReadBuffer &, size_t, double) const override { throwNoSerialization(); }
void serializeText(const IColumn &, size_t, WriteBuffer &, const FormatSettings &) const override { throwNoSerialization(); }
void deserializeText(IColumn &, ReadBuffer &, const FormatSettings &) const override { throwNoSerialization(); }
void serializeProtobuf(const IColumn &, size_t, ProtobufWriter &) const override { throwNoSerialization(); }
void serializeProtobuf(const IColumn &, size_t, ProtobufWriter &, size_t &) const override { throwNoSerialization(); }
void deserializeProtobuf(IColumn &, ProtobufReader &, bool, bool &) const override { throwNoSerialization(); }
MutableColumnPtr createColumn() const override

View File

@ -12,26 +12,12 @@ BlockOutputStreamFromRowOutputStream::BlockOutputStreamFromRowOutputStream(RowOu
void BlockOutputStreamFromRowOutputStream::write(const Block & block)
{
size_t rows = block.rows();
size_t columns = block.columns();
for (size_t i = 0; i < rows; ++i)
{
if (!first_row)
row_output->writeRowBetweenDelimiter();
first_row = false;
row_output->writeRowStartDelimiter();
for (size_t j = 0; j < columns; ++j)
{
if (j != 0)
row_output->writeFieldDelimiter();
auto & col = block.getByPosition(j);
row_output->writeField(*col.column, *col.type, i);
}
row_output->writeRowEndDelimiter();
row_output->write(block, i);
}
}

View File

@ -1,9 +1,15 @@
#include <Common/Exception.h>
#include <Core/Block.h>
#include <Formats/IRowOutputStream.h>
namespace DB
{
namespace ErrorCodes
{
extern const int NOT_IMPLEMENTED;
}
void IRowOutputStream::write(const Block & block, size_t row_num)
{
@ -23,4 +29,9 @@ void IRowOutputStream::write(const Block & block, size_t row_num)
writeRowEndDelimiter();
}
void IRowOutputStream::writeField(const IColumn &, const IDataType &, size_t)
{
throw Exception("Method writeField is not implemented for output format", ErrorCodes::NOT_IMPLEMENTED);
}
}

View File

@ -28,7 +28,7 @@ public:
virtual void write(const Block & block, size_t row_num);
/** Write single value. */
virtual void writeField(const IColumn & column, const IDataType & type, size_t row_num) = 0;
virtual void writeField(const IColumn & column, const IDataType & type, size_t row_num);
/** Write delimiter. */
virtual void writeFieldDelimiter() {} /// delimiter between values

View File

@ -1,91 +0,0 @@
#include <Formats/FormatFactory.h>
#include <Common/config.h>
#if USE_PROTOBUF
#include "ProtobufBlockOutputStream.h"
#include <Core/Block.h>
#include <Formats/FormatSchemaInfo.h>
#include <Formats/ProtobufSchemas.h>
#include <Interpreters/Context.h>
#include <google/protobuf/descriptor.h>
namespace DB
{
namespace ErrorCodes
{
extern const int NOT_IMPLEMENTED;
extern const int NO_DATA_FOR_REQUIRED_PROTOBUF_FIELD;
}
ProtobufBlockOutputStream::ProtobufBlockOutputStream(
WriteBuffer & buffer_,
const Block & header_,
const google::protobuf::Descriptor * message_type,
const FormatSettings & format_settings_)
: writer(buffer_, message_type), header(header_), format_settings(format_settings_)
{
}
void ProtobufBlockOutputStream::write(const Block & block)
{
std::vector<const ColumnWithTypeAndName *> columns_in_write_order;
const auto & fields_in_write_order = writer.fieldsInWriteOrder();
columns_in_write_order.reserve(fields_in_write_order.size());
for (size_t i = 0; i != fields_in_write_order.size(); ++i)
{
const auto * field = fields_in_write_order[i];
const ColumnWithTypeAndName * column = nullptr;
if (block.has(field->name()))
{
column = &block.getByName(field->name());
}
else if (field->is_required())
{
throw Exception(
"Output doesn't have a column named '" + field->name() + "' which is required to write the output in the protobuf format.",
ErrorCodes::NO_DATA_FOR_REQUIRED_PROTOBUF_FIELD);
}
columns_in_write_order.emplace_back(column);
}
for (size_t row_num = 0; row_num != block.rows(); ++row_num)
{
writer.newMessage();
for (const auto * column : columns_in_write_order)
{
if (column)
{
assert(column->name == writer.currentField()->name());
column->type->serializeProtobuf(*(column->column), row_num, writer);
}
writer.nextField();
}
}
}
void registerOutputFormatProtobuf(FormatFactory & factory)
{
factory.registerOutputFormat(
"Protobuf", [](WriteBuffer & buf, const Block & header, const Context & context, const FormatSettings & format_settings)
{
const auto * message_type = ProtobufSchemas::instance().getMessageTypeForFormatSchema(FormatSchemaInfo(context, "proto"));
return std::make_shared<ProtobufBlockOutputStream>(buf, header, message_type, format_settings);
});
}
}
#else
namespace DB
{
class FormatFactory;
void registerOutputFormatProtobuf(FormatFactory &) {}
}
#endif

View File

@ -69,6 +69,12 @@ namespace ProtobufColumnMatcher
const std::vector<String> & column_names,
const google::protobuf::Descriptor * message_type);
template <typename Traits = DefaultTraits>
static std::unique_ptr<Message<Traits>> matchColumns(
const std::vector<String> & column_names,
const google::protobuf::Descriptor * message_type,
std::vector<const google::protobuf::FieldDescriptor *> & field_descriptors_without_match);
namespace details
{
void throwNoCommonColumns();
@ -88,7 +94,8 @@ namespace ProtobufColumnMatcher
std::unique_ptr<Message<Traits>> matchColumnsRecursive(
ColumnNameMatcher & name_matcher,
const google::protobuf::Descriptor * message_type,
const String & field_name_prefix)
const String & field_name_prefix,
std::vector<const google::protobuf::FieldDescriptor *> * field_descriptors_without_match)
{
auto message = std::make_unique<Message<Traits>>();
for (int i = 0; i != message_type->field_count(); ++i)
@ -98,7 +105,10 @@ namespace ProtobufColumnMatcher
|| (field_descriptor->type() == google::protobuf::FieldDescriptor::TYPE_GROUP))
{
auto nested_message = matchColumnsRecursive<Traits>(
name_matcher, field_descriptor->message_type(), field_name_prefix + field_descriptor->name() + ".");
name_matcher,
field_descriptor->message_type(),
field_name_prefix + field_descriptor->name() + ".",
field_descriptors_without_match);
if (nested_message)
{
message->fields.emplace_back();
@ -112,7 +122,12 @@ namespace ProtobufColumnMatcher
else
{
size_t column_index = name_matcher.findColumn(field_name_prefix + field_descriptor->name());
if (column_index != static_cast<size_t>(-1))
if (column_index == static_cast<size_t>(-1))
{
if (field_descriptors_without_match)
field_descriptors_without_match->emplace_back(field_descriptor);
}
else
{
message->fields.emplace_back();
auto & current_field = message->fields.back();
@ -143,16 +158,34 @@ namespace ProtobufColumnMatcher
}
}
template <typename Data>
static std::unique_ptr<Message<Data>> matchColumnsImpl(
const std::vector<String> & column_names,
const google::protobuf::Descriptor * message_type,
std::vector<const google::protobuf::FieldDescriptor *> * field_descriptors_without_match)
{
details::ColumnNameMatcher name_matcher(column_names);
auto message = details::matchColumnsRecursive<Data>(name_matcher, message_type, "", field_descriptors_without_match);
if (!message)
details::throwNoCommonColumns();
return message;
}
template <typename Data>
static std::unique_ptr<Message<Data>> matchColumns(
const std::vector<String> & column_names,
const google::protobuf::Descriptor * message_type)
{
details::ColumnNameMatcher name_matcher(column_names);
auto message = details::matchColumnsRecursive<Data>(name_matcher, message_type, "");
if (!message)
details::throwNoCommonColumns();
return message;
return matchColumnsImpl<Data>(column_names, message_type, nullptr);
}
template <typename Data>
static std::unique_ptr<Message<Data>> matchColumns(
const std::vector<String> & column_names,
const google::protobuf::Descriptor * message_type,
std::vector<const google::protobuf::FieldDescriptor *> & field_descriptors_without_match)
{
return matchColumnsImpl<Data>(column_names, message_type, &field_descriptors_without_match);
}
}

View File

@ -48,7 +48,8 @@ namespace
}
// SimpleReader is an utility class to deserialize protobufs.
// Knows nothing about protobuf schemas, just provides useful functions to deserialize data.
ProtobufReader::SimpleReader::SimpleReader(ReadBuffer & in_)
: in(in_)
, cursor(1 /* We starts at cursor == 1 to keep any cursor value > REACHED_END, this allows to simplify conditions */)
@ -374,6 +375,7 @@ void ProtobufReader::SimpleReader::ignoreGroup()
}
// Implementation for a converter from any protobuf field type to any DB data type.
class ProtobufReader::ConverterBaseImpl : public ProtobufReader::IConverter
{
public:
@ -688,19 +690,19 @@ private:
std::optional<std::unordered_map<StringRef, Int16>> enum_name_to_value_map;
};
#define PROTOBUF_READER_CONVERTER_IMPL_SPECIALIZATION_FOR_STRINGS(field_type_id) \
template<> \
class ProtobufReader::ConverterImpl<field_type_id> : public ConverterFromString \
#define PROTOBUF_READER_CREATE_CONVERTER_SPECIALIZATION_FOR_STRINGS(field_type_id) \
template <> \
std::unique_ptr<ProtobufReader::IConverter> ProtobufReader::createConverter<field_type_id>( \
const google::protobuf::FieldDescriptor * field) \
{ \
using ConverterFromString::ConverterFromString; \
return std::make_unique<ConverterFromString>(simple_reader, field); \
}
PROTOBUF_READER_CONVERTER_IMPL_SPECIALIZATION_FOR_STRINGS(google::protobuf::FieldDescriptor::TYPE_STRING);
PROTOBUF_READER_CONVERTER_IMPL_SPECIALIZATION_FOR_STRINGS(google::protobuf::FieldDescriptor::TYPE_BYTES);
#undef PROTOBUF_READER_CONVERTER_IMPL_SPECIALIZATION_FOR_STRINGS
PROTOBUF_READER_CREATE_CONVERTER_SPECIALIZATION_FOR_STRINGS(google::protobuf::FieldDescriptor::TYPE_STRING)
PROTOBUF_READER_CREATE_CONVERTER_SPECIALIZATION_FOR_STRINGS(google::protobuf::FieldDescriptor::TYPE_BYTES)
#undef PROTOBUF_READER_CREATE_CONVERTER_SPECIALIZATION_FOR_STRINGS
template <int field_type_id, typename T>
template <int field_type_id, typename FromType>
class ProtobufReader::ConverterFromNumber : public ConverterBaseImpl
{
public:
@ -708,7 +710,7 @@ public:
bool readStringInto(PaddedPODArray<UInt8> & str) override
{
T number;
FromType number;
if (!readField(number))
return false;
WriteBufferFromVector<PaddedPODArray<UInt8>> buf(str);
@ -765,7 +767,7 @@ private:
template <typename To>
bool readNumeric(To & value)
{
T number;
FromType number;
if (!readField(number))
return false;
value = numericCast<To>(number);
@ -775,9 +777,9 @@ private:
template<typename EnumType>
bool readEnum(EnumType & value)
{
if constexpr (!std::is_integral_v<T>)
if constexpr (!std::is_integral_v<FromType>)
cannotConvertType("Enum"); // It's not correct to convert floating point to enum.
T number;
FromType number;
if (!readField(number))
return false;
value = numericCast<EnumType>(number);
@ -799,40 +801,39 @@ private:
template <typename S>
bool readDecimal(Decimal<S> & decimal, UInt32 scale)
{
T number;
FromType number;
if (!readField(number))
return false;
decimal.value = convertToDecimal<DataTypeNumber<T>, DataTypeDecimal<Decimal<S>>>(number, scale);
decimal.value = convertToDecimal<DataTypeNumber<FromType>, DataTypeDecimal<Decimal<S>>>(number, scale);
return true;
}
bool readField(T & value)
bool readField(FromType & value)
{
if constexpr ((field_type_id == google::protobuf::FieldDescriptor::TYPE_INT32) && std::is_same_v<T, Int64>)
if constexpr (((field_type_id == google::protobuf::FieldDescriptor::TYPE_INT32) && std::is_same_v<FromType, Int64>)
|| ((field_type_id == google::protobuf::FieldDescriptor::TYPE_INT64) && std::is_same_v<FromType, Int64>))
{
return simple_reader.readInt(value);
else if constexpr ((field_type_id == google::protobuf::FieldDescriptor::TYPE_SINT32) && std::is_same_v<T, Int64>)
return simple_reader.readSInt(value);
else if constexpr ((field_type_id == google::protobuf::FieldDescriptor::TYPE_UINT32) && std::is_same_v<T, UInt64>)
}
else if constexpr (((field_type_id == google::protobuf::FieldDescriptor::TYPE_UINT32) && std::is_same_v<FromType, UInt64>)
|| ((field_type_id == google::protobuf::FieldDescriptor::TYPE_UINT64) && std::is_same_v<FromType, UInt64>))
{
return simple_reader.readUInt(value);
else if constexpr ((field_type_id == google::protobuf::FieldDescriptor::TYPE_INT64) && std::is_same_v<T, Int64>)
return simple_reader.readInt(value);
else if constexpr ((field_type_id == google::protobuf::FieldDescriptor::TYPE_SINT64) && std::is_same_v<T, Int64>)
}
else if constexpr (((field_type_id == google::protobuf::FieldDescriptor::TYPE_SINT32) && std::is_same_v<FromType, Int64>)
|| ((field_type_id == google::protobuf::FieldDescriptor::TYPE_SINT64) && std::is_same_v<FromType, Int64>))
{
return simple_reader.readSInt(value);
else if constexpr ((field_type_id == google::protobuf::FieldDescriptor::TYPE_UINT64) && std::is_same_v<T, UInt64>)
return simple_reader.readUInt(value);
else if constexpr ((field_type_id == google::protobuf::FieldDescriptor::TYPE_FIXED32) && std::is_same_v<T, UInt32>)
return simple_reader.readFixed(value);
else if constexpr ((field_type_id == google::protobuf::FieldDescriptor::TYPE_SFIXED32) && std::is_same_v<T, Int32>)
return simple_reader.readFixed(value);
else if constexpr ((field_type_id == google::protobuf::FieldDescriptor::TYPE_FIXED64) && std::is_same_v<T, UInt64>)
return simple_reader.readFixed(value);
else if constexpr ((field_type_id == google::protobuf::FieldDescriptor::TYPE_SFIXED64) && std::is_same_v<T, Int64>)
return simple_reader.readFixed(value);
else if constexpr ((field_type_id == google::protobuf::FieldDescriptor::TYPE_FLOAT) && std::is_same_v<T, float>)
return simple_reader.readFixed(value);
}
else
{
static_assert((field_type_id == google::protobuf::FieldDescriptor::TYPE_DOUBLE) && std::is_same_v<T, double>);
static_assert(((field_type_id == google::protobuf::FieldDescriptor::TYPE_FIXED32) && std::is_same_v<FromType, UInt32>)
|| ((field_type_id == google::protobuf::FieldDescriptor::TYPE_SFIXED32) && std::is_same_v<FromType, Int32>)
|| ((field_type_id == google::protobuf::FieldDescriptor::TYPE_FIXED64) && std::is_same_v<FromType, UInt64>)
|| ((field_type_id == google::protobuf::FieldDescriptor::TYPE_SFIXED64) && std::is_same_v<FromType, Int64>)
|| ((field_type_id == google::protobuf::FieldDescriptor::TYPE_FLOAT) && std::is_same_v<FromType, float>)
|| ((field_type_id == google::protobuf::FieldDescriptor::TYPE_DOUBLE) && std::is_same_v<FromType, double>));
return simple_reader.readFixed(value);
}
}
@ -840,30 +841,30 @@ private:
std::optional<std::unordered_set<Int16>> set_of_enum_values;
};
#define PROTOBUF_READER_CONVERTER_IMPL_SPECIALIZATION_FOR_NUMBERS(field_type_id, field_type) \
template<> \
class ProtobufReader::ConverterImpl<field_type_id> : public ConverterFromNumber<field_type_id, field_type> \
#define PROTOBUF_READER_CREATE_CONVERTER_SPECIALIZATION_FOR_NUMBERS(field_type_id, field_type) \
template <> \
std::unique_ptr<ProtobufReader::IConverter> ProtobufReader::createConverter<field_type_id>( \
const google::protobuf::FieldDescriptor * field) \
{ \
using ConverterFromNumber::ConverterFromNumber; \
return std::make_unique<ConverterFromNumber<field_type_id, field_type>>(simple_reader, field); \
}
PROTOBUF_READER_CONVERTER_IMPL_SPECIALIZATION_FOR_NUMBERS(google::protobuf::FieldDescriptor::TYPE_INT32, Int64);
PROTOBUF_READER_CONVERTER_IMPL_SPECIALIZATION_FOR_NUMBERS(google::protobuf::FieldDescriptor::TYPE_SINT32, Int64);
PROTOBUF_READER_CONVERTER_IMPL_SPECIALIZATION_FOR_NUMBERS(google::protobuf::FieldDescriptor::TYPE_UINT32, UInt64);
PROTOBUF_READER_CONVERTER_IMPL_SPECIALIZATION_FOR_NUMBERS(google::protobuf::FieldDescriptor::TYPE_INT64, Int64);
PROTOBUF_READER_CONVERTER_IMPL_SPECIALIZATION_FOR_NUMBERS(google::protobuf::FieldDescriptor::TYPE_SINT64, Int64);
PROTOBUF_READER_CONVERTER_IMPL_SPECIALIZATION_FOR_NUMBERS(google::protobuf::FieldDescriptor::TYPE_UINT64, UInt64);
PROTOBUF_READER_CONVERTER_IMPL_SPECIALIZATION_FOR_NUMBERS(google::protobuf::FieldDescriptor::TYPE_FIXED32, UInt32);
PROTOBUF_READER_CONVERTER_IMPL_SPECIALIZATION_FOR_NUMBERS(google::protobuf::FieldDescriptor::TYPE_SFIXED32, Int32);
PROTOBUF_READER_CONVERTER_IMPL_SPECIALIZATION_FOR_NUMBERS(google::protobuf::FieldDescriptor::TYPE_FIXED64, UInt64);
PROTOBUF_READER_CONVERTER_IMPL_SPECIALIZATION_FOR_NUMBERS(google::protobuf::FieldDescriptor::TYPE_SFIXED64, Int64);
PROTOBUF_READER_CONVERTER_IMPL_SPECIALIZATION_FOR_NUMBERS(google::protobuf::FieldDescriptor::TYPE_FLOAT, float);
PROTOBUF_READER_CONVERTER_IMPL_SPECIALIZATION_FOR_NUMBERS(google::protobuf::FieldDescriptor::TYPE_DOUBLE, double);
#undef PROTOBUF_READER_CONVERTER_IMPL_SPECIALIZATION_FOR_NUMBERS
PROTOBUF_READER_CREATE_CONVERTER_SPECIALIZATION_FOR_NUMBERS(google::protobuf::FieldDescriptor::TYPE_INT32, Int64);
PROTOBUF_READER_CREATE_CONVERTER_SPECIALIZATION_FOR_NUMBERS(google::protobuf::FieldDescriptor::TYPE_SINT32, Int64);
PROTOBUF_READER_CREATE_CONVERTER_SPECIALIZATION_FOR_NUMBERS(google::protobuf::FieldDescriptor::TYPE_UINT32, UInt64);
PROTOBUF_READER_CREATE_CONVERTER_SPECIALIZATION_FOR_NUMBERS(google::protobuf::FieldDescriptor::TYPE_INT64, Int64);
PROTOBUF_READER_CREATE_CONVERTER_SPECIALIZATION_FOR_NUMBERS(google::protobuf::FieldDescriptor::TYPE_SINT64, Int64);
PROTOBUF_READER_CREATE_CONVERTER_SPECIALIZATION_FOR_NUMBERS(google::protobuf::FieldDescriptor::TYPE_UINT64, UInt64);
PROTOBUF_READER_CREATE_CONVERTER_SPECIALIZATION_FOR_NUMBERS(google::protobuf::FieldDescriptor::TYPE_FIXED32, UInt32);
PROTOBUF_READER_CREATE_CONVERTER_SPECIALIZATION_FOR_NUMBERS(google::protobuf::FieldDescriptor::TYPE_SFIXED32, Int32);
PROTOBUF_READER_CREATE_CONVERTER_SPECIALIZATION_FOR_NUMBERS(google::protobuf::FieldDescriptor::TYPE_FIXED64, UInt64);
PROTOBUF_READER_CREATE_CONVERTER_SPECIALIZATION_FOR_NUMBERS(google::protobuf::FieldDescriptor::TYPE_SFIXED64, Int64);
PROTOBUF_READER_CREATE_CONVERTER_SPECIALIZATION_FOR_NUMBERS(google::protobuf::FieldDescriptor::TYPE_FLOAT, float);
PROTOBUF_READER_CREATE_CONVERTER_SPECIALIZATION_FOR_NUMBERS(google::protobuf::FieldDescriptor::TYPE_DOUBLE, double);
#undef PROTOBUF_READER_CREATE_CONVERTER_SPECIALIZATION_FOR_NUMBERS
template<>
class ProtobufReader::ConverterImpl<google::protobuf::FieldDescriptor::TYPE_BOOL> : public ConverterBaseImpl
class ProtobufReader::ConverterFromBool : public ConverterBaseImpl
{
public:
using ConverterBaseImpl::ConverterBaseImpl;
@ -913,10 +914,15 @@ private:
}
};
template <>
std::unique_ptr<ProtobufReader::IConverter> ProtobufReader::createConverter<google::protobuf::FieldDescriptor::TYPE_BOOL>(
const google::protobuf::FieldDescriptor * field)
{
return std::make_unique<ConverterFromBool>(simple_reader, field);
}
template<>
class ProtobufReader::ConverterImpl<google::protobuf::FieldDescriptor::TYPE_ENUM> : public ConverterBaseImpl
class ProtobufReader::ConverterFromEnum : public ConverterBaseImpl
{
public:
using ConverterBaseImpl::ConverterBaseImpl;
@ -973,10 +979,15 @@ private:
Int64 pbnumber;
if (!readField(pbnumber))
return false;
auto it = enum_pbnumber_to_value_map->find(pbnumber);
if (it == enum_pbnumber_to_value_map->end())
cannotConvertValue(toString(pbnumber), "Enum");
value = static_cast<T>(it->second);
if (enum_pbnumber_always_equals_value)
value = static_cast<T>(pbnumber);
else
{
auto it = enum_pbnumber_to_value_map->find(pbnumber);
if (it == enum_pbnumber_to_value_map->end())
cannotConvertValue(toString(pbnumber), "Enum");
value = static_cast<T>(it->second);
}
return true;
}
@ -999,12 +1010,19 @@ private:
if (likely(enum_pbnumber_to_value_map.has_value()))
return;
enum_pbnumber_to_value_map.emplace();
enum_pbnumber_always_equals_value = true;
for (const auto & name_value_pair : name_value_pairs)
{
Int16 value = name_value_pair.second;
const auto * enum_descriptor = field->enum_type()->FindValueByName(name_value_pair.first);
if (enum_descriptor)
{
enum_pbnumber_to_value_map->emplace(enum_descriptor->number(), value);
if (enum_descriptor->number() != value)
enum_pbnumber_always_equals_value = false;
}
else
enum_pbnumber_always_equals_value = false;
}
}
@ -1015,8 +1033,16 @@ private:
std::optional<std::unordered_map<Int64, StringRef>> enum_pbnumber_to_name_map;
std::optional<std::unordered_map<Int64, Int16>> enum_pbnumber_to_value_map;
bool enum_pbnumber_always_equals_value;
};
template <>
std::unique_ptr<ProtobufReader::IConverter> ProtobufReader::createConverter<google::protobuf::FieldDescriptor::TYPE_ENUM>(
const google::protobuf::FieldDescriptor * field)
{
return std::make_unique<ConverterFromEnum>(simple_reader, field);
}
ProtobufReader::ProtobufReader(
ReadBuffer & in_, const google::protobuf::Descriptor * message_type, const std::vector<String> & column_names)
@ -1041,7 +1067,7 @@ void ProtobufReader::setTraitsDataAfterMatchingColumns(Message * message)
{
#define PROTOBUF_READER_CONVERTER_CREATING_CASE(field_type_id) \
case field_type_id: \
field.data.converter = std::make_unique<ConverterImpl<field_type_id>>(simple_reader, field.field_descriptor); \
field.data.converter = createConverter<field_type_id>(field.field_descriptor); \
break
PROTOBUF_READER_CONVERTER_CREATING_CASE(google::protobuf::FieldDescriptor::TYPE_STRING);
PROTOBUF_READER_CONVERTER_CREATING_CASE(google::protobuf::FieldDescriptor::TYPE_BYTES);

View File

@ -158,9 +158,10 @@ private:
};
class ConverterBaseImpl;
template <int type_id> class ConverterImpl;
class ConverterFromString;
template<int field_type_id, typename FromType> class ConverterFromNumber;
class ConverterFromBool;
class ConverterFromEnum;
struct ColumnMatcherTraits
{
@ -178,6 +179,9 @@ private:
void setTraitsDataAfterMatchingColumns(Message * message);
template <int field_type_id>
std::unique_ptr<IConverter> createConverter(const google::protobuf::FieldDescriptor * field);
SimpleReader simple_reader;
std::unique_ptr<Message> root_message;
Message* current_message = nullptr;
@ -202,30 +206,30 @@ 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 readColumnIndex(size_t &) { return false; }
bool readNumber(Int8 &) { return false; }
bool readNumber(UInt8 &) { return false; }
bool readNumber(Int16 &) { return false; }
bool readNumber(UInt16 &) { return false; }
bool readNumber(Int32 &) { return false; }
bool readNumber(UInt32 &) { return false; }
bool readNumber(Int64 &) { return false; }
bool readNumber(UInt64 &) { return false; }
bool readNumber(UInt128 &) { return false; }
bool readNumber(Float32 &) { return false; }
bool readNumber(Float64 &) { return false; }
bool readStringInto(PaddedPODArray<UInt8> &) { return false; }
void prepareEnumMapping(const std::vector<std::pair<std::string, Int8>> &) {}
void prepareEnumMapping(const std::vector<std::pair<std::string, Int16>> &) {}
bool readEnum(Int8 &) { return false; }
bool readEnum(Int16 &) { return false; }
bool readUUID(UUID &) { return false; }
bool readDate(DayNum &) { return false; }
bool readDateTime(time_t &) { return false; }
bool readDecimal(Decimal32 &, UInt32, UInt32) { return false; }
bool readDecimal(Decimal64 &, UInt32, UInt32) { return false; }
bool readDecimal(Decimal128 &, UInt32, UInt32) { return false; }
bool readAggregateFunction(const AggregateFunctionPtr &, AggregateDataPtr, Arena &) { return false; }
bool maybeCanReadValue() const { return false; }
};

View File

@ -12,8 +12,8 @@
namespace DB
{
ProtobufRowInputStream::ProtobufRowInputStream(ReadBuffer & in_, const Block & header, const FormatSchemaInfo & info)
: data_types(header.getDataTypes()), reader(in_, ProtobufSchemas::instance().getMessageTypeForFormatSchema(info), header.getNames())
ProtobufRowInputStream::ProtobufRowInputStream(ReadBuffer & in_, const Block & header, const FormatSchemaInfo & format_schema)
: data_types(header.getDataTypes()), reader(in_, ProtobufSchemas::instance().getMessageTypeForFormatSchema(format_schema), header.getNames())
{
}
@ -87,7 +87,7 @@ void registerInputFormatProtobuf(FormatFactory & factory)
namespace DB
{
class FormatFactory;
void registerInputFormatProtobuf(FormatFactory & factory) {}
void registerInputFormatProtobuf(FormatFactory &) {}
}
#endif

View File

@ -13,12 +13,18 @@ class Block;
class FormatSchemaInfo;
/** Interface of stream, that allows to read data by rows.
/** Stream designed to deserialize data from the google protobuf format.
* Each row is read as a separated message.
* These messages are delimited according to documentation
* https://github.com/protocolbuffers/protobuf/blob/master/src/google/protobuf/util/delimited_message_util.h
* Serializing in 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 ProtobufRowInputStream : public IRowInputStream
{
public:
ProtobufRowInputStream(ReadBuffer & in_, const Block & header, const FormatSchemaInfo & info);
ProtobufRowInputStream(ReadBuffer & in_, const Block & header, const FormatSchemaInfo & format_schema);
~ProtobufRowInputStream() override;
bool read(MutableColumns & columns, RowReadExtension & extra) override;

View File

@ -0,0 +1,55 @@
#include <Formats/FormatFactory.h>
#include <Common/config.h>
#if USE_PROTOBUF
#include "ProtobufRowOutputStream.h"
#include <Core/Block.h>
#include <Formats/BlockOutputStreamFromRowOutputStream.h>
#include <Formats/FormatSchemaInfo.h>
#include <Formats/ProtobufSchemas.h>
#include <google/protobuf/descriptor.h>
namespace DB
{
ProtobufRowOutputStream::ProtobufRowOutputStream(WriteBuffer & out, const Block & header, const FormatSchemaInfo & format_schema)
: data_types(header.getDataTypes()), writer(out, ProtobufSchemas::instance().getMessageTypeForFormatSchema(format_schema), header.getNames())
{
value_indices.resize(header.columns());
}
void ProtobufRowOutputStream::write(const Block & block, size_t row_num)
{
writer.startMessage();
std::fill(value_indices.begin(), value_indices.end(), 0);
size_t column_index;
while (writer.writeField(column_index))
data_types[column_index]->serializeProtobuf(
*block.getByPosition(column_index).column, row_num, writer, value_indices[column_index]);
writer.endMessage();
}
void registerOutputFormatProtobuf(FormatFactory & factory)
{
factory.registerOutputFormat(
"Protobuf", [](WriteBuffer & buf, const Block & header, const Context & context, const FormatSettings &)
{
return std::make_shared<BlockOutputStreamFromRowOutputStream>(
std::make_shared<ProtobufRowOutputStream>(buf, header, FormatSchemaInfo(context, "proto")), header);
});
}
}
#else
namespace DB
{
class FormatFactory;
void registerOutputFormatProtobuf(FormatFactory &) {}
}
#endif

View File

@ -1,8 +1,7 @@
#pragma once
#include <Core/Block.h>
#include <DataStreams/IBlockOutputStream.h>
#include <Formats/FormatSettings.h>
#include <DataTypes/IDataType.h>
#include <Formats/IRowOutputStream.h>
#include <Formats/ProtobufWriter.h>
@ -17,6 +16,9 @@ namespace protobuf
namespace DB
{
class Block;
class FormatSchemaInfo;
/** Stream designed to serialize data in the google protobuf format.
* Each row is written as a separated message.
* These messages are delimited according to documentation
@ -25,23 +27,18 @@ namespace DB
* SELECT * from table FORMAT Protobuf SETTINGS format_schema = 'schema:Message'
* where schema is the name of "schema.proto" file specifying protobuf schema.
*/
class ProtobufBlockOutputStream : public IBlockOutputStream
class ProtobufRowOutputStream : public IRowOutputStream
{
public:
ProtobufBlockOutputStream(
WriteBuffer & buffer_,
const Block & header_,
const google::protobuf::Descriptor * message_prototype_,
const FormatSettings & format_settings_);
ProtobufRowOutputStream(WriteBuffer & out, const Block & header, const FormatSchemaInfo & format_schema);
Block getHeader() const override { return header; }
void write(const Block & block) override;
void write(const Block & block, size_t row_num) override;
std::string getContentType() const override { return "application/octet-stream"; }
private:
DataTypes data_types;
ProtobufWriter writer;
const Block header;
const FormatSettings format_settings;
std::vector<size_t> value_indices;
};
}

View File

@ -2,8 +2,8 @@
#if USE_PROTOBUF
#include <Formats/FormatSchemaInfo.h>
#include <Formats/ProtobufSchemas.h>
#include <google/protobuf/compiler/importer.h>
#include <Formats/ProtobufSchemas.h> // Y_IGNORE
#include <google/protobuf/compiler/importer.h> // Y_IGNORE
#include <Common/Exception.h>
@ -12,7 +12,6 @@ namespace DB
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
extern const int NOT_IMPLEMENTED;
extern const int CANNOT_PARSE_PROTOBUF_SCHEMA;
}
@ -74,11 +73,6 @@ const google::protobuf::Descriptor * ProtobufSchemas::getMessageTypeForFormatSch
return importer->import(info.schemaPath(), info.messageName());
}
const google::protobuf::Descriptor * ProtobufSchemas::getMessageTypeForColumns(const std::vector<ColumnWithTypeAndName> & /*columns*/)
{
throw Exception("Using the 'Protobuf' format without schema is not implemented", ErrorCodes::NOT_IMPLEMENTED);
}
}
#endif

View File

@ -5,7 +5,6 @@
#include <memory>
#include <unordered_map>
#include <vector>
#include <Core/Types.h>
#include <ext/singleton.h>
@ -21,9 +20,8 @@ namespace protobuf
namespace DB
{
class FormatSchemaInfo;
struct ColumnWithTypeAndName;
/** Keeps parsed google protobuf schemas either parsed from files or generated from DB columns.
/** Keeps parsed google protobuf schemas parsed from files.
* This class is used to handle the "Protobuf" input/output formats.
*/
class ProtobufSchemas : public ext::singleton<ProtobufSchemas>
@ -36,10 +34,6 @@ public:
/// 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);
/// Generates a message type with suitable types of fields to store a block with |header|, then returns the descriptor
/// of the generated message type.
const google::protobuf::Descriptor * getMessageTypeForColumns(const std::vector<ColumnWithTypeAndName> & columns);
private:
class ImporterWithSourceTree;
std::unordered_map<String, std::unique_ptr<ImporterWithSourceTree>> importers;

View File

@ -1,363 +0,0 @@
#include <cassert>
#include <Formats/ProtobufSimpleWriter.h>
#include <IO/WriteHelpers.h>
namespace DB
{
namespace
{
// Note: We cannot simply use writeVarUInt() from IO/VarInt.h here because there is one small difference:
// Google protobuf's representation of 64-bit integer contains from 1 to 10 bytes, whileas writeVarUInt() writes from 1 to 9 bytes
// because it omits the tenth byte (which is not necessary to decode actually).
void writePbVarUInt(UInt64 value, WriteBuffer & buf)
{
while (value >= 0x80)
{
buf.write(static_cast<char>(value | 0x80));
value >>= 7;
}
buf.write(static_cast<char>(value));
}
void writePbVarInt(Int64 value, WriteBuffer & buf)
{
writePbVarUInt((static_cast<UInt64>(value) << 1) ^ static_cast<UInt64>(value >> 63), buf);
}
void writePbVarIntNoZigZagEncoding(Int64 value, WriteBuffer & buf) { writePbVarUInt(static_cast<UInt64>(value), buf); }
}
enum ProtobufSimpleWriter::WireType : UInt32
{
VARINT = 0,
BITS64 = 1,
LENGTH_DELIMITED = 2,
BITS32 = 5
};
ProtobufSimpleWriter::ProtobufSimpleWriter(WriteBuffer & out_) : out(out_)
{
}
ProtobufSimpleWriter::~ProtobufSimpleWriter()
{
finishCurrentMessage();
}
void ProtobufSimpleWriter::newMessage()
{
finishCurrentMessage();
were_messages = true;
}
void ProtobufSimpleWriter::finishCurrentMessage()
{
if (!were_messages)
return;
finishCurrentField();
current_field_number = 0;
StringRef str = message_buffer.stringRef();
writePbVarUInt(str.size, out);
out.write(str.data, str.size);
message_buffer.restart();
}
void ProtobufSimpleWriter::setCurrentField(UInt32 field_number)
{
finishCurrentField();
assert(current_field_number < field_number);
current_field_number = field_number;
num_normal_values = 0;
num_packed_values = 0;
}
void ProtobufSimpleWriter::finishCurrentField()
{
if (num_packed_values)
{
assert(!num_normal_values);
StringRef str = repeated_packing_buffer.stringRef();
if (str.size)
{
writeKey(LENGTH_DELIMITED, message_buffer);
writePbVarUInt(str.size, message_buffer);
message_buffer.write(str.data, str.size);
repeated_packing_buffer.restart();
}
}
}
void ProtobufSimpleWriter::writeKey(WireType wire_type, WriteBuffer & buf)
{
writePbVarUInt((current_field_number << 3) | wire_type, buf);
}
void ProtobufSimpleWriter::writeInt32(Int32 value)
{
assert(current_field_number);
writeKey(VARINT, message_buffer);
writePbVarIntNoZigZagEncoding(value, message_buffer);
++num_normal_values;
}
void ProtobufSimpleWriter::writeUInt32(UInt32 value)
{
assert(current_field_number);
writeKey(VARINT, message_buffer);
writePbVarUInt(value, message_buffer);
++num_normal_values;
}
void ProtobufSimpleWriter::writeSInt32(Int32 value)
{
assert(current_field_number);
writeKey(VARINT, message_buffer);
writePbVarInt(value, message_buffer);
++num_normal_values;
}
void ProtobufSimpleWriter::writeInt64(Int64 value)
{
assert(current_field_number);
writeKey(VARINT, message_buffer);
writePbVarIntNoZigZagEncoding(value, message_buffer);
++num_normal_values;
}
void ProtobufSimpleWriter::writeUInt64(UInt64 value)
{
assert(current_field_number);
writeKey(VARINT, message_buffer);
writePbVarUInt(value, message_buffer);
++num_normal_values;
}
void ProtobufSimpleWriter::writeSInt64(Int64 value)
{
assert(current_field_number);
writeKey(VARINT, message_buffer);
writePbVarInt(value, message_buffer);
++num_normal_values;
}
void ProtobufSimpleWriter::writeFixed32(UInt32 value)
{
assert(current_field_number);
writeKey(BITS32, message_buffer);
writePODBinary(value, message_buffer);
++num_normal_values;
}
void ProtobufSimpleWriter::writeSFixed32(Int32 value)
{
assert(current_field_number);
writeKey(BITS32, message_buffer);
writePODBinary(value, message_buffer);
++num_normal_values;
}
void ProtobufSimpleWriter::writeFloat(float value)
{
assert(current_field_number);
writeKey(BITS32, message_buffer);
writePODBinary(value, message_buffer);
++num_normal_values;
}
void ProtobufSimpleWriter::writeFixed64(UInt64 value)
{
assert(current_field_number);
writeKey(BITS64, message_buffer);
writePODBinary(value, message_buffer);
++num_normal_values;
}
void ProtobufSimpleWriter::writeSFixed64(Int64 value)
{
assert(current_field_number);
writeKey(BITS64, message_buffer);
writePODBinary(value, message_buffer);
++num_normal_values;
}
void ProtobufSimpleWriter::writeDouble(double value)
{
assert(current_field_number);
writeKey(BITS64, message_buffer);
writePODBinary(value, message_buffer);
++num_normal_values;
}
void ProtobufSimpleWriter::writeString(const StringRef & str)
{
assert(current_field_number);
++num_normal_values;
writeKey(LENGTH_DELIMITED, message_buffer);
writePbVarUInt(str.size, message_buffer);
message_buffer.write(str.data, str.size);
}
void ProtobufSimpleWriter::writeInt32IfNonZero(Int32 value)
{
if (value)
writeInt32(value);
}
void ProtobufSimpleWriter::writeUInt32IfNonZero(UInt32 value)
{
if (value)
writeUInt32(value);
}
void ProtobufSimpleWriter::writeSInt32IfNonZero(Int32 value)
{
if (value)
writeSInt32(value);
}
void ProtobufSimpleWriter::writeInt64IfNonZero(Int64 value)
{
if (value)
writeInt64(value);
}
void ProtobufSimpleWriter::writeUInt64IfNonZero(UInt64 value)
{
if (value)
writeUInt64(value);
}
void ProtobufSimpleWriter::writeSInt64IfNonZero(Int64 value)
{
if (value)
writeSInt64(value);
}
void ProtobufSimpleWriter::writeFixed32IfNonZero(UInt32 value)
{
if (value)
writeFixed32(value);
}
void ProtobufSimpleWriter::writeSFixed32IfNonZero(Int32 value)
{
if (value)
writeSFixed32(value);
}
void ProtobufSimpleWriter::writeFloatIfNonZero(float value)
{
if (value != 0)
writeFloat(value);
}
void ProtobufSimpleWriter::writeFixed64IfNonZero(UInt64 value)
{
if (value)
writeFixed64(value);
}
void ProtobufSimpleWriter::writeSFixed64IfNonZero(Int64 value)
{
if (value)
writeSFixed64(value);
}
void ProtobufSimpleWriter::writeDoubleIfNonZero(double value)
{
if (value != 0)
writeDouble(value);
}
void ProtobufSimpleWriter::writeStringIfNotEmpty(const StringRef & str)
{
if (str.size)
writeString(str);
}
void ProtobufSimpleWriter::packRepeatedInt32(Int32 value)
{
assert(current_field_number);
writePbVarIntNoZigZagEncoding(value, repeated_packing_buffer);
++num_packed_values;
}
void ProtobufSimpleWriter::packRepeatedUInt32(UInt32 value)
{
assert(current_field_number);
writePbVarUInt(value, repeated_packing_buffer);
++num_packed_values;
}
void ProtobufSimpleWriter::packRepeatedSInt32(Int32 value)
{
assert(current_field_number);
writePbVarInt(value, repeated_packing_buffer);
++num_packed_values;
}
void ProtobufSimpleWriter::packRepeatedInt64(Int64 value)
{
assert(current_field_number);
writePbVarIntNoZigZagEncoding(value, repeated_packing_buffer);
++num_packed_values;
}
void ProtobufSimpleWriter::packRepeatedUInt64(UInt64 value)
{
assert(current_field_number);
writePbVarUInt(value, repeated_packing_buffer);
++num_packed_values;
}
void ProtobufSimpleWriter::packRepeatedSInt64(Int64 value)
{
assert(current_field_number);
writePbVarInt(value, repeated_packing_buffer);
++num_packed_values;
}
void ProtobufSimpleWriter::packRepeatedFixed32(UInt32 value)
{
assert(current_field_number);
writePODBinary(value, repeated_packing_buffer);
++num_packed_values;
}
void ProtobufSimpleWriter::packRepeatedSFixed32(Int32 value)
{
assert(current_field_number);
writePODBinary(value, repeated_packing_buffer);
++num_packed_values;
}
void ProtobufSimpleWriter::packRepeatedFloat(float value)
{
assert(current_field_number);
writePODBinary(value, repeated_packing_buffer);
++num_packed_values;
}
void ProtobufSimpleWriter::packRepeatedFixed64(UInt64 value)
{
assert(current_field_number);
writePODBinary(value, repeated_packing_buffer);
++num_packed_values;
}
void ProtobufSimpleWriter::packRepeatedSFixed64(Int64 value)
{
assert(current_field_number);
writePODBinary(value, repeated_packing_buffer);
++num_packed_values;
}
void ProtobufSimpleWriter::packRepeatedDouble(double value)
{
assert(current_field_number);
writePODBinary(value, repeated_packing_buffer);
++num_packed_values;
}
}

View File

@ -1,88 +0,0 @@
#pragma once
#include <Core/Types.h>
#include <boost/noncopyable.hpp>
#include "IO/WriteBufferFromString.h"
namespace DB
{
/** Utility class to serialize protobufs.
* Knows nothing about protobuf schemas, just provides useful functions to serialize data.
* This class is written following the documentation: https://developers.google.com/protocol-buffers/docs/encoding
*/
class ProtobufSimpleWriter : private boost::noncopyable
{
public:
ProtobufSimpleWriter(WriteBuffer & out_);
~ProtobufSimpleWriter();
/// Should be called when we start writing a new message.
void newMessage();
/// Should be called when we start writing a new field.
/// A passed 'field_number' should be positive and greater than any previous 'field_number'.
void setCurrentField(UInt32 field_number);
UInt32 currentFieldNumber() const { return current_field_number; }
/// Returns number of values added to the current field.
size_t numValues() const { return num_normal_values + num_packed_values; }
void writeInt32(Int32 value);
void writeUInt32(UInt32 value);
void writeSInt32(Int32 value);
void writeInt64(Int64 value);
void writeUInt64(UInt64 value);
void writeSInt64(Int64 value);
void writeFixed32(UInt32 value);
void writeSFixed32(Int32 value);
void writeFloat(float value);
void writeFixed64(UInt64 value);
void writeSFixed64(Int64 value);
void writeDouble(double value);
void writeString(const StringRef & str);
void writeInt32IfNonZero(Int32 value);
void writeUInt32IfNonZero(UInt32 value);
void writeSInt32IfNonZero(Int32 value);
void writeInt64IfNonZero(Int64 value);
void writeUInt64IfNonZero(UInt64 value);
void writeSInt64IfNonZero(Int64 value);
void writeFixed32IfNonZero(UInt32 value);
void writeSFixed32IfNonZero(Int32 value);
void writeFloatIfNonZero(float value);
void writeFixed64IfNonZero(UInt64 value);
void writeSFixed64IfNonZero(Int64 value);
void writeDoubleIfNonZero(double value);
void writeStringIfNotEmpty(const StringRef & str);
void packRepeatedInt32(Int32 value);
void packRepeatedUInt32(UInt32 value);
void packRepeatedSInt32(Int32 value);
void packRepeatedInt64(Int64 value);
void packRepeatedUInt64(UInt64 value);
void packRepeatedSInt64(Int64 value);
void packRepeatedFixed32(UInt32 value);
void packRepeatedSFixed32(Int32 value);
void packRepeatedFloat(float value);
void packRepeatedFixed64(UInt64 value);
void packRepeatedSFixed64(Int64 value);
void packRepeatedDouble(double value);
private:
void finishCurrentMessage();
void finishCurrentField();
enum WireType : UInt32;
void writeKey(WireType wire_type, WriteBuffer & buf);
WriteBuffer & out;
bool were_messages = false;
WriteBufferFromOwnString message_buffer;
UInt32 current_field_number = 0;
size_t num_normal_values = 0;
size_t num_packed_values = 0;
WriteBufferFromOwnString repeated_packing_buffer;
};
}

File diff suppressed because it is too large Load Diff

View File

@ -1,12 +1,16 @@
#pragma once
#include <Core/UUID.h>
#include <Formats/ProtobufSimpleWriter.h>
#include <boost/noncopyable.hpp>
#include <Common/PODArray.h>
#include <Common/UInt128.h>
#include <common/DayNum.h>
#include <Common/config.h>
#if USE_PROTOBUF
#include <Formats/ProtobufColumnMatcher.h>
#include <IO/WriteBufferFromString.h>
#include <boost/noncopyable.hpp>
#include <Common/PODArray.h>
namespace google
@ -18,20 +22,6 @@ namespace protobuf
}
}
#if USE_PROTOBUF
# define EMPTY_DEF
# define EMPTY_DEF_RET(a)
#else
# define EMPTY_DEF {}
# define EMPTY_DEF_RET(a) {return a;}
# pragma GCC diagnostic push
# pragma GCC diagnostic ignored "-Wunused-parameter"
# if defined(__clang__)
# pragma GCC diagnostic ignored "-Wextra-semi"
# endif
#endif
namespace DB
{
class IAggregateFunction;
@ -44,85 +34,234 @@ using ConstAggregateDataPtr = const char *;
class ProtobufWriter : private boost::noncopyable
{
public:
ProtobufWriter(WriteBuffer & out, const google::protobuf::Descriptor * message_type) EMPTY_DEF;
~ProtobufWriter() EMPTY_DEF;
ProtobufWriter(WriteBuffer & out, const google::protobuf::Descriptor * message_type, const std::vector<String> & column_names);
~ProtobufWriter();
/// Returns fields of the protobuf schema sorted by their numbers.
const std::vector<const google::protobuf::FieldDescriptor *> & fieldsInWriteOrder() const;
/// Should be called at the beginning of writing a message.
void startMessage();
/// Should be called when we start writing a new message.
void newMessage() EMPTY_DEF;
/// Should be called at the end of writing a message.
void endMessage();
/// Should be called when we start writing a new field.
/// Returns false if there is no more fields in the message type.
bool nextField() EMPTY_DEF_RET(false);
/// Prepares for writing values of a field.
/// Returns true and sets 'column_index' to the corresponding column's index.
/// Returns false if there are no more fields to write in the message type (call endMessage() in this case).
bool writeField(size_t & column_index);
/// Returns the current field of the message type.
/// The value returned by this function changes after calling nextField() or newMessage().
#if USE_PROTOBUF
const google::protobuf::FieldDescriptor * currentField() const { return current_field; }
#endif
void writeNumber(Int8 value) EMPTY_DEF;
void writeNumber(UInt8 value) EMPTY_DEF;
void writeNumber(Int16 value) EMPTY_DEF;
void writeNumber(UInt16 value) EMPTY_DEF;
void writeNumber(Int32 value) EMPTY_DEF;
void writeNumber(UInt32 value) EMPTY_DEF;
void writeNumber(Int64 value) EMPTY_DEF;
void writeNumber(UInt64 value) EMPTY_DEF;
void writeNumber(UInt128 value) EMPTY_DEF;
void writeNumber(Float32 value) EMPTY_DEF;
void writeNumber(Float64 value) EMPTY_DEF;
void writeString(const StringRef & value) EMPTY_DEF;
void prepareEnumMapping(const std::vector<std::pair<std::string, Int8>> & name_value_pairs) EMPTY_DEF;
void prepareEnumMapping(const std::vector<std::pair<std::string, Int16>> & name_value_pairs) EMPTY_DEF;
void writeEnum(Int8 value) EMPTY_DEF;
void writeEnum(Int16 value) EMPTY_DEF;
void writeUUID(const UUID & value) EMPTY_DEF;
void writeDate(DayNum date) EMPTY_DEF;
void writeDateTime(time_t tm) EMPTY_DEF;
void writeDecimal(Decimal32 decimal, UInt32 scale) EMPTY_DEF;
void writeDecimal(Decimal64 decimal, UInt32 scale) EMPTY_DEF;
void writeDecimal(const Decimal128 & decimal, UInt32 scale) EMPTY_DEF;
void writeAggregateFunction(const AggregateFunctionPtr & function, ConstAggregateDataPtr place) EMPTY_DEF;
/// Writes a value. This function should be called one or multiple times after writeField().
/// Returns false if there are no more place for the values in the protobuf's field.
/// This can happen if the protobuf's field is not declared as repeated in the protobuf schema.
bool writeNumber(Int8 value) { return writeValueIfPossible(&IConverter::writeInt8, value); }
bool writeNumber(UInt8 value) { return writeValueIfPossible(&IConverter::writeUInt8, value); }
bool writeNumber(Int16 value) { return writeValueIfPossible(&IConverter::writeInt16, value); }
bool writeNumber(UInt16 value) { return writeValueIfPossible(&IConverter::writeUInt16, value); }
bool writeNumber(Int32 value) { return writeValueIfPossible(&IConverter::writeInt32, value); }
bool writeNumber(UInt32 value) { return writeValueIfPossible(&IConverter::writeUInt32, value); }
bool writeNumber(Int64 value) { return writeValueIfPossible(&IConverter::writeInt64, value); }
bool writeNumber(UInt64 value) { return writeValueIfPossible(&IConverter::writeUInt64, value); }
bool writeNumber(UInt128 value) { return writeValueIfPossible(&IConverter::writeUInt128, value); }
bool writeNumber(Float32 value) { return writeValueIfPossible(&IConverter::writeFloat32, value); }
bool writeNumber(Float64 value) { return writeValueIfPossible(&IConverter::writeFloat64, value); }
bool writeString(const StringRef & str) { return writeValueIfPossible(&IConverter::writeString, str); }
void prepareEnumMapping(const std::vector<std::pair<std::string, Int8>> & enum_values) { current_converter->prepareEnumMapping8(enum_values); }
void prepareEnumMapping(const std::vector<std::pair<std::string, Int16>> & enum_values) { current_converter->prepareEnumMapping16(enum_values); }
bool writeEnum(Int8 value) { return writeValueIfPossible(&IConverter::writeEnum8, value); }
bool writeEnum(Int16 value) { return writeValueIfPossible(&IConverter::writeEnum16, value); }
bool writeUUID(const UUID & uuid) { return writeValueIfPossible(&IConverter::writeUUID, uuid); }
bool writeDate(DayNum date) { return writeValueIfPossible(&IConverter::writeDate, date); }
bool writeDateTime(time_t tm) { return writeValueIfPossible(&IConverter::writeDateTime, tm); }
bool writeDecimal(Decimal32 decimal, UInt32 scale) { return writeValueIfPossible(&IConverter::writeDecimal32, decimal, scale); }
bool writeDecimal(Decimal64 decimal, UInt32 scale) { return writeValueIfPossible(&IConverter::writeDecimal64, decimal, scale); }
bool writeDecimal(const Decimal128 & decimal, UInt32 scale) { return writeValueIfPossible(&IConverter::writeDecimal128, decimal, scale); }
bool writeAggregateFunction(const AggregateFunctionPtr & function, ConstAggregateDataPtr place) { return writeValueIfPossible(&IConverter::writeAggregateFunction, function, place); }
private:
#if USE_PROTOBUF
class SimpleWriter
{
public:
SimpleWriter(WriteBuffer & out_);
~SimpleWriter();
void enumerateFieldsInWriteOrder(const google::protobuf::Descriptor * message_type);
void createConverters();
void startMessage();
void endMessage();
void finishCurrentMessage();
void finishCurrentField();
void startNestedMessage();
void endNestedMessage(UInt32 field_number, bool is_group, bool skip_if_empty);
ProtobufSimpleWriter simple_writer;
std::vector<const google::protobuf::FieldDescriptor *> fields_in_write_order;
size_t current_field_index = -1;
const google::protobuf::FieldDescriptor * current_field = nullptr;
void writeInt(UInt32 field_number, Int64 value);
void writeUInt(UInt32 field_number, UInt64 value);
void writeSInt(UInt32 field_number, Int64 value);
template <typename T>
void writeFixed(UInt32 field_number, T value);
void writeString(UInt32 field_number, const StringRef & str);
class Converter;
class ToStringConverter;
template <typename T>
class ToNumberConverter;
class ToBoolConverter;
class ToEnumConverter;
void startRepeatedPack();
void addIntToRepeatedPack(Int64 value);
void addUIntToRepeatedPack(UInt64 value);
void addSIntToRepeatedPack(Int64 value);
template <typename T>
void addFixedToRepeatedPack(T value);
void endRepeatedPack(UInt32 field_number);
std::vector<std::unique_ptr<Converter>> converters;
Converter * current_converter = nullptr;
private:
struct Piece
{
size_t start;
size_t end;
Piece(size_t start, size_t end) : start(start), end(end) {}
Piece() = default;
};
#endif
struct NestedInfo
{
size_t num_pieces_at_start;
size_t num_bytes_skipped_at_start;
NestedInfo(size_t num_pieces_at_start, size_t num_bytes_skipped_at_start)
: num_pieces_at_start(num_pieces_at_start), num_bytes_skipped_at_start(num_bytes_skipped_at_start)
{
}
};
WriteBuffer & out;
PODArray<UInt8> buffer;
std::vector<Piece> pieces;
size_t current_piece_start;
size_t num_bytes_skipped;
std::vector<NestedInfo> nested_infos;
};
class IConverter
{
public:
virtual ~IConverter() = default;
virtual void writeString(const StringRef &) = 0;
virtual void writeInt8(Int8) = 0;
virtual void writeUInt8(UInt8) = 0;
virtual void writeInt16(Int16) = 0;
virtual void writeUInt16(UInt16) = 0;
virtual void writeInt32(Int32) = 0;
virtual void writeUInt32(UInt32) = 0;
virtual void writeInt64(Int64) = 0;
virtual void writeUInt64(UInt64) = 0;
virtual void writeUInt128(const UInt128 &) = 0;
virtual void writeFloat32(Float32) = 0;
virtual void writeFloat64(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 void writeEnum8(Int8) = 0;
virtual void writeEnum16(Int16) = 0;
virtual void writeUUID(const UUID &) = 0;
virtual void writeDate(DayNum) = 0;
virtual void writeDateTime(time_t) = 0;
virtual void writeDecimal32(Decimal32, UInt32) = 0;
virtual void writeDecimal64(Decimal64, UInt32) = 0;
virtual void writeDecimal128(const Decimal128 &, UInt32) = 0;
virtual void writeAggregateFunction(const AggregateFunctionPtr &, ConstAggregateDataPtr) = 0;
};
class ConverterBaseImpl;
template <bool skip_null_value>
class ConverterToString;
template <int field_type_id, typename ToType, bool skip_null_value, bool pack_repeated>
class ConverterToNumber;
template <bool skip_null_value, bool pack_repeated>
class ConverterToBool;
template <bool skip_null_value, bool pack_repeated>
class ConverterToEnum;
struct ColumnMatcherTraits
{
struct FieldData
{
std::unique_ptr<IConverter> converter;
bool is_required;
bool is_repeatable;
bool should_pack_repeated;
ProtobufColumnMatcher::Message<ColumnMatcherTraits> * repeatable_container_message;
};
struct MessageData
{
UInt32 parent_field_number;
bool is_group;
bool is_required;
ProtobufColumnMatcher::Message<ColumnMatcherTraits> * repeatable_container_message;
bool need_repeat;
};
};
using Message = ProtobufColumnMatcher::Message<ColumnMatcherTraits>;
using Field = ProtobufColumnMatcher::Field<ColumnMatcherTraits>;
void setTraitsDataAfterMatchingColumns(Message * message);
template <int field_type_id>
std::unique_ptr<IConverter> createConverter(const google::protobuf::FieldDescriptor * field);
template <typename... Params>
using WriteValueFunctionPtr = void (IConverter::*)(Params...);
template <typename... Params, typename... Args>
bool writeValueIfPossible(WriteValueFunctionPtr<Params...> func, Args &&... args)
{
if (num_values && !current_field->data.is_repeatable)
{
setNestedMessageNeedsRepeat();
return false;
}
(current_converter->*func)(std::forward<Args>(args)...);
++num_values;
return true;
}
void setNestedMessageNeedsRepeat();
void endWritingField();
SimpleWriter simple_writer;
std::unique_ptr<Message> root_message;
Message * current_message;
size_t current_field_index = 0;
const Field * current_field = nullptr;
IConverter * current_converter = nullptr;
size_t num_values = 0;
};
}
#if !USE_PROTOBUF
# undef EMPTY_DEF
# undef EMPTY_DEF_RET
# pragma GCC diagnostic pop
#else
namespace DB
{
class IAggregateFunction;
using AggregateFunctionPtr = std::shared_ptr<IAggregateFunction>;
using ConstAggregateDataPtr = const char *;
class ProtobufWriter
{
public:
bool writeNumber(Int8 value) { return false; }
bool writeNumber(UInt8 value) { return false; }
bool writeNumber(Int16 value) { return false; }
bool writeNumber(UInt16 value) { return false; }
bool writeNumber(Int32 value) { return false; }
bool writeNumber(UInt32 value) { return false; }
bool writeNumber(Int64 value) { return false; }
bool writeNumber(UInt64 value) { return false; }
bool writeNumber(UInt128 value) { return false; }
bool writeNumber(Float32 value) { return false; }
bool writeNumber(Float64 value) { return false; }
bool writeString(const StringRef & value) { 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 writeEnum(Int8 value) { return false; }
bool writeEnum(Int16 value) { return false; }
bool writeUUID(const UUID & value) { return false; }
bool writeDate(DayNum date) { return false; }
bool writeDateTime(time_t tm) { return false; }
bool writeDecimal(Decimal32 decimal, UInt32 scale) { return false; }
bool writeDecimal(Decimal64 decimal, UInt32 scale) { return false; }
bool writeDecimal(const Decimal128 & decimal, UInt32 scale) { return false; }
bool writeAggregateFunction(const AggregateFunctionPtr & function, ConstAggregateDataPtr place) { return false; }
};
}
#endif

View File

@ -7,7 +7,7 @@
#include <Functions/FunctionHelpers.h>
#include <Functions/GatherUtils/Algorithms.h>
#include <IO/WriteHelpers.h>
#include <libbase64.h>
#include <libbase64.h> // Y_IGNORE
namespace DB

View File

@ -14,7 +14,7 @@
#include <Common/config.h>
#if USE_XXHASH
#include <xxhash.h>
#include <xxhash.h> // Y_IGNORE
#endif
#include <Poco/ByteOrder.h>

View File

@ -2,7 +2,7 @@
#if USE_BROTLI
#include "BrotliReadBuffer.h"
#include <brotli/decode.h>
#include <brotli/decode.h> // Y_IGNORE
namespace DB
{

View File

@ -4,7 +4,7 @@
#include <Poco/URI.h>
#if USE_HDFS
#include <hdfs/hdfs.h>
#include <hdfs/hdfs.h> // Y_IGNORE
namespace DB
{

View File

@ -2,10 +2,10 @@
#if USE_HDFS
#include <IO/ReadBufferFromHDFS.h>
#include <IO/ReadBufferFromHDFS.h> // Y_IGNORE
#include <IO/HDFSCommon.h>
#include <Poco/URI.h>
#include <hdfs/hdfs.h>
#include <hdfs/hdfs.h> // Y_IGNORE
namespace DB

View File

@ -3,9 +3,9 @@
#if USE_HDFS
#include <Poco/URI.h>
#include <IO/WriteBufferFromHDFS.h>
#include <IO/WriteBufferFromHDFS.h> // Y_IGNORE
#include <IO/HDFSCommon.h>
#include <hdfs/hdfs.h>
#include <hdfs/hdfs.h> // Y_IGNORE
namespace DB

View File

@ -3,12 +3,12 @@
#if USE_HDFS
#include <Storages/StorageFactory.h>
#include <Storages/StorageHDFS.h>
#include <Storages/StorageHDFS.h> // Y_IGNORE
#include <Interpreters/Context.h>
#include <Interpreters/evaluateConstantExpression.h>
#include <Parsers/ASTLiteral.h>
#include <IO/ReadBufferFromHDFS.h>
#include <IO/WriteBufferFromHDFS.h>
#include <IO/ReadBufferFromHDFS.h> // Y_IGNORE
#include <IO/WriteBufferFromHDFS.h> // Y_IGNORE
#include <Formats/FormatFactory.h>
#include <DataStreams/IBlockOutputStream.h>
#include <DataStreams/UnionBlockInputStream.h>

View File

@ -1,9 +1,9 @@
#include <Common/config.h>
#if USE_HDFS
#include <Storages/StorageHDFS.h>
#include <Storages/StorageHDFS.h> // Y_IGNORE
#include <TableFunctions/TableFunctionFactory.h>
#include <TableFunctions/TableFunctionHDFS.h>
#include <TableFunctions/TableFunctionHDFS.h> // Y_IGNORE
namespace DB
{

View File

@ -20,7 +20,27 @@ enum ZodiacSign {
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 Person {
message MeasureUnit
{
string unit = 1;
float coef = 2;
};
string uuid = 1;
string name = 2;
string surname = 3;
@ -41,6 +61,8 @@ message Person {
float someRatio = 18;
float temperature = 19;
sint64 randomBigNumber = 20;
repeated MeasureUnit measureUnits = 21;
Nestiness nestiness = 22;
};
enum OnlineStatus {
@ -53,6 +75,11 @@ message AltPerson {
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;
@ -65,6 +92,7 @@ message AltPerson {
OnlineStatus isOnline = 1;
double someRatio = 100;
fixed64 visitTime = 15;
Dummy newMessage = 1000;
sfixed64 randomBigNumber = 140;
repeated int32 newFieldInt = 104;
repeated float color = 14;
@ -73,9 +101,26 @@ message AltPerson {
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;
};
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;
@ -95,4 +140,6 @@ message StrPerson {
string someRatio = 18;
string temperature = 19;
string randomBigNumber = 20;
MeasureUnits measureUnits = 21;
NestinessA nestiness_a = 22;
};

View File

@ -1,4 +1,4 @@
echo -ne '\xe0\x01\x0a\x24\x61\x37\x35\x32\x32\x31\x35\x38\x2d\x33\x64\x34\x31\x2d\x34\x62\x37\x37\x2d\x61\x64\x36\x39\x2d\x36\x63\x35\x39\x38\x65\x65\x35\x35\x63\x34\x39\x12\x04\x49\x76\x61\x6e\x1a\x06\x50\x65\x74\x72\x6f\x76\x20\x01\x28\xaf\x1f\x32\x03\x70\x6e\x67\x3a\x0c\x2b\x37\x34\x39\x35\x31\x32\x33\x34\x35\x36\x37\x40\x01\x4d\xfc\xd0\x30\x5c\x50\x26\x58\x09\x62\x09\x59\x65\x73\x74\x65\x72\x64\x61\x79\x62\x07\x46\x6c\x6f\x77\x65\x72\x73\x6a\x04\xff\x01\x00\x00\x72\x06\x4d\x6f\x73\x63\x6f\x77\x7a\x08\x4b\x03\x5f\x42\x72\x7d\x16\x42\x81\x01\x1f\x85\xeb\x51\xb8\x1e\x09\x40\x89\x01\x33\x33\x33\x33\x33\xc3\x6a\x40\x95\x01\xcd\xcc\xcc\x3d\x9d\x01\x9a\x99\xb9\x40\xa0\x01\x80\xc4\xd7\x8d\x7f\xaa\x01\x0c\x0a\x05\x6d\x65\x74\x65\x72\x15\x00\x00\x80\x3f\xaa\x01\x11\x0a\x0a\x63\x65\x6e\x74\x69\x6d\x65\x74\x65\x72\x15\x0a\xd7\x23\x3c\xaa\x01\x10\x0a\x09\x6b\x69\x6c\x6f\x6d\x65\x74\x65\x72\x15\x00\x00\x7a\x44\x7e\x0a\x24\x63\x36\x39\x34\x61\x64\x38\x61\x2d\x66\x37\x31\x34\x2d\x34\x65\x61\x33\x2d\x39\x30\x37\x64\x2d\x66\x64\x35\x34\x66\x62\x32\x35\x64\x39\x62\x35\x12\x07\x4e\x61\x74\x61\x6c\x69\x61\x1a\x08\x53\x6f\x6b\x6f\x6c\x6f\x76\x61\x28\xa6\x3f\x32\x03\x6a\x70\x67\x50\x1a\x58\x0b\x6a\x04\x64\xc8\x01\x32\x72\x08\x50\x6c\x79\x6d\x6f\x75\x74\x68\x7a\x08\x6a\x9d\x49\x42\x46\x8c\x84\xc0\x81\x01\x6e\x86\x1b\xf0\xf9\x21\x09\x40\x95\x01\x42\x60\xe5\x3b\x9d\x01\xcd\xcc\xac\x40\xa0\x01\xff\xff\xa9\xce\x93\x8c\x09\xb3\x01\x0a\x24\x61\x37\x64\x61\x31\x61\x61\x36\x2d\x66\x34\x32\x35\x2d\x34\x37\x38\x39\x2d\x38\x39\x34\x37\x2d\x62\x30\x33\x34\x37\x38\x36\x65\x64\x33\x37\x34\x12\x06\x56\x61\x73\x69\x6c\x79\x1a\x07\x53\x69\x64\x6f\x72\x6f\x76\x20\x01\x28\xfb\x48\x32\x03\x62\x6d\x70\x3a\x0d\x2b\x34\x34\x32\x30\x31\x32\x33\x34\x35\x36\x37\x38\x40\x01\x4d\x50\xe0\x27\x5c\x50\x17\x58\x04\x62\x05\x53\x75\x6e\x6e\x79\x6a\x05\xfa\x01\xf4\x01\x0a\x72\x08\x4d\x75\x72\x6d\x61\x6e\x73\x6b\x7a\x08\xfd\xf0\x89\x42\xc8\x4c\x04\x42\x81\x01\x11\x2d\x44\x54\xfb\x21\x09\x40\x89\x01\x00\x00\x00\xe8\x76\x48\x37\x42\x95\x01\x00\x00\x48\x44\x9d\x01\xcd\xcc\x4c\xc0\xa0\x01\x80\xd4\x9f\x93\x01\xaa\x01\x0c\x0a\x05\x70\x6f\x75\x6e\x64\x15\x00\x00\x80\x41' | $CLICKHOUSE_CLIENT --query="INSERT INTO test.table FORMAT Protobuf SETTINGS format_schema = '$CURDIR/00825_protobuf_format_with_nested:Person'"
echo -ne '\xaa\x01\x12\x05\x46\x72\x69\x64\x61\x28\x99\xe1\xf3\xd1\x0b\x52\x08\x45\x72\x6d\x61\x6b\x6f\x76\x61\x72\x0c\x00\x00\xdc\x42\x00\x00\x52\x43\x00\x00\x94\x42\x79\x48\xce\x3d\x51\x00\x00\x00\x00\xc8\x02\x14\xc2\x05\x08\x00\x00\x80\x44\x00\x00\x80\x49\x9a\x06\x02\x4b\x42\x9a\x06\x02\x4d\x42\xa1\x06\x00\x00\x00\x00\x00\x00\xe0\x3f\xa8\x06\x2a\xa8\x06\xa8\xff\xff\xff\xff\xff\xff\xff\xff\x01\xb0\x06\x01\xbd\x06\x25\x06\x49\x40\xfa\x06\x02\x34\x30\x90\x08\xe2\x08\xe1\x08\x89\xe6\x6e\xdd\x01\x00\x00\x00\xb0\x09\xc3\x19\xd0\x0c\xb7\x02\xe2\x12\x24\x32\x30\x66\x63\x64\x39\x35\x61\x2d\x33\x33\x32\x64\x2d\x34\x31\x64\x62\x2d\x61\x39\x65\x63\x2d\x31\x36\x31\x66\x36\x34\x34\x64\x30\x35\x39\x63\xb0\x01\x08\x01\x12\x06\x49\x73\x6f\x6c\x64\x65\x52\x07\x4c\x61\x76\x72\x6f\x76\x61\x72\x0c\x00\x00\x7f\x43\x00\x00\x00\x00\x00\x00\x7f\x43\xaa\x01\x03\x61\x62\x63\xc8\x02\x32\xc2\x05\x08\x00\x00\x00\x41\x00\x00\x80\x3f\x9a\x06\x04\x42\x79\x74\x65\x9a\x06\x03\x42\x69\x74\xa1\x06\x00\x00\x00\x00\x00\x00\x12\x40\xa8\x06\x1a\xa8\x06\xb0\xff\xff\xff\xff\xff\xff\xff\xff\x01\xb0\x06\x01\xbd\x06\xf9\x0f\x49\x40\xc2\x06\x01\x2c\xfa\x06\x02\x33\x32\x90\x08\x78\xe1\x08\x39\x4e\x2b\xfe\xe4\xf5\xff\xff\xb0\x09\xe8\x30\xd8\x12\x01\xe2\x12\x24\x37\x63\x66\x61\x36\x38\x35\x36\x2d\x61\x35\x34\x61\x2d\x34\x37\x38\x36\x2d\x62\x38\x65\x35\x2d\x37\x34\x35\x31\x35\x39\x64\x35\x32\x32\x37\x38\xc2\x3e\x05\x15\x00\x00\xb6\x42' | $CLICKHOUSE_CLIENT --query="INSERT INTO test.table FORMAT Protobuf SETTINGS format_schema = '$CURDIR/00825_protobuf_format_with_nested:AltPerson'"
echo -ne '\x9a\x02\x0a\x24\x61\x61\x30\x65\x35\x61\x30\x36\x2d\x63\x61\x62\x32\x2d\x34\x30\x33\x34\x2d\x61\x36\x61\x32\x2d\x34\x38\x65\x38\x32\x62\x39\x31\x36\x36\x34\x65\x12\x06\x4c\x65\x6f\x6e\x69\x64\x1a\x08\x4b\x69\x72\x69\x6c\x6c\x6f\x76\x22\x04\x6d\x61\x6c\x65\x2a\x0a\x31\x39\x38\x33\x2d\x30\x36\x2d\x32\x34\x3a\x0c\x2b\x37\x34\x39\x35\x30\x32\x37\x35\x38\x36\x34\x42\x01\x31\x4a\x13\x32\x30\x31\x39\x2d\x30\x32\x2d\x30\x34\x20\x30\x39\x3a\x34\x35\x3a\x30\x30\x52\x02\x33\x35\x5a\x06\x63\x61\x6e\x63\x65\x72\x62\x07\x37\x20\x72\x69\x6e\x67\x73\x62\x08\x45\x61\x73\x74\x73\x69\x64\x65\x62\x0b\x4c\x61\x73\x74\x20\x48\x75\x72\x72\x61\x68\x6a\x01\x30\x6a\x01\x30\x6a\x03\x32\x35\x35\x72\x09\x53\x61\x6e\x20\x44\x69\x65\x67\x6f\x7a\x09\x33\x32\x2e\x38\x32\x33\x39\x34\x33\x7a\x0b\x2d\x31\x31\x37\x2e\x30\x38\x31\x33\x32\x37\x82\x01\x09\x33\x2e\x31\x34\x31\x35\x39\x32\x37\x8a\x01\x08\x31\x35\x30\x30\x30\x30\x30\x30\x92\x01\x06\x31\x38\x36\x2e\x37\x35\x9a\x01\x04\x2d\x32\x2e\x31\xa2\x01\x0b\x32\x30\x36\x35\x39\x38\x32\x39\x33\x33\x31\xaa\x01\x18\x0a\x06\x6d\x69\x6e\x75\x74\x65\x0a\x04\x68\x6f\x75\x72\x12\x02\x36\x30\x12\x04\x33\x36\x30\x30' | $CLICKHOUSE_CLIENT --query="INSERT INTO test.table FORMAT Protobuf SETTINGS format_schema = '$CURDIR/00825_protobuf_format_with_nested:StrPerson'"
echo -ne '\xcf\x01\x0a\x24\x33\x66\x61\x65\x65\x30\x36\x34\x2d\x63\x34\x66\x37\x2d\x34\x64\x33\x34\x2d\x62\x36\x66\x33\x2d\x38\x64\x38\x31\x63\x32\x62\x36\x61\x31\x35\x64\x12\x04\x4e\x69\x63\x6b\x1a\x0a\x4b\x6f\x6c\x65\x73\x6e\x69\x6b\x6f\x76\x20\x01\x28\xda\x52\x32\x03\x62\x6d\x70\x3a\x0c\x34\x31\x32\x2d\x36\x38\x37\x2d\x35\x30\x30\x37\x40\x01\x4d\x2f\x27\xf2\x5b\x50\x14\x58\x09\x62\x06\x48\x61\x76\x61\x6e\x61\x68\x80\x01\x68\x00\x68\x80\x01\x72\x0a\x50\x69\x74\x74\x73\x62\x75\x72\x67\x68\x7a\x08\x9b\x11\x22\x42\x1f\xe6\x9f\xc2\x81\x01\x28\x2d\x44\x54\xfb\x21\x09\x40\x89\x01\x00\x00\x00\xe8\x76\x48\x27\x42\x95\x01\x00\x00\x43\x44\x9d\x01\x66\x66\x92\x41\xa0\x01\xce\xdf\xb8\xba\x01\xab\x01\x0d\xcd\xcc\xe2\x41\x0d\xcd\xcc\x4c\x3e\x0d\x00\x00\x80\x3f\x12\x05\x6f\x75\x6e\x63\x65\x12\x05\x63\x61\x72\x61\x74\x12\x04\x67\x72\x61\x6d\xac\x01' | $CLICKHOUSE_CLIENT --query="INSERT INTO test.table FORMAT Protobuf SETTINGS format_schema = '$CURDIR/00825_protobuf_format_syntax2_with_nested:Syntax2Person'"
echo -ne '\xf3\x01\x0a\x24\x61\x37\x35\x32\x32\x31\x35\x38\x2d\x33\x64\x34\x31\x2d\x34\x62\x37\x37\x2d\x61\x64\x36\x39\x2d\x36\x63\x35\x39\x38\x65\x65\x35\x35\x63\x34\x39\x12\x04\x49\x76\x61\x6e\x1a\x06\x50\x65\x74\x72\x6f\x76\x20\x01\x28\xaf\x1f\x32\x03\x70\x6e\x67\x3a\x0c\x2b\x37\x34\x39\x35\x31\x32\x33\x34\x35\x36\x37\x40\x01\x4d\xfc\xd0\x30\x5c\x50\x26\x58\x09\x62\x09\x59\x65\x73\x74\x65\x72\x64\x61\x79\x62\x07\x46\x6c\x6f\x77\x65\x72\x73\x6a\x04\xff\x01\x00\x00\x72\x06\x4d\x6f\x73\x63\x6f\x77\x7a\x08\x4b\x03\x5f\x42\x72\x7d\x16\x42\x81\x01\x1f\x85\xeb\x51\xb8\x1e\x09\x40\x89\x01\x33\x33\x33\x33\x33\xc3\x6a\x40\x95\x01\xcd\xcc\xcc\x3d\x9d\x01\x9a\x99\xb9\x40\xa0\x01\x80\xc4\xd7\x8d\x7f\xaa\x01\x0c\x0a\x05\x6d\x65\x74\x65\x72\x15\x00\x00\x80\x3f\xaa\x01\x11\x0a\x0a\x63\x65\x6e\x74\x69\x6d\x65\x74\x65\x72\x15\x0a\xd7\x23\x3c\xaa\x01\x10\x0a\x09\x6b\x69\x6c\x6f\x6d\x65\x74\x65\x72\x15\x00\x00\x7a\x44\xb2\x01\x10\x0a\x0e\xa2\x06\x0b\x0a\x09\x08\xf4\x03\x12\x04\xf5\x03\xf6\x03\x7e\x0a\x24\x63\x36\x39\x34\x61\x64\x38\x61\x2d\x66\x37\x31\x34\x2d\x34\x65\x61\x33\x2d\x39\x30\x37\x64\x2d\x66\x64\x35\x34\x66\x62\x32\x35\x64\x39\x62\x35\x12\x07\x4e\x61\x74\x61\x6c\x69\x61\x1a\x08\x53\x6f\x6b\x6f\x6c\x6f\x76\x61\x28\xa6\x3f\x32\x03\x6a\x70\x67\x50\x1a\x58\x0b\x6a\x04\x64\xc8\x01\x32\x72\x08\x50\x6c\x79\x6d\x6f\x75\x74\x68\x7a\x08\x6a\x9d\x49\x42\x46\x8c\x84\xc0\x81\x01\x6e\x86\x1b\xf0\xf9\x21\x09\x40\x95\x01\x42\x60\xe5\x3b\x9d\x01\xcd\xcc\xac\x40\xa0\x01\xff\xff\xa9\xce\x93\x8c\x09\xc0\x01\x0a\x24\x61\x37\x64\x61\x31\x61\x61\x36\x2d\x66\x34\x32\x35\x2d\x34\x37\x38\x39\x2d\x38\x39\x34\x37\x2d\x62\x30\x33\x34\x37\x38\x36\x65\x64\x33\x37\x34\x12\x06\x56\x61\x73\x69\x6c\x79\x1a\x07\x53\x69\x64\x6f\x72\x6f\x76\x20\x01\x28\xfb\x48\x32\x03\x62\x6d\x70\x3a\x0d\x2b\x34\x34\x32\x30\x31\x32\x33\x34\x35\x36\x37\x38\x40\x01\x4d\x50\xe0\x27\x5c\x50\x17\x58\x04\x62\x05\x53\x75\x6e\x6e\x79\x6a\x05\xfa\x01\xf4\x01\x0a\x72\x08\x4d\x75\x72\x6d\x61\x6e\x73\x6b\x7a\x08\xfd\xf0\x89\x42\xc8\x4c\x04\x42\x81\x01\x11\x2d\x44\x54\xfb\x21\x09\x40\x89\x01\x00\x00\x00\xe8\x76\x48\x37\x42\x95\x01\x00\x00\x48\x44\x9d\x01\xcd\xcc\x4c\xc0\xa0\x01\x80\xd4\x9f\x93\x01\xaa\x01\x0c\x0a\x05\x70\x6f\x75\x6e\x64\x15\x00\x00\x80\x41\xb2\x01\x0a\x0a\x08\xa2\x06\x05\x0a\x03\x08\xf7\x03' | $CLICKHOUSE_CLIENT --query="INSERT INTO test.table FORMAT Protobuf SETTINGS format_schema = '$CURDIR/00825_protobuf_format:Person'"
echo -ne '\xb3\x01\x12\x05\x46\x72\x69\x64\x61\x28\x99\xe1\xf3\xd1\x0b\x52\x08\x45\x72\x6d\x61\x6b\x6f\x76\x61\x72\x0c\x00\x00\xdc\x42\x00\x00\x52\x43\x00\x00\x94\x42\x79\x48\xce\x3d\x51\x00\x00\x00\x00\xc8\x02\x14\xc2\x05\x08\x00\x00\x80\x44\x00\x00\x80\x49\x9a\x06\x02\x4b\x42\x9a\x06\x02\x4d\x42\xa1\x06\x00\x00\x00\x00\x00\x00\xe0\x3f\xa8\x06\x2a\xa8\x06\xa8\xff\xff\xff\xff\xff\xff\xff\xff\x01\xb0\x06\x01\xbd\x06\x25\x06\x49\x40\xfa\x06\x02\x34\x30\x90\x08\xe2\x08\xe1\x08\x89\xe6\x6e\xdd\x01\x00\x00\x00\xb0\x09\xc3\x19\xd0\x0c\xb7\x02\xe2\x12\x24\x32\x30\x66\x63\x64\x39\x35\x61\x2d\x33\x33\x32\x64\x2d\x34\x31\x64\x62\x2d\x61\x39\x65\x63\x2d\x31\x36\x31\x66\x36\x34\x34\x64\x30\x35\x39\x63\xa0\x38\xbc\x05\xaa\x38\x02\xbd\x05\xb4\x01\x08\x01\x12\x06\x49\x73\x6f\x6c\x64\x65\x52\x07\x4c\x61\x76\x72\x6f\x76\x61\x72\x0c\x00\x00\x7f\x43\x00\x00\x00\x00\x00\x00\x7f\x43\xaa\x01\x03\x61\x62\x63\xc8\x02\x32\xc2\x05\x08\x00\x00\x00\x41\x00\x00\x80\x3f\x9a\x06\x04\x42\x79\x74\x65\x9a\x06\x03\x42\x69\x74\xa1\x06\x00\x00\x00\x00\x00\x00\x12\x40\xa8\x06\x1a\xa8\x06\xb0\xff\xff\xff\xff\xff\xff\xff\xff\x01\xb0\x06\x01\xbd\x06\xf9\x0f\x49\x40\xc2\x06\x01\x2c\xfa\x06\x02\x33\x32\x90\x08\x78\xe1\x08\x39\x4e\x2b\xfe\xe4\xf5\xff\xff\xb0\x09\xe8\x30\xd8\x12\x01\xe2\x12\x24\x37\x63\x66\x61\x36\x38\x35\x36\x2d\x61\x35\x34\x61\x2d\x34\x37\x38\x36\x2d\x62\x38\x65\x35\x2d\x37\x34\x35\x31\x35\x39\x64\x35\x32\x32\x37\x38\xa0\x38\xbe\x05\xc2\x3e\x05\x15\x00\x00\xb6\x42' | $CLICKHOUSE_CLIENT --query="INSERT INTO test.table FORMAT Protobuf SETTINGS format_schema = '$CURDIR/00825_protobuf_format:AltPerson'"
echo -ne '\xa5\x02\x0a\x24\x61\x61\x30\x65\x35\x61\x30\x36\x2d\x63\x61\x62\x32\x2d\x34\x30\x33\x34\x2d\x61\x36\x61\x32\x2d\x34\x38\x65\x38\x32\x62\x39\x31\x36\x36\x34\x65\x12\x06\x4c\x65\x6f\x6e\x69\x64\x1a\x08\x4b\x69\x72\x69\x6c\x6c\x6f\x76\x22\x04\x6d\x61\x6c\x65\x2a\x0a\x31\x39\x38\x33\x2d\x30\x36\x2d\x32\x34\x3a\x0c\x2b\x37\x34\x39\x35\x30\x32\x37\x35\x38\x36\x34\x42\x01\x31\x4a\x13\x32\x30\x31\x39\x2d\x30\x32\x2d\x30\x34\x20\x30\x39\x3a\x34\x35\x3a\x30\x30\x52\x02\x33\x35\x5a\x06\x63\x61\x6e\x63\x65\x72\x62\x07\x37\x20\x72\x69\x6e\x67\x73\x62\x08\x45\x61\x73\x74\x73\x69\x64\x65\x62\x0b\x4c\x61\x73\x74\x20\x48\x75\x72\x72\x61\x68\x6a\x01\x30\x6a\x01\x30\x6a\x03\x32\x35\x35\x72\x09\x53\x61\x6e\x20\x44\x69\x65\x67\x6f\x7a\x09\x33\x32\x2e\x38\x32\x33\x39\x34\x33\x7a\x0b\x2d\x31\x31\x37\x2e\x30\x38\x31\x33\x32\x37\x82\x01\x09\x33\x2e\x31\x34\x31\x35\x39\x32\x37\x8a\x01\x08\x31\x35\x30\x30\x30\x30\x30\x30\x92\x01\x06\x31\x38\x36\x2e\x37\x35\x9a\x01\x04\x2d\x32\x2e\x31\xa2\x01\x0b\x32\x30\x36\x35\x39\x38\x32\x39\x33\x33\x31\xaa\x01\x18\x0a\x06\x6d\x69\x6e\x75\x74\x65\x0a\x04\x68\x6f\x75\x72\x12\x02\x36\x30\x12\x04\x33\x36\x30\x30\xb2\x01\x08\x0a\x06\x12\x04\x31\x38\x30\x30' | $CLICKHOUSE_CLIENT --query="INSERT INTO test.table FORMAT Protobuf SETTINGS format_schema = '$CURDIR/00825_protobuf_format:StrPerson'"
echo -ne '\xdd\x01\x0a\x24\x33\x66\x61\x65\x65\x30\x36\x34\x2d\x63\x34\x66\x37\x2d\x34\x64\x33\x34\x2d\x62\x36\x66\x33\x2d\x38\x64\x38\x31\x63\x32\x62\x36\x61\x31\x35\x64\x12\x04\x4e\x69\x63\x6b\x1a\x0a\x4b\x6f\x6c\x65\x73\x6e\x69\x6b\x6f\x76\x20\x01\x28\xda\x52\x32\x03\x62\x6d\x70\x3a\x0c\x34\x31\x32\x2d\x36\x38\x37\x2d\x35\x30\x30\x37\x40\x01\x4d\x2f\x27\xf2\x5b\x50\x14\x58\x09\x62\x06\x48\x61\x76\x61\x6e\x61\x68\x80\x01\x68\x00\x68\x80\x01\x72\x0a\x50\x69\x74\x74\x73\x62\x75\x72\x67\x68\x7a\x08\x9b\x11\x22\x42\x1f\xe6\x9f\xc2\x81\x01\x28\x2d\x44\x54\xfb\x21\x09\x40\x89\x01\x00\x00\x00\xe8\x76\x48\x27\x42\x95\x01\x00\x00\x43\x44\x9d\x01\x66\x66\x92\x41\xa0\x01\xce\xdf\xb8\xba\x01\xab\x01\x0d\xcd\xcc\xe2\x41\x0d\xcd\xcc\x4c\x3e\x0d\x00\x00\x80\x3f\x12\x05\x6f\x75\x6e\x63\x65\x12\x05\x63\x61\x72\x61\x74\x12\x04\x67\x72\x61\x6d\xac\x01\xb3\x01\x0b\xa2\x06\x05\x0b\x08\x96\x4a\x0c\x0c\xb4\x01' | $CLICKHOUSE_CLIENT --query="INSERT INTO test.table FORMAT Protobuf SETTINGS format_schema = '$CURDIR/00825_protobuf_format_syntax2:Syntax2Person'"

View File

@ -1,7 +1,7 @@
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.970680,33.074982] 3.14159265358979 100000000000.00 800 -3.2 154400000 ['pound'] [16]
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 [] []
aa0e5a06-cab2-4034-a6a2-48e82b91664e Leonid Kirillov male 1983-06-24 \N +74950275864\0 1 2019-02-04 09:45:00 35 cancer ['7 rings','Eastside','Last Hurrah'] [0,0,255] San Diego [32.823943,-117.081327] 3.1415927 15000000.00 186.75 -2.1 20659829331 ['minute','hour'] [60,3600]
20fcd95a-332d-41db-a9ec-161f644d059c Frida Ermakova female 1978-12-12 \N 3124555929\0\0\0 0 2013-03-11 16:30:00 40 sagittarius [] [110,210,74] [42.000000,-88.000000] 3.1410000324249268 311.00 0.5 10.0 8010000009 ['KB','MB'] [1024,1048576]
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.10 0.1 5.8 17060000000 ['meter','centimeter','kilometer'] [1,0.01,1000]
3faee064-c4f7-4d34-b6f3-8d81c2b6a15d Nick Kolesnikov male 1998-12-26 bmp 412-687-5007\0 1 2018-11-19 05:59:59 20 capricorn ['Havana'] [128,0,128] Pittsburgh [40.517192,-79.949456] 3.1415926535898 50000000000.00 780 18.3 195500007 ['ounce','carat','gram'] [28.35,0.2,1]
7cfa6856-a54a-4786-b8e5-745159d52278 Isolde Lavrova female 1987-02-09 \N \N 1 \N 32 aquarius [] [255,0,255] [26.000000,-80.000000] 3.1415998935699463 \N 4.5 25.0 -11111111111111 ['Byte','Bit'] [8,1]
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.970680,33.074982] 3.14159265358979 100000000000.00 800 -3.2 154400000 ['pound'] [16] 503 []
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 []
aa0e5a06-cab2-4034-a6a2-48e82b91664e Leonid Kirillov male 1983-06-24 \N +74950275864\0 1 2019-02-04 09:45:00 35 cancer ['7 rings','Eastside','Last Hurrah'] [0,0,255] San Diego [32.823943,-117.081327] 3.1415927 15000000.00 186.75 -2.1 20659829331 ['minute','hour'] [60,3600] \N [1800]
20fcd95a-332d-41db-a9ec-161f644d059c Frida Ermakova female 1978-12-12 \N 3124555929\0\0\0 0 2013-03-11 16:30:00 40 sagittarius [] [110,210,74] [42.000000,-88.000000] 3.1410000324249268 311.00 0.5 10.0 8010000009 ['KB','MB'] [1024,1048576] 700 [701]
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.10 0.1 5.8 17060000000 ['meter','centimeter','kilometer'] [1,0.01,1000] 500 [501,502]
3faee064-c4f7-4d34-b6f3-8d81c2b6a15d Nick Kolesnikov male 1998-12-26 bmp 412-687-5007\0 1 2018-11-19 05:59:59 20 capricorn ['Havana'] [128,0,128] Pittsburgh [40.517192,-79.949456] 3.1415926535898 50000000000.00 780 18.3 195500007 ['ounce','carat','gram'] [28.35,0.2,1] 9494 []
7cfa6856-a54a-4786-b8e5-745159d52278 Isolde Lavrova female 1987-02-09 \N \N 1 \N 32 aquarius [] [255,0,255] [26.000000,-80.000000] 3.1415998935699463 \N 4.5 25.0 -11111111111111 ['Byte','Bit'] [8,1] 702 []

View File

@ -6,7 +6,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
set -e -o pipefail
# Run the client.
$CLICKHOUSE_CLIENT --multiquery <<EOF
$CLICKHOUSE_CLIENT --multiquery <<'EOF'
CREATE DATABASE IF NOT EXISTS test;
DROP TABLE IF EXISTS test.table;
@ -32,7 +32,9 @@ CREATE TABLE test.table (uuid UUID,
someRatio Float32,
temperature Decimal32(1),
randomBigNumber Int64,
measureUnits Nested (unit String, coef Float32)
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();
EOF

View File

@ -35,12 +35,15 @@ CREATE TABLE test.table (uuid UUID,
lotteryWin Nullable(Decimal64(2)),
someRatio Float32,
temperature Decimal32(1),
randomBigNumber Int64
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 test.table VALUES (toUUID('a7522158-3d41-4b77-ad69-6c598ee55c49'), 'Ivan', 'Petrov', 'male', toDate('1980-12-29'), 'png', '+74951234567', 1, toDateTime('2019-01-05 18:45:00'), 38, 'capricorn', ['Yesterday', 'Flowers'], [255, 0, 0], 'Moscow', [55.753215, 37.622504], 3.14, 214.10, 0.1, 5.8, 17060000000);
INSERT INTO test.table 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);
INSERT INTO test.table VALUES (toUUID('a7da1aa6-f425-4789-8947-b034786ed374'), 'Vasily', 'Sidorov', 'male', toDate('1995-07-28'), 'bmp', '+442012345678', 1, toDateTime('2018-12-30 00:00:00'), 23, 'leo', ['Sunny'], [250, 244, 10], 'Murmansk', [68.970682, 33.074981], 3.14159265358979, 100000000000, 800, -3.2, 154400000);
INSERT INTO test.table VALUES (toUUID('a7522158-3d41-4b77-ad69-6c598ee55c49'), 'Ivan', 'Petrov', 'male', toDate('1980-12-29'), 'png', '+74951234567', 1, toDateTime('2019-01-05 18:45:00'), 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 test.table 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 test.table VALUES (toUUID('a7da1aa6-f425-4789-8947-b034786ed374'), 'Vasily', 'Sidorov', 'male', toDate('1995-07-28'), 'bmp', '+442012345678', 1, toDateTime('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, []);
SELECT * FROM test.table ORDER BY name FORMAT Protobuf SETTINGS format_schema = '$CURDIR/00825_protobuf_format:Person';
SELECT 'ALTERNATIVE->';

View File

@ -41,7 +41,23 @@ message Syntax2Person {
optional float someRatio = 18;
optional float temperature = 19;
optional sint64 randomBigNumber = 20;
optional string newFieldStr = 21 [default='abc'];
optional int32 newFieldInt = 22 [default=-11];
optional bool newBool = 23 [default=true];
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

@ -1,51 +0,0 @@
syntax = "proto2";
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 string newFieldStr = 22 [default='abc'];
optional int32 newFieldInt = 23 [default=-11];
optional bool newBool = 24 [default=true];
};

View File

@ -1,118 +0,0 @@
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 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;
};
enum OnlineStatus {
offline = 0;
online = 1;
};
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;
};
message StrPerson {
message MeasureUnits
{
repeated string unit = 1;
repeated string coef = 2;
};
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;
};

View File

@ -0,0 +1,9 @@
[1.00]
[1.00000]
[1.0000000000]
[499500.00]
[499500.00000]
[499500.0000000000]
1545081300 [('ed87e57c-9331-462a-80b4-9f0c005e88c8',0.4400)]
4341757 5657967 2018-11-01 16:47:46 txt 321.380000000000 315.080000000000 0.000000000000 2018-11-02 00:00:00
4360430 5681495 2018-11-02 09:00:07 txt 274.350000000000 268.970000000000 0.000000000000 2018-11-02 00:00:00

View File

@ -0,0 +1,64 @@
USE test;
select groupArray(s) from (select sum(n) s from (select toDecimal32(1, 2) as n));
select groupArray(s) from (select sum(n) s from (select toDecimal64(1, 5) as n));
select groupArray(s) from (select sum(n) s from (select toDecimal128(1, 10) as n));
select groupArray(s) from (select sum(n) s from (select toDecimal32(number, 2) as n from numbers(1000)));
select groupArray(s) from (select sum(n) s from (select toDecimal64(number, 5) as n from numbers(1000)));
select groupArray(s) from (select sum(n) s from (select toDecimal128(number, 10) as n from numbers(1000)));
DROP TABLE IF EXISTS sensor_value;
CREATE TABLE sensor_value (
received_at DateTime,
device_id UUID,
sensor_id UUID,
value Nullable(Decimal(18, 4)),
low_warning Nullable(Decimal(18, 4)),
low_critical Nullable(Decimal(18, 4)),
high_warning Nullable(Decimal(18, 4)),
high_critical Nullable(Decimal(18, 4))
) ENGINE = MergeTree
PARTITION BY toDate(received_at)
ORDER BY (device_id, sensor_id);
INSERT INTO sensor_value (received_at, device_id, sensor_id, value, low_warning, low_critical, high_warning, high_critical) VALUES ('2018-12-18 00:16:07', 'a4d92414-09aa-4dbd-80b2-124ddaacf333', 'ed87e57c-9331-462a-80b4-9f0c005e88c8', '0.4400', '-10000000.0000', '-10000000.0000', '10000000.0000', '10000000.0000');
SELECT `time`, groupArray((sensor_id, volume)) AS groupArr FROM (
SELECT (intDiv(toUInt32(received_at), 900) * 900) AS `time`, sensor_id, avg(value) AS volume
FROM sensor_value
WHERE received_at BETWEEN '2018-12-12 00:00:00' AND '2018-12-30 00:00:00'
GROUP BY `time`,sensor_id
ORDER BY `time`
) GROUP BY `time`;
DROP TABLE sensor_value;
select s.a, s.b, max(s.dt1) dt1, s.c, s.d, s.f, s.i, max(s.dt2) dt2 from (
select toUInt64(4360430) a
, toUInt64(5681495) b
, toDateTime('2018-11-01 10:44:58') dt1
, 'txt' c
, toDecimal128('274.350000000000', 12) d
, toDecimal128(268.970000000000, 12) f
, toDecimal128(0.000000000000, 12) i
, toDateTime('2018-11-02 00:00:00') dt2
union all
select toUInt64(4341757) a
, toUInt64(5657967) b
, toDateTime('2018-11-01 16:47:46') dt1
, 'txt' c
, toDecimal128('321.380000000000', 12) d
, toDecimal128(315.080000000000, 12) f
, toDecimal128(0.000000000000, 12) i
, toDateTime('2018-11-02 00:00:00') dt2
union all
select toUInt64(4360430) a
, toUInt64(5681495) b
, toDateTime('2018-11-02 09:00:07') dt1
, 'txt' c
, toDecimal128('274.350000000000', 12) d
, toDecimal128(268.970000000000, 12) f
, toDecimal128(0.000000000000, 12) i
, toDateTime('2018-11-02 00:00:00') dt2
) s group by s.a, s.b, s.c, s.d, s.f, s.i;

View File

@ -23,11 +23,12 @@ The table below lists supported formats and how they can be used in `INSERT` and
| [PrettyCompactMonoBlock](#prettycompactmonoblock) | ✗ | ✔ |
| [PrettyNoEscapes](#prettynoescapes) | ✗ | ✔ |
| [PrettySpace](#prettyspace) | ✗ | ✔ |
| [Protobuf](#protobuf) | ✔ | ✔ |
| [RowBinary](#rowbinary) | ✔ | ✔ |
| [Native](#native) | ✔ | ✔ |
| [Null](#null) | ✗ | ✔ |
| [XML](#xml) | ✗ | ✔ |
| [CapnProto](#capnproto) | ✔ | |
| [CapnProto](#capnproto) | ✔ | |
## TabSeparated {#tabseparated}
@ -575,9 +576,8 @@ Cap'n Proto is a binary message format similar to Protocol Buffers and Thrift, b
Cap'n Proto messages are strictly typed and not self-describing, meaning they need an external schema description. The schema is applied on the fly and cached for each query.
``` sql
SELECT SearchPhrase, count() AS c FROM test.hits
GROUP BY SearchPhrase FORMAT CapnProto SETTINGS schema = 'schema:Message'
```bash
cat capnproto_messages.bin | clickhouse-client --query "INSERT INTO test.hits FORMAT CapnProto SETTINGS format_schema='schema:Message'"
```
Where `schema.capnp` looks like this:
@ -589,8 +589,82 @@ struct Message {
}
```
Schema files are in the file that is located in the directory specified in [ format_schema_path](../operations/server_settings/settings.md) in the server configuration.
Deserialization is effective and usually doesn't increase the system load.
See also [Format Schema](#formatschema).
## Protobuf {#protobuf}
Protobuf - is a [Protocol Buffers](https://developers.google.com/protocol-buffers/) format.
This format requires an external format schema. The schema is cached between queries.
Usage examples:
```sql
SELECT * FROM test.table FORMAT Protobuf SETTINGS format_schema = 'schemafile:MessageType'
```
```bash
cat protobuf_messages.bin | clickhouse-client --query "INSERT INTO test.table FORMAT Protobuf SETTINGS format_schema='schemafile:MessageType'"
```
Where the file `schemafile.proto` looks like this:
```
syntax = "proto3";
message MessageType {
string name = 1;
string surname = 2;
uint32 birthDate = 3;
repeated string phoneNumbers = 4;
};
```
To find the correspondence between table columns and fields of Protocol Buffers' message type ClickHouse compares their names.
This comparison is case-insensitive and the characters `_` (underscore) and `.` (dot) are considered as equal.
If types of a column and a field of Protocol Buffers' message are different the necessary conversion is applied.
Nested messages are supported. For example, for the field `z` in the following message type
```
message MessageType {
message XType {
message YType {
int32 z;
};
repeated YType y;
};
XType x;
};
```
ClickHouse tries to find a column named `x.y.z` (or `x_y_z` or `X.y_Z` and so on).
Nested messages are suitable to input or output a [nested data structures](../data_types/nested_data_structures/nested/).
Default values defined in the protobuf schema like this
```
message MessageType {
optional int32 result_per_page = 3 [default = 10];
}
```
are not applied; the [table defaults](../query_language/create.md#create-default-values) are used instead of them.
## Format Schema {#formatschema}
The file name containing the format schema is set by the setting `format_schema`.
It's required to set this setting when it is used one of the formats `Cap'n Proto` and `Protobuf`.
The format schema is a combination of a file name and the name of a message type in this file, delimited by colon,
e.g. `schemafile.proto:MessageType`.
If the file has the standard extension for the format (for example, `.proto` for `Protobuf`),
it can be omitted and in this case the format schema looks like `schemafile:MessageType`.
If you input or output data via the [client](../interfaces/cli/) the file name specified in the format schema
can contain an absolute path or a path relative to the current directory on the client.
If you input or output data via the [HTTP interface](../interfaces/http/) the file name specified in the format schema
should be located in the directory specified in [format_schema_path](../operations/server_settings/settings.md)
in the server configuration.
[Original article](https://clickhouse.yandex/docs/en/interfaces/formats/) <!--hide-->

View File

@ -123,7 +123,7 @@ The `Sign` is required because the merging algorithm doesn't guarantee that all
To finalize collapsing write a query with `GROUP BY` clause and aggregate functions that account for the sign. For example, to calculate quantity, use `sum(Sign)` instead of `count()`. To calculate the sum of something, use `sum(Sign * x)` instead of `sum(x)`, and so on, and also add `HAVING sum(Sign) > 0`.
The aggregates `count`, `sum` and `avg` could be calculated this way. The aggregate `uniq` could be calculated if an object has at list one state not collapsed. The aggregates `min` and `max` could not be calculated because `CollapsingMergeTree` does not save values history of the collapsed states.
The aggregates `count`, `sum` and `avg` could be calculated this way. The aggregate `uniq` could be calculated if an object has at least one state not collapsed. The aggregates `min` and `max` could not be calculated because `CollapsingMergeTree` does not save values history of the collapsed states.
If you need to extract data without aggregation (for example, to check whether rows are present whose newest values match certain conditions), you can use the `FINAL` modifier for the `FROM` clause. This approach is significantly less efficient.

View File

@ -45,7 +45,7 @@ In all cases, if `IF NOT EXISTS` is specified, the query won't return an error i
There can be other clauses after the `ENGINE` clause in the query. See detailed documentation on how to create tables in the descriptions of [table engines](../operations/table_engines/index.md#table_engines).
### Default Values
### Default Values {#create-default-values}
The column description can specify an expression for a default value, in one of the following ways:`DEFAULT expr`, `MATERIALIZED expr`, `ALIAS expr`.
Example: `URLDomain String DEFAULT domain(URL)`.

View File

@ -24,11 +24,12 @@ ClickHouse может принимать (`INSERT`) и отдавать (`SELECT
[PrettyCompactMonoBlock](#prettycompactmonoblock) | ✗ | ✔ |
[PrettyNoEscapes](#prettynoescapes) | ✗ | ✔ |
[PrettySpace](#prettyspace) | ✗ | ✔ |
[Protobuf](#protobuf) | ✔ | ✔ |
[RowBinary](#rowbinary) | ✔ | ✔ |
[Native](#native) | ✔ | ✔ |
[Null](#null) | ✗ | ✔ |
[XML](#xml) | ✗ | ✔ |
[CapnProto](#capnproto) | ✔ | |
[CapnProto](#capnproto) | ✔ | |
## TabSeparated {#tabseparated}
@ -565,11 +566,10 @@ test: string with \'quotes\' and \t with some special \n characters
Cap'n Proto - формат бинарных сообщений, похож на Protocol Buffers и Thrift, но не похож на JSON или MessagePack.
Сообщения Cap'n Proto строго типизированы и не самоописывающиеся, т.е. нуждаются во внешнем описании схемы. Схема применяется "на лету" и кешируется для каждого запроса.
Сообщения Cap'n Proto строго типизированы и не самоописывающиеся, т.е. нуждаются во внешнем описании схемы. Схема применяется "на лету" и кешируется между запросами.
``` sql
SELECT SearchPhrase, count() AS c FROM test.hits
GROUP BY SearchPhrase FORMAT CapnProto SETTINGS schema = 'schema:Message'
```bash
cat capnproto_messages.bin | clickhouse-client --query "INSERT INTO test.hits FORMAT CapnProto SETTINGS format_schema='schema:Message'"
```
Где `schema.capnp` выглядит следующим образом:
@ -581,9 +581,81 @@ struct Message {
}
```
Файлы со схемами находятся в файле, который находится в каталоге указанном в параметре [format_schema_path](../operations/server_settings/settings.md) конфигурации сервера.
Десериализация эффективна и обычно не повышает нагрузку на систему.
См. также [схема формата](#formatschema).
## Protobuf {#protobuf}
Protobuf - формат [Protocol Buffers](https://developers.google.com/protocol-buffers/).
Формат нуждается во внешнем описании схемы. Схема кэшируется между запросами.
Пример использования формата:
```sql
SELECT * FROM test.table FORMAT Protobuf SETTINGS format_schema = 'schemafile:MessageType'
```
или
```bash
cat protobuf_messages.bin | clickhouse-client --query "INSERT INTO test.table FORMAT Protobuf SETTINGS format_schema='schemafile:MessageType'"
```
Где файл `schemafile.proto` может выглядеть так:
```
syntax = "proto3";
message MessageType {
string name = 1;
string surname = 2;
uint32 birthDate = 3;
repeated string phoneNumbers = 4;
};
```
Соответствие между столбцами таблицы и полями сообщения Protocol Buffers устанавливается по имени,
при этом игнорируется регистр букв и символы `_` (подчеркивание) и `.` (точка) считаются одинаковыми.
Если типы столбцов не соответствуют точно типам полей сообщения Protocol Buffers, производится необходимая конвертация.
Вложенные сообщения поддерживаются, например, для поля `z` в таком сообщении
```
message MessageType {
message XType {
message YType {
int32 z;
};
repeated YType y;
};
XType x;
};
```
ClickHouse попытается найти столбец с именем `x.y.z` (или `x_y_z`, или `X.y_Z` и т.п.).
Вложенные сообщения удобно использовать в качестве соответствия для [вложенной структуры данных](../data_types/nested_data_structures/nested/).
Значения по умолчанию, определенные в схеме, например,
```
message MessageType {
optional int32 result_per_page = 3 [default = 10];
}
```
не применяются; вместо них используются определенные в таблице [значения по умолчанию](../query_language/create.md#create-default-values).
## Схема формата {#formatschema}
Имя файла со схемой записывается в настройке `format_schema`. При использовании форматов `Cap'n Proto` и `Protobuf` требуется указать схему.
Схема представляет собой имя файла и имя типа в этом файле, разделенные двоеточием, например `schemafile.proto:MessageType`.
Если файл имеет стандартное расширение для данного формата (например `.proto` для `Protobuf`),
то можно его не указывать и записывать схему так `schemafile:MessageType`.
Если для ввода/вывода данных используется [клиент](../interfaces/cli/), то при записи схемы можно использовать абсолютный путь или записывать путь
относительно текущей директории на клиенте.
Если для ввода/вывода данных используется [HTTP-интерфейс](../interfaces/http/), то файл со схемой должен располагаться на сервере в каталоге,
указанном в параметре [format_schema_path](../data_types/nested_data_structures/nested/) конфигурации сервера.
[Оригинальная статья](https://clickhouse.yandex/docs/ru/interfaces/formats/) <!--hide-->

View File

@ -46,7 +46,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name ENGINE = engine AS SELECT ...
После секции `ENGINE` в запросе могут использоваться и другие секции в зависимости от движка. Подробную документацию по созданию таблиц смотрите в описаниях [движков](../operations/table_engines/index.md#table_engines).
### Значения по умолчанию
### Значения по умолчанию {#create-default-values}
В описании столбца, может быть указано выражение для значения по умолчанию, одного из следующих видов:
`DEFAULT expr`, `MATERIALIZED expr`, `ALIAS expr`.

View File

@ -1,13 +1,13 @@
#if __has_include(<cctz/civil_time.h>)
#include <cctz/civil_time.h> // bundled, debian
#else
#include <civil_time.h> // freebsd
#include <civil_time.h> // Y_IGNORE // freebsd
#endif
#if __has_include(<cctz/time_zone.h>)
#include <cctz/time_zone.h>
#else
#include <time_zone.h>
#include <time_zone.h> // Y_IGNORE
#endif
#include <common/DateLUTImpl.h>

View File

@ -21,7 +21,7 @@ add_check(local_date_time_comparison)
if(USE_GTEST)
add_executable(unit_tests_libcommon gtest_json_test.cpp gtest_strong_typedef.cpp gtest_find_symbols.cpp)
target_link_libraries(unit_tests_libcommon common ${GTEST_MAIN_LIBRARIES})
target_link_libraries(unit_tests_libcommon common ${GTEST_MAIN_LIBRARIES} ${GTEST_LIBRARIES})
add_check(unit_tests_libcommon)
endif()

View File

@ -5,8 +5,8 @@ add_executable (markov-model markov-model.cpp)
target_link_libraries(markov-model PRIVATE clickhouse_common_io ${Boost_PROGRAM_OPTIONS_LIBRARY})
if(USE_PROTOBUF)
protobuf_generate_cpp(ProtobufDelimitedMessagesSerializer_Srcs ProtobufDelimitedMessagesSerializer_Hdrs ${CMAKE_CURRENT_SOURCE_DIR}/../../dbms/tests/queries/0_stateless/00825_protobuf_format_with_nested.proto)
protobuf_generate_cpp(ProtobufDelimitedMessagesSerializer_Srcs2 ProtobufDelimitedMessagesSerializer_Hdrs2 ${CMAKE_CURRENT_SOURCE_DIR}/../../dbms/tests/queries/0_stateless/00825_protobuf_format_syntax2_with_nested.proto)
protobuf_generate_cpp(ProtobufDelimitedMessagesSerializer_Srcs ProtobufDelimitedMessagesSerializer_Hdrs ${CMAKE_CURRENT_SOURCE_DIR}/../../dbms/tests/queries/0_stateless/00825_protobuf_format.proto)
protobuf_generate_cpp(ProtobufDelimitedMessagesSerializer_Srcs2 ProtobufDelimitedMessagesSerializer_Hdrs2 ${CMAKE_CURRENT_SOURCE_DIR}/../../dbms/tests/queries/0_stateless/00825_protobuf_format_syntax2.proto)
add_executable (ProtobufDelimitedMessagesSerializer ProtobufDelimitedMessagesSerializer.cpp ${ProtobufDelimitedMessagesSerializer_Srcs} ${ProtobufDelimitedMessagesSerializer_Hdrs} ${ProtobufDelimitedMessagesSerializer_Srcs2} ${ProtobufDelimitedMessagesSerializer_Hdrs2})
target_include_directories (ProtobufDelimitedMessagesSerializer SYSTEM BEFORE PRIVATE ${Protobuf_INCLUDE_DIR} ${CMAKE_CURRENT_BINARY_DIR})
target_link_libraries (ProtobufDelimitedMessagesSerializer PRIVATE ${Protobuf_LIBRARY} ${Boost_PROGRAM_OPTIONS_LIBRARY})

View File

@ -5,8 +5,8 @@
#include <fstream>
#include <iostream>
#include <google/protobuf/util/delimited_message_util.h>
#include "00825_protobuf_format_with_nested.pb.h"
#include "00825_protobuf_format_syntax2_with_nested.pb.h"
#include "00825_protobuf_format.pb.h"
#include "00825_protobuf_format_syntax2.pb.h"
void writeInsertQueryCommand(std::ostream & out, const std::string & format_schema, std::stringstream & delimited_messages)
@ -63,6 +63,9 @@ void writeInputInsertQueries(std::ostream & out)
mu = person.add_measureunits();
mu->set_unit("kilometer");
mu->set_coef(1000);
person.mutable_nestiness()->mutable_a()->mutable_b()->mutable_c()->set_d(500);
person.mutable_nestiness()->mutable_a()->mutable_b()->mutable_c()->add_e(501);
person.mutable_nestiness()->mutable_a()->mutable_b()->mutable_c()->add_e(502);
google::protobuf::util::SerializeDelimitedToOstream(person, &ss);
}
@ -118,10 +121,11 @@ void writeInputInsertQueries(std::ostream & out)
auto* mu = person.add_measureunits();
mu->set_unit("pound");
mu->set_coef(16);
person.mutable_nestiness()->mutable_a()->mutable_b()->mutable_c()->set_d(503);
google::protobuf::util::SerializeDelimitedToOstream(person, &ss);
}
writeInsertQueryCommand(out, "00825_protobuf_format_with_nested:Person", ss);
writeInsertQueryCommand(out, "00825_protobuf_format:Person", ss);
{
AltPerson person;
@ -149,6 +153,8 @@ void writeInputInsertQueries(std::ostream & out)
person.add_measureunits_coef(1024);
person.add_measureunits_unit("MB");
person.add_measureunits_coef(1048576);
person.set_nestiness_a_b_c_d(700);
person.add_nestiness_a_b_c_e(701);
google::protobuf::util::SerializeDelimitedToOstream(person, &ss);
}
@ -179,10 +185,11 @@ void writeInputInsertQueries(std::ostream & out)
person.add_measureunits_unit("Bit");
person.add_measureunits_coef(1);
person.mutable_newmessage()->set_z(91);
person.set_nestiness_a_b_c_d(702);
google::protobuf::util::SerializeDelimitedToOstream(person, &ss);
}
writeInsertQueryCommand(out, "00825_protobuf_format_with_nested:AltPerson", ss);
writeInsertQueryCommand(out, "00825_protobuf_format:AltPerson", ss);
{
StrPerson person;
@ -214,10 +221,11 @@ void writeInputInsertQueries(std::ostream & out)
person.mutable_measureunits()->add_coef("60");
person.mutable_measureunits()->add_unit("hour");
person.mutable_measureunits()->add_coef("3600");
person.mutable_nestiness_a()->mutable_b_c()->add_e("1800");
google::protobuf::util::SerializeDelimitedToOstream(person, &ss);
}
writeInsertQueryCommand(out, "00825_protobuf_format_with_nested:StrPerson", ss);
writeInsertQueryCommand(out, "00825_protobuf_format:StrPerson", ss);
{
Syntax2Person person;
@ -250,10 +258,11 @@ void writeInputInsertQueries(std::ostream & out)
person.mutable_measureunits()->add_coef(0.2);
person.mutable_measureunits()->add_unit("gram");
person.mutable_measureunits()->add_coef(1);
person.mutable_nestiness()->mutable_a()->mutable_b()->mutable_c()->set_d(9494);
google::protobuf::util::SerializeDelimitedToOstream(person, &ss);
}
writeInsertQueryCommand(out, "00825_protobuf_format_syntax2_with_nested:Syntax2Person", ss);
writeInsertQueryCommand(out, "00825_protobuf_format_syntax2:Syntax2Person", ss);
}
@ -285,6 +294,18 @@ void writeOutputReference(std::ostream & out)
person.set_someratio(0.1);
person.set_temperature(5.8);
person.set_randombignumber(17060000000);
auto* mu = person.add_measureunits();
mu->set_unit("meter");
mu->set_coef(1);
mu = person.add_measureunits();
mu->set_unit("centimeter");
mu->set_coef(0.01);
mu = person.add_measureunits();
mu->set_unit("kilometer");
mu->set_coef(1000);
person.mutable_nestiness()->mutable_a()->mutable_b()->mutable_c()->set_d(500);
person.mutable_nestiness()->mutable_a()->mutable_b()->mutable_c()->add_e(501);
person.mutable_nestiness()->mutable_a()->mutable_b()->mutable_c()->add_e(502);
google::protobuf::util::SerializeDelimitedToOstream(person, &out);
}
@ -337,6 +358,10 @@ void writeOutputReference(std::ostream & out)
person.set_someratio(800);
person.set_temperature(-3.2);
person.set_randombignumber(154400000);
auto* mu = person.add_measureunits();
mu->set_unit("pound");
mu->set_coef(16);
person.mutable_nestiness()->mutable_a()->mutable_b()->mutable_c()->set_d(503);
google::protobuf::util::SerializeDelimitedToOstream(person, &out);
}
@ -364,6 +389,15 @@ void writeOutputReference(std::ostream & out)
person.set_surname("Petrov");
person.set_phonenumber(+74951234567);
person.set_temperature(5);
person.add_measureunits_unit("meter");
person.add_measureunits_coef(1);
person.add_measureunits_unit("centimeter");
person.add_measureunits_coef(0.01);
person.add_measureunits_unit("kilometer");
person.add_measureunits_coef(1000);
person.set_nestiness_a_b_c_d(500);
person.add_nestiness_a_b_c_e(501);
person.add_nestiness_a_b_c_e(502);
google::protobuf::util::SerializeDelimitedToOstream(person, &out);
}
@ -411,6 +445,9 @@ void writeOutputReference(std::ostream & out)
person.set_surname("Sidorov");
person.set_phonenumber(+442012345678);
person.set_temperature(-3);
person.add_measureunits_unit("pound");
person.add_measureunits_coef(16);
person.set_nestiness_a_b_c_d(503);
google::protobuf::util::SerializeDelimitedToOstream(person, &out);
}
@ -441,6 +478,15 @@ void writeOutputReference(std::ostream & out)
person.set_someratio("0.1");
person.set_temperature("5.8");
person.set_randombignumber("17060000000");
person.mutable_measureunits()->add_unit("meter");
person.mutable_measureunits()->add_coef("1");
person.mutable_measureunits()->add_unit("centimeter");
person.mutable_measureunits()->add_coef("0.01");
person.mutable_measureunits()->add_unit("kilometer");
person.mutable_measureunits()->add_coef("1000");
person.mutable_nestiness_a()->mutable_b_c()->set_d("500");
person.mutable_nestiness_a()->mutable_b_c()->add_e("501");
person.mutable_nestiness_a()->mutable_b_c()->add_e("502");
google::protobuf::util::SerializeDelimitedToOstream(person, &out);
}
@ -491,6 +537,9 @@ void writeOutputReference(std::ostream & out)
person.set_someratio("800");
person.set_temperature("-3.2");
person.set_randombignumber("154400000");
person.mutable_measureunits()->add_unit("pound");
person.mutable_measureunits()->add_coef("16");
person.mutable_nestiness_a()->mutable_b_c()->set_d("503");
google::protobuf::util::SerializeDelimitedToOstream(person, &out);
}
@ -522,6 +571,15 @@ void writeOutputReference(std::ostream & out)
person.set_someratio(0.1);
person.set_temperature(5.8);
person.set_randombignumber(17060000000);
person.mutable_measureunits()->add_unit("meter");
person.mutable_measureunits()->add_coef(1);
person.mutable_measureunits()->add_unit("centimeter");
person.mutable_measureunits()->add_coef(0.01);
person.mutable_measureunits()->add_unit("kilometer");
person.mutable_measureunits()->add_coef(1000);
person.mutable_nestiness()->mutable_a()->mutable_b()->mutable_c()->set_d(500);
person.mutable_nestiness()->mutable_a()->mutable_b()->mutable_c()->add_e(501);
person.mutable_nestiness()->mutable_a()->mutable_b()->mutable_c()->add_e(502);
google::protobuf::util::SerializeDelimitedToOstream(person, &out);
}
@ -574,6 +632,9 @@ void writeOutputReference(std::ostream & out)
person.set_someratio(800);
person.set_temperature(-3.2);
person.set_randombignumber(154400000);
person.mutable_measureunits()->add_unit("pound");
person.mutable_measureunits()->add_coef(16);
person.mutable_nestiness()->mutable_a()->mutable_b()->mutable_c()->set_d(503);
google::protobuf::util::SerializeDelimitedToOstream(person, &out);
}
}