Skip to content

Commit 83f8e17

Browse files
authored
Even more typing in operators (template_fields/ext) (#20608)
Part of #19891 There were few more places where I missed adding Sequence typing - including examples (also converted to tuples) and also template_ext. Also in a few places iterable was left
1 parent 57ed3ab commit 83f8e17

File tree

89 files changed

+150
-146
lines changed

Some content is hidden

Large Commits have some content hidden by default. Use the searchbox below for content that may be hidden.

89 files changed

+150
-146
lines changed

airflow/decorators/python.py

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -15,7 +15,7 @@
1515
# specific language governing permissions and limitations
1616
# under the License.
1717

18-
from typing import Callable, Optional, TypeVar
18+
from typing import Callable, Optional, Sequence, TypeVar
1919

2020
from airflow.decorators.base import DecoratedOperator, task_decorator_factory
2121
from airflow.operators.python import PythonOperator
@@ -39,12 +39,12 @@ class _PythonDecoratedOperator(DecoratedOperator, PythonOperator):
3939
:type multiple_outputs: bool
4040
"""
4141

42-
template_fields = ('op_args', 'op_kwargs')
42+
template_fields: Sequence[str] = ('op_args', 'op_kwargs')
4343
template_fields_renderers = {"op_args": "py", "op_kwargs": "py"}
4444

4545
# since we won't mutate the arguments, we should just do the shallow copy
4646
# there are some cases we can't deepcopy the objects (e.g protobuf).
47-
shallow_copy_attrs = ('python_callable',)
47+
shallow_copy_attrs: Sequence[str] = ('python_callable',)
4848

4949
def __init__(
5050
self,

airflow/decorators/python_virtualenv.py

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -17,7 +17,7 @@
1717

1818
import inspect
1919
from textwrap import dedent
20-
from typing import Callable, Optional, TypeVar
20+
from typing import Callable, Optional, Sequence, TypeVar
2121

2222
from airflow.decorators.base import DecoratedOperator, task_decorator_factory
2323
from airflow.operators.python import PythonVirtualenvOperator
@@ -42,12 +42,12 @@ class _PythonVirtualenvDecoratedOperator(DecoratedOperator, PythonVirtualenvOper
4242
:type multiple_outputs: bool
4343
"""
4444

45-
template_fields = ('op_args', 'op_kwargs')
45+
template_fields: Sequence[str] = ('op_args', 'op_kwargs')
4646
template_fields_renderers = {"op_args": "py", "op_kwargs": "py"}
4747

4848
# since we won't mutate the arguments, we should just do the shallow copy
4949
# there are some cases we can't deepcopy the objects (e.g protobuf).
50-
shallow_copy_attrs = ('python_callable',)
50+
shallow_copy_attrs: Sequence[str] = ('python_callable',)
5151

5252
def __init__(
5353
self,

airflow/operators/bash.py

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -16,7 +16,7 @@
1616
# specific language governing permissions and limitations
1717
# under the License.
1818
import os
19-
from typing import Dict, Optional
19+
from typing import Dict, Optional, Sequence
2020

2121
from airflow.compat.functools import cached_property
2222
from airflow.exceptions import AirflowException, AirflowSkipException
@@ -128,9 +128,9 @@ class BashOperator(BaseOperator):
128128
129129
"""
130130

131-
template_fields = ('bash_command', 'env')
131+
template_fields: Sequence[str] = ('bash_command', 'env')
132132
template_fields_renderers = {'bash_command': 'bash', 'env': 'json'}
133-
template_ext = (
133+
template_ext: Sequence[str] = (
134134
'.sh',
135135
'.bash',
136136
)

airflow/operators/email.py

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -15,7 +15,7 @@
1515
# KIND, either express or implied. See the License for the
1616
# specific language governing permissions and limitations
1717
# under the License.
18-
from typing import Any, Dict, List, Optional, Union
18+
from typing import Any, Dict, List, Optional, Sequence, Union
1919

2020
from airflow.models import BaseOperator
2121
from airflow.utils.context import Context
@@ -48,9 +48,9 @@ class EmailOperator(BaseOperator):
4848
:type custom_headers: dict
4949
"""
5050

51-
template_fields = ('to', 'subject', 'html_content', 'files')
51+
template_fields: Sequence[str] = ('to', 'subject', 'html_content', 'files')
5252
template_fields_renderers = {"html_content": "html"}
53-
template_ext = ('.html',)
53+
template_ext: Sequence[str] = ('.html',)
5454
ui_color = '#e6faf9'
5555

5656
def __init__(

airflow/operators/generic_transfer.py

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -15,7 +15,7 @@
1515
# KIND, either express or implied. See the License for the
1616
# specific language governing permissions and limitations
1717
# under the License.
18-
from typing import List, Optional, Union
18+
from typing import List, Optional, Sequence, Union
1919

2020
from airflow.hooks.base import BaseHook
2121
from airflow.models import BaseOperator
@@ -46,8 +46,8 @@ class GenericTransfer(BaseOperator):
4646
:type insert_args: dict
4747
"""
4848

49-
template_fields = ('sql', 'destination_table', 'preoperator')
50-
template_ext = (
49+
template_fields: Sequence[str] = ('sql', 'destination_table', 'preoperator')
50+
template_ext: Sequence[str] = (
5151
'.sql',
5252
'.hql',
5353
)

airflow/operators/sql.py

Lines changed: 14 additions & 14 deletions
Original file line numberDiff line numberDiff line change
@@ -15,7 +15,7 @@
1515
# KIND, either express or implied. See the License for the
1616
# specific language governing permissions and limitations
1717
# under the License.
18-
from typing import Any, Dict, Iterable, List, Mapping, Optional, SupportsAbs, Union
18+
from typing import Any, Dict, Iterable, List, Mapping, Optional, Sequence, SupportsAbs, Union
1919

2020
from airflow.compat.functools import cached_property
2121
from airflow.exceptions import AirflowException
@@ -124,8 +124,8 @@ class SQLCheckOperator(BaseSQLOperator):
124124
:type database: str
125125
"""
126126

127-
template_fields: Iterable[str] = ("sql",)
128-
template_ext: Iterable[str] = (
127+
template_fields: Sequence[str] = ("sql",)
128+
template_ext: Sequence[str] = (
129129
".hql",
130130
".sql",
131131
)
@@ -178,14 +178,14 @@ class SQLValueCheckOperator(BaseSQLOperator):
178178
"""
179179

180180
__mapper_args__ = {"polymorphic_identity": "SQLValueCheckOperator"}
181-
template_fields = (
181+
template_fields: Sequence[str] = (
182182
"sql",
183183
"pass_value",
184-
) # type: Iterable[str]
185-
template_ext = (
184+
)
185+
template_ext: Sequence[str] = (
186186
".hql",
187187
".sql",
188-
) # type: Iterable[str]
188+
)
189189
ui_color = "#fff7e6"
190190

191191
def __init__(
@@ -289,8 +289,8 @@ class SQLIntervalCheckOperator(BaseSQLOperator):
289289
"""
290290

291291
__mapper_args__ = {"polymorphic_identity": "SQLIntervalCheckOperator"}
292-
template_fields: Iterable[str] = ("sql1", "sql2")
293-
template_ext: Iterable[str] = (
292+
template_fields: Sequence[str] = ("sql1", "sql2")
293+
template_ext: Sequence[str] = (
294294
".hql",
295295
".sql",
296296
)
@@ -418,11 +418,11 @@ class SQLThresholdCheckOperator(BaseSQLOperator):
418418
:type max_threshold: numeric or str
419419
"""
420420

421-
template_fields = ("sql", "min_threshold", "max_threshold")
422-
template_ext = (
421+
template_fields: Sequence[str] = ("sql", "min_threshold", "max_threshold")
422+
template_ext: Sequence[str] = (
423423
".hql",
424424
".sql",
425-
) # type: Iterable[str]
425+
)
426426

427427
def __init__(
428428
self,
@@ -505,8 +505,8 @@ class BranchSQLOperator(BaseSQLOperator, SkipMixin):
505505
:type parameters: mapping or iterable
506506
"""
507507

508-
template_fields = ("sql",)
509-
template_ext = (".sql",)
508+
template_fields: Sequence[str] = ("sql",)
509+
template_ext: Sequence[str] = (".sql",)
510510
ui_color = "#a22034"
511511
ui_fgcolor = "#F7F7F7"
512512

airflow/operators/trigger_dagrun.py

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -19,7 +19,7 @@
1919
import datetime
2020
import json
2121
import time
22-
from typing import Dict, List, Optional, Union
22+
from typing import Dict, List, Optional, Sequence, Union
2323

2424
from airflow.api.common.trigger_dag import trigger_dag
2525
from airflow.exceptions import AirflowException, DagNotFound, DagRunAlreadyExists
@@ -83,7 +83,7 @@ class TriggerDagRunOperator(BaseOperator):
8383
:type failed_states: list
8484
"""
8585

86-
template_fields = ("trigger_dag_id", "trigger_run_id", "execution_date", "conf")
86+
template_fields: Sequence[str] = ("trigger_dag_id", "trigger_run_id", "execution_date", "conf")
8787
template_fields_renderers = {"conf": "py"}
8888
ui_color = "#ffefeb"
8989

airflow/providers/amazon/aws/operators/athena.py

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -65,7 +65,7 @@ class AthenaOperator(BaseOperator):
6565

6666
ui_color = '#44b5e2'
6767
template_fields: Sequence[str] = ('query', 'database', 'output_location')
68-
template_ext = ('.sql',)
68+
template_ext: Sequence[str] = ('.sql',)
6969
template_fields_renderers = {"query": "sql"}
7070

7171
def __init__(

airflow/providers/amazon/aws/operators/cloud_formation.py

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -41,7 +41,7 @@ class CloudFormationCreateStackOperator(BaseOperator):
4141
"""
4242

4343
template_fields: Sequence[str] = ('stack_name',)
44-
template_ext = ()
44+
template_ext: Sequence[str] = ()
4545
ui_color = '#6b9659'
4646

4747
def __init__(self, *, stack_name: str, params: dict, aws_conn_id: str = 'aws_default', **kwargs):
@@ -73,7 +73,7 @@ class CloudFormationDeleteStackOperator(BaseOperator):
7373
"""
7474

7575
template_fields: Sequence[str] = ('stack_name',)
76-
template_ext = ()
76+
template_ext: Sequence[str] = ()
7777
ui_color = '#1d472b'
7878
ui_fgcolor = '#FFF'
7979

airflow/providers/amazon/aws/operators/dms.py

Lines changed: 5 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -65,7 +65,7 @@ class DmsCreateTaskOperator(BaseOperator):
6565
'migration_type',
6666
'create_task_kwargs',
6767
)
68-
template_ext = ()
68+
template_ext: Sequence[str] = ()
6969
template_fields_renderers = {
7070
"table_mappings": "json",
7171
"create_task_kwargs": "json",
@@ -135,7 +135,7 @@ class DmsDeleteTaskOperator(BaseOperator):
135135
"""
136136

137137
template_fields: Sequence[str] = ('replication_task_arn',)
138-
template_ext = ()
138+
template_ext: Sequence[str] = ()
139139
template_fields_renderers: Dict[str, str] = {}
140140

141141
def __init__(
@@ -175,7 +175,7 @@ class DmsDescribeTasksOperator(BaseOperator):
175175
"""
176176

177177
template_fields: Sequence[str] = ('describe_tasks_kwargs',)
178-
template_ext = ()
178+
template_ext: Sequence[str] = ()
179179
template_fields_renderers: Dict[str, str] = {'describe_tasks_kwargs': 'json'}
180180

181181
def __init__(
@@ -228,7 +228,7 @@ class DmsStartTaskOperator(BaseOperator):
228228
'start_replication_task_type',
229229
'start_task_kwargs',
230230
)
231-
template_ext = ()
231+
template_ext: Sequence[str] = ()
232232
template_fields_renderers = {'start_task_kwargs': 'json'}
233233

234234
def __init__(
@@ -277,7 +277,7 @@ class DmsStopTaskOperator(BaseOperator):
277277
"""
278278

279279
template_fields: Sequence[str] = ('replication_task_arn',)
280-
template_ext = ()
280+
template_ext: Sequence[str] = ()
281281
template_fields_renderers: Dict[str, str] = {}
282282

283283
def __init__(

0 commit comments

Comments
 (0)