Skip to content

Commit

Permalink
Add note about using dag_run.conf in BashOperator (#9143)
Browse files Browse the repository at this point in the history
(cherry picked from commit 4d8599e)
  • Loading branch information
ashb authored and kaxil committed Jul 1, 2020
1 parent 4918b85 commit 2fa5157
Show file tree
Hide file tree
Showing 3 changed files with 65 additions and 3 deletions.
4 changes: 2 additions & 2 deletions airflow/example_dags/example_trigger_target_dag.py
Expand Up @@ -66,7 +66,7 @@ def run_this_func(ds, **kwargs):
# You can also access the DagRun object in templates
bash_task = BashOperator(
task_id="bash_task",
bash_command='echo "Here is the message: '
'{{ dag_run.conf["message"] if dag_run else "" }}" ',
bash_command='echo "Here is the message: $message"',
env={'message': '{{ dag_run.conf["message"] if dag_run else "" }}'},
dag=dag,
)
33 changes: 32 additions & 1 deletion airflow/operators/bash_operator.py
Expand Up @@ -33,7 +33,7 @@


class BashOperator(BaseOperator):
"""
r"""
Execute a Bash script, command or set of commands.
.. seealso::
Expand All @@ -53,6 +53,37 @@ class BashOperator(BaseOperator):
:type env: dict
:param output_encoding: Output encoding of bash command
:type output_encoding: str
.. warning::
Care should be taken with "user" input or when using Jinja templates in the
``bash_command``, as this bash operator does not perform any escaping or
sanitization of the command.
This applies mostly to using "dag_run" conf, as that can be submitted via
users in the Web UI. Most of the default template variables are not at
risk.
For example, do **not** do this:
.. code-block:: python
bash_task = BashOperator(
task_id="bash_task",
bash_command='echo "Here is the message: \'{{ dag_run.conf["message"] if dag_run else "" }}\'"',
)
Instead, you should pass this via the ``env`` kwarg and use double-quotes
inside the bash_command, as below:
.. code-block:: python
bash_task = BashOperator(
task_id="bash_task",
bash_command='echo "here is the message: \'$message\'"',
env={'message': '{{ dag_run.conf["message"] if dag_run else "" }}'},
)
"""
template_fields = ('bash_command', 'env')
template_ext = ('.sh', '.bash',)
Expand Down
31 changes: 31 additions & 0 deletions docs/howto/operator/bash.rst
Expand Up @@ -41,6 +41,37 @@ You can use :ref:`Jinja templates <jinja-templating>` to parameterize the
:start-after: [START howto_operator_bash_template]
:end-before: [END howto_operator_bash_template]


.. warning::

Care should be taken with "user" input or when using Jinja templates in the
``bash_command``, as this bash operator does not perform any escaping or
sanitization of the command.

This applies mostly to using "dag_run" conf, as that can be submitted via
users in the Web UI. Most of the default template variables are not at
risk.

For example, do **not** do this:

.. code-block:: python
bash_task = BashOperator(
task_id="bash_task",
bash_command='echo "Here is the message: \'{{ dag_run.conf["message"] if dag_run else "" }}\'"',
)
Instead, you should pass this via the ``env`` kwarg and use double-quotes
inside the bash_command, as below:

.. code-block:: python
bash_task = BashOperator(
task_id="bash_task",
bash_command='echo "here is the message: \'$message\'"',
env={'message': '{{ dag_run.conf["message"] if dag_run else "" }}'},
)
Troubleshooting
---------------

Expand Down

0 comments on commit 2fa5157

Please sign in to comment.