Add support for references

This commit is contained in:
Andrew Onyshchuk 2020-01-31 22:13:12 -06:00
parent 1b308142d6
commit b84d5c2139
8 changed files with 102 additions and 3 deletions

View File

@ -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);
}

View File

@ -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

View File

@ -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

View File

@ -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'

View File

@ -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

View 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"
}
]
}

View 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"
}