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

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

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

    https://github.com/apache/flink/pull/6323#discussion_r202535180
  
    --- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/TableFactoryService.scala
 ---
    @@ -18,143 +18,358 @@
     
     package org.apache.flink.table.factories
     
    -import java.util.{ServiceConfigurationError, ServiceLoader}
    +import java.util.{ServiceConfigurationError, ServiceLoader, Map => JMap}
     
     import org.apache.flink.table.api._
     import org.apache.flink.table.descriptors.ConnectorDescriptorValidator._
     import org.apache.flink.table.descriptors.FormatDescriptorValidator._
     import org.apache.flink.table.descriptors.MetadataValidator._
     import org.apache.flink.table.descriptors.StatisticsValidator._
    -import org.apache.flink.table.descriptors.{DescriptorProperties, 
TableDescriptor, TableDescriptorValidator}
    +import org.apache.flink.table.descriptors._
     import org.apache.flink.table.util.Logging
    +import org.apache.flink.util.Preconditions
     
     import _root_.scala.collection.JavaConverters._
     import _root_.scala.collection.mutable
     
     /**
    -  * Unified interface to search for TableFactoryDiscoverable of provided 
type and properties.
    +  * Unified interface to search for a [[TableFactory]] of provided type 
and properties.
       */
     object TableFactoryService extends Logging {
     
       private lazy val defaultLoader = 
ServiceLoader.load(classOf[TableFactory])
     
    -  def find(clz: Class[_], descriptor: TableDescriptor): TableFactory = {
    -    find(clz, descriptor, null)
    +  /**
    +    * Finds a table factory of the given class and descriptor.
    +    *
    +    * @param factoryClass desired factory class
    +    * @param descriptor descriptor describing the factory configuration
    +    * @tparam T factory class type
    +    * @return the matching factory
    +    */
    +  def find[T](factoryClass: Class[T], descriptor: Descriptor): T = {
    +    Preconditions.checkNotNull(factoryClass)
    +    Preconditions.checkNotNull(descriptor)
    +
    +    val descriptorProperties = new DescriptorProperties()
    +    descriptor.addProperties(descriptorProperties)
    +    findInternal(factoryClass, descriptorProperties.asMap, None)
       }
     
    -  def find(clz: Class[_], descriptor: TableDescriptor, classLoader: 
ClassLoader)
    -  : TableFactory = {
    +  /**
    +    * Finds a table factory of the given class, descriptor, and 
classloader.
    +    *
    +    * @param factoryClass desired factory class
    +    * @param descriptor descriptor describing the factory configuration
    +    * @param classLoader classloader for service loading
    +    * @tparam T factory class type
    +    * @return the matching factory
    +    */
    +  def find[T](factoryClass: Class[T], descriptor: Descriptor, classLoader: 
ClassLoader): T = {
    +    Preconditions.checkNotNull(factoryClass)
    +    Preconditions.checkNotNull(descriptor)
    +    Preconditions.checkNotNull(classLoader)
     
    -    val properties = new DescriptorProperties()
    -    descriptor.addProperties(properties)
    -    find(clz, properties.asMap.asScala.toMap, classLoader)
    +    val descriptorProperties = new DescriptorProperties()
    +    descriptor.addProperties(descriptorProperties)
    +    findInternal(factoryClass, descriptorProperties.asMap, None)
       }
     
    -  def find(clz: Class[_], properties: Map[String, String]): TableFactory = 
{
    -    find(clz: Class[_], properties, null)
    +  /**
    +    * Finds a table factory of the given class and property map.
    +    *
    +    * @param factoryClass desired factory class
    +    * @param propertyMap properties that describe the factory configuration
    +    * @tparam T factory class type
    +    * @return the matching factory
    +    */
    +  def find[T](factoryClass: Class[T], propertyMap: JMap[String, String]): 
T = {
    +    Preconditions.checkNotNull(factoryClass)
    +    Preconditions.checkNotNull(propertyMap)
    +
    +    findInternal(factoryClass, propertyMap, None)
       }
     
    -  def find(clz: Class[_], properties: Map[String, String],
    -           classLoader: ClassLoader): TableFactory = {
    +  /**
    +    * Finds a table factory of the given class, property map, and 
classloader.
    +    *
    +    * @param factoryClass desired factory class
    +    * @param propertyMap properties that describe the factory configuration
    +    * @param classLoader classloader for service loading
    +    * @tparam T factory class type
    +    * @return the matching factory
    +    */
    +  def find[T](
    +      factoryClass: Class[T],
    +      propertyMap: JMap[String, String],
    +      classLoader: ClassLoader)
    +    : T = {
    +    Preconditions.checkNotNull(factoryClass)
    +    Preconditions.checkNotNull(propertyMap)
    +    Preconditions.checkNotNull(classLoader)
    +
    +    findInternal(factoryClass, propertyMap, Some(classLoader))
    +  }
    +
    +  /**
    +    * Finds a table factory of the given class, property map, and 
classloader.
    +    *
    +    * @param factoryClass desired factory class
    +    * @param propertyMap properties that describe the factory configuration
    +    * @param classLoader classloader for service loading
    +    * @tparam T factory class type
    +    * @return the matching factory
    +    */
    +  private def findInternal[T](
    +      factoryClass: Class[T],
    +      propertyMap: JMap[String, String],
    +      classLoader: Option[ClassLoader])
    +    : T = {
    +
    +    val properties = propertyMap.asScala.toMap
    +
    +    // discover table factories
    +    val foundFactories = discoverFactories(classLoader)
     
    -    var matchingFactory: Option[(TableFactory, Seq[String])] = None
    +    // filter by factory class
    +    val classFactories = filterByFactoryClass(
    +      factoryClass,
    +      properties,
    +      foundFactories)
    +
    +    // find matching context
    +    val contextFactories = filterByContext(
    +      factoryClass,
    +      properties,
    +      foundFactories,
    +      classFactories)
    +
    +    // filter by supported keys
    +    filterBySupportedProperties(
    +      factoryClass,
    +      properties,
    +      foundFactories,
    +      contextFactories)
    +  }
    +
    +  /**
    +    * Searches for factories using Java service providers.
    +    *
    +    * @return all factories in the classpath
    +    */
    +  private def discoverFactories[T](classLoader: Option[ClassLoader]): 
Seq[TableFactory] = {
    +    val foundFactories = mutable.ArrayBuffer[TableFactory]()
         try {
    -      val iter = if (classLoader == null) {
    -        defaultLoader.iterator()
    -      } else {
    -        val customLoader = ServiceLoader.load(classOf[TableFactory], 
classLoader)
    -        customLoader.iterator()
    +      val iterator = classLoader match {
    +        case Some(customClassLoader) =>
    +          val customLoader = ServiceLoader.load(classOf[TableFactory], 
customClassLoader)
    +          customLoader.iterator()
    +        case None =>
    +          defaultLoader.iterator()
           }
    -      while (iter.hasNext) {
    -        val factory = iter.next()
    -
    -        if (clz.isAssignableFrom(factory.getClass)) {
    -          val requiredContextJava = try {
    -            factory.requiredContext()
    -          } catch {
    -            case t: Throwable =>
    -              throw new TableException(
    -                s"Table source factory 
'${factory.getClass.getCanonicalName}' caused an exception.",
    -                t)
    -          }
    -
    -          val requiredContext = if (requiredContextJava != null) {
    -            // normalize properties
    -            requiredContextJava.asScala.map(e => (e._1.toLowerCase, e._2))
    -          } else {
    -            Map[String, String]()
    -          }
    -
    -          val plainContext = mutable.Map[String, String]()
    -          plainContext ++= requiredContext
    -          // we remove the versions for now until we have the first 
backwards compatibility case
    -          // with the version we can provide mappings in case the format 
changes
    -          plainContext.remove(CONNECTOR_PROPERTY_VERSION)
    -          plainContext.remove(FORMAT_PROPERTY_VERSION)
    -          plainContext.remove(METADATA_PROPERTY_VERSION)
    -          plainContext.remove(STATISTICS_PROPERTY_VERSION)
    -
    -          if (plainContext.forall(e => properties.contains(e._1) && 
properties(e._1) == e._2)) {
    -            matchingFactory match {
    -              case Some(_) => throw new 
AmbiguousTableFactoryException(properties)
    -              case None => matchingFactory =
    -                Some((factory.asInstanceOf[TableFactory], 
requiredContext.keys.toSeq))
    -            }
    -          }
    -        }
    +
    +      while (iterator.hasNext) {
    +        val factory = iterator.next()
    +        foundFactories += factory
           }
    +
    +      foundFactories
         } catch {
           case e: ServiceConfigurationError =>
             LOG.error("Could not load service provider for table factories.", 
e)
             throw new TableException("Could not load service provider for 
table factories.", e)
         }
    +  }
    +
    +  /**
    +    * Filters for factories with matching context.
    +    *
    +    * @return all matching factories
    +    */
    +  private def filterByContext[T](
    +      factoryClass: Class[T],
    +      properties: Map[String, String],
    +      foundFactories: Seq[TableFactory],
    +      classFactories: Seq[TableFactory])
    +    : Seq[TableFactory] = {
    +
    +    val matchingFactories = mutable.ArrayBuffer[TableFactory]()
    +
    +    classFactories.foreach { factory =>
    +      val requestedContext = normalizeContext(factory)
    +
    +      val plainContext = mutable.Map[String, String]()
    +      plainContext ++= requestedContext
    +      // we remove the version for now until we have the first backwards 
compatibility case
    +      // with the version we can provide mappings in case the format 
changes
    --- End diff --
    
    I opened FLINK-9851 for that.


> Add unified format interfaces and format discovery
> --------------------------------------------------
>
>                 Key: FLINK-8558
>                 URL: https://issues.apache.org/jira/browse/FLINK-8558
>             Project: Flink
>          Issue Type: New Feature
>          Components: Streaming Connectors
>            Reporter: Timo Walther
>            Assignee: Timo Walther
>            Priority: Major
>              Labels: pull-request-available
>
> In the last release, we introduced a new module {{flink-formats}}. Currently 
> only {{flink-avro}} is located there but we will add more formats such as 
> {{flink-json}}, {{flink-protobuf}}, and so on. For better separation of 
> concerns we want decouple connectors from formats: e.g., remove 
> {{KafkaAvroTableSource}} and {{KafkaJsonTableSource}}.
> A newly introduced {{FormatFactory}} will use Java service loaders to 
> discovery available formats in the classpath (similar to how file systems are 
> discovered now). A {{Format}} will provide a method for converting {{byte[]}} 
> to target record type.



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

Reply via email to