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

Guowei Ma edited comment on FLINK-15786 at 2/28/20 6:40 AM:
------------------------------------------------------------

Thanks [~pnowojski] for your detailed explanation.

I agree with you that we could change the DataStream API to postpone the 
construction of the source/sink operator. This could avoid the user to use the 
Plugin mechanism directly to create a source/sink object at the client-side for 
separating the connector class loader from the user code classloader.

What I concern about is that what the relationship between the 
FlinkUserClassLoader and the connector Classloader. In theory, there might be 
interdependent scenarios such as FlinkKafkaPartitioner/KafkaConnector. This 
interface is provided by the Kafka connector. And the user could implement it. 
The implemented class is also passed to the Kafka connector and would be 
constructed at runtime. One option for this scenario is to provide a 
classloader that knows both the user implemented FlinkKafkaPartitioner class 
and the Kafka connector class. We could call it XClassloader at first. 

Do you think that we need to resolve this scenario?  Do you have any concerns 
about the XClassloader? A related open question is that the datastream users 
could also enjoy the benefits of classloader isolation without changing any 
code if we choose to use the XClassLoader solution.


was (Author: maguowei):
Thanks [~pnowojski] for your detailed explanation.

I agree with you that we could change the DataStream API to postpone the 
construction of the source/sink operator. This could avoid the user to use the 
Plugin mechanism directly to create a source/sink object at the client-side for 
separating the connector class loader from the user code classloader.

What I concern about is that what the relationship between the 
FlinkUserClassLoader and the connector Classloader. In theory, there might be 
interdependent scenarios such as FlinkKafkaPartitioner/KafkaConnector. This 
interface is provided by the Kafka connector. And the user could implement it. 
The implemented class is also passed to the Kafka connector and would be 
constructed at runtime. One option for this scenario is to provide a 
classloader that knows both the user implemented FlinkKafkaPartitioner class 
and the Kafka connector class. We could call it XClassloader at first. 

Do you think that we need to resolve this scenario?  A related open question is 
that we might not need to change(or add) the DataStream API for separating the 
connector classloader if we choose to use the XClassloader.

> Load connector code with separate classloader
> ---------------------------------------------
>
>                 Key: FLINK-15786
>                 URL: https://issues.apache.org/jira/browse/FLINK-15786
>             Project: Flink
>          Issue Type: Improvement
>          Components: Runtime / Task
>            Reporter: Guowei Ma
>            Priority: Major
>              Labels: usability
>
> Currently, connector code can be seen as part of user code. Usually, users 
> only need to add the corresponding connector as a dependency and package it 
> in the user jar. This is convenient enough.
> However, connectors usually need to interact with external systems and often 
> introduce heavy dependencies, there is a high possibility of a class conflict 
> of different connectors or the user code of the same job. For example, every 
> one or two weeks, we will receive issue reports relevant with connector class 
> conflict from our users. The problem can get worse when users want to analyze 
> data from different sources and write output to different sinks.
> Using separate classloader to load the different connector code could resolve 
> the problem.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to