You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "ASF GitHub Bot (JIRA)" <ji...@apache.org> on 2018/01/26 17:55:01 UTC

***UNCHECKED*** [jira] [Commented] (FLINK-8240) Create unified interfaces to configure and instatiate TableSources

    [ https://issues.apache.org/jira/browse/FLINK-8240?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16341345#comment-16341345 ] 

ASF GitHub Bot commented on FLINK-8240:
---------------------------------------

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

    https://github.com/apache/flink/pull/5240#discussion_r164122214
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/NormalizedProperties.scala ---
    @@ -0,0 +1,328 @@
    +/*
    + * 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.table.descriptors
    +
    +import java.io.Serializable
    +import java.lang.{Boolean => JBoolean, Double => JDouble, Integer => JInt, Long => JLong}
    +
    +import org.apache.commons.codec.binary.Base64
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.table.api.{TableSchema, ValidationException}
    +import org.apache.flink.table.descriptors.DescriptorUtils._
    +import org.apache.flink.table.descriptors.NormalizedProperties.normalizeTableSchema
    +import org.apache.flink.table.plan.stats.ColumnStats
    +import org.apache.flink.table.sources.tsextractors.{ExistingField, StreamRecordTimestamp, TimestampExtractor}
    +import org.apache.flink.table.sources.wmstrategies.{AscendingTimestamps, BoundedOutOfOrderTimestamps, PreserveWatermarks, WatermarkStrategy}
    +import org.apache.flink.table.typeutils.TypeStringUtils
    +import org.apache.flink.util.InstantiationUtil
    +import org.apache.flink.util.Preconditions.checkNotNull
    +
    +import scala.collection.mutable
    +
    +/**
    +  * Utility class for having a unified string-based representation of Table API related classes
    +  * such as [[TableSchema]], [[TypeInformation]], [[WatermarkStrategy]], etc.
    +  */
    +class NormalizedProperties(
    --- End diff --
    
    Rename to `TableSourceProperties`? `NormalizedProperties` is quite generic


> Create unified interfaces to configure and instatiate TableSources
> ------------------------------------------------------------------
>
>                 Key: FLINK-8240
>                 URL: https://issues.apache.org/jira/browse/FLINK-8240
>             Project: Flink
>          Issue Type: New Feature
>          Components: Table API &amp; SQL
>            Reporter: Timo Walther
>            Assignee: Timo Walther
>            Priority: Major
>
> At the moment every table source has different ways for configuration and instantiation. Some table source are tailored to a specific encoding (e.g., {{KafkaAvroTableSource}}, {{KafkaJsonTableSource}}) or only support one encoding for reading (e.g., {{CsvTableSource}}). Each of them might implement a builder or support table source converters for external catalogs.
> The table sources should have a unified interface for discovery, defining common properties, and instantiation. The {{TableSourceConverters}} provide a similar functionality but use an external catalog. We might generialize this interface.
> In general a table source declaration depends on the following parts:
> {code}
> - Source
>   - Type (e.g. Kafka, Custom)
>   - Properties (e.g. topic, connection info)
> - Encoding
>   - Type (e.g. Avro, JSON, CSV)
>   - Schema (e.g. Avro class, JSON field names/types)
> - Rowtime descriptor/Proctime
>   - Watermark strategy and Watermark properties
>   - Time attribute info
> - Bucketization
> {code}
> This issue needs a design document before implementation. Any discussion is very welcome.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)