Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

ExternalPythonOperator: AttributeError: 'python_path' is configured as a template field but ExternalPythonOperator does not have this attribute. #27232

Closed
2 tasks done
enisnazif opened this issue Oct 24, 2022 · 0 comments · Fixed by #27256
Labels
area:core kind:bug This is a clearly a bug

Comments

@enisnazif
Copy link
Contributor

enisnazif commented Oct 24, 2022

Apache Airflow version

Other Airflow 2 version (please specify below)

What happened

Using the ExternalPythonOperator directly in v2.4.2 as opposed to via the @task.external decorator described in https://airflow.apache.org/docs/apache-airflow/stable/howto/operator/python.html#externalpythonoperator causes the following error:

AttributeError: 'python_path' is configured as a template field but ExternalPythonOperator does not have this attribute.

This seems to be due to https://github.com/apache/airflow/blob/main/airflow/operators/python.py#L624 having 'python_path' as an additional template field, instead of 'python', which is the correct additional keyword argument for the operator

What you think should happen instead

We should change https://github.com/apache/airflow/blob/main/airflow/operators/python.py#L624 to
read:

template_fields: Sequence[str] = tuple({'python'} | set(PythonOperator.template_fields))

instead of

template_fields: Sequence[str] = tuple({'python_path'} | set(PythonOperator.template_fields))

This has been verified by adding:

ExternalPythonOperator.template_fields = tuple({'python'} | set(PythonOperator.template_fields))

in the sample DAG code below, which causes the DAG to run successfully

How to reproduce

import airflow
from airflow.models import DAG
from airflow.operators.python import ExternalPythonOperator

args = dict(
    start_date=airflow.utils.dates.days_ago(3),
    email=["x@y.com"],
    email_on_failure=False,
    email_on_retry=False,
    retries=0
)

dag = DAG(
    dag_id='test_dag',
    default_args=args,
    schedule_interval='0 20 * * *',
    catchup=False,
)

def print_kwargs(*args, **kwargs):
    print('args', args)
    print('kwargs', kwargs)


with dag:

    def print_hello():
        print('hello')

    # Due to a typo in the airflow library :(
    # ExternalPythonOperator.template_fields = tuple({'python'} | set(PythonOperator.template_fields))

    t1 = ExternalPythonOperator(
        task_id='test_task',
        python='/opt/airflow/miniconda/envs/nexus/bin/python',
        python_callable=print_kwargs
    )

Operating System

Ubuntu 18.04

Versions of Apache Airflow Providers

No response

Deployment

Docker-Compose

Deployment details

No response

Anything else

No response

Are you willing to submit PR?

  • Yes I am willing to submit a PR!

Code of Conduct

@enisnazif enisnazif added area:core kind:bug This is a clearly a bug labels Oct 24, 2022
enisnazif added a commit to enisnazif/airflow that referenced this issue Oct 25, 2022
potiuk added a commit to potiuk/airflow that referenced this issue Oct 31, 2022
This might be useful as sometims classic operators differ in
behaviour and it is great to test both.

Related: apache#27232
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
area:core kind:bug This is a clearly a bug
Projects
None yet
Development

Successfully merging a pull request may close this issue.

1 participant