Uploaded image for project: 'Flink'
  1. Flink
  2. FLINK-15992

Incorrect classloader when finding TableFactory

    XMLWordPrintableJSON

Details

    Description

      Background

      As a streaming service maintainer in our company, to ensure our users depend on the correct version of Kafka and flink-kafka, we add "flink-connector-kafka" into "fink-dist/lib" directory.

      Problem

      When submitting flink-sql jobs, we encountered below exceptions:

      Caused by: org.apache.flink.table.api.NoMatchingTableFactoryException: Could not find a suitable table factory for 'org.apache.flink.table.factories.DeserializationSchemaFactory' in
      the classpath.
      

      But we have add "org.apache.flink.formats.json.JsonRowFormatFactory" in "META-INF/services/org.apache.flink.table.factories.TableFactory", which implements DeserializationSchemaFactory.

      Debug

      We find that it was caused by this:

      // org.apache.flink.streaming.connectors.kafka.KafkaTableSourceSinkFactoryBase#getSerializationSchema
      
      		final SerializationSchemaFactory<Row> formatFactory = TableFactoryService.find(
      			SerializationSchemaFactory.class,
      			properties,
      			this.getClass().getClassLoader());
      

      It uses `this.getClass().getClassLoader()`, which will be BootStrapClassLoader of flink.
      I think we could replace it with `Thread.currentThread().getContextClassLoader()` to solve this.

      There is a related issue: https://issues.apache.org/jira/browse/FLINK-15552

      Attachments

        Issue Links

          Activity

            People

              Unassigned Unassigned
              victor-wong jiasheng55
              Votes:
              0 Vote for this issue
              Watchers:
              4 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved:

                Time Tracking

                  Estimated:
                  Original Estimate - Not Specified
                  Not Specified
                  Remaining:
                  Remaining Estimate - 0h
                  0h
                  Logged:
                  Time Spent - 20m
                  20m