Nuno Afonso created FLINK-30803: ----------------------------------- Summary: PyFlink mishandles script dependencies Key: FLINK-30803 URL: https://issues.apache.org/jira/browse/FLINK-30803 Project: Flink Issue Type: Bug Components: API / Python Affects Versions: 1.15.3, 1.15.2, 1.16.0 Reporter: Nuno Afonso Attachments: word_count_split.zip
h2. Summary Since Flink 1.15, PyFlink is unable to run scripts that import other scripts under other directories. For instance, if _main.py_ imports {_}job/word_count.py{_}, PyFlink will fail due to not finding the _job_ directory. The issue seems to have started after a [refactoring of _PythonDriver_|https://github.com/apache/flink/commit/330aae0c6e0811f50888d17830f10f7a29efe7d7] to address FLINK-26847. The path to the Python script is removed, which forces PyFlink to use the copy in its temporary directory. When files are copied to this directory, the original directory structure is not maintained and ends up breaking the imports. h2. Testing To confirm the regression, I ran the attached application in both a Flink 1.14.6 and 1.15.3 clusters. h3. Flink 1.14.6 Application was able to start after being submitted via CLI: {code:java} % ./bin/flink run --python ~/sandbox/word_count_split/main.py WARNING: An illegal reflective access operation has occurred WARNING: Illegal reflective access by org.apache.flink.api.java.ClosureCleaner (file:/.../flink-1.14.6/lib/flink-dist_2.12-1.14.6.jar) to field java.lang.String.value WARNING: Please consider reporting this to the maintainers of org.apache.flink.api.java.ClosureCleaner WARNING: Use --illegal-access=warn to enable warnings of further illegal reflective access operations WARNING: All illegal access operations will be denied in a future release Job has been submitted with JobID 6f7be21072384ca3a314af10860c4ba8 {code} h3. Flink 1.15.3 Application did not start due to not finding the _job_ directory: {code:java} % ./bin/flink run --python ~/sandbox/word_count_split/main.py Traceback (most recent call last): File "/usr/lib64/python3.7/runpy.py", line 193, in _run_module_as_main "__main__", mod_spec) File "/usr/lib64/python3.7/runpy.py", line 85, in _run_code exec(code, run_globals) File "/tmp/pyflink/40c649c3-24af-46ef-ae27-e0019cb55769/3673dd18-adff-40e0-bb11-06a3f00ba29c/main.py", line 5, in <module> from job.word_count import word_count ModuleNotFoundError: No module named 'job' org.apache.flink.client.program.ProgramAbortException: java.lang.RuntimeException: Python process exits with code: 1 at org.apache.flink.client.python.PythonDriver.main(PythonDriver.java:140) at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.base/java.lang.reflect.Method.invoke(Method.java:566) at org.apache.flink.client.program.PackagedProgram.callMainMethod(PackagedProgram.java:355) at org.apache.flink.client.program.PackagedProgram.invokeInteractiveModeForExecution(PackagedProgram.java:222) at org.apache.flink.client.ClientUtils.executeProgram(ClientUtils.java:114) at org.apache.flink.client.cli.CliFrontend.executeProgram(CliFrontend.java:841) at org.apache.flink.client.cli.CliFrontend.run(CliFrontend.java:240) at org.apache.flink.client.cli.CliFrontend.parseAndRun(CliFrontend.java:1085) at org.apache.flink.client.cli.CliFrontend.lambda$main$10(CliFrontend.java:1163) at org.apache.flink.runtime.security.contexts.NoOpSecurityContext.runSecured(NoOpSecurityContext.java:28) at org.apache.flink.client.cli.CliFrontend.main(CliFrontend.java:1163) Caused by: java.lang.RuntimeException: Python process exits with code: 1 at org.apache.flink.client.python.PythonDriver.main(PythonDriver.java:130) ... 13 more {code} -- This message was sent by Atlassian Jira (v8.20.10#820010)