Re: 关于Flink1.10 Standalone 模式任务提交

2020-04-23 文章 宇张
嗯嗯,刚刚把blink包改为provided后程序能正常运行了,但接下来工程打包的时候都要手动加下面这些感觉比较麻烦,不过能运行就好。

感谢答疑

org.apache.flink:flink-connector-kafka-base_2.11


On Thu, Apr 23, 2020 at 4:36 PM Jingsong Li  wrote:

> 不能把lib下有的jar打进去。
>
> 比如flink-table-planner-blink,lib下也有一份flink-table-planner-blink
>
> 把这一堆去掉吧:
> org.apache.flink:flink-table-common
> org.apache.flink:flink-table-api-java
>
> org.apache.flink:flink-table-api-java-bridge_2.11
> org.apache.flink:flink-table-planner-blink_2.11
>
> Best,
> Jingsong Lee
>
> On Thu, Apr 23, 2020 at 4:24 PM 宇张  wrote:
>
> > 》》加上  >
> >
> combine.children="append">这部分配置之后对应的TableFactory文件里面有对应的KafkaFactory信息了,虽说程序还是无法运行,但是错误变为jar包冲突了,也就不是先前加载不到的错误;
> > 但是感觉每次都配置这些貌似对用户不太友好。
> >
> > org.codehaus.janino.CompilerFactory cannot be cast to
> > org.codehaus.commons.compiler.ICompilerFactory
> >
> >
> > 
> > 
> > 
> > org.apache.flink:flink-table-common
> > org.apache.flink:flink-table-api-java
> >
> > org.apache.flink:flink-table-api-java-bridge_2.11
> >
>  org.apache.flink:flink-table-planner-blink_2.11
> >
>  org.apache.flink:flink-connector-kafka-0.11_2.11
> >
>  org.apache.flink:flink-connector-kafka-0.9_2.11
> >
>  org.apache.flink:flink-connector-kafka-0.10_2.11
> >
>  org.apache.flink:flink-connector-kafka-base_2.11
> > org.apache.flink:flink-jdbc_2.11
> > org.apache.flink:flink-json
> > 
> > 
> >
> > 
> > 
> >  >
> >
> implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer"/>
> > 
> >  >
> >
> implementation="org.apache.maven.plugins.shade.resource.ApacheNoticeResourceTransformer">
> > Apache Flink
> > UTF-8
> > 
> >  >
> >
> implementation="org.apache.maven.plugins.shade.resource.ManifestResourceTransformer">
> > com.akulaku.data.main.StreamMain
> > 
> > 
> >
> >
> > On Thu, Apr 23, 2020 at 4:07 PM Jingsong Li 
> > wrote:
> >
> > > Hi 张,
> > >
> > > 加上这个[1]试试:
> > >
> > > 
> > >   
> > >> >
> >
> implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer"/>
> > >   
> > >> >
> >
> implementation="org.apache.maven.plugins.shade.resource.ApacheNoticeResourceTransformer">
> > > Apache Flink
> > > UTF-8
> > >   
> > > 
> > >
> > >
> > > [1]https://github.com/apache/flink/blob/master/pom.xml#L1654
> > >
> > > Best,
> > > Jingsong Lee
> > >
> > > On Thu, Apr 23, 2020 at 3:56 PM 宇张  wrote:
> > >
> > > > 下面配置中,如果不加flink-json模块是可以打出kafkafactory的,加了flink-json模块就只剩下
> > > > JsonRowFormatFactory一个类,kafka的Factory就打印不出来了,所以是不是某一部分导致冲突了,
> > > > 但我看我先前flink1.9的工程,里面也无法打印kafkaFactory类,只有一个
> > > > GenericInMemoryCatalogFactory类,但flink1.9和1.10对比,看发布文档类加载策略有过改动
> > > >
> > > > org.apache.flink:flink-connector-kafka-0.11_2.11
> > > > org.apache.flink:flink-connector-kafka-base_2.11
> > > > org.apache.flink:flink-json
> > > >
> > > >
> > > > On Thu, Apr 23, 2020 at 3:43 PM Jingsong Li 
> > > > wrote:
> > > >
> > > > > > 如果是这样,听起来 client 的 classloading 策略没啥问题,似乎是 SPI 加载那边的 ClassLoader
> > > > > 有问题。之前FileSystem 相关解析就出过类似的 ClassLoader 的 BUG
> > > > >
> > > > > @tison 不管怎么样,也得保证jar里的SPI文件包含Kafka的类,不然SPI没法找
> > > > >
> > > > > @宇张 建议你仔细看下[1],这个pom是能打出正确的SPI文件的
> > > > >
> > > > > [1]
> > > > >
> > > > >
> > > >
> > >
> >
> https://github.com/apache/flink/blob/master/flink-table/flink-table-uber-blink/pom.xml#L104
> > > > >
> > > > > Best,
> > > > > Jingsong Lee
> > > > >
> > > > > On Thu, Apr 23, 2020 at 3:35 PM tison 
> wrote:
> > > > >
> > > > > > 另外你 shaded 里面去 shaded com.ibm.icu 也意义不明...
> > > > > >
> > > > > > Best,
> > > > > > tison.
> > > > > >
> > > > > >
> > > > > > tison  于2020年4月23日周四 下午3:34写道:
> > > > > >
> > > > > > > 这个问题我建议你记一个 JIRA 然后提供一个可复现的程序。因为你如果是 Flink Standalone Session
> > 模式,在
> > > > > Client
> > > > > > > 端编译失败抛出如上异常,不应该跟放不放在 lib 下有什么关系。这边听你说感觉也很奇怪,可能需要本地复现一下比较好判断。
> > > > > > >
> > > > > > > Best,
> > > > > > > tison.
> > > > > > >
> > > > > > >
> > > > > > > 宇张  于2020年4月23日周四 上午11:53写道:
> > > > > > >
> > > > > > >> 你的意思是,UberJar 不放在 lib 里,在用户程序里通过线程上下文 ClassLoader 能加载到
> > > > > > >> KafkaTableSourceSinkFactory
> > > > > > >> 吗?(同时 class loading 为 child-first)
> > > > > > >> 》》是的
> > > > > > >>
> > > > > > >> On Thu, Apr 23, 2020 at 11:42 AM tison 
> > > > wrote:
> > > > > > >>
> > > > > > >> > >》拿到ClassLoader后看下能不能取到KafkaTableSourceSinkFactory的class
> > > > > > >> > >这个能拿到
> > > > > > >> >
> > > > > > >> > 你的意思是,UberJar 不放在 lib 里,在用户程序里通过线程上下文 ClassLoader 能加载到
> > > > > > >> > KafkaTableSourceSinkFactory
> > > > > > >> > 吗?(同时 class loading 为 child-first)
> > > > > > >> >
> > > > > > >> > 如果是这样,听起来 client 的 classloading 策略没啥问题,似乎是 SPI 加载那边的
> > ClassLoader
> > > > > > 有问题。之前
> > > > > > >> > FileSystem 相关解析就出过类似的 ClassLoader 的 BUG
> > > > > > >> >
> > > > > > >> > Best,
> > > > > > >> > tison.
> > > > > > >> >
> > > > > > >> >
> > > > > > >> > 宇张  于2020年4月23日周四 上午11:36写道:
> > > > > > >> >
> > > > > > >> > > 我尝试进行了添加,程序依然无法运行,异常信息和上面一致,下面是我的shade配置:
> > > > > > >> > >
> > > > > > >> > > 
> > > > > >

Re: 关于Flink1.10 Standalone 模式任务提交

2020-04-23 文章 Jingsong Li
不能把lib下有的jar打进去。

比如flink-table-planner-blink,lib下也有一份flink-table-planner-blink

把这一堆去掉吧:
org.apache.flink:flink-table-common
org.apache.flink:flink-table-api-java
org.apache.flink:flink-table-api-java-bridge_2.11
org.apache.flink:flink-table-planner-blink_2.11

Best,
Jingsong Lee

On Thu, Apr 23, 2020 at 4:24 PM 宇张  wrote:

> 》》加上 
> combine.children="append">这部分配置之后对应的TableFactory文件里面有对应的KafkaFactory信息了,虽说程序还是无法运行,但是错误变为jar包冲突了,也就不是先前加载不到的错误;
> 但是感觉每次都配置这些貌似对用户不太友好。
>
> org.codehaus.janino.CompilerFactory cannot be cast to
> org.codehaus.commons.compiler.ICompilerFactory
>
>
> 
> 
> 
> org.apache.flink:flink-table-common
> org.apache.flink:flink-table-api-java
>
> org.apache.flink:flink-table-api-java-bridge_2.11
> org.apache.flink:flink-table-planner-blink_2.11
> org.apache.flink:flink-connector-kafka-0.11_2.11
> org.apache.flink:flink-connector-kafka-0.9_2.11
> org.apache.flink:flink-connector-kafka-0.10_2.11
> org.apache.flink:flink-connector-kafka-base_2.11
> org.apache.flink:flink-jdbc_2.11
> org.apache.flink:flink-json
> 
> 
>
> 
> 
> 
> implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer"/>
> 
> 
> implementation="org.apache.maven.plugins.shade.resource.ApacheNoticeResourceTransformer">
> Apache Flink
> UTF-8
> 
> 
> implementation="org.apache.maven.plugins.shade.resource.ManifestResourceTransformer">
> com.akulaku.data.main.StreamMain
> 
> 
>
>
> On Thu, Apr 23, 2020 at 4:07 PM Jingsong Li 
> wrote:
>
> > Hi 张,
> >
> > 加上这个[1]试试:
> >
> > 
> >   
> >>
> implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer"/>
> >   
> >>
> implementation="org.apache.maven.plugins.shade.resource.ApacheNoticeResourceTransformer">
> > Apache Flink
> > UTF-8
> >   
> > 
> >
> >
> > [1]https://github.com/apache/flink/blob/master/pom.xml#L1654
> >
> > Best,
> > Jingsong Lee
> >
> > On Thu, Apr 23, 2020 at 3:56 PM 宇张  wrote:
> >
> > > 下面配置中,如果不加flink-json模块是可以打出kafkafactory的,加了flink-json模块就只剩下
> > > JsonRowFormatFactory一个类,kafka的Factory就打印不出来了,所以是不是某一部分导致冲突了,
> > > 但我看我先前flink1.9的工程,里面也无法打印kafkaFactory类,只有一个
> > > GenericInMemoryCatalogFactory类,但flink1.9和1.10对比,看发布文档类加载策略有过改动
> > >
> > > org.apache.flink:flink-connector-kafka-0.11_2.11
> > > org.apache.flink:flink-connector-kafka-base_2.11
> > > org.apache.flink:flink-json
> > >
> > >
> > > On Thu, Apr 23, 2020 at 3:43 PM Jingsong Li 
> > > wrote:
> > >
> > > > > 如果是这样,听起来 client 的 classloading 策略没啥问题,似乎是 SPI 加载那边的 ClassLoader
> > > > 有问题。之前FileSystem 相关解析就出过类似的 ClassLoader 的 BUG
> > > >
> > > > @tison 不管怎么样,也得保证jar里的SPI文件包含Kafka的类,不然SPI没法找
> > > >
> > > > @宇张 建议你仔细看下[1],这个pom是能打出正确的SPI文件的
> > > >
> > > > [1]
> > > >
> > > >
> > >
> >
> https://github.com/apache/flink/blob/master/flink-table/flink-table-uber-blink/pom.xml#L104
> > > >
> > > > Best,
> > > > Jingsong Lee
> > > >
> > > > On Thu, Apr 23, 2020 at 3:35 PM tison  wrote:
> > > >
> > > > > 另外你 shaded 里面去 shaded com.ibm.icu 也意义不明...
> > > > >
> > > > > Best,
> > > > > tison.
> > > > >
> > > > >
> > > > > tison  于2020年4月23日周四 下午3:34写道:
> > > > >
> > > > > > 这个问题我建议你记一个 JIRA 然后提供一个可复现的程序。因为你如果是 Flink Standalone Session
> 模式,在
> > > > Client
> > > > > > 端编译失败抛出如上异常,不应该跟放不放在 lib 下有什么关系。这边听你说感觉也很奇怪,可能需要本地复现一下比较好判断。
> > > > > >
> > > > > > Best,
> > > > > > tison.
> > > > > >
> > > > > >
> > > > > > 宇张  于2020年4月23日周四 上午11:53写道:
> > > > > >
> > > > > >> 你的意思是,UberJar 不放在 lib 里,在用户程序里通过线程上下文 ClassLoader 能加载到
> > > > > >> KafkaTableSourceSinkFactory
> > > > > >> 吗?(同时 class loading 为 child-first)
> > > > > >> 》》是的
> > > > > >>
> > > > > >> On Thu, Apr 23, 2020 at 11:42 AM tison 
> > > wrote:
> > > > > >>
> > > > > >> > >》拿到ClassLoader后看下能不能取到KafkaTableSourceSinkFactory的class
> > > > > >> > >这个能拿到
> > > > > >> >
> > > > > >> > 你的意思是,UberJar 不放在 lib 里,在用户程序里通过线程上下文 ClassLoader 能加载到
> > > > > >> > KafkaTableSourceSinkFactory
> > > > > >> > 吗?(同时 class loading 为 child-first)
> > > > > >> >
> > > > > >> > 如果是这样,听起来 client 的 classloading 策略没啥问题,似乎是 SPI 加载那边的
> ClassLoader
> > > > > 有问题。之前
> > > > > >> > FileSystem 相关解析就出过类似的 ClassLoader 的 BUG
> > > > > >> >
> > > > > >> > Best,
> > > > > >> > tison.
> > > > > >> >
> > > > > >> >
> > > > > >> > 宇张  于2020年4月23日周四 上午11:36写道:
> > > > > >> >
> > > > > >> > > 我尝试进行了添加,程序依然无法运行,异常信息和上面一致,下面是我的shade配置:
> > > > > >> > >
> > > > > >> > > 
> > > > > >> > > org.apache.maven.plugins
> > > > > >> > > maven-shade-plugin
> > > > > >> > > 
> > > > > >> > > 
> > > > > >> > > 
> > > > > >> > > package
> > > > > >> > > 
> > > > > >> > > shade
> > > > > >> > > 
> > > > > >> > > 
> > > > > >> > > 
> > > > > >> > >  > > > > >> > >
> > > > > >> > >
> > > > > >> > >
> > > > > >> >
> > > > > >>
> > > > >
> > > >
> > >
> >
> implement

Re: 关于Flink1.10 Standalone 模式任务提交

2020-04-23 文章 宇张
》》加上 这部分配置之后对应的TableFactory文件里面有对应的KafkaFactory信息了,虽说程序还是无法运行,但是错误变为jar包冲突了,也就不是先前加载不到的错误;
但是感觉每次都配置这些貌似对用户不太友好。

org.codehaus.janino.CompilerFactory cannot be cast to
org.codehaus.commons.compiler.ICompilerFactory





org.apache.flink:flink-table-common
org.apache.flink:flink-table-api-java
org.apache.flink:flink-table-api-java-bridge_2.11
org.apache.flink:flink-table-planner-blink_2.11
org.apache.flink:flink-connector-kafka-0.11_2.11
org.apache.flink:flink-connector-kafka-0.9_2.11
org.apache.flink:flink-connector-kafka-0.10_2.11
org.apache.flink:flink-connector-kafka-base_2.11
org.apache.flink:flink-jdbc_2.11
org.apache.flink:flink-json








Apache Flink
UTF-8


com.akulaku.data.main.StreamMain




On Thu, Apr 23, 2020 at 4:07 PM Jingsong Li  wrote:

> Hi 张,
>
> 加上这个[1]试试:
>
> 
>   
>implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer"/>
>   
>implementation="org.apache.maven.plugins.shade.resource.ApacheNoticeResourceTransformer">
> Apache Flink
> UTF-8
>   
> 
>
>
> [1]https://github.com/apache/flink/blob/master/pom.xml#L1654
>
> Best,
> Jingsong Lee
>
> On Thu, Apr 23, 2020 at 3:56 PM 宇张  wrote:
>
> > 下面配置中,如果不加flink-json模块是可以打出kafkafactory的,加了flink-json模块就只剩下
> > JsonRowFormatFactory一个类,kafka的Factory就打印不出来了,所以是不是某一部分导致冲突了,
> > 但我看我先前flink1.9的工程,里面也无法打印kafkaFactory类,只有一个
> > GenericInMemoryCatalogFactory类,但flink1.9和1.10对比,看发布文档类加载策略有过改动
> >
> > org.apache.flink:flink-connector-kafka-0.11_2.11
> > org.apache.flink:flink-connector-kafka-base_2.11
> > org.apache.flink:flink-json
> >
> >
> > On Thu, Apr 23, 2020 at 3:43 PM Jingsong Li 
> > wrote:
> >
> > > > 如果是这样,听起来 client 的 classloading 策略没啥问题,似乎是 SPI 加载那边的 ClassLoader
> > > 有问题。之前FileSystem 相关解析就出过类似的 ClassLoader 的 BUG
> > >
> > > @tison 不管怎么样,也得保证jar里的SPI文件包含Kafka的类,不然SPI没法找
> > >
> > > @宇张 建议你仔细看下[1],这个pom是能打出正确的SPI文件的
> > >
> > > [1]
> > >
> > >
> >
> https://github.com/apache/flink/blob/master/flink-table/flink-table-uber-blink/pom.xml#L104
> > >
> > > Best,
> > > Jingsong Lee
> > >
> > > On Thu, Apr 23, 2020 at 3:35 PM tison  wrote:
> > >
> > > > 另外你 shaded 里面去 shaded com.ibm.icu 也意义不明...
> > > >
> > > > Best,
> > > > tison.
> > > >
> > > >
> > > > tison  于2020年4月23日周四 下午3:34写道:
> > > >
> > > > > 这个问题我建议你记一个 JIRA 然后提供一个可复现的程序。因为你如果是 Flink Standalone Session 模式,在
> > > Client
> > > > > 端编译失败抛出如上异常,不应该跟放不放在 lib 下有什么关系。这边听你说感觉也很奇怪,可能需要本地复现一下比较好判断。
> > > > >
> > > > > Best,
> > > > > tison.
> > > > >
> > > > >
> > > > > 宇张  于2020年4月23日周四 上午11:53写道:
> > > > >
> > > > >> 你的意思是,UberJar 不放在 lib 里,在用户程序里通过线程上下文 ClassLoader 能加载到
> > > > >> KafkaTableSourceSinkFactory
> > > > >> 吗?(同时 class loading 为 child-first)
> > > > >> 》》是的
> > > > >>
> > > > >> On Thu, Apr 23, 2020 at 11:42 AM tison 
> > wrote:
> > > > >>
> > > > >> > >》拿到ClassLoader后看下能不能取到KafkaTableSourceSinkFactory的class
> > > > >> > >这个能拿到
> > > > >> >
> > > > >> > 你的意思是,UberJar 不放在 lib 里,在用户程序里通过线程上下文 ClassLoader 能加载到
> > > > >> > KafkaTableSourceSinkFactory
> > > > >> > 吗?(同时 class loading 为 child-first)
> > > > >> >
> > > > >> > 如果是这样,听起来 client 的 classloading 策略没啥问题,似乎是 SPI 加载那边的 ClassLoader
> > > > 有问题。之前
> > > > >> > FileSystem 相关解析就出过类似的 ClassLoader 的 BUG
> > > > >> >
> > > > >> > Best,
> > > > >> > tison.
> > > > >> >
> > > > >> >
> > > > >> > 宇张  于2020年4月23日周四 上午11:36写道:
> > > > >> >
> > > > >> > > 我尝试进行了添加,程序依然无法运行,异常信息和上面一致,下面是我的shade配置:
> > > > >> > >
> > > > >> > > 
> > > > >> > > org.apache.maven.plugins
> > > > >> > > maven-shade-plugin
> > > > >> > > 
> > > > >> > > 
> > > > >> > > 
> > > > >> > > package
> > > > >> > > 
> > > > >> > > shade
> > > > >> > > 
> > > > >> > > 
> > > > >> > > 
> > > > >> > >  > > > >> > >
> > > > >> > >
> > > > >> > >
> > > > >> >
> > > > >>
> > > >
> > >
> >
> implementation="org.apache.maven.plugins.shade.resource.ManifestResourceTransformer">
> > > > >> > >
> > > > >> > > com.akulaku.data.main.StreamMain
> > > > >> > > 
> > > > >> > > 
> > > > >> > >
> > > > >> > > 
> > > > >> > > 
> > > > >> > > *:*
> > > > >> > > 
> > > > >> > > META-INF/*.SF
> > > > >> > > META-INF/*.DSA
> > > > >> > > META-INF/*.RSA
> > > > >> > > 
> > > > >> > > 
> > > > >> > > 
> > > > >> > >
> > > > >> > > 
> > > > >> > > 
> > > > >> > > 
> > > > >> > >
> > > > >> > > org.apache.flink:flink-table-common
> > > > >> > >
> > > > >> > > org.apache.flink:flink-table-api-java
> > > > >> > >
> > > > >> > >
> > > org.apache.flink:flink-table-api

Re: 关于Flink1.10 Standalone 模式任务提交

2020-04-23 文章 Jingsong Li
Hi 张,

加上这个[1]试试:


  
  
  
  
Apache Flink
UTF-8
  



[1]https://github.com/apache/flink/blob/master/pom.xml#L1654

Best,
Jingsong Lee

On Thu, Apr 23, 2020 at 3:56 PM 宇张  wrote:

> 下面配置中,如果不加flink-json模块是可以打出kafkafactory的,加了flink-json模块就只剩下
> JsonRowFormatFactory一个类,kafka的Factory就打印不出来了,所以是不是某一部分导致冲突了,
> 但我看我先前flink1.9的工程,里面也无法打印kafkaFactory类,只有一个
> GenericInMemoryCatalogFactory类,但flink1.9和1.10对比,看发布文档类加载策略有过改动
>
> org.apache.flink:flink-connector-kafka-0.11_2.11
> org.apache.flink:flink-connector-kafka-base_2.11
> org.apache.flink:flink-json
>
>
> On Thu, Apr 23, 2020 at 3:43 PM Jingsong Li 
> wrote:
>
> > > 如果是这样,听起来 client 的 classloading 策略没啥问题,似乎是 SPI 加载那边的 ClassLoader
> > 有问题。之前FileSystem 相关解析就出过类似的 ClassLoader 的 BUG
> >
> > @tison 不管怎么样,也得保证jar里的SPI文件包含Kafka的类,不然SPI没法找
> >
> > @宇张 建议你仔细看下[1],这个pom是能打出正确的SPI文件的
> >
> > [1]
> >
> >
> https://github.com/apache/flink/blob/master/flink-table/flink-table-uber-blink/pom.xml#L104
> >
> > Best,
> > Jingsong Lee
> >
> > On Thu, Apr 23, 2020 at 3:35 PM tison  wrote:
> >
> > > 另外你 shaded 里面去 shaded com.ibm.icu 也意义不明...
> > >
> > > Best,
> > > tison.
> > >
> > >
> > > tison  于2020年4月23日周四 下午3:34写道:
> > >
> > > > 这个问题我建议你记一个 JIRA 然后提供一个可复现的程序。因为你如果是 Flink Standalone Session 模式,在
> > Client
> > > > 端编译失败抛出如上异常,不应该跟放不放在 lib 下有什么关系。这边听你说感觉也很奇怪,可能需要本地复现一下比较好判断。
> > > >
> > > > Best,
> > > > tison.
> > > >
> > > >
> > > > 宇张  于2020年4月23日周四 上午11:53写道:
> > > >
> > > >> 你的意思是,UberJar 不放在 lib 里,在用户程序里通过线程上下文 ClassLoader 能加载到
> > > >> KafkaTableSourceSinkFactory
> > > >> 吗?(同时 class loading 为 child-first)
> > > >> 》》是的
> > > >>
> > > >> On Thu, Apr 23, 2020 at 11:42 AM tison 
> wrote:
> > > >>
> > > >> > >》拿到ClassLoader后看下能不能取到KafkaTableSourceSinkFactory的class
> > > >> > >这个能拿到
> > > >> >
> > > >> > 你的意思是,UberJar 不放在 lib 里,在用户程序里通过线程上下文 ClassLoader 能加载到
> > > >> > KafkaTableSourceSinkFactory
> > > >> > 吗?(同时 class loading 为 child-first)
> > > >> >
> > > >> > 如果是这样,听起来 client 的 classloading 策略没啥问题,似乎是 SPI 加载那边的 ClassLoader
> > > 有问题。之前
> > > >> > FileSystem 相关解析就出过类似的 ClassLoader 的 BUG
> > > >> >
> > > >> > Best,
> > > >> > tison.
> > > >> >
> > > >> >
> > > >> > 宇张  于2020年4月23日周四 上午11:36写道:
> > > >> >
> > > >> > > 我尝试进行了添加,程序依然无法运行,异常信息和上面一致,下面是我的shade配置:
> > > >> > >
> > > >> > > 
> > > >> > > org.apache.maven.plugins
> > > >> > > maven-shade-plugin
> > > >> > > 
> > > >> > > 
> > > >> > > 
> > > >> > > package
> > > >> > > 
> > > >> > > shade
> > > >> > > 
> > > >> > > 
> > > >> > > 
> > > >> > >  > > >> > >
> > > >> > >
> > > >> > >
> > > >> >
> > > >>
> > >
> >
> implementation="org.apache.maven.plugins.shade.resource.ManifestResourceTransformer">
> > > >> > >
> > > >> > > com.akulaku.data.main.StreamMain
> > > >> > > 
> > > >> > > 
> > > >> > >
> > > >> > > 
> > > >> > > 
> > > >> > > *:*
> > > >> > > 
> > > >> > > META-INF/*.SF
> > > >> > > META-INF/*.DSA
> > > >> > > META-INF/*.RSA
> > > >> > > 
> > > >> > > 
> > > >> > > 
> > > >> > >
> > > >> > > 
> > > >> > > 
> > > >> > > 
> > > >> > >
> > > >> > > org.apache.flink:flink-table-common
> > > >> > >
> > > >> > > org.apache.flink:flink-table-api-java
> > > >> > >
> > > >> > >
> > org.apache.flink:flink-table-api-java-bridge_2.11
> > > >> > >
> > > >> > >
> org.apache.flink:flink-table-planner-blink_2.11
> > > >> > >
> > > >> > >
> > org.apache.flink:flink-connector-kafka-0.11_2.11
> > > >> > >
> > > >> > >
> > org.apache.flink:flink-connector-kafka-base_2.11
> > > >> > >
> > >  org.apache.flink:flink-json
> > > >> > > 
> > > >> > > 
> > > >> > > 
> > > >> > > 
> > > >> > > 
> > > >> > > com.ibm.icu
> > > >> > >
> > > >> > >
> > > >>
> > org.apache.flink.table.shaded.com.ibm.icu
> > > >> > > 
> > > >> > > 
> > > >> > > 
> > > >> > > 
> > > >> > > 
> > > >> > > 
> > > >> > >
> > > >> > >
> > > >> > > On Thu, Apr 23, 2020 at 10:53 AM Jingsong Li <
> > > jingsongl...@gmail.com>
> > > >> > > wrote:
> > > >> > >
> > > >> > > > Hi,
> > > >> > > >
> > > >> > > > Flink的connector发现机制是通过java
> > > >> > spi服务发现机制的,所以你的services下文件不包含Kafka相关的内容就不会加载到。
> > > >> > > >
> > > >> > > > > 而且两种打包方式运行时是都能加载到KafkaFactory类文件的
> > > >> > > >
> > > >> > > > 只是类文件是没有用的,没地方引用到它。
> > > >> > > >
> > > >> > > > 你试试[1]中的方法?添加combine.children
> > > >> > > >
> > > >> > > > [1]
> > > >> > > >
> > > >> > > >
> > > >> > >
> > > >> >
> > > >>
> > >
> >
> https://github.com/apache/flink/blob/master/flink-table/flink-table-uber-

Re: 关于Flink1.10 Standalone 模式任务提交

2020-04-23 文章 宇张
下面配置中,如果不加flink-json模块是可以打出kafkafactory的,加了flink-json模块就只剩下
JsonRowFormatFactory一个类,kafka的Factory就打印不出来了,所以是不是某一部分导致冲突了,
但我看我先前flink1.9的工程,里面也无法打印kafkaFactory类,只有一个
GenericInMemoryCatalogFactory类,但flink1.9和1.10对比,看发布文档类加载策略有过改动

org.apache.flink:flink-connector-kafka-0.11_2.11
org.apache.flink:flink-connector-kafka-base_2.11
org.apache.flink:flink-json


On Thu, Apr 23, 2020 at 3:43 PM Jingsong Li  wrote:

> > 如果是这样,听起来 client 的 classloading 策略没啥问题,似乎是 SPI 加载那边的 ClassLoader
> 有问题。之前FileSystem 相关解析就出过类似的 ClassLoader 的 BUG
>
> @tison 不管怎么样,也得保证jar里的SPI文件包含Kafka的类,不然SPI没法找
>
> @宇张 建议你仔细看下[1],这个pom是能打出正确的SPI文件的
>
> [1]
>
> https://github.com/apache/flink/blob/master/flink-table/flink-table-uber-blink/pom.xml#L104
>
> Best,
> Jingsong Lee
>
> On Thu, Apr 23, 2020 at 3:35 PM tison  wrote:
>
> > 另外你 shaded 里面去 shaded com.ibm.icu 也意义不明...
> >
> > Best,
> > tison.
> >
> >
> > tison  于2020年4月23日周四 下午3:34写道:
> >
> > > 这个问题我建议你记一个 JIRA 然后提供一个可复现的程序。因为你如果是 Flink Standalone Session 模式,在
> Client
> > > 端编译失败抛出如上异常,不应该跟放不放在 lib 下有什么关系。这边听你说感觉也很奇怪,可能需要本地复现一下比较好判断。
> > >
> > > Best,
> > > tison.
> > >
> > >
> > > 宇张  于2020年4月23日周四 上午11:53写道:
> > >
> > >> 你的意思是,UberJar 不放在 lib 里,在用户程序里通过线程上下文 ClassLoader 能加载到
> > >> KafkaTableSourceSinkFactory
> > >> 吗?(同时 class loading 为 child-first)
> > >> 》》是的
> > >>
> > >> On Thu, Apr 23, 2020 at 11:42 AM tison  wrote:
> > >>
> > >> > >》拿到ClassLoader后看下能不能取到KafkaTableSourceSinkFactory的class
> > >> > >这个能拿到
> > >> >
> > >> > 你的意思是,UberJar 不放在 lib 里,在用户程序里通过线程上下文 ClassLoader 能加载到
> > >> > KafkaTableSourceSinkFactory
> > >> > 吗?(同时 class loading 为 child-first)
> > >> >
> > >> > 如果是这样,听起来 client 的 classloading 策略没啥问题,似乎是 SPI 加载那边的 ClassLoader
> > 有问题。之前
> > >> > FileSystem 相关解析就出过类似的 ClassLoader 的 BUG
> > >> >
> > >> > Best,
> > >> > tison.
> > >> >
> > >> >
> > >> > 宇张  于2020年4月23日周四 上午11:36写道:
> > >> >
> > >> > > 我尝试进行了添加,程序依然无法运行,异常信息和上面一致,下面是我的shade配置:
> > >> > >
> > >> > > 
> > >> > > org.apache.maven.plugins
> > >> > > maven-shade-plugin
> > >> > > 
> > >> > > 
> > >> > > 
> > >> > > package
> > >> > > 
> > >> > > shade
> > >> > > 
> > >> > > 
> > >> > > 
> > >> > >  > >> > >
> > >> > >
> > >> > >
> > >> >
> > >>
> >
> implementation="org.apache.maven.plugins.shade.resource.ManifestResourceTransformer">
> > >> > >
> > >> > > com.akulaku.data.main.StreamMain
> > >> > > 
> > >> > > 
> > >> > >
> > >> > > 
> > >> > > 
> > >> > > *:*
> > >> > > 
> > >> > > META-INF/*.SF
> > >> > > META-INF/*.DSA
> > >> > > META-INF/*.RSA
> > >> > > 
> > >> > > 
> > >> > > 
> > >> > >
> > >> > > 
> > >> > > 
> > >> > > 
> > >> > >
> > >> > > org.apache.flink:flink-table-common
> > >> > >
> > >> > > org.apache.flink:flink-table-api-java
> > >> > >
> > >> > >
> org.apache.flink:flink-table-api-java-bridge_2.11
> > >> > >
> > >> > > org.apache.flink:flink-table-planner-blink_2.11
> > >> > >
> > >> > >
> org.apache.flink:flink-connector-kafka-0.11_2.11
> > >> > >
> > >> > >
> org.apache.flink:flink-connector-kafka-base_2.11
> > >> > >
> >  org.apache.flink:flink-json
> > >> > > 
> > >> > > 
> > >> > > 
> > >> > > 
> > >> > > 
> > >> > > com.ibm.icu
> > >> > >
> > >> > >
> > >>
> org.apache.flink.table.shaded.com.ibm.icu
> > >> > > 
> > >> > > 
> > >> > > 
> > >> > > 
> > >> > > 
> > >> > > 
> > >> > >
> > >> > >
> > >> > > On Thu, Apr 23, 2020 at 10:53 AM Jingsong Li <
> > jingsongl...@gmail.com>
> > >> > > wrote:
> > >> > >
> > >> > > > Hi,
> > >> > > >
> > >> > > > Flink的connector发现机制是通过java
> > >> > spi服务发现机制的,所以你的services下文件不包含Kafka相关的内容就不会加载到。
> > >> > > >
> > >> > > > > 而且两种打包方式运行时是都能加载到KafkaFactory类文件的
> > >> > > >
> > >> > > > 只是类文件是没有用的,没地方引用到它。
> > >> > > >
> > >> > > > 你试试[1]中的方法?添加combine.children
> > >> > > >
> > >> > > > [1]
> > >> > > >
> > >> > > >
> > >> > >
> > >> >
> > >>
> >
> https://github.com/apache/flink/blob/master/flink-table/flink-table-uber-blink/pom.xml#L104
> > >> > > >
> > >> > > > Best,
> > >> > > > Jingsong Lee
> > >> > > >
> > >> > > > On Thu, Apr 23, 2020 at 10:37 AM 宇张 
> wrote:
> > >> > > >
> > >> > > > >
> > >> > > > >
> > >> > > >
> > >> > >
> > >> >
> > >>
> >
> 我这面采用shade打包方式进行了尝试,发现依然运行出错,运行错误日志与assembly打包产生的错误日志一致,就是上面提到的错误,而且shade和assembly打包产生的
> > >> > > > >
> > >> > > > >
> > >> > > >
> > >> > >
> > >> >
> > >>
> >
> META-INF/services/org.apache.flink.table.factories.TableFactory文件及里面的内容一致,而且两种打包方式运行时是都能加载到KafkaFactory类文件的,所以貌似不是打包导致的问题,而更像是bug
> > >

Re: 关于Flink1.10 Standalone 模式任务提交

2020-04-23 文章 Jingsong Li
> 如果是这样,听起来 client 的 classloading 策略没啥问题,似乎是 SPI 加载那边的 ClassLoader
有问题。之前FileSystem 相关解析就出过类似的 ClassLoader 的 BUG

@tison 不管怎么样,也得保证jar里的SPI文件包含Kafka的类,不然SPI没法找

@宇张 建议你仔细看下[1],这个pom是能打出正确的SPI文件的

[1]
https://github.com/apache/flink/blob/master/flink-table/flink-table-uber-blink/pom.xml#L104

Best,
Jingsong Lee

On Thu, Apr 23, 2020 at 3:35 PM tison  wrote:

> 另外你 shaded 里面去 shaded com.ibm.icu 也意义不明...
>
> Best,
> tison.
>
>
> tison  于2020年4月23日周四 下午3:34写道:
>
> > 这个问题我建议你记一个 JIRA 然后提供一个可复现的程序。因为你如果是 Flink Standalone Session 模式,在 Client
> > 端编译失败抛出如上异常,不应该跟放不放在 lib 下有什么关系。这边听你说感觉也很奇怪,可能需要本地复现一下比较好判断。
> >
> > Best,
> > tison.
> >
> >
> > 宇张  于2020年4月23日周四 上午11:53写道:
> >
> >> 你的意思是,UberJar 不放在 lib 里,在用户程序里通过线程上下文 ClassLoader 能加载到
> >> KafkaTableSourceSinkFactory
> >> 吗?(同时 class loading 为 child-first)
> >> 》》是的
> >>
> >> On Thu, Apr 23, 2020 at 11:42 AM tison  wrote:
> >>
> >> > >》拿到ClassLoader后看下能不能取到KafkaTableSourceSinkFactory的class
> >> > >这个能拿到
> >> >
> >> > 你的意思是,UberJar 不放在 lib 里,在用户程序里通过线程上下文 ClassLoader 能加载到
> >> > KafkaTableSourceSinkFactory
> >> > 吗?(同时 class loading 为 child-first)
> >> >
> >> > 如果是这样,听起来 client 的 classloading 策略没啥问题,似乎是 SPI 加载那边的 ClassLoader
> 有问题。之前
> >> > FileSystem 相关解析就出过类似的 ClassLoader 的 BUG
> >> >
> >> > Best,
> >> > tison.
> >> >
> >> >
> >> > 宇张  于2020年4月23日周四 上午11:36写道:
> >> >
> >> > > 我尝试进行了添加,程序依然无法运行,异常信息和上面一致,下面是我的shade配置:
> >> > >
> >> > > 
> >> > > org.apache.maven.plugins
> >> > > maven-shade-plugin
> >> > > 
> >> > > 
> >> > > 
> >> > > package
> >> > > 
> >> > > shade
> >> > > 
> >> > > 
> >> > > 
> >> > >  >> > >
> >> > >
> >> > >
> >> >
> >>
> implementation="org.apache.maven.plugins.shade.resource.ManifestResourceTransformer">
> >> > >
> >> > > com.akulaku.data.main.StreamMain
> >> > > 
> >> > > 
> >> > >
> >> > > 
> >> > > 
> >> > > *:*
> >> > > 
> >> > > META-INF/*.SF
> >> > > META-INF/*.DSA
> >> > > META-INF/*.RSA
> >> > > 
> >> > > 
> >> > > 
> >> > >
> >> > > 
> >> > > 
> >> > > 
> >> > >
> >> > > org.apache.flink:flink-table-common
> >> > >
> >> > > org.apache.flink:flink-table-api-java
> >> > >
> >> > > org.apache.flink:flink-table-api-java-bridge_2.11
> >> > >
> >> > > org.apache.flink:flink-table-planner-blink_2.11
> >> > >
> >> > > org.apache.flink:flink-connector-kafka-0.11_2.11
> >> > >
> >> > > org.apache.flink:flink-connector-kafka-base_2.11
> >> > >
>  org.apache.flink:flink-json
> >> > > 
> >> > > 
> >> > > 
> >> > > 
> >> > > 
> >> > > com.ibm.icu
> >> > >
> >> > >
> >> org.apache.flink.table.shaded.com.ibm.icu
> >> > > 
> >> > > 
> >> > > 
> >> > > 
> >> > > 
> >> > > 
> >> > >
> >> > >
> >> > > On Thu, Apr 23, 2020 at 10:53 AM Jingsong Li <
> jingsongl...@gmail.com>
> >> > > wrote:
> >> > >
> >> > > > Hi,
> >> > > >
> >> > > > Flink的connector发现机制是通过java
> >> > spi服务发现机制的,所以你的services下文件不包含Kafka相关的内容就不会加载到。
> >> > > >
> >> > > > > 而且两种打包方式运行时是都能加载到KafkaFactory类文件的
> >> > > >
> >> > > > 只是类文件是没有用的,没地方引用到它。
> >> > > >
> >> > > > 你试试[1]中的方法?添加combine.children
> >> > > >
> >> > > > [1]
> >> > > >
> >> > > >
> >> > >
> >> >
> >>
> https://github.com/apache/flink/blob/master/flink-table/flink-table-uber-blink/pom.xml#L104
> >> > > >
> >> > > > Best,
> >> > > > Jingsong Lee
> >> > > >
> >> > > > On Thu, Apr 23, 2020 at 10:37 AM 宇张  wrote:
> >> > > >
> >> > > > >
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
> 我这面采用shade打包方式进行了尝试,发现依然运行出错,运行错误日志与assembly打包产生的错误日志一致,就是上面提到的错误,而且shade和assembly打包产生的
> >> > > > >
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
> META-INF/services/org.apache.flink.table.factories.TableFactory文件及里面的内容一致,而且两种打包方式运行时是都能加载到KafkaFactory类文件的,所以貌似不是打包导致的问题,而更像是bug
> >> > > > > 下面是我maven插件配置:
> >> > > > >
> >> > > > > 
> >> > > > > 
> >> > > > > 
> >> > > > > 
> >> > > > > 
> >> > > > > 
> >> > > > > 
> >> > > > > 
> >> > > > > 
> >> > > > > 
> >> > > > > 
> >> > > > > 
> >> > > > > 
> >> > > > > 
> >> > > > > 
> >> > > > > 
> >> > > > > 
> >> > > > > 
> >> > > > > 
> >> > > > > 
> >> > > > > 
> >> > > > > 
> >> > > > > 
> >> > > > > 
> >> > > > > 
> >> > > > > 
> >> > > > > 
> >> > > > > 
> >> > > > > 
> >> > > > > 
> >> > > > > 
> >> > > > > 
> >> > > > > 
> >> > > > > org.apache.maven.plugins
> >> > > > > maven-shade-plugin
> >> > > > > 
> >> > > > > 
> >> > > > > 
> >> > > > > package
> >> > > > > 
> >> > > > > shade
> >> > > > > 
> >> > > >

Re: 关于Flink1.10 Standalone 模式任务提交

2020-04-23 文章 tison
另外你 shaded 里面去 shaded com.ibm.icu 也意义不明...

Best,
tison.


tison  于2020年4月23日周四 下午3:34写道:

> 这个问题我建议你记一个 JIRA 然后提供一个可复现的程序。因为你如果是 Flink Standalone Session 模式,在 Client
> 端编译失败抛出如上异常,不应该跟放不放在 lib 下有什么关系。这边听你说感觉也很奇怪,可能需要本地复现一下比较好判断。
>
> Best,
> tison.
>
>
> 宇张  于2020年4月23日周四 上午11:53写道:
>
>> 你的意思是,UberJar 不放在 lib 里,在用户程序里通过线程上下文 ClassLoader 能加载到
>> KafkaTableSourceSinkFactory
>> 吗?(同时 class loading 为 child-first)
>> 》》是的
>>
>> On Thu, Apr 23, 2020 at 11:42 AM tison  wrote:
>>
>> > >》拿到ClassLoader后看下能不能取到KafkaTableSourceSinkFactory的class
>> > >这个能拿到
>> >
>> > 你的意思是,UberJar 不放在 lib 里,在用户程序里通过线程上下文 ClassLoader 能加载到
>> > KafkaTableSourceSinkFactory
>> > 吗?(同时 class loading 为 child-first)
>> >
>> > 如果是这样,听起来 client 的 classloading 策略没啥问题,似乎是 SPI 加载那边的 ClassLoader 有问题。之前
>> > FileSystem 相关解析就出过类似的 ClassLoader 的 BUG
>> >
>> > Best,
>> > tison.
>> >
>> >
>> > 宇张  于2020年4月23日周四 上午11:36写道:
>> >
>> > > 我尝试进行了添加,程序依然无法运行,异常信息和上面一致,下面是我的shade配置:
>> > >
>> > > 
>> > > org.apache.maven.plugins
>> > > maven-shade-plugin
>> > > 
>> > > 
>> > > 
>> > > package
>> > > 
>> > > shade
>> > > 
>> > > 
>> > > 
>> > > > > >
>> > >
>> > >
>> >
>> implementation="org.apache.maven.plugins.shade.resource.ManifestResourceTransformer">
>> > >
>> > > com.akulaku.data.main.StreamMain
>> > > 
>> > > 
>> > >
>> > > 
>> > > 
>> > > *:*
>> > > 
>> > > META-INF/*.SF
>> > > META-INF/*.DSA
>> > > META-INF/*.RSA
>> > > 
>> > > 
>> > > 
>> > >
>> > > 
>> > > 
>> > > 
>> > >
>> > > org.apache.flink:flink-table-common
>> > >
>> > > org.apache.flink:flink-table-api-java
>> > >
>> > > org.apache.flink:flink-table-api-java-bridge_2.11
>> > >
>> > > org.apache.flink:flink-table-planner-blink_2.11
>> > >
>> > > org.apache.flink:flink-connector-kafka-0.11_2.11
>> > >
>> > > org.apache.flink:flink-connector-kafka-base_2.11
>> > > org.apache.flink:flink-json
>> > > 
>> > > 
>> > > 
>> > > 
>> > > 
>> > > com.ibm.icu
>> > >
>> > >
>> org.apache.flink.table.shaded.com.ibm.icu
>> > > 
>> > > 
>> > > 
>> > > 
>> > > 
>> > > 
>> > >
>> > >
>> > > On Thu, Apr 23, 2020 at 10:53 AM Jingsong Li 
>> > > wrote:
>> > >
>> > > > Hi,
>> > > >
>> > > > Flink的connector发现机制是通过java
>> > spi服务发现机制的,所以你的services下文件不包含Kafka相关的内容就不会加载到。
>> > > >
>> > > > > 而且两种打包方式运行时是都能加载到KafkaFactory类文件的
>> > > >
>> > > > 只是类文件是没有用的,没地方引用到它。
>> > > >
>> > > > 你试试[1]中的方法?添加combine.children
>> > > >
>> > > > [1]
>> > > >
>> > > >
>> > >
>> >
>> https://github.com/apache/flink/blob/master/flink-table/flink-table-uber-blink/pom.xml#L104
>> > > >
>> > > > Best,
>> > > > Jingsong Lee
>> > > >
>> > > > On Thu, Apr 23, 2020 at 10:37 AM 宇张  wrote:
>> > > >
>> > > > >
>> > > > >
>> > > >
>> > >
>> >
>> 我这面采用shade打包方式进行了尝试,发现依然运行出错,运行错误日志与assembly打包产生的错误日志一致,就是上面提到的错误,而且shade和assembly打包产生的
>> > > > >
>> > > > >
>> > > >
>> > >
>> >
>> META-INF/services/org.apache.flink.table.factories.TableFactory文件及里面的内容一致,而且两种打包方式运行时是都能加载到KafkaFactory类文件的,所以貌似不是打包导致的问题,而更像是bug
>> > > > > 下面是我maven插件配置:
>> > > > >
>> > > > > 
>> > > > > 
>> > > > > 
>> > > > > 
>> > > > > 
>> > > > > 
>> > > > > 
>> > > > > 
>> > > > > 
>> > > > > 
>> > > > > 
>> > > > > 
>> > > > > 
>> > > > > 
>> > > > > 
>> > > > > 
>> > > > > 
>> > > > > 
>> > > > > 
>> > > > > 
>> > > > > 
>> > > > > 
>> > > > > 
>> > > > > 
>> > > > > 
>> > > > > 
>> > > > > 
>> > > > > 
>> > > > > 
>> > > > > 
>> > > > > 
>> > > > > 
>> > > > > 
>> > > > > org.apache.maven.plugins
>> > > > > maven-shade-plugin
>> > > > > 
>> > > > > 
>> > > > > 
>> > > > > package
>> > > > > 
>> > > > > shade
>> > > > > 
>> > > > > 
>> > > > > 
>> > > > > > > > > >
>> > > > >
>> > > > >
>> > > >
>> > >
>> >
>> implementation="org.apache.maven.plugins.shade.resource.ManifestResourceTransformer">
>> > > > >
>> > > > > com.akulaku.data.main.StreamMain
>> > > > > 
>> > > > > 
>> > > > >
>> > > > > 
>> > > > > 
>> > > > > *:*
>> > > > > 
>> > > > > META-INF/*.SF
>> > > > > META-INF/*.DSA
>> > > > > META-INF/*.RSA
>> > > > > 
>> > > > > 
>> > > > >   

Re: 关于Flink1.10 Standalone 模式任务提交

2020-04-23 文章 tison
这个问题我建议你记一个 JIRA 然后提供一个可复现的程序。因为你如果是 Flink Standalone Session 模式,在 Client
端编译失败抛出如上异常,不应该跟放不放在 lib 下有什么关系。这边听你说感觉也很奇怪,可能需要本地复现一下比较好判断。

Best,
tison.


宇张  于2020年4月23日周四 上午11:53写道:

> 你的意思是,UberJar 不放在 lib 里,在用户程序里通过线程上下文 ClassLoader 能加载到
> KafkaTableSourceSinkFactory
> 吗?(同时 class loading 为 child-first)
> 》》是的
>
> On Thu, Apr 23, 2020 at 11:42 AM tison  wrote:
>
> > >》拿到ClassLoader后看下能不能取到KafkaTableSourceSinkFactory的class
> > >这个能拿到
> >
> > 你的意思是,UberJar 不放在 lib 里,在用户程序里通过线程上下文 ClassLoader 能加载到
> > KafkaTableSourceSinkFactory
> > 吗?(同时 class loading 为 child-first)
> >
> > 如果是这样,听起来 client 的 classloading 策略没啥问题,似乎是 SPI 加载那边的 ClassLoader 有问题。之前
> > FileSystem 相关解析就出过类似的 ClassLoader 的 BUG
> >
> > Best,
> > tison.
> >
> >
> > 宇张  于2020年4月23日周四 上午11:36写道:
> >
> > > 我尝试进行了添加,程序依然无法运行,异常信息和上面一致,下面是我的shade配置:
> > >
> > > 
> > > org.apache.maven.plugins
> > > maven-shade-plugin
> > > 
> > > 
> > > 
> > > package
> > > 
> > > shade
> > > 
> > > 
> > > 
> > >  > >
> > >
> > >
> >
> implementation="org.apache.maven.plugins.shade.resource.ManifestResourceTransformer">
> > >
> > > com.akulaku.data.main.StreamMain
> > > 
> > > 
> > >
> > > 
> > > 
> > > *:*
> > > 
> > > META-INF/*.SF
> > > META-INF/*.DSA
> > > META-INF/*.RSA
> > > 
> > > 
> > > 
> > >
> > > 
> > > 
> > > 
> > >
> > > org.apache.flink:flink-table-common
> > >
> > > org.apache.flink:flink-table-api-java
> > >
> > > org.apache.flink:flink-table-api-java-bridge_2.11
> > >
> > > org.apache.flink:flink-table-planner-blink_2.11
> > >
> > > org.apache.flink:flink-connector-kafka-0.11_2.11
> > >
> > > org.apache.flink:flink-connector-kafka-base_2.11
> > > org.apache.flink:flink-json
> > > 
> > > 
> > > 
> > > 
> > > 
> > > com.ibm.icu
> > >
> > >
> org.apache.flink.table.shaded.com.ibm.icu
> > > 
> > > 
> > > 
> > > 
> > > 
> > > 
> > >
> > >
> > > On Thu, Apr 23, 2020 at 10:53 AM Jingsong Li 
> > > wrote:
> > >
> > > > Hi,
> > > >
> > > > Flink的connector发现机制是通过java
> > spi服务发现机制的,所以你的services下文件不包含Kafka相关的内容就不会加载到。
> > > >
> > > > > 而且两种打包方式运行时是都能加载到KafkaFactory类文件的
> > > >
> > > > 只是类文件是没有用的,没地方引用到它。
> > > >
> > > > 你试试[1]中的方法?添加combine.children
> > > >
> > > > [1]
> > > >
> > > >
> > >
> >
> https://github.com/apache/flink/blob/master/flink-table/flink-table-uber-blink/pom.xml#L104
> > > >
> > > > Best,
> > > > Jingsong Lee
> > > >
> > > > On Thu, Apr 23, 2020 at 10:37 AM 宇张  wrote:
> > > >
> > > > >
> > > > >
> > > >
> > >
> >
> 我这面采用shade打包方式进行了尝试,发现依然运行出错,运行错误日志与assembly打包产生的错误日志一致,就是上面提到的错误,而且shade和assembly打包产生的
> > > > >
> > > > >
> > > >
> > >
> >
> META-INF/services/org.apache.flink.table.factories.TableFactory文件及里面的内容一致,而且两种打包方式运行时是都能加载到KafkaFactory类文件的,所以貌似不是打包导致的问题,而更像是bug
> > > > > 下面是我maven插件配置:
> > > > >
> > > > > 
> > > > > 
> > > > > 
> > > > > 
> > > > > 
> > > > > 
> > > > > 
> > > > > 
> > > > > 
> > > > > 
> > > > > 
> > > > > 
> > > > > 
> > > > > 
> > > > > 
> > > > > 
> > > > > 
> > > > > 
> > > > > 
> > > > > 
> > > > > 
> > > > > 
> > > > > 
> > > > > 
> > > > > 
> > > > > 
> > > > > 
> > > > > 
> > > > > 
> > > > > 
> > > > > 
> > > > > 
> > > > > 
> > > > > org.apache.maven.plugins
> > > > > maven-shade-plugin
> > > > > 
> > > > > 
> > > > > 
> > > > > package
> > > > > 
> > > > > shade
> > > > > 
> > > > > 
> > > > > 
> > > > >  > > > >
> > > > >
> > > > >
> > > >
> > >
> >
> implementation="org.apache.maven.plugins.shade.resource.ManifestResourceTransformer">
> > > > >
> > > > > com.akulaku.data.main.StreamMain
> > > > > 
> > > > > 
> > > > >
> > > > > 
> > > > > 
> > > > > *:*
> > > > > 
> > > > > META-INF/*.SF
> > > > > META-INF/*.DSA
> > > > > META-INF/*.RSA
> > > > > 
> > > > > 
> > > > > 
> > > > > 
> > > > > 
> > > > > 
> > > > > 
> > > > >
> > > > >
> > > > > On Wed, Apr 22, 2020 at 8:00 PM Jingsong Li <
> jingsongl...@gmail.com>
> > > > > wrote:
> > > > >
> > > > > > Hi,
> > > > > >
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> 如果org.

Re: 关于Flink1.10 Standalone 模式任务提交

2020-04-22 文章 宇张
你的意思是,UberJar 不放在 lib 里,在用户程序里通过线程上下文 ClassLoader 能加载到
KafkaTableSourceSinkFactory
吗?(同时 class loading 为 child-first)
》》是的

On Thu, Apr 23, 2020 at 11:42 AM tison  wrote:

> >》拿到ClassLoader后看下能不能取到KafkaTableSourceSinkFactory的class
> >这个能拿到
>
> 你的意思是,UberJar 不放在 lib 里,在用户程序里通过线程上下文 ClassLoader 能加载到
> KafkaTableSourceSinkFactory
> 吗?(同时 class loading 为 child-first)
>
> 如果是这样,听起来 client 的 classloading 策略没啥问题,似乎是 SPI 加载那边的 ClassLoader 有问题。之前
> FileSystem 相关解析就出过类似的 ClassLoader 的 BUG
>
> Best,
> tison.
>
>
> 宇张  于2020年4月23日周四 上午11:36写道:
>
> > 我尝试进行了添加,程序依然无法运行,异常信息和上面一致,下面是我的shade配置:
> >
> > 
> > org.apache.maven.plugins
> > maven-shade-plugin
> > 
> > 
> > 
> > package
> > 
> > shade
> > 
> > 
> > 
> >  >
> >
> >
> implementation="org.apache.maven.plugins.shade.resource.ManifestResourceTransformer">
> >
> > com.akulaku.data.main.StreamMain
> > 
> > 
> >
> > 
> > 
> > *:*
> > 
> > META-INF/*.SF
> > META-INF/*.DSA
> > META-INF/*.RSA
> > 
> > 
> > 
> >
> > 
> > 
> > 
> >
> > org.apache.flink:flink-table-common
> >
> > org.apache.flink:flink-table-api-java
> >
> > org.apache.flink:flink-table-api-java-bridge_2.11
> >
> > org.apache.flink:flink-table-planner-blink_2.11
> >
> > org.apache.flink:flink-connector-kafka-0.11_2.11
> >
> > org.apache.flink:flink-connector-kafka-base_2.11
> > org.apache.flink:flink-json
> > 
> > 
> > 
> > 
> > 
> > com.ibm.icu
> >
> > org.apache.flink.table.shaded.com.ibm.icu
> > 
> > 
> > 
> > 
> > 
> > 
> >
> >
> > On Thu, Apr 23, 2020 at 10:53 AM Jingsong Li 
> > wrote:
> >
> > > Hi,
> > >
> > > Flink的connector发现机制是通过java
> spi服务发现机制的,所以你的services下文件不包含Kafka相关的内容就不会加载到。
> > >
> > > > 而且两种打包方式运行时是都能加载到KafkaFactory类文件的
> > >
> > > 只是类文件是没有用的,没地方引用到它。
> > >
> > > 你试试[1]中的方法?添加combine.children
> > >
> > > [1]
> > >
> > >
> >
> https://github.com/apache/flink/blob/master/flink-table/flink-table-uber-blink/pom.xml#L104
> > >
> > > Best,
> > > Jingsong Lee
> > >
> > > On Thu, Apr 23, 2020 at 10:37 AM 宇张  wrote:
> > >
> > > >
> > > >
> > >
> >
> 我这面采用shade打包方式进行了尝试,发现依然运行出错,运行错误日志与assembly打包产生的错误日志一致,就是上面提到的错误,而且shade和assembly打包产生的
> > > >
> > > >
> > >
> >
> META-INF/services/org.apache.flink.table.factories.TableFactory文件及里面的内容一致,而且两种打包方式运行时是都能加载到KafkaFactory类文件的,所以貌似不是打包导致的问题,而更像是bug
> > > > 下面是我maven插件配置:
> > > >
> > > > 
> > > > 
> > > > 
> > > > 
> > > > 
> > > > 
> > > > 
> > > > 
> > > > 
> > > > 
> > > > 
> > > > 
> > > > 
> > > > 
> > > > 
> > > > 
> > > > 
> > > > 
> > > > 
> > > > 
> > > > 
> > > > 
> > > > 
> > > > 
> > > > 
> > > > 
> > > > 
> > > > 
> > > > 
> > > > 
> > > > 
> > > > 
> > > > 
> > > > org.apache.maven.plugins
> > > > maven-shade-plugin
> > > > 
> > > > 
> > > > 
> > > > package
> > > > 
> > > > shade
> > > > 
> > > > 
> > > > 
> > > >  > > >
> > > >
> > > >
> > >
> >
> implementation="org.apache.maven.plugins.shade.resource.ManifestResourceTransformer">
> > > >
> > > > com.akulaku.data.main.StreamMain
> > > > 
> > > > 
> > > >
> > > > 
> > > > 
> > > > *:*
> > > > 
> > > > META-INF/*.SF
> > > > META-INF/*.DSA
> > > > META-INF/*.RSA
> > > > 
> > > > 
> > > > 
> > > > 
> > > > 
> > > > 
> > > > 
> > > >
> > > >
> > > > On Wed, Apr 22, 2020 at 8:00 PM Jingsong Li 
> > > > wrote:
> > > >
> > > > > Hi,
> > > > >
> > > > >
> > > > >
> > > >
> > >
> >
> 如果org.apache.flink.table.factories.TableFactory里面没有KafkaTableSourceSinkFactory,那就是打包有问题。不清楚1.9的是怎么运行起来的,但是所有的jar的meta-inf-services文件都没有KafkaTableSourceSinkFactory,那也不应该能运行起来的。
> > > > >
> > > > > 推荐打包方式用shade,shade会merge meta-inf-services的文件的。
> > > > >
> > > > > Best,
> > > > > Jingsong Lee
> > > > >
> > > > > On Wed, Apr 22, 2020 at 7:31 PM 宇张  wrote:
> > > > >
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> 》也确认下org.apache.flink.table.factories.TableFactory的内容,里面有没有KafkaTableSourceSinkFactory
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> 这个我看了一下我先前flink1.9的工程,应用程序Jar里面也是没有这个类的,但是程序运行加载是没问题的,这么对比貌似就不是maven打包的问题了。
> > >

Re: 关于Flink1.10 Standalone 模式任务提交

2020-04-22 文章 tison
>》拿到ClassLoader后看下能不能取到KafkaTableSourceSinkFactory的class
>这个能拿到

你的意思是,UberJar 不放在 lib 里,在用户程序里通过线程上下文 ClassLoader 能加载到
KafkaTableSourceSinkFactory
吗?(同时 class loading 为 child-first)

如果是这样,听起来 client 的 classloading 策略没啥问题,似乎是 SPI 加载那边的 ClassLoader 有问题。之前
FileSystem 相关解析就出过类似的 ClassLoader 的 BUG

Best,
tison.


宇张  于2020年4月23日周四 上午11:36写道:

> 我尝试进行了添加,程序依然无法运行,异常信息和上面一致,下面是我的shade配置:
>
> 
> org.apache.maven.plugins
> maven-shade-plugin
> 
> 
> 
> package
> 
> shade
> 
> 
> 
> 
>
> implementation="org.apache.maven.plugins.shade.resource.ManifestResourceTransformer">
>
> com.akulaku.data.main.StreamMain
> 
> 
>
> 
> 
> *:*
> 
> META-INF/*.SF
> META-INF/*.DSA
> META-INF/*.RSA
> 
> 
> 
>
> 
> 
> 
>
> org.apache.flink:flink-table-common
>
> org.apache.flink:flink-table-api-java
>
> org.apache.flink:flink-table-api-java-bridge_2.11
>
> org.apache.flink:flink-table-planner-blink_2.11
>
> org.apache.flink:flink-connector-kafka-0.11_2.11
>
> org.apache.flink:flink-connector-kafka-base_2.11
> org.apache.flink:flink-json
> 
> 
> 
> 
> 
> com.ibm.icu
>
> org.apache.flink.table.shaded.com.ibm.icu
> 
> 
> 
> 
> 
> 
>
>
> On Thu, Apr 23, 2020 at 10:53 AM Jingsong Li 
> wrote:
>
> > Hi,
> >
> > Flink的connector发现机制是通过java spi服务发现机制的,所以你的services下文件不包含Kafka相关的内容就不会加载到。
> >
> > > 而且两种打包方式运行时是都能加载到KafkaFactory类文件的
> >
> > 只是类文件是没有用的,没地方引用到它。
> >
> > 你试试[1]中的方法?添加combine.children
> >
> > [1]
> >
> >
> https://github.com/apache/flink/blob/master/flink-table/flink-table-uber-blink/pom.xml#L104
> >
> > Best,
> > Jingsong Lee
> >
> > On Thu, Apr 23, 2020 at 10:37 AM 宇张  wrote:
> >
> > >
> > >
> >
> 我这面采用shade打包方式进行了尝试,发现依然运行出错,运行错误日志与assembly打包产生的错误日志一致,就是上面提到的错误,而且shade和assembly打包产生的
> > >
> > >
> >
> META-INF/services/org.apache.flink.table.factories.TableFactory文件及里面的内容一致,而且两种打包方式运行时是都能加载到KafkaFactory类文件的,所以貌似不是打包导致的问题,而更像是bug
> > > 下面是我maven插件配置:
> > >
> > > 
> > > 
> > > 
> > > 
> > > 
> > > 
> > > 
> > > 
> > > 
> > > 
> > > 
> > > 
> > > 
> > > 
> > > 
> > > 
> > > 
> > > 
> > > 
> > > 
> > > 
> > > 
> > > 
> > > 
> > > 
> > > 
> > > 
> > > 
> > > 
> > > 
> > > 
> > > 
> > > 
> > > org.apache.maven.plugins
> > > maven-shade-plugin
> > > 
> > > 
> > > 
> > > package
> > > 
> > > shade
> > > 
> > > 
> > > 
> > >  > >
> > >
> > >
> >
> implementation="org.apache.maven.plugins.shade.resource.ManifestResourceTransformer">
> > >
> > > com.akulaku.data.main.StreamMain
> > > 
> > > 
> > >
> > > 
> > > 
> > > *:*
> > > 
> > > META-INF/*.SF
> > > META-INF/*.DSA
> > > META-INF/*.RSA
> > > 
> > > 
> > > 
> > > 
> > > 
> > > 
> > > 
> > >
> > >
> > > On Wed, Apr 22, 2020 at 8:00 PM Jingsong Li 
> > > wrote:
> > >
> > > > Hi,
> > > >
> > > >
> > > >
> > >
> >
> 如果org.apache.flink.table.factories.TableFactory里面没有KafkaTableSourceSinkFactory,那就是打包有问题。不清楚1.9的是怎么运行起来的,但是所有的jar的meta-inf-services文件都没有KafkaTableSourceSinkFactory,那也不应该能运行起来的。
> > > >
> > > > 推荐打包方式用shade,shade会merge meta-inf-services的文件的。
> > > >
> > > > Best,
> > > > Jingsong Lee
> > > >
> > > > On Wed, Apr 22, 2020 at 7:31 PM 宇张  wrote:
> > > >
> > > > >
> > > > >
> > > >
> > >
> >
> 》也确认下org.apache.flink.table.factories.TableFactory的内容,里面有没有KafkaTableSourceSinkFactory
> > > > >
> > > > >
> > > >
> > >
> >
> 这个我看了一下我先前flink1.9的工程,应用程序Jar里面也是没有这个类的,但是程序运行加载是没问题的,这么对比貌似就不是maven打包的问题了。
> > > > >
> > > > > On Wed, Apr 22, 2020 at 7:22 PM 宇张  wrote:
> > > > >
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> 》也确认下org.apache.flink.table.factories.TableFactory的内容,里面有没有KafkaTableSourceSinkFactory
> > > > > > 这个没有,只有org.apache.flink.formats.json.JsonRowFormatFactory
> > > > > > 》拿到ClassLoader后看下能不能取到KafkaTableSourceSinkFactory的class
> > > > > > 这个能拿到
> > > > > >
> > > > > > 这么看来 貌似是 mvn打包有问题:
> > > > > > mvn clean package -DskipTests
> > > > > > 依赖范围为默认
> > > > > >
> > > > > >
> > > > > > On Wed, Apr 22, 2020 at 7:05 PM Jingsong Li <
> > jingsongl...@gmail.com>
> > > > > > wrote:
> > > > > >
> > >

Re: 关于Flink1.10 Standalone 模式任务提交

2020-04-22 文章 宇张
我尝试进行了添加,程序依然无法运行,异常信息和上面一致,下面是我的shade配置:


org.apache.maven.plugins
maven-shade-plugin



package

shade




com.akulaku.data.main.StreamMain





*:*

META-INF/*.SF
META-INF/*.DSA
META-INF/*.RSA







org.apache.flink:flink-table-common
org.apache.flink:flink-table-api-java

org.apache.flink:flink-table-api-java-bridge_2.11

org.apache.flink:flink-table-planner-blink_2.11

org.apache.flink:flink-connector-kafka-0.11_2.11

org.apache.flink:flink-connector-kafka-base_2.11
org.apache.flink:flink-json





com.ibm.icu

org.apache.flink.table.shaded.com.ibm.icu








On Thu, Apr 23, 2020 at 10:53 AM Jingsong Li  wrote:

> Hi,
>
> Flink的connector发现机制是通过java spi服务发现机制的,所以你的services下文件不包含Kafka相关的内容就不会加载到。
>
> > 而且两种打包方式运行时是都能加载到KafkaFactory类文件的
>
> 只是类文件是没有用的,没地方引用到它。
>
> 你试试[1]中的方法?添加combine.children
>
> [1]
>
> https://github.com/apache/flink/blob/master/flink-table/flink-table-uber-blink/pom.xml#L104
>
> Best,
> Jingsong Lee
>
> On Thu, Apr 23, 2020 at 10:37 AM 宇张  wrote:
>
> >
> >
> 我这面采用shade打包方式进行了尝试,发现依然运行出错,运行错误日志与assembly打包产生的错误日志一致,就是上面提到的错误,而且shade和assembly打包产生的
> >
> >
> META-INF/services/org.apache.flink.table.factories.TableFactory文件及里面的内容一致,而且两种打包方式运行时是都能加载到KafkaFactory类文件的,所以貌似不是打包导致的问题,而更像是bug
> > 下面是我maven插件配置:
> >
> > 
> > 
> > 
> > 
> > 
> > 
> > 
> > 
> > 
> > 
> > 
> > 
> > 
> > 
> > 
> > 
> > 
> > 
> > 
> > 
> > 
> > 
> > 
> > 
> > 
> > 
> > 
> > 
> > 
> > 
> > 
> > 
> > 
> > org.apache.maven.plugins
> > maven-shade-plugin
> > 
> > 
> > 
> > package
> > 
> > shade
> > 
> > 
> > 
> >  >
> >
> >
> implementation="org.apache.maven.plugins.shade.resource.ManifestResourceTransformer">
> >
> > com.akulaku.data.main.StreamMain
> > 
> > 
> >
> > 
> > 
> > *:*
> > 
> > META-INF/*.SF
> > META-INF/*.DSA
> > META-INF/*.RSA
> > 
> > 
> > 
> > 
> > 
> > 
> > 
> >
> >
> > On Wed, Apr 22, 2020 at 8:00 PM Jingsong Li 
> > wrote:
> >
> > > Hi,
> > >
> > >
> > >
> >
> 如果org.apache.flink.table.factories.TableFactory里面没有KafkaTableSourceSinkFactory,那就是打包有问题。不清楚1.9的是怎么运行起来的,但是所有的jar的meta-inf-services文件都没有KafkaTableSourceSinkFactory,那也不应该能运行起来的。
> > >
> > > 推荐打包方式用shade,shade会merge meta-inf-services的文件的。
> > >
> > > Best,
> > > Jingsong Lee
> > >
> > > On Wed, Apr 22, 2020 at 7:31 PM 宇张  wrote:
> > >
> > > >
> > > >
> > >
> >
> 》也确认下org.apache.flink.table.factories.TableFactory的内容,里面有没有KafkaTableSourceSinkFactory
> > > >
> > > >
> > >
> >
> 这个我看了一下我先前flink1.9的工程,应用程序Jar里面也是没有这个类的,但是程序运行加载是没问题的,这么对比貌似就不是maven打包的问题了。
> > > >
> > > > On Wed, Apr 22, 2020 at 7:22 PM 宇张  wrote:
> > > >
> > > > >
> > > > >
> > > >
> > >
> >
> 》也确认下org.apache.flink.table.factories.TableFactory的内容,里面有没有KafkaTableSourceSinkFactory
> > > > > 这个没有,只有org.apache.flink.formats.json.JsonRowFormatFactory
> > > > > 》拿到ClassLoader后看下能不能取到KafkaTableSourceSinkFactory的class
> > > > > 这个能拿到
> > > > >
> > > > > 这么看来 貌似是 mvn打包有问题:
> > > > > mvn clean package -DskipTests
> > > > > 依赖范围为默认
> > > > >
> > > > >
> > > > > On Wed, Apr 22, 2020 at 7:05 PM Jingsong Li <
> jingsongl...@gmail.com>
> > > > > wrote:
> > > > >
> > > > >> Hi,
> > > > >>
> > > > >>
> > > > >>
> > > >
> > >
> >
> 也确认下org.apache.flink.table.factories.TableFactory的内容,里面有没有KafkaTableSourceSinkFactory
> > > > >>
> > > > >> >
> > > 这个指的是打印当前线程的classloader嘛Thread.currentThread().getContextClassLoader()
> > > > >> 是的,拿到ClassLoader后看下能不能取到KafkaTableSourceSinkFactory的class
> > > > >>
> > > > >> Best,
> > > > >> Jingsong Lee
> > > > >>
> > > > >> On Wed, Apr 22, 2020 at 7:00 PM 宇张  wrote:
> > > > >>
> > > > >> > 看下你打包的 UberJar 里有没一个内容包括
> > > > >> > 1、下面这个文件是存在的
> > > > >> >
> > > >
> org.apache.flink.streaming.connectors.kafka.KafkaTableSourceSinkFactory
> > > > >> > 的文件
> > > > >> > META-INF/services/org.apache.flink.table.factories.TableFactory
> > > > >> > 2、flink版本1.10,Standalone模式启动服务(start-cluster.sh),flink
> > > > >> > run运行(/s

Re: 关于Flink1.10 Standalone 模式任务提交

2020-04-22 文章 Jingsong Li
Hi,

Flink的connector发现机制是通过java spi服务发现机制的,所以你的services下文件不包含Kafka相关的内容就不会加载到。

> 而且两种打包方式运行时是都能加载到KafkaFactory类文件的

只是类文件是没有用的,没地方引用到它。

你试试[1]中的方法?添加combine.children

[1]
https://github.com/apache/flink/blob/master/flink-table/flink-table-uber-blink/pom.xml#L104

Best,
Jingsong Lee

On Thu, Apr 23, 2020 at 10:37 AM 宇张  wrote:

>
> 我这面采用shade打包方式进行了尝试,发现依然运行出错,运行错误日志与assembly打包产生的错误日志一致,就是上面提到的错误,而且shade和assembly打包产生的
>
> META-INF/services/org.apache.flink.table.factories.TableFactory文件及里面的内容一致,而且两种打包方式运行时是都能加载到KafkaFactory类文件的,所以貌似不是打包导致的问题,而更像是bug
> 下面是我maven插件配置:
>
> 
> 
> 
> 
> 
> 
> 
> 
> 
> 
> 
> 
> 
> 
> 
> 
> 
> 
> 
> 
> 
> 
> 
> 
> 
> 
> 
> 
> 
> 
> 
> 
> 
> org.apache.maven.plugins
> maven-shade-plugin
> 
> 
> 
> package
> 
> shade
> 
> 
> 
> 
>
> implementation="org.apache.maven.plugins.shade.resource.ManifestResourceTransformer">
>
> com.akulaku.data.main.StreamMain
> 
> 
>
> 
> 
> *:*
> 
> META-INF/*.SF
> META-INF/*.DSA
> META-INF/*.RSA
> 
> 
> 
> 
> 
> 
> 
>
>
> On Wed, Apr 22, 2020 at 8:00 PM Jingsong Li 
> wrote:
>
> > Hi,
> >
> >
> >
> 如果org.apache.flink.table.factories.TableFactory里面没有KafkaTableSourceSinkFactory,那就是打包有问题。不清楚1.9的是怎么运行起来的,但是所有的jar的meta-inf-services文件都没有KafkaTableSourceSinkFactory,那也不应该能运行起来的。
> >
> > 推荐打包方式用shade,shade会merge meta-inf-services的文件的。
> >
> > Best,
> > Jingsong Lee
> >
> > On Wed, Apr 22, 2020 at 7:31 PM 宇张  wrote:
> >
> > >
> > >
> >
> 》也确认下org.apache.flink.table.factories.TableFactory的内容,里面有没有KafkaTableSourceSinkFactory
> > >
> > >
> >
> 这个我看了一下我先前flink1.9的工程,应用程序Jar里面也是没有这个类的,但是程序运行加载是没问题的,这么对比貌似就不是maven打包的问题了。
> > >
> > > On Wed, Apr 22, 2020 at 7:22 PM 宇张  wrote:
> > >
> > > >
> > > >
> > >
> >
> 》也确认下org.apache.flink.table.factories.TableFactory的内容,里面有没有KafkaTableSourceSinkFactory
> > > > 这个没有,只有org.apache.flink.formats.json.JsonRowFormatFactory
> > > > 》拿到ClassLoader后看下能不能取到KafkaTableSourceSinkFactory的class
> > > > 这个能拿到
> > > >
> > > > 这么看来 貌似是 mvn打包有问题:
> > > > mvn clean package -DskipTests
> > > > 依赖范围为默认
> > > >
> > > >
> > > > On Wed, Apr 22, 2020 at 7:05 PM Jingsong Li 
> > > > wrote:
> > > >
> > > >> Hi,
> > > >>
> > > >>
> > > >>
> > >
> >
> 也确认下org.apache.flink.table.factories.TableFactory的内容,里面有没有KafkaTableSourceSinkFactory
> > > >>
> > > >> >
> > 这个指的是打印当前线程的classloader嘛Thread.currentThread().getContextClassLoader()
> > > >> 是的,拿到ClassLoader后看下能不能取到KafkaTableSourceSinkFactory的class
> > > >>
> > > >> Best,
> > > >> Jingsong Lee
> > > >>
> > > >> On Wed, Apr 22, 2020 at 7:00 PM 宇张  wrote:
> > > >>
> > > >> > 看下你打包的 UberJar 里有没一个内容包括
> > > >> > 1、下面这个文件是存在的
> > > >> >
> > > org.apache.flink.streaming.connectors.kafka.KafkaTableSourceSinkFactory
> > > >> > 的文件
> > > >> > META-INF/services/org.apache.flink.table.factories.TableFactory
> > > >> > 2、flink版本1.10,Standalone模式启动服务(start-cluster.sh),flink
> > > >> > run运行(/software/flink-1.10.0/bin/flink run -c
> > com.data.main.StreamMain
> > > >> > ./flink_1.10_test-1.0-jar-with-dependencies.jar)
> > > >> > 3、再确认下"TableEnvironmentImpl.sqlQuery"调用时候的ThreadClassLoader?
> > > >> >
> > 这个指的是打印当前线程的classloader嘛Thread.currentThread().getContextClassLoader()
> > > >> >
> > > >> >
> > > >> >
> > > >> > On Wed, Apr 22, 2020 at 6:00 PM Jingsong Li <
> jingsongl...@gmail.com
> > >
> > > >> > wrote:
> > > >> >
> > > >> > > Hi,
> > > >> > >
> > > >> > > 先确认下你的Jar包里有没有 meta-inf-services的文件?里面确定有Kafka?
> > > >> > >
> > > >> > > 如果有,再确认下"TableEnvironmentImpl.sqlQuery"调用时候的ThreadClassLoader?
> > > >> > > 因为现在默认是通过ThreadClassLoader来获取Factory的。
> > > >> > >
> > > >> > > Best,
> > > >> > > Jingsong Lee
> > > >> > >
> > > >> > > On Wed, Apr 22, 2020 at 5:30 PM 宇张  wrote:
> > > >> > >
> > > >> > > > 我这面使用Standalone模式运行Flink任务,但是Uber
> > > >> > > > Jar里面的TableSourceFactory不能被加载,即使设置了classloader.resolve-order:
> > > >> > > > child-first,只有放在lib目录才能加载得到,我看发布文档跟改了类加载策略,但是我不知道为什么Uber
> > > >> > > Jar里面的Factory不能被加载
> > > >> > > > Flink Client respects Classloading Policy (FLINK-13749
> > > >> > > > )
> > > >> > > > <
> > > >> > > >
> > > >> > >
> > > >> >
> > > >>
> > >
> >
> https://ci.apache.org/projects/flink/flink-docs-release-1.10/release-notes/flink-1.10.html#flink-client-respects-classloading-policy-flink-13749
> > > >> > > > >
> > > >> > > >
> > > >> > > > The Flink client now also respects the configured classloading
> > > >> policy,
> > > >> > > > i.e., parent-first or child-first classloading. Previously,
> only
> > > >> > cluster
> > > >> > > > components such as the job manager or task manager supported
> > this
> > >

Re: 关于Flink1.10 Standalone 模式任务提交

2020-04-22 文章 宇张
我这面采用shade打包方式进行了尝试,发现依然运行出错,运行错误日志与assembly打包产生的错误日志一致,就是上面提到的错误,而且shade和assembly打包产生的
META-INF/services/org.apache.flink.table.factories.TableFactory文件及里面的内容一致,而且两种打包方式运行时是都能加载到KafkaFactory类文件的,所以貌似不是打包导致的问题,而更像是bug
下面是我maven插件配置:


































org.apache.maven.plugins
maven-shade-plugin



package

shade




com.akulaku.data.main.StreamMain





*:*

META-INF/*.SF
META-INF/*.DSA
META-INF/*.RSA









On Wed, Apr 22, 2020 at 8:00 PM Jingsong Li  wrote:

> Hi,
>
>
> 如果org.apache.flink.table.factories.TableFactory里面没有KafkaTableSourceSinkFactory,那就是打包有问题。不清楚1.9的是怎么运行起来的,但是所有的jar的meta-inf-services文件都没有KafkaTableSourceSinkFactory,那也不应该能运行起来的。
>
> 推荐打包方式用shade,shade会merge meta-inf-services的文件的。
>
> Best,
> Jingsong Lee
>
> On Wed, Apr 22, 2020 at 7:31 PM 宇张  wrote:
>
> >
> >
> 》也确认下org.apache.flink.table.factories.TableFactory的内容,里面有没有KafkaTableSourceSinkFactory
> >
> >
> 这个我看了一下我先前flink1.9的工程,应用程序Jar里面也是没有这个类的,但是程序运行加载是没问题的,这么对比貌似就不是maven打包的问题了。
> >
> > On Wed, Apr 22, 2020 at 7:22 PM 宇张  wrote:
> >
> > >
> > >
> >
> 》也确认下org.apache.flink.table.factories.TableFactory的内容,里面有没有KafkaTableSourceSinkFactory
> > > 这个没有,只有org.apache.flink.formats.json.JsonRowFormatFactory
> > > 》拿到ClassLoader后看下能不能取到KafkaTableSourceSinkFactory的class
> > > 这个能拿到
> > >
> > > 这么看来 貌似是 mvn打包有问题:
> > > mvn clean package -DskipTests
> > > 依赖范围为默认
> > >
> > >
> > > On Wed, Apr 22, 2020 at 7:05 PM Jingsong Li 
> > > wrote:
> > >
> > >> Hi,
> > >>
> > >>
> > >>
> >
> 也确认下org.apache.flink.table.factories.TableFactory的内容,里面有没有KafkaTableSourceSinkFactory
> > >>
> > >> >
> 这个指的是打印当前线程的classloader嘛Thread.currentThread().getContextClassLoader()
> > >> 是的,拿到ClassLoader后看下能不能取到KafkaTableSourceSinkFactory的class
> > >>
> > >> Best,
> > >> Jingsong Lee
> > >>
> > >> On Wed, Apr 22, 2020 at 7:00 PM 宇张  wrote:
> > >>
> > >> > 看下你打包的 UberJar 里有没一个内容包括
> > >> > 1、下面这个文件是存在的
> > >> >
> > org.apache.flink.streaming.connectors.kafka.KafkaTableSourceSinkFactory
> > >> > 的文件
> > >> > META-INF/services/org.apache.flink.table.factories.TableFactory
> > >> > 2、flink版本1.10,Standalone模式启动服务(start-cluster.sh),flink
> > >> > run运行(/software/flink-1.10.0/bin/flink run -c
> com.data.main.StreamMain
> > >> > ./flink_1.10_test-1.0-jar-with-dependencies.jar)
> > >> > 3、再确认下"TableEnvironmentImpl.sqlQuery"调用时候的ThreadClassLoader?
> > >> >
> 这个指的是打印当前线程的classloader嘛Thread.currentThread().getContextClassLoader()
> > >> >
> > >> >
> > >> >
> > >> > On Wed, Apr 22, 2020 at 6:00 PM Jingsong Li  >
> > >> > wrote:
> > >> >
> > >> > > Hi,
> > >> > >
> > >> > > 先确认下你的Jar包里有没有 meta-inf-services的文件?里面确定有Kafka?
> > >> > >
> > >> > > 如果有,再确认下"TableEnvironmentImpl.sqlQuery"调用时候的ThreadClassLoader?
> > >> > > 因为现在默认是通过ThreadClassLoader来获取Factory的。
> > >> > >
> > >> > > Best,
> > >> > > Jingsong Lee
> > >> > >
> > >> > > On Wed, Apr 22, 2020 at 5:30 PM 宇张  wrote:
> > >> > >
> > >> > > > 我这面使用Standalone模式运行Flink任务,但是Uber
> > >> > > > Jar里面的TableSourceFactory不能被加载,即使设置了classloader.resolve-order:
> > >> > > > child-first,只有放在lib目录才能加载得到,我看发布文档跟改了类加载策略,但是我不知道为什么Uber
> > >> > > Jar里面的Factory不能被加载
> > >> > > > Flink Client respects Classloading Policy (FLINK-13749
> > >> > > > )
> > >> > > > <
> > >> > > >
> > >> > >
> > >> >
> > >>
> >
> https://ci.apache.org/projects/flink/flink-docs-release-1.10/release-notes/flink-1.10.html#flink-client-respects-classloading-policy-flink-13749
> > >> > > > >
> > >> > > >
> > >> > > > The Flink client now also respects the configured classloading
> > >> policy,
> > >> > > > i.e., parent-first or child-first classloading. Previously, only
> > >> > cluster
> > >> > > > components such as the job manager or task manager supported
> this
> > >> > > setting.
> > >> > > > This does mean that users might get different behaviour in their
> > >> > > programs,
> > >> > > > in which case they should configure the classloading policy
> > >> explicitly
> > >> > to
> > >> > > > use parent-first classloading, which was the previous
> (hard-coded)
> > >> > > > behaviour.
> > >> > > >
> > >> > > > 异常信息:
> > >> > > >
> > >> > > >   rg.apache.flink.client.program.ProgramInvocationException: The
> > >> main
> > >> > > > method caused an error: findAndCreateTableSource failed.
> > >> > > > at
> > >> > > >
> > >> > > >
> > >> > >
> > >> >
> > >>
> >
> org.apache.flink.client.program.PackagedProgram.callMainMethod(PackagedProgram.java:335)
> > >> > > > at
> > >> > > >
> > >> > > >
> > >> > >
> > >> >
> > >>
> >
> org.apache.flink.client.progr

Re: 关于Flink1.10 Standalone 模式任务提交

2020-04-22 文章 Jingsong Li
Hi,

如果org.apache.flink.table.factories.TableFactory里面没有KafkaTableSourceSinkFactory,那就是打包有问题。不清楚1.9的是怎么运行起来的,但是所有的jar的meta-inf-services文件都没有KafkaTableSourceSinkFactory,那也不应该能运行起来的。

推荐打包方式用shade,shade会merge meta-inf-services的文件的。

Best,
Jingsong Lee

On Wed, Apr 22, 2020 at 7:31 PM 宇张  wrote:

>
> 》也确认下org.apache.flink.table.factories.TableFactory的内容,里面有没有KafkaTableSourceSinkFactory
>
> 这个我看了一下我先前flink1.9的工程,应用程序Jar里面也是没有这个类的,但是程序运行加载是没问题的,这么对比貌似就不是maven打包的问题了。
>
> On Wed, Apr 22, 2020 at 7:22 PM 宇张  wrote:
>
> >
> >
> 》也确认下org.apache.flink.table.factories.TableFactory的内容,里面有没有KafkaTableSourceSinkFactory
> > 这个没有,只有org.apache.flink.formats.json.JsonRowFormatFactory
> > 》拿到ClassLoader后看下能不能取到KafkaTableSourceSinkFactory的class
> > 这个能拿到
> >
> > 这么看来 貌似是 mvn打包有问题:
> > mvn clean package -DskipTests
> > 依赖范围为默认
> >
> >
> > On Wed, Apr 22, 2020 at 7:05 PM Jingsong Li 
> > wrote:
> >
> >> Hi,
> >>
> >>
> >>
> 也确认下org.apache.flink.table.factories.TableFactory的内容,里面有没有KafkaTableSourceSinkFactory
> >>
> >> > 这个指的是打印当前线程的classloader嘛Thread.currentThread().getContextClassLoader()
> >> 是的,拿到ClassLoader后看下能不能取到KafkaTableSourceSinkFactory的class
> >>
> >> Best,
> >> Jingsong Lee
> >>
> >> On Wed, Apr 22, 2020 at 7:00 PM 宇张  wrote:
> >>
> >> > 看下你打包的 UberJar 里有没一个内容包括
> >> > 1、下面这个文件是存在的
> >> >
> org.apache.flink.streaming.connectors.kafka.KafkaTableSourceSinkFactory
> >> > 的文件
> >> > META-INF/services/org.apache.flink.table.factories.TableFactory
> >> > 2、flink版本1.10,Standalone模式启动服务(start-cluster.sh),flink
> >> > run运行(/software/flink-1.10.0/bin/flink run -c com.data.main.StreamMain
> >> > ./flink_1.10_test-1.0-jar-with-dependencies.jar)
> >> > 3、再确认下"TableEnvironmentImpl.sqlQuery"调用时候的ThreadClassLoader?
> >> > 这个指的是打印当前线程的classloader嘛Thread.currentThread().getContextClassLoader()
> >> >
> >> >
> >> >
> >> > On Wed, Apr 22, 2020 at 6:00 PM Jingsong Li 
> >> > wrote:
> >> >
> >> > > Hi,
> >> > >
> >> > > 先确认下你的Jar包里有没有 meta-inf-services的文件?里面确定有Kafka?
> >> > >
> >> > > 如果有,再确认下"TableEnvironmentImpl.sqlQuery"调用时候的ThreadClassLoader?
> >> > > 因为现在默认是通过ThreadClassLoader来获取Factory的。
> >> > >
> >> > > Best,
> >> > > Jingsong Lee
> >> > >
> >> > > On Wed, Apr 22, 2020 at 5:30 PM 宇张  wrote:
> >> > >
> >> > > > 我这面使用Standalone模式运行Flink任务,但是Uber
> >> > > > Jar里面的TableSourceFactory不能被加载,即使设置了classloader.resolve-order:
> >> > > > child-first,只有放在lib目录才能加载得到,我看发布文档跟改了类加载策略,但是我不知道为什么Uber
> >> > > Jar里面的Factory不能被加载
> >> > > > Flink Client respects Classloading Policy (FLINK-13749
> >> > > > )
> >> > > > <
> >> > > >
> >> > >
> >> >
> >>
> https://ci.apache.org/projects/flink/flink-docs-release-1.10/release-notes/flink-1.10.html#flink-client-respects-classloading-policy-flink-13749
> >> > > > >
> >> > > >
> >> > > > The Flink client now also respects the configured classloading
> >> policy,
> >> > > > i.e., parent-first or child-first classloading. Previously, only
> >> > cluster
> >> > > > components such as the job manager or task manager supported this
> >> > > setting.
> >> > > > This does mean that users might get different behaviour in their
> >> > > programs,
> >> > > > in which case they should configure the classloading policy
> >> explicitly
> >> > to
> >> > > > use parent-first classloading, which was the previous (hard-coded)
> >> > > > behaviour.
> >> > > >
> >> > > > 异常信息:
> >> > > >
> >> > > >   rg.apache.flink.client.program.ProgramInvocationException: The
> >> main
> >> > > > method caused an error: findAndCreateTableSource failed.
> >> > > > at
> >> > > >
> >> > > >
> >> > >
> >> >
> >>
> org.apache.flink.client.program.PackagedProgram.callMainMethod(PackagedProgram.java:335)
> >> > > > at
> >> > > >
> >> > > >
> >> > >
> >> >
> >>
> org.apache.flink.client.program.PackagedProgram.invokeInteractiveModeForExecution(PackagedProgram.java:205)
> >> > > > at
> >> > >
> >> org.apache.flink.client.ClientUtils.executeProgram(ClientUtils.java:138)
> >> > > > at
> >> > > >
> >> > > >
> >> > >
> >> >
> >>
> org.apache.flink.client.cli.CliFrontend.executeProgram(CliFrontend.java:664)
> >> > > > at
> org.apache.flink.client.cli.CliFrontend.run(CliFrontend.java:213)
> >> > > > at
> >> > > >
> >> > > >
> >> > >
> >> >
> >>
> org.apache.flink.client.cli.CliFrontend.parseParameters(CliFrontend.java:895)
> >> > > > at
> >> > > >
> >> > > >
> >> > >
> >> >
> >>
> org.apache.flink.client.cli.CliFrontend.lambda$main$10(CliFrontend.java:968)
> >> > > > at
> >> > > >
> >> > > >
> >> > >
> >> >
> >>
> org.apache.flink.runtime.security.NoOpSecurityContext.runSecured(NoOpSecurityContext.java:30)
> >> > > > at
> >> org.apache.flink.client.cli.CliFrontend.main(CliFrontend.java:968)
> >> > > > Caused by: org.apache.flink.table.api.TableException:
> >> > > > findAndCreateTableSource failed.
> >> > > > at
> >> > > >
> >> > > >
> >> > >
> >> >
> >>
> org.apache.flink.table.factories.TableFactoryUtil.findAndCreateTableSource(TableFactoryUtil.java:55)
> >> > > > at
> >> > > >
> >> > > >
> >> > >
> >

Re: 关于Flink1.10 Standalone 模式任务提交

2020-04-22 文章 宇张
》也确认下org.apache.flink.table.factories.TableFactory的内容,里面有没有KafkaTableSourceSinkFactory
这个我看了一下我先前flink1.9的工程,应用程序Jar里面也是没有这个类的,但是程序运行加载是没问题的,这么对比貌似就不是maven打包的问题了。

On Wed, Apr 22, 2020 at 7:22 PM 宇张  wrote:

>
> 》也确认下org.apache.flink.table.factories.TableFactory的内容,里面有没有KafkaTableSourceSinkFactory
> 这个没有,只有org.apache.flink.formats.json.JsonRowFormatFactory
> 》拿到ClassLoader后看下能不能取到KafkaTableSourceSinkFactory的class
> 这个能拿到
>
> 这么看来 貌似是 mvn打包有问题:
> mvn clean package -DskipTests
> 依赖范围为默认
>
>
> On Wed, Apr 22, 2020 at 7:05 PM Jingsong Li 
> wrote:
>
>> Hi,
>>
>>
>> 也确认下org.apache.flink.table.factories.TableFactory的内容,里面有没有KafkaTableSourceSinkFactory
>>
>> > 这个指的是打印当前线程的classloader嘛Thread.currentThread().getContextClassLoader()
>> 是的,拿到ClassLoader后看下能不能取到KafkaTableSourceSinkFactory的class
>>
>> Best,
>> Jingsong Lee
>>
>> On Wed, Apr 22, 2020 at 7:00 PM 宇张  wrote:
>>
>> > 看下你打包的 UberJar 里有没一个内容包括
>> > 1、下面这个文件是存在的
>> > org.apache.flink.streaming.connectors.kafka.KafkaTableSourceSinkFactory
>> > 的文件
>> > META-INF/services/org.apache.flink.table.factories.TableFactory
>> > 2、flink版本1.10,Standalone模式启动服务(start-cluster.sh),flink
>> > run运行(/software/flink-1.10.0/bin/flink run -c com.data.main.StreamMain
>> > ./flink_1.10_test-1.0-jar-with-dependencies.jar)
>> > 3、再确认下"TableEnvironmentImpl.sqlQuery"调用时候的ThreadClassLoader?
>> > 这个指的是打印当前线程的classloader嘛Thread.currentThread().getContextClassLoader()
>> >
>> >
>> >
>> > On Wed, Apr 22, 2020 at 6:00 PM Jingsong Li 
>> > wrote:
>> >
>> > > Hi,
>> > >
>> > > 先确认下你的Jar包里有没有 meta-inf-services的文件?里面确定有Kafka?
>> > >
>> > > 如果有,再确认下"TableEnvironmentImpl.sqlQuery"调用时候的ThreadClassLoader?
>> > > 因为现在默认是通过ThreadClassLoader来获取Factory的。
>> > >
>> > > Best,
>> > > Jingsong Lee
>> > >
>> > > On Wed, Apr 22, 2020 at 5:30 PM 宇张  wrote:
>> > >
>> > > > 我这面使用Standalone模式运行Flink任务,但是Uber
>> > > > Jar里面的TableSourceFactory不能被加载,即使设置了classloader.resolve-order:
>> > > > child-first,只有放在lib目录才能加载得到,我看发布文档跟改了类加载策略,但是我不知道为什么Uber
>> > > Jar里面的Factory不能被加载
>> > > > Flink Client respects Classloading Policy (FLINK-13749
>> > > > )
>> > > > <
>> > > >
>> > >
>> >
>> https://ci.apache.org/projects/flink/flink-docs-release-1.10/release-notes/flink-1.10.html#flink-client-respects-classloading-policy-flink-13749
>> > > > >
>> > > >
>> > > > The Flink client now also respects the configured classloading
>> policy,
>> > > > i.e., parent-first or child-first classloading. Previously, only
>> > cluster
>> > > > components such as the job manager or task manager supported this
>> > > setting.
>> > > > This does mean that users might get different behaviour in their
>> > > programs,
>> > > > in which case they should configure the classloading policy
>> explicitly
>> > to
>> > > > use parent-first classloading, which was the previous (hard-coded)
>> > > > behaviour.
>> > > >
>> > > > 异常信息:
>> > > >
>> > > >   rg.apache.flink.client.program.ProgramInvocationException: The
>> main
>> > > > method caused an error: findAndCreateTableSource failed.
>> > > > at
>> > > >
>> > > >
>> > >
>> >
>> org.apache.flink.client.program.PackagedProgram.callMainMethod(PackagedProgram.java:335)
>> > > > at
>> > > >
>> > > >
>> > >
>> >
>> org.apache.flink.client.program.PackagedProgram.invokeInteractiveModeForExecution(PackagedProgram.java:205)
>> > > > at
>> > >
>> org.apache.flink.client.ClientUtils.executeProgram(ClientUtils.java:138)
>> > > > at
>> > > >
>> > > >
>> > >
>> >
>> org.apache.flink.client.cli.CliFrontend.executeProgram(CliFrontend.java:664)
>> > > > at org.apache.flink.client.cli.CliFrontend.run(CliFrontend.java:213)
>> > > > at
>> > > >
>> > > >
>> > >
>> >
>> org.apache.flink.client.cli.CliFrontend.parseParameters(CliFrontend.java:895)
>> > > > at
>> > > >
>> > > >
>> > >
>> >
>> org.apache.flink.client.cli.CliFrontend.lambda$main$10(CliFrontend.java:968)
>> > > > at
>> > > >
>> > > >
>> > >
>> >
>> org.apache.flink.runtime.security.NoOpSecurityContext.runSecured(NoOpSecurityContext.java:30)
>> > > > at
>> org.apache.flink.client.cli.CliFrontend.main(CliFrontend.java:968)
>> > > > Caused by: org.apache.flink.table.api.TableException:
>> > > > findAndCreateTableSource failed.
>> > > > at
>> > > >
>> > > >
>> > >
>> >
>> org.apache.flink.table.factories.TableFactoryUtil.findAndCreateTableSource(TableFactoryUtil.java:55)
>> > > > at
>> > > >
>> > > >
>> > >
>> >
>> org.apache.flink.table.factories.TableFactoryUtil.findAndCreateTableSource(TableFactoryUtil.java:92)
>> > > > at
>> > > >
>> > > >
>> > >
>> >
>> org.apache.flink.table.planner.plan.schema.CatalogSourceTable.findAndCreateTableSource(CatalogSourceTable.scala:156)
>> > > > at
>> > > >
>> > > >
>> > >
>> >
>> org.apache.flink.table.planner.plan.schema.CatalogSourceTable.tableSource$lzycompute(CatalogSourceTable.scala:65)
>> > > > at
>> > > >
>> > > >
>> > >
>> >
>> org.apache.flink.table.planner.plan.schema.CatalogSourceTable.tableSource(CatalogSourceTable.scala:65)
>> > > > at
>> > > >
>> > > >
>> > >
>

Re: 关于Flink1.10 Standalone 模式任务提交

2020-04-22 文章 宇张
》也确认下org.apache.flink.table.factories.TableFactory的内容,里面有没有KafkaTableSourceSinkFactory
这个没有,只有org.apache.flink.formats.json.JsonRowFormatFactory
》拿到ClassLoader后看下能不能取到KafkaTableSourceSinkFactory的class
这个能拿到

这么看来 貌似是 mvn打包有问题:
mvn clean package -DskipTests
依赖范围为默认


On Wed, Apr 22, 2020 at 7:05 PM Jingsong Li  wrote:

> Hi,
>
>
> 也确认下org.apache.flink.table.factories.TableFactory的内容,里面有没有KafkaTableSourceSinkFactory
>
> > 这个指的是打印当前线程的classloader嘛Thread.currentThread().getContextClassLoader()
> 是的,拿到ClassLoader后看下能不能取到KafkaTableSourceSinkFactory的class
>
> Best,
> Jingsong Lee
>
> On Wed, Apr 22, 2020 at 7:00 PM 宇张  wrote:
>
> > 看下你打包的 UberJar 里有没一个内容包括
> > 1、下面这个文件是存在的
> > org.apache.flink.streaming.connectors.kafka.KafkaTableSourceSinkFactory
> > 的文件
> > META-INF/services/org.apache.flink.table.factories.TableFactory
> > 2、flink版本1.10,Standalone模式启动服务(start-cluster.sh),flink
> > run运行(/software/flink-1.10.0/bin/flink run -c com.data.main.StreamMain
> > ./flink_1.10_test-1.0-jar-with-dependencies.jar)
> > 3、再确认下"TableEnvironmentImpl.sqlQuery"调用时候的ThreadClassLoader?
> > 这个指的是打印当前线程的classloader嘛Thread.currentThread().getContextClassLoader()
> >
> >
> >
> > On Wed, Apr 22, 2020 at 6:00 PM Jingsong Li 
> > wrote:
> >
> > > Hi,
> > >
> > > 先确认下你的Jar包里有没有 meta-inf-services的文件?里面确定有Kafka?
> > >
> > > 如果有,再确认下"TableEnvironmentImpl.sqlQuery"调用时候的ThreadClassLoader?
> > > 因为现在默认是通过ThreadClassLoader来获取Factory的。
> > >
> > > Best,
> > > Jingsong Lee
> > >
> > > On Wed, Apr 22, 2020 at 5:30 PM 宇张  wrote:
> > >
> > > > 我这面使用Standalone模式运行Flink任务,但是Uber
> > > > Jar里面的TableSourceFactory不能被加载,即使设置了classloader.resolve-order:
> > > > child-first,只有放在lib目录才能加载得到,我看发布文档跟改了类加载策略,但是我不知道为什么Uber
> > > Jar里面的Factory不能被加载
> > > > Flink Client respects Classloading Policy (FLINK-13749
> > > > )
> > > > <
> > > >
> > >
> >
> https://ci.apache.org/projects/flink/flink-docs-release-1.10/release-notes/flink-1.10.html#flink-client-respects-classloading-policy-flink-13749
> > > > >
> > > >
> > > > The Flink client now also respects the configured classloading
> policy,
> > > > i.e., parent-first or child-first classloading. Previously, only
> > cluster
> > > > components such as the job manager or task manager supported this
> > > setting.
> > > > This does mean that users might get different behaviour in their
> > > programs,
> > > > in which case they should configure the classloading policy
> explicitly
> > to
> > > > use parent-first classloading, which was the previous (hard-coded)
> > > > behaviour.
> > > >
> > > > 异常信息:
> > > >
> > > >   rg.apache.flink.client.program.ProgramInvocationException: The main
> > > > method caused an error: findAndCreateTableSource failed.
> > > > at
> > > >
> > > >
> > >
> >
> org.apache.flink.client.program.PackagedProgram.callMainMethod(PackagedProgram.java:335)
> > > > at
> > > >
> > > >
> > >
> >
> org.apache.flink.client.program.PackagedProgram.invokeInteractiveModeForExecution(PackagedProgram.java:205)
> > > > at
> > >
> org.apache.flink.client.ClientUtils.executeProgram(ClientUtils.java:138)
> > > > at
> > > >
> > > >
> > >
> >
> org.apache.flink.client.cli.CliFrontend.executeProgram(CliFrontend.java:664)
> > > > at org.apache.flink.client.cli.CliFrontend.run(CliFrontend.java:213)
> > > > at
> > > >
> > > >
> > >
> >
> org.apache.flink.client.cli.CliFrontend.parseParameters(CliFrontend.java:895)
> > > > at
> > > >
> > > >
> > >
> >
> org.apache.flink.client.cli.CliFrontend.lambda$main$10(CliFrontend.java:968)
> > > > at
> > > >
> > > >
> > >
> >
> org.apache.flink.runtime.security.NoOpSecurityContext.runSecured(NoOpSecurityContext.java:30)
> > > > at org.apache.flink.client.cli.CliFrontend.main(CliFrontend.java:968)
> > > > Caused by: org.apache.flink.table.api.TableException:
> > > > findAndCreateTableSource failed.
> > > > at
> > > >
> > > >
> > >
> >
> org.apache.flink.table.factories.TableFactoryUtil.findAndCreateTableSource(TableFactoryUtil.java:55)
> > > > at
> > > >
> > > >
> > >
> >
> org.apache.flink.table.factories.TableFactoryUtil.findAndCreateTableSource(TableFactoryUtil.java:92)
> > > > at
> > > >
> > > >
> > >
> >
> org.apache.flink.table.planner.plan.schema.CatalogSourceTable.findAndCreateTableSource(CatalogSourceTable.scala:156)
> > > > at
> > > >
> > > >
> > >
> >
> org.apache.flink.table.planner.plan.schema.CatalogSourceTable.tableSource$lzycompute(CatalogSourceTable.scala:65)
> > > > at
> > > >
> > > >
> > >
> >
> org.apache.flink.table.planner.plan.schema.CatalogSourceTable.tableSource(CatalogSourceTable.scala:65)
> > > > at
> > > >
> > > >
> > >
> >
> org.apache.flink.table.planner.plan.schema.CatalogSourceTable.toRel(CatalogSourceTable.scala:76)
> > > > at
> > > >
> > > >
> > >
> >
> org.apache.calcite.sql2rel.SqlToRelConverter.toRel(SqlToRelConverter.java:3328)
> > > > at
> > > >
> > > >
> > >
> >
> org.apache.calcite.sql2rel.SqlToRelConverter.convertIdentifier(SqlToRelConverter.java:2357)
> > > > at
> > > >
> > > >
> > >
> >
> 

Re: 关于Flink1.10 Standalone 模式任务提交

2020-04-22 文章 Jingsong Li
Hi,

也确认下org.apache.flink.table.factories.TableFactory的内容,里面有没有KafkaTableSourceSinkFactory

> 这个指的是打印当前线程的classloader嘛Thread.currentThread().getContextClassLoader()
是的,拿到ClassLoader后看下能不能取到KafkaTableSourceSinkFactory的class

Best,
Jingsong Lee

On Wed, Apr 22, 2020 at 7:00 PM 宇张  wrote:

> 看下你打包的 UberJar 里有没一个内容包括
> 1、下面这个文件是存在的
> org.apache.flink.streaming.connectors.kafka.KafkaTableSourceSinkFactory
> 的文件
> META-INF/services/org.apache.flink.table.factories.TableFactory
> 2、flink版本1.10,Standalone模式启动服务(start-cluster.sh),flink
> run运行(/software/flink-1.10.0/bin/flink run -c com.data.main.StreamMain
> ./flink_1.10_test-1.0-jar-with-dependencies.jar)
> 3、再确认下"TableEnvironmentImpl.sqlQuery"调用时候的ThreadClassLoader?
> 这个指的是打印当前线程的classloader嘛Thread.currentThread().getContextClassLoader()
>
>
>
> On Wed, Apr 22, 2020 at 6:00 PM Jingsong Li 
> wrote:
>
> > Hi,
> >
> > 先确认下你的Jar包里有没有 meta-inf-services的文件?里面确定有Kafka?
> >
> > 如果有,再确认下"TableEnvironmentImpl.sqlQuery"调用时候的ThreadClassLoader?
> > 因为现在默认是通过ThreadClassLoader来获取Factory的。
> >
> > Best,
> > Jingsong Lee
> >
> > On Wed, Apr 22, 2020 at 5:30 PM 宇张  wrote:
> >
> > > 我这面使用Standalone模式运行Flink任务,但是Uber
> > > Jar里面的TableSourceFactory不能被加载,即使设置了classloader.resolve-order:
> > > child-first,只有放在lib目录才能加载得到,我看发布文档跟改了类加载策略,但是我不知道为什么Uber
> > Jar里面的Factory不能被加载
> > > Flink Client respects Classloading Policy (FLINK-13749
> > > )
> > > <
> > >
> >
> https://ci.apache.org/projects/flink/flink-docs-release-1.10/release-notes/flink-1.10.html#flink-client-respects-classloading-policy-flink-13749
> > > >
> > >
> > > The Flink client now also respects the configured classloading policy,
> > > i.e., parent-first or child-first classloading. Previously, only
> cluster
> > > components such as the job manager or task manager supported this
> > setting.
> > > This does mean that users might get different behaviour in their
> > programs,
> > > in which case they should configure the classloading policy explicitly
> to
> > > use parent-first classloading, which was the previous (hard-coded)
> > > behaviour.
> > >
> > > 异常信息:
> > >
> > >   rg.apache.flink.client.program.ProgramInvocationException: The main
> > > method caused an error: findAndCreateTableSource failed.
> > > at
> > >
> > >
> >
> org.apache.flink.client.program.PackagedProgram.callMainMethod(PackagedProgram.java:335)
> > > at
> > >
> > >
> >
> org.apache.flink.client.program.PackagedProgram.invokeInteractiveModeForExecution(PackagedProgram.java:205)
> > > at
> > org.apache.flink.client.ClientUtils.executeProgram(ClientUtils.java:138)
> > > at
> > >
> > >
> >
> org.apache.flink.client.cli.CliFrontend.executeProgram(CliFrontend.java:664)
> > > at org.apache.flink.client.cli.CliFrontend.run(CliFrontend.java:213)
> > > at
> > >
> > >
> >
> org.apache.flink.client.cli.CliFrontend.parseParameters(CliFrontend.java:895)
> > > at
> > >
> > >
> >
> org.apache.flink.client.cli.CliFrontend.lambda$main$10(CliFrontend.java:968)
> > > at
> > >
> > >
> >
> org.apache.flink.runtime.security.NoOpSecurityContext.runSecured(NoOpSecurityContext.java:30)
> > > at org.apache.flink.client.cli.CliFrontend.main(CliFrontend.java:968)
> > > Caused by: org.apache.flink.table.api.TableException:
> > > findAndCreateTableSource failed.
> > > at
> > >
> > >
> >
> org.apache.flink.table.factories.TableFactoryUtil.findAndCreateTableSource(TableFactoryUtil.java:55)
> > > at
> > >
> > >
> >
> org.apache.flink.table.factories.TableFactoryUtil.findAndCreateTableSource(TableFactoryUtil.java:92)
> > > at
> > >
> > >
> >
> org.apache.flink.table.planner.plan.schema.CatalogSourceTable.findAndCreateTableSource(CatalogSourceTable.scala:156)
> > > at
> > >
> > >
> >
> org.apache.flink.table.planner.plan.schema.CatalogSourceTable.tableSource$lzycompute(CatalogSourceTable.scala:65)
> > > at
> > >
> > >
> >
> org.apache.flink.table.planner.plan.schema.CatalogSourceTable.tableSource(CatalogSourceTable.scala:65)
> > > at
> > >
> > >
> >
> org.apache.flink.table.planner.plan.schema.CatalogSourceTable.toRel(CatalogSourceTable.scala:76)
> > > at
> > >
> > >
> >
> org.apache.calcite.sql2rel.SqlToRelConverter.toRel(SqlToRelConverter.java:3328)
> > > at
> > >
> > >
> >
> org.apache.calcite.sql2rel.SqlToRelConverter.convertIdentifier(SqlToRelConverter.java:2357)
> > > at
> > >
> > >
> >
> org.apache.calcite.sql2rel.SqlToRelConverter.convertFrom(SqlToRelConverter.java:2051)
> > > at
> > >
> > >
> >
> org.apache.calcite.sql2rel.SqlToRelConverter.convertFrom(SqlToRelConverter.java:2005)
> > > at
> > >
> > >
> >
> org.apache.calcite.sql2rel.SqlToRelConverter.convertFrom(SqlToRelConverter.java:2083)
> > > at
> > >
> > >
> >
> org.apache.calcite.sql2rel.SqlToRelConverter.convertSelectImpl(SqlToRelConverter.java:646)
> > > at
> > >
> > >
> >
> org.apache.calcite.sql2rel.SqlToRelConverter.convertSelect(SqlToRelConverter.java:627)
> > > at
> > >
> > >
> >
> org.apache.calcite.sql2rel.SqlToRelConverter.convertQueryRecursive(SqlToRelConvert

Re: 关于Flink1.10 Standalone 模式任务提交

2020-04-22 文章 宇张
看下你打包的 UberJar 里有没一个内容包括
1、下面这个文件是存在的
org.apache.flink.streaming.connectors.kafka.KafkaTableSourceSinkFactory
的文件
META-INF/services/org.apache.flink.table.factories.TableFactory
2、flink版本1.10,Standalone模式启动服务(start-cluster.sh),flink
run运行(/software/flink-1.10.0/bin/flink run -c com.data.main.StreamMain
./flink_1.10_test-1.0-jar-with-dependencies.jar)
3、再确认下"TableEnvironmentImpl.sqlQuery"调用时候的ThreadClassLoader?
这个指的是打印当前线程的classloader嘛Thread.currentThread().getContextClassLoader()



On Wed, Apr 22, 2020 at 6:00 PM Jingsong Li  wrote:

> Hi,
>
> 先确认下你的Jar包里有没有 meta-inf-services的文件?里面确定有Kafka?
>
> 如果有,再确认下"TableEnvironmentImpl.sqlQuery"调用时候的ThreadClassLoader?
> 因为现在默认是通过ThreadClassLoader来获取Factory的。
>
> Best,
> Jingsong Lee
>
> On Wed, Apr 22, 2020 at 5:30 PM 宇张  wrote:
>
> > 我这面使用Standalone模式运行Flink任务,但是Uber
> > Jar里面的TableSourceFactory不能被加载,即使设置了classloader.resolve-order:
> > child-first,只有放在lib目录才能加载得到,我看发布文档跟改了类加载策略,但是我不知道为什么Uber
> Jar里面的Factory不能被加载
> > Flink Client respects Classloading Policy (FLINK-13749
> > )
> > <
> >
> https://ci.apache.org/projects/flink/flink-docs-release-1.10/release-notes/flink-1.10.html#flink-client-respects-classloading-policy-flink-13749
> > >
> >
> > The Flink client now also respects the configured classloading policy,
> > i.e., parent-first or child-first classloading. Previously, only cluster
> > components such as the job manager or task manager supported this
> setting.
> > This does mean that users might get different behaviour in their
> programs,
> > in which case they should configure the classloading policy explicitly to
> > use parent-first classloading, which was the previous (hard-coded)
> > behaviour.
> >
> > 异常信息:
> >
> >   rg.apache.flink.client.program.ProgramInvocationException: The main
> > method caused an error: findAndCreateTableSource failed.
> > at
> >
> >
> org.apache.flink.client.program.PackagedProgram.callMainMethod(PackagedProgram.java:335)
> > at
> >
> >
> org.apache.flink.client.program.PackagedProgram.invokeInteractiveModeForExecution(PackagedProgram.java:205)
> > at
> org.apache.flink.client.ClientUtils.executeProgram(ClientUtils.java:138)
> > at
> >
> >
> org.apache.flink.client.cli.CliFrontend.executeProgram(CliFrontend.java:664)
> > at org.apache.flink.client.cli.CliFrontend.run(CliFrontend.java:213)
> > at
> >
> >
> org.apache.flink.client.cli.CliFrontend.parseParameters(CliFrontend.java:895)
> > at
> >
> >
> org.apache.flink.client.cli.CliFrontend.lambda$main$10(CliFrontend.java:968)
> > at
> >
> >
> org.apache.flink.runtime.security.NoOpSecurityContext.runSecured(NoOpSecurityContext.java:30)
> > at org.apache.flink.client.cli.CliFrontend.main(CliFrontend.java:968)
> > Caused by: org.apache.flink.table.api.TableException:
> > findAndCreateTableSource failed.
> > at
> >
> >
> org.apache.flink.table.factories.TableFactoryUtil.findAndCreateTableSource(TableFactoryUtil.java:55)
> > at
> >
> >
> org.apache.flink.table.factories.TableFactoryUtil.findAndCreateTableSource(TableFactoryUtil.java:92)
> > at
> >
> >
> org.apache.flink.table.planner.plan.schema.CatalogSourceTable.findAndCreateTableSource(CatalogSourceTable.scala:156)
> > at
> >
> >
> org.apache.flink.table.planner.plan.schema.CatalogSourceTable.tableSource$lzycompute(CatalogSourceTable.scala:65)
> > at
> >
> >
> org.apache.flink.table.planner.plan.schema.CatalogSourceTable.tableSource(CatalogSourceTable.scala:65)
> > at
> >
> >
> org.apache.flink.table.planner.plan.schema.CatalogSourceTable.toRel(CatalogSourceTable.scala:76)
> > at
> >
> >
> org.apache.calcite.sql2rel.SqlToRelConverter.toRel(SqlToRelConverter.java:3328)
> > at
> >
> >
> org.apache.calcite.sql2rel.SqlToRelConverter.convertIdentifier(SqlToRelConverter.java:2357)
> > at
> >
> >
> org.apache.calcite.sql2rel.SqlToRelConverter.convertFrom(SqlToRelConverter.java:2051)
> > at
> >
> >
> org.apache.calcite.sql2rel.SqlToRelConverter.convertFrom(SqlToRelConverter.java:2005)
> > at
> >
> >
> org.apache.calcite.sql2rel.SqlToRelConverter.convertFrom(SqlToRelConverter.java:2083)
> > at
> >
> >
> org.apache.calcite.sql2rel.SqlToRelConverter.convertSelectImpl(SqlToRelConverter.java:646)
> > at
> >
> >
> org.apache.calcite.sql2rel.SqlToRelConverter.convertSelect(SqlToRelConverter.java:627)
> > at
> >
> >
> org.apache.calcite.sql2rel.SqlToRelConverter.convertQueryRecursive(SqlToRelConverter.java:3181)
> > at
> >
> >
> org.apache.calcite.sql2rel.SqlToRelConverter.convertQuery(SqlToRelConverter.java:563)
> > at org.apache.flink.table.planner.calcite.FlinkPlannerImpl.org
> >
> >
> $apache$flink$table$planner$calcite$FlinkPlannerImpl$$rel(FlinkPlannerImpl.scala:148)
> > at
> >
> >
> org.apache.flink.table.planner.calcite.FlinkPlannerImpl.rel(FlinkPlannerImpl.scala:135)
> > at
> >
> >
> org.apache.flink.table.planner.operations.SqlToOperationConverter.toQueryOperation(SqlToOperationConverter.java:522)
> > at
> >
> >
> org.apache.flink.table.planner.operations.SqlToOperationConverter.conver

Re: 关于Flink1.10 Standalone 模式任务提交

2020-04-22 文章 Jingsong Li
Hi,

先确认下你的Jar包里有没有 meta-inf-services的文件?里面确定有Kafka?

如果有,再确认下"TableEnvironmentImpl.sqlQuery"调用时候的ThreadClassLoader?
因为现在默认是通过ThreadClassLoader来获取Factory的。

Best,
Jingsong Lee

On Wed, Apr 22, 2020 at 5:30 PM 宇张  wrote:

> 我这面使用Standalone模式运行Flink任务,但是Uber
> Jar里面的TableSourceFactory不能被加载,即使设置了classloader.resolve-order:
> child-first,只有放在lib目录才能加载得到,我看发布文档跟改了类加载策略,但是我不知道为什么Uber Jar里面的Factory不能被加载
> Flink Client respects Classloading Policy (FLINK-13749
> )
> <
> https://ci.apache.org/projects/flink/flink-docs-release-1.10/release-notes/flink-1.10.html#flink-client-respects-classloading-policy-flink-13749
> >
>
> The Flink client now also respects the configured classloading policy,
> i.e., parent-first or child-first classloading. Previously, only cluster
> components such as the job manager or task manager supported this setting.
> This does mean that users might get different behaviour in their programs,
> in which case they should configure the classloading policy explicitly to
> use parent-first classloading, which was the previous (hard-coded)
> behaviour.
>
> 异常信息:
>
>   rg.apache.flink.client.program.ProgramInvocationException: The main
> method caused an error: findAndCreateTableSource failed.
> at
>
> org.apache.flink.client.program.PackagedProgram.callMainMethod(PackagedProgram.java:335)
> at
>
> org.apache.flink.client.program.PackagedProgram.invokeInteractiveModeForExecution(PackagedProgram.java:205)
> at org.apache.flink.client.ClientUtils.executeProgram(ClientUtils.java:138)
> at
>
> org.apache.flink.client.cli.CliFrontend.executeProgram(CliFrontend.java:664)
> at org.apache.flink.client.cli.CliFrontend.run(CliFrontend.java:213)
> at
>
> org.apache.flink.client.cli.CliFrontend.parseParameters(CliFrontend.java:895)
> at
>
> org.apache.flink.client.cli.CliFrontend.lambda$main$10(CliFrontend.java:968)
> at
>
> org.apache.flink.runtime.security.NoOpSecurityContext.runSecured(NoOpSecurityContext.java:30)
> at org.apache.flink.client.cli.CliFrontend.main(CliFrontend.java:968)
> Caused by: org.apache.flink.table.api.TableException:
> findAndCreateTableSource failed.
> at
>
> org.apache.flink.table.factories.TableFactoryUtil.findAndCreateTableSource(TableFactoryUtil.java:55)
> at
>
> org.apache.flink.table.factories.TableFactoryUtil.findAndCreateTableSource(TableFactoryUtil.java:92)
> at
>
> org.apache.flink.table.planner.plan.schema.CatalogSourceTable.findAndCreateTableSource(CatalogSourceTable.scala:156)
> at
>
> org.apache.flink.table.planner.plan.schema.CatalogSourceTable.tableSource$lzycompute(CatalogSourceTable.scala:65)
> at
>
> org.apache.flink.table.planner.plan.schema.CatalogSourceTable.tableSource(CatalogSourceTable.scala:65)
> at
>
> org.apache.flink.table.planner.plan.schema.CatalogSourceTable.toRel(CatalogSourceTable.scala:76)
> at
>
> org.apache.calcite.sql2rel.SqlToRelConverter.toRel(SqlToRelConverter.java:3328)
> at
>
> org.apache.calcite.sql2rel.SqlToRelConverter.convertIdentifier(SqlToRelConverter.java:2357)
> at
>
> org.apache.calcite.sql2rel.SqlToRelConverter.convertFrom(SqlToRelConverter.java:2051)
> at
>
> org.apache.calcite.sql2rel.SqlToRelConverter.convertFrom(SqlToRelConverter.java:2005)
> at
>
> org.apache.calcite.sql2rel.SqlToRelConverter.convertFrom(SqlToRelConverter.java:2083)
> at
>
> org.apache.calcite.sql2rel.SqlToRelConverter.convertSelectImpl(SqlToRelConverter.java:646)
> at
>
> org.apache.calcite.sql2rel.SqlToRelConverter.convertSelect(SqlToRelConverter.java:627)
> at
>
> org.apache.calcite.sql2rel.SqlToRelConverter.convertQueryRecursive(SqlToRelConverter.java:3181)
> at
>
> org.apache.calcite.sql2rel.SqlToRelConverter.convertQuery(SqlToRelConverter.java:563)
> at org.apache.flink.table.planner.calcite.FlinkPlannerImpl.org
>
> $apache$flink$table$planner$calcite$FlinkPlannerImpl$$rel(FlinkPlannerImpl.scala:148)
> at
>
> org.apache.flink.table.planner.calcite.FlinkPlannerImpl.rel(FlinkPlannerImpl.scala:135)
> at
>
> org.apache.flink.table.planner.operations.SqlToOperationConverter.toQueryOperation(SqlToOperationConverter.java:522)
> at
>
> org.apache.flink.table.planner.operations.SqlToOperationConverter.convertSqlQuery(SqlToOperationConverter.java:436)
> at
>
> org.apache.flink.table.planner.operations.SqlToOperationConverter.convert(SqlToOperationConverter.java:154)
> at
>
> org.apache.flink.table.planner.delegation.ParserImpl.parse(ParserImpl.java:66)
> at
>
> org.apache.flink.table.api.internal.TableEnvironmentImpl.sqlQuery(TableEnvironmentImpl.java:464)
> at com.akulaku.data.main.StreamMain.main(StreamMain.java:87)
> at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
> at
>
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
> at
>
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> at java.lang.reflect.Method.invoke(Method.java:498)
> at
>
> org.apache.flink.client.program.PackagedProgram.callMainMethod(PackagedProgram.java:321)
> ...

Re: 关于Flink1.10 Standalone 模式任务提交

2020-04-22 文章 tison
虽然你放到 lib 下就能行了听起来是个 BUG,能不能说明一下你的 Flink 版本还有具体的启动命令。

FLINK-13749 可能在早期版本上没有,另外 Standalone 的类加载如果是 PerJob 有更改过。

Best,
tison.


tison  于2020年4月22日周三 下午5:48写道:

> 看下你打包的 UberJar 里有没一个内容包括
>
> org.apache.flink.streaming.connectors.kafka.KafkaTableSourceSinkFactory
>
> 的文件
>
> META-INF/services/org.apache.flink.table.factories.TableFactory
>
> Best,
> tison.
>
>
> 宇张  于2020年4月22日周三 下午5:30写道:
>
>> 我这面使用Standalone模式运行Flink任务,但是Uber
>> Jar里面的TableSourceFactory不能被加载,即使设置了classloader.resolve-order:
>> child-first,只有放在lib目录才能加载得到,我看发布文档跟改了类加载策略,但是我不知道为什么Uber
>> Jar里面的Factory不能被加载
>> Flink Client respects Classloading Policy (FLINK-13749
>> )
>> <
>> https://ci.apache.org/projects/flink/flink-docs-release-1.10/release-notes/flink-1.10.html#flink-client-respects-classloading-policy-flink-13749
>> >
>>
>> The Flink client now also respects the configured classloading policy,
>> i.e., parent-first or child-first classloading. Previously, only cluster
>> components such as the job manager or task manager supported this setting.
>> This does mean that users might get different behaviour in their programs,
>> in which case they should configure the classloading policy explicitly to
>> use parent-first classloading, which was the previous (hard-coded)
>> behaviour.
>>
>> 异常信息:
>>
>>   rg.apache.flink.client.program.ProgramInvocationException: The main
>> method caused an error: findAndCreateTableSource failed.
>> at
>>
>> org.apache.flink.client.program.PackagedProgram.callMainMethod(PackagedProgram.java:335)
>> at
>>
>> org.apache.flink.client.program.PackagedProgram.invokeInteractiveModeForExecution(PackagedProgram.java:205)
>> at
>> org.apache.flink.client.ClientUtils.executeProgram(ClientUtils.java:138)
>> at
>>
>> org.apache.flink.client.cli.CliFrontend.executeProgram(CliFrontend.java:664)
>> at org.apache.flink.client.cli.CliFrontend.run(CliFrontend.java:213)
>> at
>>
>> org.apache.flink.client.cli.CliFrontend.parseParameters(CliFrontend.java:895)
>> at
>>
>> org.apache.flink.client.cli.CliFrontend.lambda$main$10(CliFrontend.java:968)
>> at
>>
>> org.apache.flink.runtime.security.NoOpSecurityContext.runSecured(NoOpSecurityContext.java:30)
>> at org.apache.flink.client.cli.CliFrontend.main(CliFrontend.java:968)
>> Caused by: org.apache.flink.table.api.TableException:
>> findAndCreateTableSource failed.
>> at
>>
>> org.apache.flink.table.factories.TableFactoryUtil.findAndCreateTableSource(TableFactoryUtil.java:55)
>> at
>>
>> org.apache.flink.table.factories.TableFactoryUtil.findAndCreateTableSource(TableFactoryUtil.java:92)
>> at
>>
>> org.apache.flink.table.planner.plan.schema.CatalogSourceTable.findAndCreateTableSource(CatalogSourceTable.scala:156)
>> at
>>
>> org.apache.flink.table.planner.plan.schema.CatalogSourceTable.tableSource$lzycompute(CatalogSourceTable.scala:65)
>> at
>>
>> org.apache.flink.table.planner.plan.schema.CatalogSourceTable.tableSource(CatalogSourceTable.scala:65)
>> at
>>
>> org.apache.flink.table.planner.plan.schema.CatalogSourceTable.toRel(CatalogSourceTable.scala:76)
>> at
>>
>> org.apache.calcite.sql2rel.SqlToRelConverter.toRel(SqlToRelConverter.java:3328)
>> at
>>
>> org.apache.calcite.sql2rel.SqlToRelConverter.convertIdentifier(SqlToRelConverter.java:2357)
>> at
>>
>> org.apache.calcite.sql2rel.SqlToRelConverter.convertFrom(SqlToRelConverter.java:2051)
>> at
>>
>> org.apache.calcite.sql2rel.SqlToRelConverter.convertFrom(SqlToRelConverter.java:2005)
>> at
>>
>> org.apache.calcite.sql2rel.SqlToRelConverter.convertFrom(SqlToRelConverter.java:2083)
>> at
>>
>> org.apache.calcite.sql2rel.SqlToRelConverter.convertSelectImpl(SqlToRelConverter.java:646)
>> at
>>
>> org.apache.calcite.sql2rel.SqlToRelConverter.convertSelect(SqlToRelConverter.java:627)
>> at
>>
>> org.apache.calcite.sql2rel.SqlToRelConverter.convertQueryRecursive(SqlToRelConverter.java:3181)
>> at
>>
>> org.apache.calcite.sql2rel.SqlToRelConverter.convertQuery(SqlToRelConverter.java:563)
>> at org.apache.flink.table.planner.calcite.FlinkPlannerImpl.org
>>
>> $apache$flink$table$planner$calcite$FlinkPlannerImpl$$rel(FlinkPlannerImpl.scala:148)
>> at
>>
>> org.apache.flink.table.planner.calcite.FlinkPlannerImpl.rel(FlinkPlannerImpl.scala:135)
>> at
>>
>> org.apache.flink.table.planner.operations.SqlToOperationConverter.toQueryOperation(SqlToOperationConverter.java:522)
>> at
>>
>> org.apache.flink.table.planner.operations.SqlToOperationConverter.convertSqlQuery(SqlToOperationConverter.java:436)
>> at
>>
>> org.apache.flink.table.planner.operations.SqlToOperationConverter.convert(SqlToOperationConverter.java:154)
>> at
>>
>> org.apache.flink.table.planner.delegation.ParserImpl.parse(ParserImpl.java:66)
>> at
>>
>> org.apache.flink.table.api.internal.TableEnvironmentImpl.sqlQuery(TableEnvironmentImpl.java:464)
>> at com.akulaku.data.main.StreamMain.main(StreamMain.java:87)
>> at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>> at
>>
>> sun.reflect.NativeMethodAccessorImp

Re: 关于Flink1.10 Standalone 模式任务提交

2020-04-22 文章 tison
看下你打包的 UberJar 里有没一个内容包括

org.apache.flink.streaming.connectors.kafka.KafkaTableSourceSinkFactory

的文件

META-INF/services/org.apache.flink.table.factories.TableFactory

Best,
tison.


宇张  于2020年4月22日周三 下午5:30写道:

> 我这面使用Standalone模式运行Flink任务,但是Uber
> Jar里面的TableSourceFactory不能被加载,即使设置了classloader.resolve-order:
> child-first,只有放在lib目录才能加载得到,我看发布文档跟改了类加载策略,但是我不知道为什么Uber Jar里面的Factory不能被加载
> Flink Client respects Classloading Policy (FLINK-13749
> )
> <
> https://ci.apache.org/projects/flink/flink-docs-release-1.10/release-notes/flink-1.10.html#flink-client-respects-classloading-policy-flink-13749
> >
>
> The Flink client now also respects the configured classloading policy,
> i.e., parent-first or child-first classloading. Previously, only cluster
> components such as the job manager or task manager supported this setting.
> This does mean that users might get different behaviour in their programs,
> in which case they should configure the classloading policy explicitly to
> use parent-first classloading, which was the previous (hard-coded)
> behaviour.
>
> 异常信息:
>
>   rg.apache.flink.client.program.ProgramInvocationException: The main
> method caused an error: findAndCreateTableSource failed.
> at
>
> org.apache.flink.client.program.PackagedProgram.callMainMethod(PackagedProgram.java:335)
> at
>
> org.apache.flink.client.program.PackagedProgram.invokeInteractiveModeForExecution(PackagedProgram.java:205)
> at org.apache.flink.client.ClientUtils.executeProgram(ClientUtils.java:138)
> at
>
> org.apache.flink.client.cli.CliFrontend.executeProgram(CliFrontend.java:664)
> at org.apache.flink.client.cli.CliFrontend.run(CliFrontend.java:213)
> at
>
> org.apache.flink.client.cli.CliFrontend.parseParameters(CliFrontend.java:895)
> at
>
> org.apache.flink.client.cli.CliFrontend.lambda$main$10(CliFrontend.java:968)
> at
>
> org.apache.flink.runtime.security.NoOpSecurityContext.runSecured(NoOpSecurityContext.java:30)
> at org.apache.flink.client.cli.CliFrontend.main(CliFrontend.java:968)
> Caused by: org.apache.flink.table.api.TableException:
> findAndCreateTableSource failed.
> at
>
> org.apache.flink.table.factories.TableFactoryUtil.findAndCreateTableSource(TableFactoryUtil.java:55)
> at
>
> org.apache.flink.table.factories.TableFactoryUtil.findAndCreateTableSource(TableFactoryUtil.java:92)
> at
>
> org.apache.flink.table.planner.plan.schema.CatalogSourceTable.findAndCreateTableSource(CatalogSourceTable.scala:156)
> at
>
> org.apache.flink.table.planner.plan.schema.CatalogSourceTable.tableSource$lzycompute(CatalogSourceTable.scala:65)
> at
>
> org.apache.flink.table.planner.plan.schema.CatalogSourceTable.tableSource(CatalogSourceTable.scala:65)
> at
>
> org.apache.flink.table.planner.plan.schema.CatalogSourceTable.toRel(CatalogSourceTable.scala:76)
> at
>
> org.apache.calcite.sql2rel.SqlToRelConverter.toRel(SqlToRelConverter.java:3328)
> at
>
> org.apache.calcite.sql2rel.SqlToRelConverter.convertIdentifier(SqlToRelConverter.java:2357)
> at
>
> org.apache.calcite.sql2rel.SqlToRelConverter.convertFrom(SqlToRelConverter.java:2051)
> at
>
> org.apache.calcite.sql2rel.SqlToRelConverter.convertFrom(SqlToRelConverter.java:2005)
> at
>
> org.apache.calcite.sql2rel.SqlToRelConverter.convertFrom(SqlToRelConverter.java:2083)
> at
>
> org.apache.calcite.sql2rel.SqlToRelConverter.convertSelectImpl(SqlToRelConverter.java:646)
> at
>
> org.apache.calcite.sql2rel.SqlToRelConverter.convertSelect(SqlToRelConverter.java:627)
> at
>
> org.apache.calcite.sql2rel.SqlToRelConverter.convertQueryRecursive(SqlToRelConverter.java:3181)
> at
>
> org.apache.calcite.sql2rel.SqlToRelConverter.convertQuery(SqlToRelConverter.java:563)
> at org.apache.flink.table.planner.calcite.FlinkPlannerImpl.org
>
> $apache$flink$table$planner$calcite$FlinkPlannerImpl$$rel(FlinkPlannerImpl.scala:148)
> at
>
> org.apache.flink.table.planner.calcite.FlinkPlannerImpl.rel(FlinkPlannerImpl.scala:135)
> at
>
> org.apache.flink.table.planner.operations.SqlToOperationConverter.toQueryOperation(SqlToOperationConverter.java:522)
> at
>
> org.apache.flink.table.planner.operations.SqlToOperationConverter.convertSqlQuery(SqlToOperationConverter.java:436)
> at
>
> org.apache.flink.table.planner.operations.SqlToOperationConverter.convert(SqlToOperationConverter.java:154)
> at
>
> org.apache.flink.table.planner.delegation.ParserImpl.parse(ParserImpl.java:66)
> at
>
> org.apache.flink.table.api.internal.TableEnvironmentImpl.sqlQuery(TableEnvironmentImpl.java:464)
> at com.akulaku.data.main.StreamMain.main(StreamMain.java:87)
> at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
> at
>
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
> at
>
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> at java.lang.reflect.Method.invoke(Method.java:498)
> at
>
> org.apache.flink.client.program.PackagedProgram.callMainMethod(PackagedProgram.java:321)
> ... 8 more