26
26
from airflow .exceptions import AirflowException
27
27
from airflow .models .expandinput import NotFullyPopulated
28
28
from airflow .models .taskmixin import DAGNode
29
+ from airflow .template .templater import Templater
29
30
from airflow .utils .context import Context
30
- from airflow .utils .helpers import render_template_as_native , render_template_to_string
31
- from airflow .utils .log .logging_mixin import LoggingMixin
32
- from airflow .utils .mixins import ResolveMixin
33
31
from airflow .utils .session import NEW_SESSION , provide_session
34
32
from airflow .utils .sqlalchemy import skip_locked , with_row_locks
35
33
from airflow .utils .state import State , TaskInstanceState
@@ -76,7 +74,7 @@ class NotMapped(Exception):
76
74
"""Raise if a task is neither mapped nor has any parent mapped groups."""
77
75
78
76
79
- class AbstractOperator (LoggingMixin , DAGNode ):
77
+ class AbstractOperator (Templater , DAGNode ):
80
78
"""Common implementation for operators, including unmapped and mapped.
81
79
82
80
This base class is more about sharing implementations, not defining a common
@@ -96,10 +94,6 @@ class AbstractOperator(LoggingMixin, DAGNode):
96
94
97
95
# Defines the operator level extra links.
98
96
operator_extra_links : Collection [BaseOperatorLink ]
99
- # For derived classes to define which fields will get jinjaified.
100
- template_fields : Collection [str ]
101
- # Defines which files extensions to look for in the templated fields.
102
- template_ext : Sequence [str ]
103
97
104
98
owner : str
105
99
task_id : str
@@ -153,48 +147,6 @@ def dag_id(self) -> str:
153
147
def node_id (self ) -> str :
154
148
return self .task_id
155
149
156
- def get_template_env (self ) -> jinja2 .Environment :
157
- """Fetch a Jinja template environment from the DAG or instantiate empty environment if no DAG."""
158
- # This is imported locally since Jinja2 is heavy and we don't need it
159
- # for most of the functionalities. It is imported by get_template_env()
160
- # though, so we don't need to put this after the 'if dag' check.
161
- from airflow .templates import SandboxedEnvironment
162
-
163
- dag = self .get_dag ()
164
- if dag :
165
- return dag .get_template_env (force_sandboxed = False )
166
- return SandboxedEnvironment (cache_size = 0 )
167
-
168
- def prepare_template (self ) -> None :
169
- """Hook triggered after the templated fields get replaced by their content.
170
-
171
- If you need your operator to alter the content of the file before the
172
- template is rendered, it should override this method to do so.
173
- """
174
-
175
- def resolve_template_files (self ) -> None :
176
- """Getting the content of files for template_field / template_ext."""
177
- if self .template_ext :
178
- for field in self .template_fields :
179
- content = getattr (self , field , None )
180
- if content is None :
181
- continue
182
- elif isinstance (content , str ) and any (content .endswith (ext ) for ext in self .template_ext ):
183
- env = self .get_template_env ()
184
- try :
185
- setattr (self , field , env .loader .get_source (env , content )[0 ]) # type: ignore
186
- except Exception :
187
- self .log .exception ("Failed to resolve template field %r" , field )
188
- elif isinstance (content , list ):
189
- env = self .get_template_env ()
190
- for i , item in enumerate (content ):
191
- if isinstance (item , str ) and any (item .endswith (ext ) for ext in self .template_ext ):
192
- try :
193
- content [i ] = env .loader .get_source (env , item )[0 ] # type: ignore
194
- except Exception :
195
- self .log .exception ("Failed to get source %s" , item )
196
- self .prepare_template ()
197
-
198
150
def get_direct_relative_ids (self , upstream : bool = False ) -> set [str ]:
199
151
"""Get direct relative IDs to the current task, upstream or downstream."""
200
152
if upstream :
@@ -580,6 +532,17 @@ def render_template_fields(
580
532
"""
581
533
raise NotImplementedError ()
582
534
535
+ def _render (self , template , context , dag : DAG | None = None ):
536
+ if dag is None :
537
+ dag = self .get_dag ()
538
+ return super ()._render (template , context , dag = dag )
539
+
540
+ def get_template_env (self , dag : DAG | None = None ) -> jinja2 .Environment :
541
+ """Get the template environment for rendering templates."""
542
+ if dag is None :
543
+ dag = self .get_dag ()
544
+ return super ().get_template_env (dag = dag )
545
+
583
546
@provide_session
584
547
def _do_render_template_fields (
585
548
self ,
@@ -591,6 +554,7 @@ def _do_render_template_fields(
591
554
* ,
592
555
session : Session = NEW_SESSION ,
593
556
) -> None :
557
+ """Override the base to use custom error logging."""
594
558
for attr_name in template_fields :
595
559
try :
596
560
value = getattr (parent , attr_name )
@@ -618,85 +582,3 @@ def _do_render_template_fields(
618
582
raise
619
583
else :
620
584
setattr (parent , attr_name , rendered_content )
621
-
622
- def render_template (
623
- self ,
624
- content : Any ,
625
- context : Context ,
626
- jinja_env : jinja2 .Environment | None = None ,
627
- seen_oids : set [int ] | None = None ,
628
- ) -> Any :
629
- """Render a templated string.
630
-
631
- If *content* is a collection holding multiple templated strings, strings
632
- in the collection will be templated recursively.
633
-
634
- :param content: Content to template. Only strings can be templated (may
635
- be inside a collection).
636
- :param context: Dict with values to apply on templated content
637
- :param jinja_env: Jinja environment. Can be provided to avoid
638
- re-creating Jinja environments during recursion.
639
- :param seen_oids: template fields already rendered (to avoid
640
- *RecursionError* on circular dependencies)
641
- :return: Templated content
642
- """
643
- # "content" is a bad name, but we're stuck to it being public API.
644
- value = content
645
- del content
646
-
647
- if seen_oids is not None :
648
- oids = seen_oids
649
- else :
650
- oids = set ()
651
-
652
- if id (value ) in oids :
653
- return value
654
-
655
- if not jinja_env :
656
- jinja_env = self .get_template_env ()
657
-
658
- if isinstance (value , str ):
659
- if any (value .endswith (ext ) for ext in self .template_ext ): # A filepath.
660
- template = jinja_env .get_template (value )
661
- else :
662
- template = jinja_env .from_string (value )
663
- dag = self .get_dag ()
664
- if dag and dag .render_template_as_native_obj :
665
- return render_template_as_native (template , context )
666
- return render_template_to_string (template , context )
667
-
668
- if isinstance (value , ResolveMixin ):
669
- return value .resolve (context )
670
-
671
- # Fast path for common built-in collections.
672
- if value .__class__ is tuple :
673
- return tuple (self .render_template (element , context , jinja_env , oids ) for element in value )
674
- elif isinstance (value , tuple ): # Special case for named tuples.
675
- return value .__class__ (* (self .render_template (el , context , jinja_env , oids ) for el in value ))
676
- elif isinstance (value , list ):
677
- return [self .render_template (element , context , jinja_env , oids ) for element in value ]
678
- elif isinstance (value , dict ):
679
- return {k : self .render_template (v , context , jinja_env , oids ) for k , v in value .items ()}
680
- elif isinstance (value , set ):
681
- return {self .render_template (element , context , jinja_env , oids ) for element in value }
682
-
683
- # More complex collections.
684
- self ._render_nested_template_fields (value , context , jinja_env , oids )
685
- return value
686
-
687
- def _render_nested_template_fields (
688
- self ,
689
- value : Any ,
690
- context : Context ,
691
- jinja_env : jinja2 .Environment ,
692
- seen_oids : set [int ],
693
- ) -> None :
694
- if id (value ) in seen_oids :
695
- return
696
- seen_oids .add (id (value ))
697
- try :
698
- nested_template_fields = value .template_fields
699
- except AttributeError :
700
- # content has no inner template fields
701
- return
702
- self ._do_render_template_fields (value , nested_template_fields , context , jinja_env , seen_oids )
0 commit comments