Merge pull request #48257 from Avogar/capnp-more-types

Support more types in CapnProto format, allow integer conversions
This commit is contained in:
Kruglov Pavel 2023-04-05 11:56:46 +02:00 committed by GitHub
commit 8d6b709f1a
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
10 changed files with 278 additions and 101 deletions

View File

@ -1610,29 +1610,34 @@ See also [Format Schema](#formatschema).
The table below shows supported data types and how they match ClickHouse [data types](/docs/en/sql-reference/data-types/index.md) in `INSERT` and `SELECT` queries.
| CapnProto data type (`INSERT`) | ClickHouse data type | CapnProto data type (`SELECT`) |
|----------------------------------|------------------------------------------------------------------------------------------------------------------------|------------------------------|
| `UINT8`, `BOOL` | [UInt8](/docs/en/sql-reference/data-types/int-uint.md) | `UINT8` |
| `INT8` | [Int8](/docs/en/sql-reference/data-types/int-uint.md) | `INT8` |
| `UINT16` | [UInt16](/docs/en/sql-reference/data-types/int-uint.md), [Date](/docs/en/sql-reference/data-types/date.md) | `UINT16` |
| `INT16` | [Int16](/docs/en/sql-reference/data-types/int-uint.md) | `INT16` |
| `UINT32` | [UInt32](/docs/en/sql-reference/data-types/int-uint.md), [DateTime](/docs/en/sql-reference/data-types/datetime.md) | `UINT32` |
| `INT32` | [Int32](/docs/en/sql-reference/data-types/int-uint.md) | `INT32` |
| `UINT64` | [UInt64](/docs/en/sql-reference/data-types/int-uint.md) | `UINT64` |
| `INT64` | [Int64](/docs/en/sql-reference/data-types/int-uint.md), [DateTime64](/docs/en/sql-reference/data-types/datetime.md) | `INT64` |
| `FLOAT32` | [Float32](/docs/en/sql-reference/data-types/float.md) | `FLOAT32` |
| `FLOAT64` | [Float64](/docs/en/sql-reference/data-types/float.md) | `FLOAT64` |
| `TEXT, DATA` | [String](/docs/en/sql-reference/data-types/string.md), [FixedString](/docs/en/sql-reference/data-types/fixedstring.md) | `TEXT, DATA` |
| `union(T, Void), union(Void, T)` | [Nullable(T)](/docs/en/sql-reference/data-types/date.md) | `union(T, Void), union(Void, T)` |
| `ENUM` | [Enum(8\ |16)](/docs/en/sql-reference/data-types/enum.md) | `ENUM` |
| `LIST` | [Array](/docs/en/sql-reference/data-types/array.md) | `LIST` |
| `STRUCT` | [Tuple](/docs/en/sql-reference/data-types/tuple.md) | `STRUCT` |
| `UINT32` | [IPv4](/docs/en/sql-reference/data-types/domains/ipv4.md) | `UINT32` |
| `DATA` | [IPv6](/docs/en/sql-reference/data-types/domains/ipv6.md) | `DATA` |
| CapnProto data type (`INSERT`) | ClickHouse data type | CapnProto data type (`SELECT`) |
|------------------------------------------------------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|------------------------------------------------------|
| `UINT8`, `BOOL` | [UInt8](/docs/en/sql-reference/data-types/int-uint.md) | `UINT8` |
| `INT8` | [Int8](/docs/en/sql-reference/data-types/int-uint.md) | `INT8` |
| `UINT16` | [UInt16](/docs/en/sql-reference/data-types/int-uint.md), [Date](/docs/en/sql-reference/data-types/date.md) | `UINT16` |
| `INT16` | [Int16](/docs/en/sql-reference/data-types/int-uint.md) | `INT16` |
| `UINT32` | [UInt32](/docs/en/sql-reference/data-types/int-uint.md), [DateTime](/docs/en/sql-reference/data-types/datetime.md) | `UINT32` |
| `INT32` | [Int32](/docs/en/sql-reference/data-types/int-uint.md), [Decimal32](/docs/en/sql-reference/data-types/decimal.md) | `INT32` |
| `UINT64` | [UInt64](/docs/en/sql-reference/data-types/int-uint.md) | `UINT64` |
| `INT64` | [Int64](/docs/en/sql-reference/data-types/int-uint.md), [DateTime64](/docs/en/sql-reference/data-types/datetime.md), [Decimal64](/docs/en/sql-reference/data-types/decimal.md) | `INT64` |
| `FLOAT32` | [Float32](/docs/en/sql-reference/data-types/float.md) | `FLOAT32` |
| `FLOAT64` | [Float64](/docs/en/sql-reference/data-types/float.md) | `FLOAT64` |
| `TEXT, DATA` | [String](/docs/en/sql-reference/data-types/string.md), [FixedString](/docs/en/sql-reference/data-types/fixedstring.md) | `TEXT, DATA` |
| `union(T, Void), union(Void, T)` | [Nullable(T)](/docs/en/sql-reference/data-types/date.md) | `union(T, Void), union(Void, T)` |
| `ENUM` | [Enum(8/16)](/docs/en/sql-reference/data-types/enum.md) | `ENUM` |
| `LIST` | [Array](/docs/en/sql-reference/data-types/array.md) | `LIST` |
| `STRUCT` | [Tuple](/docs/en/sql-reference/data-types/tuple.md) | `STRUCT` |
| `UINT32` | [IPv4](/docs/en/sql-reference/data-types/domains/ipv4.md) | `UINT32` |
| `DATA` | [IPv6](/docs/en/sql-reference/data-types/domains/ipv6.md) | `DATA` |
| `DATA` | [Int128/UInt128/Int256/UInt256](/docs/en/sql-reference/data-types/int-uint.md) | `DATA` |
| `DATA` | [Decimal128/Decimal256](/docs/en/sql-reference/data-types/decimal.md) | `DATA` |
| `STRUCT(entries LIST(STRUCT(key Key, value Value)))` | [Map](/docs/en/sql-reference/data-types/map.md) | `STRUCT(entries LIST(STRUCT(key Key, value Value)))` |
Integer types can be converted into each other during input/output.
For working with `Enum` in CapnProto format use the [format_capn_proto_enum_comparising_mode](/docs/en/operations/settings/settings-formats.md/#format_capn_proto_enum_comparising_mode) setting.
Arrays can be nested and can have a value of the `Nullable` type as an argument. `Tuple` type also can be nested.
Arrays can be nested and can have a value of the `Nullable` type as an argument. `Tuple` and `Map` types also can be nested.
### Inserting and Selecting Data {#inserting-and-selecting-data-capnproto}

View File

@ -9,6 +9,7 @@
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypeMap.h>
#include <DataTypes/IDataType.h>
#include <boost/algorithm/string.hpp>
#include <boost/algorithm/string/join.hpp>
@ -264,23 +265,25 @@ static bool checkTupleType(const capnp::Type & capnp_type, const DataTypePtr & d
return false;
}
if (!tuple_data_type->haveExplicitNames())
bool have_explicit_names = tuple_data_type->haveExplicitNames();
const auto & nested_names = tuple_data_type->getElementNames();
for (uint32_t i = 0; i != nested_names.size(); ++i)
{
error_message += "Only named Tuple can be converted to CapnProto Struct";
return false;
}
for (const auto & name : tuple_data_type->getElementNames())
{
KJ_IF_MAYBE(field, struct_schema.findFieldByName(name))
if (have_explicit_names)
{
if (!checkCapnProtoType(field->getType(), nested_types[tuple_data_type->getPositionByName(name)], mode, error_message, name))
KJ_IF_MAYBE (field, struct_schema.findFieldByName(nested_names[i]))
{
if (!checkCapnProtoType(field->getType(), nested_types[tuple_data_type->getPositionByName(nested_names[i])], mode, error_message, nested_names[i]))
return false;
}
else
{
error_message += "CapnProto struct doesn't contain a field with name " + nested_names[i];
return false;
}
}
else
{
error_message += "CapnProto struct doesn't contain a field with name " + name;
else if (!checkCapnProtoType(struct_schema.getFields()[i].getType(), nested_types[tuple_data_type->getPositionByName(nested_names[i])], mode, error_message, nested_names[i]))
return false;
}
}
return true;
@ -307,41 +310,129 @@ static bool checkArrayType(const capnp::Type & capnp_type, const DataTypePtr & d
return checkCapnProtoType(list_schema.getElementType(), nested_type, mode, error_message, column_name);
}
static bool checkMapType(const capnp::Type & capnp_type, const DataTypePtr & data_type, FormatSettings::EnumComparingMode mode, String & error_message)
{
/// We output/input Map type as follow CapnProto schema
///
/// struct Map {
/// struct Entry {
/// key @0: Key;
/// value @1: Value;
/// }
/// entries @0 :List(Entry);
/// }
if (!capnp_type.isStruct())
return false;
auto struct_schema = capnp_type.asStruct();
if (checkIfStructContainsUnnamedUnion(struct_schema))
{
error_message += "CapnProto struct contains unnamed union";
return false;
}
if (struct_schema.getFields().size() != 1)
{
error_message += "CapnProto struct that represents Map type can contain only one field";
return false;
}
const auto & field_type = struct_schema.getFields()[0].getType();
if (!field_type.isList())
{
error_message += "Field of CapnProto struct that represents Map is not a list";
return false;
}
auto list_element_type = field_type.asList().getElementType();
if (!list_element_type.isStruct())
{
error_message += "Field of CapnProto struct that represents Map is not a list of structs";
return false;
}
auto key_value_struct = list_element_type.asStruct();
if (checkIfStructContainsUnnamedUnion(key_value_struct))
{
error_message += "CapnProto struct contains unnamed union";
return false;
}
if (key_value_struct.getFields().size() != 2)
{
error_message += "Key-value structure for Map struct should have exactly 2 fields";
return false;
}
const auto & map_type = assert_cast<const DataTypeMap &>(*data_type);
DataTypes types = {map_type.getKeyType(), map_type.getValueType()};
Names names = {"key", "value"};
for (size_t i = 0; i != types.size(); ++i)
{
KJ_IF_MAYBE(field, key_value_struct.findFieldByName(names[i]))
{
if (!checkCapnProtoType(field->getType(), types[i], mode, error_message, names[i]))
return false;
}
else
{
error_message += R"(Key-value structure for Map struct should have exactly 2 fields with names "key" and "value")";
return false;
}
}
return true;
}
static bool isCapnInteger(const capnp::Type & capnp_type)
{
return capnp_type.isInt8() || capnp_type.isUInt8() || capnp_type.isInt16() || capnp_type.isUInt16() || capnp_type.isInt32()
|| capnp_type.isUInt32() || capnp_type.isInt64() || capnp_type.isUInt64();
}
static bool checkCapnProtoType(const capnp::Type & capnp_type, const DataTypePtr & data_type, FormatSettings::EnumComparingMode mode, String & error_message, const String & column_name)
{
switch (data_type->getTypeId())
{
case TypeIndex::UInt8:
return capnp_type.isBool() || capnp_type.isUInt8();
case TypeIndex::Date: [[fallthrough]];
case TypeIndex::UInt16:
return capnp_type.isBool() || isCapnInteger(capnp_type);
case TypeIndex::Int8: [[fallthrough]];
case TypeIndex::Int16: [[fallthrough]];
case TypeIndex::UInt16: [[fallthrough]];
case TypeIndex::Int32: [[fallthrough]];
case TypeIndex::UInt32: [[fallthrough]];
case TypeIndex::Int64: [[fallthrough]];
case TypeIndex::UInt64:
/// Allow integer conversions durin input/output.
return isCapnInteger(capnp_type);
case TypeIndex::Date:
return capnp_type.isUInt16();
case TypeIndex::DateTime: [[fallthrough]];
case TypeIndex::IPv4: [[fallthrough]];
case TypeIndex::UInt32:
case TypeIndex::IPv4:
return capnp_type.isUInt32();
case TypeIndex::UInt64:
return capnp_type.isUInt64();
case TypeIndex::Int8:
return capnp_type.isInt8();
case TypeIndex::Int16:
return capnp_type.isInt16();
case TypeIndex::Date32: [[fallthrough]];
case TypeIndex::Decimal32: [[fallthrough]];
case TypeIndex::Int32:
return capnp_type.isInt32();
case TypeIndex::Decimal32:
return capnp_type.isInt32() || capnp_type.isUInt32();
case TypeIndex::DateTime64: [[fallthrough]];
case TypeIndex::Decimal64: [[fallthrough]];
case TypeIndex::Int64:
return capnp_type.isInt64();
case TypeIndex::Float32:
return capnp_type.isFloat32();
case TypeIndex::Decimal64:
return capnp_type.isInt64() || capnp_type.isUInt64();
case TypeIndex::Float32:[[fallthrough]];
case TypeIndex::Float64:
return capnp_type.isFloat64();
/// Allow converting between Float32 and isFloat64
return capnp_type.isFloat32() || capnp_type.isFloat64();
case TypeIndex::Enum8:
return checkEnums<Int8>(capnp_type, data_type, mode, INT8_MAX, error_message);
case TypeIndex::Enum16:
return checkEnums<Int16>(capnp_type, data_type, mode, INT16_MAX, error_message);
case TypeIndex::Int128: [[fallthrough]];
case TypeIndex::UInt128: [[fallthrough]];
case TypeIndex::Int256: [[fallthrough]];
case TypeIndex::UInt256: [[fallthrough]];
case TypeIndex::Decimal128: [[fallthrough]];
case TypeIndex::Decimal256:
return capnp_type.isData();
case TypeIndex::Tuple:
return checkTupleType(capnp_type, data_type, mode, error_message);
case TypeIndex::Nullable:
@ -359,6 +450,8 @@ static bool checkCapnProtoType(const capnp::Type & capnp_type, const DataTypePtr
case TypeIndex::IPv6: [[fallthrough]];
case TypeIndex::String:
return capnp_type.isText() || capnp_type.isData();
case TypeIndex::Map:
return checkMapType(capnp_type, data_type, mode, error_message);
default:
return false;
}

View File

@ -16,6 +16,7 @@
#include <Columns/ColumnLowCardinality.h>
#include <Columns/ColumnNullable.h>
#include <Columns/ColumnDecimal.h>
#include <Columns/ColumnMap.h>
#include <DataTypes/DataTypeEnum.h>
#include <DataTypes/DataTypeArray.h>
@ -23,6 +24,7 @@
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypeLowCardinality.h>
#include <DataTypes/DataTypeMap.h>
namespace DB
{
@ -80,22 +82,39 @@ kj::Array<capnp::word> CapnProtoRowInputFormat::readMessage()
return msg;
}
static void insertSignedInteger(IColumn & column, const DataTypePtr & column_type, Int64 value)
static void insertInteger(IColumn & column, const DataTypePtr & column_type, UInt64 value)
{
switch (column_type->getTypeId())
{
case TypeIndex::Int8:
assert_cast<ColumnInt8 &>(column).insertValue(value);
break;
case TypeIndex::UInt8:
assert_cast<ColumnUInt8 &>(column).insertValue(value);
break;
case TypeIndex::Int16:
assert_cast<ColumnInt16 &>(column).insertValue(value);
break;
case TypeIndex::Date: [[fallthrough]];
case TypeIndex::UInt16:
assert_cast<ColumnUInt16 &>(column).insertValue(value);
break;
case TypeIndex::Int32:
assert_cast<ColumnInt32 &>(column).insertValue(static_cast<Int32>(value));
break;
case TypeIndex::DateTime: [[fallthrough]];
case TypeIndex::UInt32:
assert_cast<ColumnUInt32 &>(column).insertValue(static_cast<UInt32>(value));
break;
case TypeIndex::IPv4:
assert_cast<ColumnIPv4 &>(column).insertValue(IPv4(static_cast<UInt32>(value)));
break;
case TypeIndex::Int64:
assert_cast<ColumnInt64 &>(column).insertValue(value);
break;
case TypeIndex::UInt64:
assert_cast<ColumnUInt64 &>(column).insertValue(value);
break;
case TypeIndex::DateTime64:
assert_cast<ColumnDecimal<DateTime64> &>(column).insertValue(value);
break;
@ -106,33 +125,7 @@ static void insertSignedInteger(IColumn & column, const DataTypePtr & column_typ
assert_cast<ColumnDecimal<Decimal64> &>(column).insertValue(value);
break;
default:
throw Exception(ErrorCodes::LOGICAL_ERROR, "Column type is not a signed integer.");
}
}
static void insertUnsignedInteger(IColumn & column, const DataTypePtr & column_type, UInt64 value)
{
switch (column_type->getTypeId())
{
case TypeIndex::UInt8:
assert_cast<ColumnUInt8 &>(column).insertValue(value);
break;
case TypeIndex::Date: [[fallthrough]];
case TypeIndex::UInt16:
assert_cast<ColumnUInt16 &>(column).insertValue(value);
break;
case TypeIndex::DateTime: [[fallthrough]];
case TypeIndex::UInt32:
assert_cast<ColumnUInt32 &>(column).insertValue(static_cast<UInt32>(value));
break;
case TypeIndex::UInt64:
assert_cast<ColumnUInt64 &>(column).insertValue(value);
break;
case TypeIndex::IPv4:
assert_cast<ColumnIPv4 &>(column).insertValue(IPv4(static_cast<UInt32>(value)));
break;
default:
throw Exception(ErrorCodes::LOGICAL_ERROR, "Column type is not an unsigned integer.");
throw Exception(ErrorCodes::LOGICAL_ERROR, "Column type {} cannot be parsed from integer", column_type->getName());
}
}
@ -152,8 +145,11 @@ static void insertFloat(IColumn & column, const DataTypePtr & column_type, Float
}
template <typename Value>
static void insertString(IColumn & column, Value value)
static void insertData(IColumn & column, const DataTypePtr & column_type, Value value)
{
if (column_type->haveMaximumSizeOfValue() && value.size() != column_type->getSizeOfValueInMemory())
throw Exception(ErrorCodes::INCORRECT_DATA, "Unexpected size of {} value: {}", column_type->getName(), value.size());
column.insertData(reinterpret_cast<const char *>(value.begin()), value.size());
}
@ -166,10 +162,10 @@ static void insertEnum(IColumn & column, const DataTypePtr & column_type, const
switch (enum_comparing_mode)
{
case FormatSettings::EnumComparingMode::BY_VALUES:
insertSignedInteger(column, nested_type, Int64(enumerant.getOrdinal()));
insertInteger(column, nested_type, Int64(enumerant.getOrdinal()));
return;
case FormatSettings::EnumComparingMode::BY_NAMES:
insertSignedInteger(column, nested_type, Int64(enum_type->getValue(String(enumerant.getProto().getName()))));
insertInteger(column, nested_type, Int64(enum_type->getValue(String(enumerant.getProto().getName()))));
return;
case FormatSettings::EnumComparingMode::BY_NAMES_CASE_INSENSITIVE:
{
@ -179,7 +175,7 @@ static void insertEnum(IColumn & column, const DataTypePtr & column_type, const
{
if (compareEnumNames(name, enum_name, enum_comparing_mode))
{
insertSignedInteger(column, nested_type, Int64(enum_type->getValue(name)));
insertInteger(column, nested_type, Int64(enum_type->getValue(name)));
break;
}
}
@ -202,22 +198,22 @@ static void insertValue(IColumn & column, const DataTypePtr & column_type, const
switch (value.getType())
{
case capnp::DynamicValue::Type::INT:
insertSignedInteger(column, column_type, value.as<Int64>());
insertInteger(column, column_type, value.as<Int64>());
break;
case capnp::DynamicValue::Type::UINT:
insertUnsignedInteger(column, column_type, value.as<UInt64>());
insertInteger(column, column_type, value.as<UInt64>());
break;
case capnp::DynamicValue::Type::FLOAT:
insertFloat(column, column_type, value.as<Float64>());
break;
case capnp::DynamicValue::Type::BOOL:
insertUnsignedInteger(column, column_type, UInt64(value.as<bool>()));
insertInteger(column, column_type, UInt64(value.as<bool>()));
break;
case capnp::DynamicValue::Type::DATA:
insertString(column, value.as<capnp::Data>());
insertData(column, column_type, value.as<capnp::Data>());
break;
case capnp::DynamicValue::Type::TEXT:
insertString(column, value.as<capnp::Text>());
insertData(column, column_type, value.as<capnp::Text>());
break;
case capnp::DynamicValue::Type::ENUM:
if (column_type->getTypeId() == TypeIndex::Enum8)
@ -260,14 +256,26 @@ static void insertValue(IColumn & column, const DataTypePtr & column_type, const
{
auto & tuple_column = assert_cast<ColumnTuple &>(column);
const auto * tuple_type = assert_cast<const DataTypeTuple *>(column_type.get());
for (size_t i = 0; i != tuple_column.tupleSize(); ++i)
bool have_explicit_names = tuple_type->haveExplicitNames();
auto struct_schema = struct_value.getSchema();
for (uint32_t i = 0; i != tuple_column.tupleSize(); ++i)
insertValue(
tuple_column.getColumn(i),
tuple_type->getElements()[i],
tuple_type->getElementNames()[i],
struct_value.get(tuple_type->getElementNames()[i]),
struct_value.get(have_explicit_names ? struct_schema.getFieldByName(tuple_type->getElementNames()[i]) : struct_schema.getFields()[i]),
enum_comparing_mode);
}
else if (isMap(column_type))
{
const auto & map_type = assert_cast<const DataTypeMap &>(*column_type);
DataTypes key_value_types = {map_type.getKeyType(), map_type.getValueType()};
Names key_value_names = {"key", "value"};
auto entries_type = std::make_shared<DataTypeArray>(std::make_shared<DataTypeTuple>(key_value_types, key_value_names));
auto & entries_column = assert_cast<ColumnMap &>(column).getNestedColumn();
auto entries_field = struct_value.getSchema().getFields()[0];
insertValue(entries_column, entries_type, column_name, struct_value.get(entries_field), enum_comparing_mode);
}
else
{
/// It can be nested column from Nested type.

View File

@ -14,12 +14,14 @@
#include <Columns/ColumnTuple.h>
#include <Columns/ColumnLowCardinality.h>
#include <Columns/ColumnDecimal.h>
#include <Columns/ColumnMap.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeEnum.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypeLowCardinality.h>
#include <DataTypes/DataTypeMap.h>
namespace DB
{
@ -177,18 +179,46 @@ static std::optional<capnp::DynamicValue::Reader> convertToDynamicValue(
else if (isTuple(data_type))
{
const auto * tuple_data_type = assert_cast<const DataTypeTuple *>(data_type.get());
auto nested_types = tuple_data_type->getElements();
const auto & nested_types = tuple_data_type->getElements();
const auto & nested_names = tuple_data_type->getElementNames();
const auto & nested_columns = assert_cast<const ColumnTuple *>(column.get())->getColumns();
for (const auto & name : tuple_data_type->getElementNames())
bool have_explicit_names = tuple_data_type->haveExplicitNames();
for (uint32_t i = 0; i != nested_names.size(); ++i)
{
auto pos = tuple_data_type->getPositionByName(name);
auto field_builder
= initStructFieldBuilder(nested_columns[pos], row_num, struct_builder, nested_struct_schema.getFieldByName(name));
auto value = convertToDynamicValue(nested_columns[pos], nested_types[pos], row_num, column_name, field_builder, enum_comparing_mode, temporary_text_data_storage);
capnp::StructSchema::Field nested_field = have_explicit_names ? nested_struct_schema.getFieldByName(nested_names[i]) : nested_struct_schema.getFields()[i];
auto field_builder = initStructFieldBuilder(nested_columns[i], row_num, struct_builder, nested_field);
auto value = convertToDynamicValue(nested_columns[i], nested_types[i], row_num, nested_names[i], field_builder, enum_comparing_mode, temporary_text_data_storage);
if (value)
struct_builder.set(name, *value);
struct_builder.set(nested_field, *value);
}
}
else if (isMap(data_type))
{
/// We output Map type as follow CapnProto schema
///
/// struct Map {
/// struct Entry {
/// key @0: Key;
/// value @1: Value;
/// }
/// entries @0 :List(Entry);
/// }
///
/// And we don't need to check that struct have this form here because we checked it before.
const auto & map_type = assert_cast<const DataTypeMap &>(*data_type);
DataTypes key_value_types = {map_type.getKeyType(), map_type.getValueType()};
Names key_value_names = {"key", "value"};
auto entries_type = std::make_shared<DataTypeArray>(std::make_shared<DataTypeTuple>(key_value_types, key_value_names));
/// Nested column in Map is actually Array(Tuple), so we can output it according to "entries" field schema.
const auto & entries_column = assert_cast<const ColumnMap *>(column.get())->getNestedColumnPtr();
auto entries_field = nested_struct_schema.getFields()[0];
auto field_builder = initStructFieldBuilder(entries_column, row_num, struct_builder, entries_field);
auto entries_value = convertToDynamicValue(entries_column, entries_type, row_num, column_name, field_builder, enum_comparing_mode, temporary_text_data_storage);
if (entries_value)
struct_builder.set(entries_field, *entries_value);
}
else
{
/// It can be nested column from Nested type.

View File

@ -50,4 +50,3 @@ OK
OK
OK
OK
OK

View File

@ -109,8 +109,6 @@ $CLICKHOUSE_CLIENT --query="SELECT * FROM file('data.capnp', 'CapnProto', 'uint6
$CLICKHOUSE_CLIENT --query="SELECT * FROM file('data.capnp', 'CapnProto', 'uint64 Enum(\'one\' = 1)') SETTINGS format_schema='$SERVER_SCHEMADIR/02030_capnp_simple_types:Message'" 2>&1 | grep -F -q "CAPN_PROTO_BAD_CAST" && echo 'OK' || echo 'FAIL';
$CLICKHOUSE_CLIENT --query="SELECT * FROM file('data.capnp', 'CapnProto', 'uint64 Tuple(UInt64)') SETTINGS format_schema='$SERVER_SCHEMADIR/02030_capnp_simple_types:Message'" 2>&1 | grep -F -q "CAPN_PROTO_BAD_CAST" && echo 'OK' || echo 'FAIL';
$CLICKHOUSE_CLIENT --query="SELECT * FROM file('data.capnp', 'CapnProto', 'uint64 Nullable(UInt64)') SETTINGS format_schema='$SERVER_SCHEMADIR/02030_capnp_simple_types:Message'" 2>&1 | grep -F -q "CAPN_PROTO_BAD_CAST" && echo 'OK' || echo 'FAIL';
$CLICKHOUSE_CLIENT --query="SELECT * FROM file('data.capnp', 'CapnProto', 'uint64 Int32') SETTINGS format_schema='$SERVER_SCHEMADIR/02030_capnp_simple_types:Message'" 2>&1 | grep -F -q "CAPN_PROTO_BAD_CAST" && echo 'OK' || echo 'FAIL';
$CLICKHOUSE_CLIENT --query="SELECT number AS a, toString(number) as b FROM numbers(5) FORMAT CapnProto SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_unnamed_union:Message'" 2>&1 | grep -F -q "CAPN_PROTO_BAD_CAST" && echo 'OK' || echo 'FAIL';
$CLICKHOUSE_CLIENT --query="SELECT toNullable(toString(number)) as nullable1 FROM numbers(5) FORMAT CapnProto SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_fake_nullable:Message'" 2>&1 | grep -F -q "CAPN_PROTO_BAD_CAST" && echo 'OK' || echo 'FAIL';

View File

@ -0,0 +1,3 @@
42 42 42 42 42.42 42.42
{'Hello':42,'World':24}
42 42 42 42 42 42 42 42

View File

@ -0,0 +1,17 @@
#!/usr/bin/env bash
# Tags: no-fasttest, no-parallel, no-replicated-database
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
SCHEMADIR=$CURDIR/format_schemas
$CLICKHOUSE_LOCAL -q "select 42::Int128 as int128, 42::UInt128 as uint128, 42::Int256 as int256, 42::UInt256 as uint256, 42.42::Decimal128(2) as decimal128, 42.42::Decimal256(2) as decimal256 format CapnProto settings format_schema='$SCHEMADIR/02705_big_numbers:Message'" | $CLICKHOUSE_LOCAL --input-format CapnProto --structure "int128 Int128, uint128 UInt128, int256 Int256, uint256 UInt256, decimal128 Decimal128(2), decimal256 Decimal256(2)" -q "select * from table" --format_schema="$SCHEMADIR/02705_big_numbers:Message"
$CLICKHOUSE_LOCAL -q "select map('Hello', 42, 'World', 24) as map format CapnProto settings format_schema='$SCHEMADIR/02705_map:Message'" | $CLICKHOUSE_LOCAL --input-format CapnProto --structure "map Map(String, UInt32)" --format_schema="$SCHEMADIR/02705_map:Message" -q "select * from table"
$CLICKHOUSE_LOCAL -q "select 42 as int8, 42 as uint8, 42 as int16, 42 as uint16, 42 as int32, 42 as uint32, 42 as int64, 42 as uint64 format CapnProto settings format_schema='$SCHEMADIR/02030_capnp_simple_types:Message'" | $CLICKHOUSE_LOCAL --input-format CapnProto --structure "int8 UInt32, uint8 Int32, int16 Int8, uint16 UInt8, int32 UInt64, uint32 Int64, int64 UInt16, uint64 Int16" --format_schema="$SCHEMADIR/02030_capnp_simple_types:Message" -q "select * from table"

View File

@ -0,0 +1,10 @@
@0xdbb9ad1f14bf0b36;
struct Message {
int128 @0 :Data;
uint128 @1 :Data;
int256 @2 :Data;
uint256 @3 :Data;
decimal128 @4 :Data;
decimal256 @5 :Data;
}

View File

@ -0,0 +1,14 @@
@0xdbb9ad1f14bf0b36;
struct Message {
struct Map {
struct Entry {
key @0 : Text;
value @1 : UInt32;
}
entries @0 : List(Entry);
}
map @0 : Map;
}