You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by GitBox <gi...@apache.org> on 2021/03/08 11:30:26 UTC

[GitHub] [airflow] marcosmarxm opened a new issue #14660: KeyError template_fields does not exists when get_template_context

marcosmarxm opened a new issue #14660:
URL: https://github.com/apache/airflow/issues/14660


   ```python
   import unittest
   import pytest
   
   from airflow.models import TaskInstance, Variable
   from airflow.version import version
   
   from airflow.operators.python import PythonOperator
   from airflow import DAG
   from airflow.utils.dates import days_ago
   
   def someFunc(arg, **kwargs):
       print("hello word")
   
   class TestOperator(unittest.TestCase):
   
       def test_operator(self):
   
           dag_object = DAG(dag_id='dag_example',
            default_args={'owner': 'airflow'},
            schedule_interval='@daily',
            start_date=days_ago(1))
   
           test_operator = PythonOperator(
               dag=dag_object,
               task_id='test-task',
               retries=0,
               python_callable=someFunc,
               op_args=[0],
               templates_dict={
                   'database': 'some_database',
                   'query': 'some_query',
                   'output_location': 'output_location'}
           )
   
           ti = TaskInstance(task=test_operator, execution_date=dag_object.start_date)
           test_operator.execute(ti.get_template_context())
   ```
   This will raise the `KeyError: 'Variable template_fields does not exist'`
   Tracing the error:
   ```
   /airflow/operators/python.py
       self.op_kwargs = determine_kwargs(self.python_callable, self.op_args, context)
   /airflow/models/baseoperator.py:602: in __setattr__
       self.set_xcomargs_dependencies()
   /airflow/models/baseoperator.py:724: in set_xcomargs_dependencies
       apply_set_upstream(arg)
   /airflow/models/baseoperator.py:716: in apply_set_upstream
       apply_set_upstream(elem)
   /airflow/models/baseoperator.py:716: in apply_set_upstream
       apply_set_upstream(elem)
   /airflow/models/baseoperator.py:717: in apply_set_upstream
       elif hasattr(arg, "template_fields"):
   /airflow/models/taskinstance.py:1585: in __getattr__
       self.var = Variable.get(item, deserialize_json=True)
   ```
   
   In Airflow 1.0 the model Variable uses the method `get_variable` to return values.
   https://github.com/apache/airflow/blob/e6475938727941f24926c0d85f4d03a2b7064f2a/airflow/models/variable.py#L104-L122
   
   In Airflow 2.0 was changed to `get_variable_from_secrets`. This function only map variables in the `EnvironmentVariablesBackend` and the `MetastoreBackend` context.
   https://github.com/apache/airflow/blob/b718495e4caecb753742c3eb22919411a715f24a/airflow/models/variable.py#L113-L137
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ashb commented on issue #14660: KeyError template_fields does not exists when get_template_context

Posted by GitBox <gi...@apache.org>.
ashb commented on issue #14660:
URL: https://github.com/apache/airflow/issues/14660#issuecomment-808358088


   Oh actually, this may just be a "bug"/change needed in your tests for 2.0:
   
   ```
           test_operator.prepare_for_execution().execute(ti.get_template_context())
   ```
   
   This works.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ashb closed issue #14660: KeyError template_fields does not exists when get_template_context

Posted by GitBox <gi...@apache.org>.
ashb closed issue #14660:
URL: https://github.com/apache/airflow/issues/14660


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] boring-cyborg[bot] commented on issue #14660: KeyError template_fields does not exists when get_template_context

Posted by GitBox <gi...@apache.org>.
boring-cyborg[bot] commented on issue #14660:
URL: https://github.com/apache/airflow/issues/14660#issuecomment-792692523


   Thanks for opening your first issue here! Be sure to follow the issue template!
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] marcosmarxm commented on issue #14660: KeyError template_fields does not exists when get_template_context

Posted by GitBox <gi...@apache.org>.
marcosmarxm commented on issue #14660:
URL: https://github.com/apache/airflow/issues/14660#issuecomment-808378622


   Thanks @ashb I'm going to say this to the user 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ashb commented on issue #14660: KeyError template_fields does not exists when get_template_context

Posted by GitBox <gi...@apache.org>.
ashb commented on issue #14660:
URL: https://github.com/apache/airflow/issues/14660#issuecomment-808355191


   This is def a bug.
   
   It can be worked around by changing the python callable to not take `**kwargs`:
   
   ```
   def someFunc(arg):
       print("hello word")
   ```
   
   But this definately is a bug that we'll fix.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org