[ 
https://issues.apache.org/jira/browse/FLINK-24336?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Nicholas Jiang updated FLINK-24336:
-----------------------------------
    Description: 
When I run flink client to submit a python based workflow, I got the 
MalformedURLException like this:

https://gist.github.com/is/faabafc7f8750f3f3161fbb6517ed6ff

After some debug work, I found the problem is related with 
TableEvneriontment.execute_sql. The root cause is  
TableEvenriontment._add_jars_to_j_env_config in 
pyflink/table/TableEnverionment.py.

```
if j_configuration.containsKey(config_key):
  for url in j_configuration.getString(config_key, "").split(";"):
    jar_urls_set.add(url)
```

In our case, pipeline.classpaths was set by empty list value
from FromProgramOption, so the upper code block will
introduce a empty string ("") into pipeline.classpaths, for example
"a.jar;b.jar;;c.jar", and it will cause the according exception.

Another problem, the order of string set in python is not
determinate, so ";".join(jar_urls_set) does NOT keep the
classpaths order. The list is more suiteable in this case.

> PyFlink TableEnvironment executes the SQL randomly MalformedURLException with 
> the configuration for 'pipeline.classpaths'
> -------------------------------------------------------------------------------------------------------------------------
>
>                 Key: FLINK-24336
>                 URL: https://issues.apache.org/jira/browse/FLINK-24336
>             Project: Flink
>          Issue Type: Bug
>          Components: API / Python
>    Affects Versions: 1.14.0, 1.13.2, 1.14.1
>            Reporter: Nicholas Jiang
>            Priority: Major
>
> When I run flink client to submit a python based workflow, I got the 
> MalformedURLException like this:
> https://gist.github.com/is/faabafc7f8750f3f3161fbb6517ed6ff
> After some debug work, I found the problem is related with 
> TableEvneriontment.execute_sql. The root cause is  
> TableEvenriontment._add_jars_to_j_env_config in 
> pyflink/table/TableEnverionment.py.
> ```
> if j_configuration.containsKey(config_key):
>   for url in j_configuration.getString(config_key, "").split(";"):
>     jar_urls_set.add(url)
> ```
> In our case, pipeline.classpaths was set by empty list value
> from FromProgramOption, so the upper code block will
> introduce a empty string ("") into pipeline.classpaths, for example
> "a.jar;b.jar;;c.jar", and it will cause the according exception.
> Another problem, the order of string set in python is not
> determinate, so ";".join(jar_urls_set) does NOT keep the
> classpaths order. The list is more suiteable in this case.



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

Reply via email to