Aleksandr Pilipenko created FLINK-32962:
-------------------------------------------
Summary: Failure to install python dependencies from requirements
file
Key: FLINK-32962
URL: https://issues.apache.org/jira/browse/FLINK-32962
Project: Flink
Issue Type: Bug
Components: API / Python
Affects Versions: 1.17.1, 1.16.2, 1.15.4
Reporter: Aleksandr Pilipenko
We have encountered an issue when Flink fails to install python dependencies
from requirements file if python environment contains setuptools dependency
version 67.5.0 or above.
Flink job fails with following error:
{code:java}
py4j.protocol.Py4JJavaError: An error occurred while calling o118.await.:
java.util.concurrent.ExecutionException:
org.apache.flink.table.api.TableException: Failed to wait job finish at
java.base/java.util.concurrent.CompletableFuture.reportGet(CompletableFuture.java:395)
at
java.base/java.util.concurrent.CompletableFuture.get(CompletableFuture.java:1999)
at
org.apache.flink.table.api.internal.TableResultImpl.awaitInternal(TableResultImpl.java:118)
at
org.apache.flink.table.api.internal.TableResultImpl.await(TableResultImpl.java:81)
...Caused by: java.util.concurrent.ExecutionException:
org.apache.flink.client.program.ProgramInvocationException: Job failed (JobID:
2ca4026944022ac4537c503464d4c47f) at
java.base/java.util.concurrent.CompletableFuture.reportGet(CompletableFuture.java:395)
at
java.base/java.util.concurrent.CompletableFuture.get(CompletableFuture.java:1999)
at
org.apache.flink.table.api.internal.InsertResultProvider.hasNext(InsertResultProvider.java:83)
... 6 moreCaused by:
org.apache.flink.client.program.ProgramInvocationException: Job failed (JobID:
2ca4026944022ac4537c503464d4c47f) at
org.apache.flink.client.deployment.ClusterClientJobClientAdapter.lambda$null$6(ClusterClientJobClientAdapter.java:130)
at
java.base/java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:642)
at
java.base/java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:506)
at
java.base/java.util.concurrent.CompletableFuture.complete(CompletableFuture.java:2073)
...
Caused by: java.io.IOException: java.io.IOException: Failed to execute the
command: /Users/z3d1k/.pyenv/versions/3.9.17/bin/python -m pip install
--ignore-installed -r
/var/folders/rb/q_3h54_94b57gz_qkbp593vw0000gn/T/tm_localhost:63904-4178d3/blobStorage/job_2ca4026944022ac4537c503464d4c47f/blob_p-feb04bed919e628d98b1ef085111482f05bf43a1-1f5c3fda7cbdd6842e950e04d9d807c5
--install-option
--prefix=/var/folders/rb/q_3h54_94b57gz_qkbp593vw0000gn/T/python-dist-c17912db-5aba-439f-9e39-69cb8c957bec/python-requirementsoutput:Usage:
/Users/z3d1k/.pyenv/versions/3.9.17/bin/python -m pip install [options]
<requirement specifier> [package-index-options] ...
/Users/z3d1k/.pyenv/versions/3.9.17/bin/python -m pip install [options] -r
<requirements file> [package-index-options] ...
/Users/z3d1k/.pyenv/versions/3.9.17/bin/python -m pip install [options] [-e]
<vcs project url> ... /Users/z3d1k/.pyenv/versions/3.9.17/bin/python -m pip
install [options] [-e] <local project path> ...
/Users/z3d1k/.pyenv/versions/3.9.17/bin/python -m pip install [options]
<archive url/path> ...
no such option: --install-option
at
org.apache.flink.python.util.PythonEnvironmentManagerUtils.pipInstallRequirements(PythonEnvironmentManagerUtils.java:144)
at
org.apache.flink.python.env.AbstractPythonEnvironmentManager.installRequirements(AbstractPythonEnvironmentManager.java:215)
at
org.apache.flink.python.env.AbstractPythonEnvironmentManager.lambda$open$0(AbstractPythonEnvironmentManager.java:126)
at
org.apache.flink.python.env.AbstractPythonEnvironmentManager$PythonEnvResources.createResource(AbstractPythonEnvironmentManager.java:435)
at
org.apache.flink.python.env.AbstractPythonEnvironmentManager$PythonEnvResources.getOrAllocateSharedResource(AbstractPythonEnvironmentManager.java:402)
at
org.apache.flink.python.env.AbstractPythonEnvironmentManager.open(AbstractPythonEnvironmentManager.java:114)
at
org.apache.flink.streaming.api.runners.python.beam.BeamPythonFunctionRunner.open(BeamPythonFunctionRunner.java:238)
at
org.apache.flink.streaming.api.operators.python.process.AbstractExternalPythonFunctionOperator.open(AbstractExternalPythonFunctionOperator.java:57)
at
org.apache.flink.table.runtime.operators.python.AbstractStatelessFunctionOperator.open(AbstractStatelessFunctionOperator.java:92)
at
org.apache.flink.table.runtime.operators.python.scalar.AbstractPythonScalarFunctionOperator.open(AbstractPythonScalarFunctionOperator.java:101)
at
org.apache.flink.table.runtime.operators.python.scalar.PythonScalarFunctionOperator.open(PythonScalarFunctionOperator.java:71)
at
org.apache.flink.streaming.runtime.tasks.RegularOperatorChain.initializeStateAndOpenOperators(RegularOperatorChain.java:107)
at
org.apache.flink.streaming.runtime.tasks.StreamTask.restoreGates(StreamTask.java:734)
...Caused by: java.io.IOException: Failed to execute the command:
/Users/z3d1k/.pyenv/versions/3.9.17/bin/python -m pip install
--ignore-installed -r
/var/folders/rb/q_3h54_94b57gz_qkbp593vw0000gn/T/tm_localhost:63904-4178d3/blobStorage/job_2ca4026944022ac4537c503464d4c47f/blob_p-feb04bed919e628d98b1ef085111482f05bf43a1-1f5c3fda7cbdd6842e950e04d9d807c5
--install-option
--prefix=/var/folders/rb/q_3h54_94b57gz_qkbp593vw0000gn/T/python-dist-c17912db-5aba-439f-9e39-69cb8c957bec/python-requirementsoutput:Usage:
/Users/z3d1k/.pyenv/versions/3.9.17/bin/python -m pip install [options]
<requirement specifier> [package-index-options] ...
/Users/z3d1k/.pyenv/versions/3.9.17/bin/python -m pip install [options] -r
<requirements file> [package-index-options] ...
/Users/z3d1k/.pyenv/versions/3.9.17/bin/python -m pip install [options] [-e]
<vcs project url> ... /Users/z3d1k/.pyenv/versions/3.9.17/bin/python -m pip
install [options] [-e] <local project path> ...
/Users/z3d1k/.pyenv/versions/3.9.17/bin/python -m pip install [options]
<archive url/path> ...
no such option: --install-option
at
org.apache.flink.python.util.PythonEnvironmentManagerUtils.execute(PythonEnvironmentManagerUtils.java:220)
at
org.apache.flink.python.util.PythonEnvironmentManagerUtils.pipInstallRequirements(PythonEnvironmentManagerUtils.java:130)
... 20 more {code}
Before installing dependencies provided in the requirements.txt file, Flink
checks version of pip, and based on result constructs cli command to execute
[1]. Version check is performed by running following command [2]:
{code:java}
python -c "import sys;from pkg_resources import get_distribution,
parse_version;pip_version =
get_distribution('pip').version;print(parse_version(pip_version) >=
parse_version(sys.args[1])) {code}
Starting from this version, python will print deprecation warning when package
pkg_resources is used [3]:
{code:java}
<string>:1: DeprecationWarning: pkg_resources is deprecated as an API. See
https://setuptools.pypa.io/en/latest/pkg_resources.html
True {code}
This warning breaks output parsing, causing Flink to assume that python
environment has pip with version 8.0.0 or below, adding "--install-option"
argument. This argument had been deprecated and removed from recent pip
versions.
*Proposed solution:*
Remove pip version check since pip of version `8.0.0` does not support recent
python versions:
Documentation states that Flink supports python 3.7 and above for Flink 1.17
[4] and earliest version to support python 3.7 is pip 18.1 [5].
*References:*
[1]:
[https://github.com/apache/flink/blob/c1fba732738aaaa0d576f80881eb464a954e12c5/flink-python/src/main/java/org/apache/flink/python/util/PythonEnvironmentManagerUtils.java#L61-L65]
[2]:
[https://github.com/apache/flink/blob/c1fba732738aaaa0d576f80881eb464a954e12c5/flink-python/src/main/java/org/apache/flink/python/util/PythonEnvironmentManagerUtils.java#L109-L117]
[3]: [https://github.com/pypa/setuptools/pull/3843]
[4]:
[https://nightlies.apache.org/flink/flink-docs-release-1.17/docs/dev/python/installation/]
[5]: [https://pip.pypa.io/en/stable/news/#v18-1]
--
This message was sent by Atlassian Jira
(v8.20.10#820010)