Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3663#discussion_r113207820
  
    --- Diff: 
flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaAvroTableSource.java
 ---
    @@ -0,0 +1,117 @@
    +/*
    + * 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.connectors.kafka;
    +
    +import java.util.List;
    +import java.util.Properties;
    +import org.apache.avro.Schema;
    +import org.apache.avro.specific.SpecificData;
    +import org.apache.avro.specific.SpecificRecord;
    +import org.apache.avro.specific.SpecificRecordBase;
    +import org.apache.avro.util.Utf8;
    +import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.java.typeutils.AvroTypeInfo;
    +import org.apache.flink.api.java.typeutils.GenericTypeInfo;
    +import org.apache.flink.api.java.typeutils.RowTypeInfo;
    +import 
org.apache.flink.streaming.util.serialization.AvroRowDeserializationSchema;
    +import org.apache.flink.streaming.util.serialization.DeserializationSchema;
    +import org.apache.flink.table.sources.StreamTableSource;
    +
    +/**
    + * A version-agnostic Kafka Avro {@link StreamTableSource}.
    + *
    + * <p>The version-specific Kafka consumers need to extend this class and
    + * override {@link #getKafkaConsumer(String, Properties, 
DeserializationSchema)}}.
    + */
    +public abstract class KafkaAvroTableSource extends KafkaTableSource {
    +
    +   /**
    +    * Creates a generic Kafka Avro {@link StreamTableSource} using a given 
{@link SpecificRecord}.
    +    *
    +    * @param topic      Kafka topic to consume.
    +    * @param properties Properties for the Kafka consumer.
    +    * @param record     Avro specific record.
    +    */
    +   KafkaAvroTableSource(
    +           String topic,
    +           Properties properties,
    +           Class<? extends SpecificRecordBase> record) {
    +
    +           super(
    +                   topic,
    +                   properties,
    +                   createDeserializationSchema(record),
    +                   createFieldNames(record),
    +                   createFieldTypes(record));
    +   }
    +
    +   private static AvroRowDeserializationSchema 
createDeserializationSchema(Class<? extends SpecificRecordBase> record) {
    +           return new AvroRowDeserializationSchema(record);
    +   }
    +
    +   /**
    +    * Converts the extracted AvroTypeInfo into a RowTypeInfo nested 
structure with deterministic field order.
    +    * Replaces generic Utf8 with basic String type information.
    +    */
    +   private static TypeInformation<?> 
convertToRowTypeInformation(TypeInformation<?> extracted, Schema schema) {
    +           if (schema.getType() == Schema.Type.RECORD) {
    +                   final List<Schema.Field> fields = schema.getFields();
    +                   final AvroTypeInfo<?> avroTypeInfo = (AvroTypeInfo<?>) 
extracted;
    +
    +                   final TypeInformation<?>[] types = new 
TypeInformation<?>[fields.size()];
    +                   final String[] names = new String[fields.size()];
    +                   for (int i = 0; i < fields.size(); i++) {
    +                           final Schema.Field field = fields.get(i);
    +                           types[i] = 
convertToRowTypeInformation(avroTypeInfo.getTypeAt(field.name()), 
field.schema());
    +                           names[i] = field.name();
    +                   }
    +                   return new RowTypeInfo(types, names);
    +           } else if (extracted instanceof GenericTypeInfo<?>) {
    +                   final GenericTypeInfo<?> genericTypeInfo = 
(GenericTypeInfo<?>) extracted;
    +                   if (genericTypeInfo.getTypeClass() == Utf8.class) {
    +                           return BasicTypeInfo.STRING_TYPE_INFO;
    +                   }
    +           }
    +           return extracted;
    +   }
    +
    +   private static <T extends SpecificRecordBase> TypeInformation<?>[] 
createFieldTypes(Class<T> record) {
    --- End diff --
    
    This method can be removed as well, if we refactor `KafkaTableSource` as 
described below.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

Reply via email to