mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Add support for references
This commit is contained in:
parent
1b308142d6
commit
b84d5c2139
@ -44,6 +44,7 @@
|
||||
#include <avro/Reader.hh>
|
||||
#include <avro/Schema.hh>
|
||||
#include <avro/Specific.hh>
|
||||
#include <avro/Types.hh>
|
||||
#include <avro/ValidSchema.hh>
|
||||
#include <avro/Writer.hh>
|
||||
|
||||
@ -149,6 +150,12 @@ static void insertNumber(IColumn & column, WhichDataType type, T value)
|
||||
}
|
||||
}
|
||||
|
||||
static std::string nodeToJson(avro::NodePtr root_node)
|
||||
{
|
||||
std::ostringstream ss;
|
||||
root_node->printJson(ss, 0);
|
||||
return ss.str();
|
||||
}
|
||||
|
||||
AvroDeserializer::DeserializeFn AvroDeserializer::createDeserializeFn(avro::NodePtr root_node, DataTypePtr target_type)
|
||||
{
|
||||
@ -326,6 +333,8 @@ AvroDeserializer::DeserializeFn AvroDeserializer::createDeserializeFn(avro::Node
|
||||
}
|
||||
break;
|
||||
}
|
||||
case avro::AVRO_SYMBOLIC:
|
||||
return createDeserializeFn(avro::resolveSymbol(root_node), target_type);
|
||||
case avro::AVRO_MAP: [[fallthrough]];
|
||||
case avro::AVRO_RECORD: [[fallthrough]];
|
||||
default:
|
||||
@ -333,7 +342,7 @@ AvroDeserializer::DeserializeFn AvroDeserializer::createDeserializeFn(avro::Node
|
||||
}
|
||||
|
||||
throw Exception(
|
||||
"Type " + target_type->getName() + " is not compatible with Avro " + avro::ValidSchema(root_node).toJson(false),
|
||||
"Type " + target_type->getName() + " is not compatible with Avro " + avro::toString(root_node->type()) + ":\n" + nodeToJson(root_node),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
||||
@ -415,6 +424,18 @@ AvroDeserializer::SkipFn AvroDeserializer::createSkipFn(avro::NodePtr root_node)
|
||||
skip_fn(decoder);
|
||||
};
|
||||
}
|
||||
case avro::AVRO_SYMBOLIC:
|
||||
{
|
||||
auto [it, inserted] = symbolic_skip_fn_map.emplace(root_node->name(), SkipFnHolder{});
|
||||
if (inserted)
|
||||
{
|
||||
it->second.skip_fn = createSkipFn(avro::resolveSymbol(root_node));
|
||||
}
|
||||
return [&holder = it->second](avro::Decoder & decoder)
|
||||
{
|
||||
holder.skip_fn(decoder);
|
||||
};
|
||||
}
|
||||
default:
|
||||
throw Exception("Unsupported Avro type " + root_node->name().fullname() + " (" + toString(int(root_node->type())) + ")", ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
@ -4,6 +4,7 @@
|
||||
#if USE_AVRO
|
||||
|
||||
#include <unordered_map>
|
||||
#include <map>
|
||||
#include <vector>
|
||||
|
||||
#include <Core/Block.h>
|
||||
@ -27,8 +28,9 @@ public:
|
||||
private:
|
||||
using DeserializeFn = std::function<void(IColumn & column, avro::Decoder & decoder)>;
|
||||
using SkipFn = std::function<void(avro::Decoder & decoder)>;
|
||||
struct SkipFnHolder { SkipFn skip_fn; };
|
||||
static DeserializeFn createDeserializeFn(avro::NodePtr root_node, DataTypePtr target_type);
|
||||
static SkipFn createSkipFn(avro::NodePtr root_node);
|
||||
SkipFn createSkipFn(avro::NodePtr root_node);
|
||||
|
||||
/// Map from field index in Avro schema to column number in block header. Or -1 if there is no corresponding column.
|
||||
std::vector<int> field_mapping;
|
||||
@ -38,6 +40,10 @@ private:
|
||||
|
||||
/// How to deserialize the corresponding field in Avro schema.
|
||||
std::vector<DeserializeFn> deserialize_fns;
|
||||
|
||||
/// Map from name of named Avro type (record, enum, fixed) to SkipFn holder.
|
||||
/// This is to avoid infinite recursion when Avro schema contains self-references. e.g. LinkedList
|
||||
std::map<avro::Name, SkipFnHolder> symbolic_skip_fn_map;
|
||||
};
|
||||
|
||||
class AvroRowInputFormat : public IRowInputFormat
|
||||
|
@ -16,6 +16,9 @@
|
||||
= logical_types
|
||||
"2019-12-20","2020-01-10 07:31:56.227","2020-01-10 07:31:56.227000"
|
||||
18250,1578641516227,1578641516227000
|
||||
= references
|
||||
"a1","c1"
|
||||
"a2","c2"
|
||||
= compression
|
||||
1000
|
||||
1000
|
||||
|
@ -24,7 +24,8 @@ echo = logical_types
|
||||
cat $DATA_DIR/logical_types.avro | ${CLICKHOUSE_LOCAL} --input-format Avro --output-format CSV -S "a_date Date, b_timestamp_millis DateTime64(3, 'UTC'), c_timestamp_micros DateTime64(6, 'UTC')" -q 'select * from table'
|
||||
cat $DATA_DIR/logical_types.avro | ${CLICKHOUSE_LOCAL} --input-format Avro --output-format CSV -S 'a_date Int32, b_timestamp_millis Int64, c_timestamp_micros Int64' -q 'select * from table'
|
||||
|
||||
|
||||
echo = references
|
||||
cat $DATA_DIR/references.avro | ${CLICKHOUSE_LOCAL} --input-format Avro --output-format CSV -S "a String, c String" -q 'select * from table'
|
||||
|
||||
echo = compression
|
||||
cat $DATA_DIR/simple.null.avro | ${CLICKHOUSE_LOCAL} --input-format Avro --output-format CSV -S 'a Int64' -q 'select count() from table'
|
||||
|
@ -7,6 +7,7 @@ avro-tools fromjson --schema-file primitive.avsc primitive.json > primitive.avr
|
||||
avro-tools fromjson --schema-file complex.avsc complex.json > complex.avro
|
||||
avro-tools fromjson --schema-file logical_types.avsc logical_types.json > logical_types.avro
|
||||
avro-tools fromjson --schema-file empty.avsc empty.json > empty.avro
|
||||
avro-tools fromjson --schema-file references.avsc references.json > references.avro
|
||||
|
||||
#compression
|
||||
avro-tools fromjson --codec null --schema-file simple.avsc simple.json > simple.null.avro
|
||||
|
BIN
dbms/tests/queries/0_stateless/data_avro/references.avro
Normal file
BIN
dbms/tests/queries/0_stateless/data_avro/references.avro
Normal file
Binary file not shown.
39
dbms/tests/queries/0_stateless/data_avro/references.avsc
Normal file
39
dbms/tests/queries/0_stateless/data_avro/references.avsc
Normal file
@ -0,0 +1,39 @@
|
||||
{
|
||||
"type": "record",
|
||||
"name": "row",
|
||||
"fields": [
|
||||
{
|
||||
"name": "a",
|
||||
"type": {
|
||||
"name": "fixed_2",
|
||||
"type": "fixed",
|
||||
"size": 2
|
||||
}
|
||||
},
|
||||
{
|
||||
"name": "b",
|
||||
"type": {
|
||||
"type": "record",
|
||||
"name": "LongList",
|
||||
"fields": [
|
||||
{
|
||||
"name": "value",
|
||||
"type": "long"
|
||||
},
|
||||
{
|
||||
"name": "next",
|
||||
"type": [
|
||||
"null",
|
||||
"LongList",
|
||||
"fixed_2"
|
||||
]
|
||||
}
|
||||
]
|
||||
}
|
||||
},
|
||||
{
|
||||
"name": "c",
|
||||
"type": "fixed_2"
|
||||
}
|
||||
]
|
||||
}
|
28
dbms/tests/queries/0_stateless/data_avro/references.json
Normal file
28
dbms/tests/queries/0_stateless/data_avro/references.json
Normal file
@ -0,0 +1,28 @@
|
||||
{
|
||||
"a": "a1",
|
||||
"b": {
|
||||
"value": 1,
|
||||
"next": {
|
||||
"LongList": {
|
||||
"value": 2,
|
||||
"next": {
|
||||
"LongList": {
|
||||
"value": 3,
|
||||
"next": null
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
},
|
||||
"c": "c1"
|
||||
}
|
||||
{
|
||||
"a": "a2",
|
||||
"b": {
|
||||
"value": 4,
|
||||
"next": {
|
||||
"fixed_2": "ab"
|
||||
}
|
||||
},
|
||||
"c": "c2"
|
||||
}
|
Loading…
Reference in New Issue
Block a user