[
https://issues.apache.org/jira/browse/FLINK-3871?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15983211#comment-15983211
]
ASF GitHub Bot commented on FLINK-3871:
---------------------------------------
Github user fhueske commented on a diff in the pull request:
https://github.com/apache/flink/pull/3663#discussion_r113241221
--- Diff:
flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/AvroRowDeserializationSchema.java
---
@@ -0,0 +1,157 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.util.serialization;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.util.List;
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.io.DatumReader;
+import org.apache.avro.io.Decoder;
+import org.apache.avro.io.DecoderFactory;
+import org.apache.avro.reflect.ReflectDatumReader;
+import org.apache.avro.specific.SpecificData;
+import org.apache.avro.specific.SpecificRecord;
+import org.apache.avro.util.Utf8;
+import org.apache.flink.types.Row;
+import org.apache.flink.util.Preconditions;
+
+/**
+ * Deserialization schema from Avro bytes over {@link SpecificRecord} to
{@link Row}.
+ *
+ * Deserializes the <code>byte[]</code> messages into (nested) Flink Rows.
+ *
+ * {@link Utf8} is converted to regular Java Strings.
+ */
+public class AvroRowDeserializationSchema extends
AbstractDeserializationSchema<Row> {
+
+ /**
+ * Schema for deterministic field order.
+ */
+ private final Schema schema;
+
+ /**
+ * Reader that deserializes byte array into a record.
+ */
+ private final DatumReader<GenericRecord> datumReader;
--- End diff --
`GenericRecord` -> `SpecificRecord`
> Add Kafka TableSource with Avro serialization
> ---------------------------------------------
>
> Key: FLINK-3871
> URL: https://issues.apache.org/jira/browse/FLINK-3871
> Project: Flink
> Issue Type: New Feature
> Components: Table API & SQL
> Reporter: Fabian Hueske
> Assignee: Ivan Mushketyk
>
> Add a Kafka TableSource which supports Avro serialized data.
> The KafkaAvroTableSource should support two modes:
> # SpecificRecord Mode: In this case the user specifies a class which was
> code-generated by Avro depending on a schema. Flink treats these classes as
> regular POJOs. Hence, they are also natively supported by the Table API and
> SQL. Classes generated by Avro contain their Schema in a static field. The
> schema should be used to automatically derive field names and types. Hence,
> there is no additional information required than the name of the class.
> # GenericRecord Mode: In this case the user specifies an Avro Schema. The
> schema is used to deserialize the data into a GenericRecord which must be
> translated into possibly nested {{Row}} based on the schema information.
> Again, the Avro Schema is used to automatically derive the field names and
> types. This mode is less efficient than the SpecificRecord mode because the
> {{GenericRecord}} needs to be converted into {{Row}}.
> This feature depends on FLINK-5280, i.e., support for nested data in
> {{TableSource}}.
--
This message was sent by Atlassian JIRA
(v6.3.15#6346)