You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by as...@apache.org on 2021/05/18 10:51:43 UTC

[airflow] 04/04: Sandbox templates (#15912)

This is an automated email from the ASF dual-hosted git repository.

ash pushed a commit to branch v2-1-test
in repository https://gitbox.apache.org/repos/asf/airflow.git

commit 304e174674ff6921cb7ed79c0158949b50eff8fe
Author: Ash Berlin-Taylor <as...@firemirror.com>
AuthorDate: Tue May 18 11:50:47 2021 +0100

    Sandbox templates (#15912)
    
    Templates _shouldn't_ ever be taken from untrusted user input (and it's hard to
    do so without just edit the dag file, at which point you can run whatever
    python code you like _anyway_), but this is a reasonable safety measure just in
    case someone does something "clever".
    
    (cherry picked from commit 429584525fd7037215c5b6019aa7d643f2d7cb1a)
---
 airflow/models/baseoperator.py    |  7 ++++++-
 airflow/models/dag.py             |  3 ++-
 airflow/models/taskinstance.py    | 11 +++--------
 airflow/templates.py              | 32 ++++++++++++++++++++++++++++++++
 tests/core/test_templates.py      | 39 +++++++++++++++++++++++++++++++++++++++
 tests/models/test_baseoperator.py |  2 +-
 6 files changed, 83 insertions(+), 11 deletions(-)

diff --git a/airflow/models/baseoperator.py b/airflow/models/baseoperator.py
index 328e50d..f6fec77 100644
--- a/airflow/models/baseoperator.py
+++ b/airflow/models/baseoperator.py
@@ -54,6 +54,7 @@ except ImportError:
 from dateutil.relativedelta import relativedelta
 from sqlalchemy.orm import Session
 
+import airflow.templates
 from airflow.configuration import conf
 from airflow.exceptions import AirflowException
 from airflow.lineage import apply_lineage, prepare_lineage
@@ -1078,7 +1079,11 @@ class BaseOperator(Operator, LoggingMixin, TaskMixin, metaclass=BaseOperatorMeta
 
     def get_template_env(self) -> jinja2.Environment:
         """Fetch a Jinja template environment from the DAG or instantiate empty environment if no DAG."""
-        return self.dag.get_template_env() if self.has_dag() else jinja2.Environment(cache_size=0)  # noqa
+        return (
+            self.dag.get_template_env()
+            if self.has_dag()
+            else airflow.templates.SandboxedEnvironment(cache_size=0)
+        )  # noqa
 
     def prepare_template(self) -> None:
         """
diff --git a/airflow/models/dag.py b/airflow/models/dag.py
index fa86175..c90fb4f 100644
--- a/airflow/models/dag.py
+++ b/airflow/models/dag.py
@@ -53,6 +53,7 @@ from sqlalchemy import Boolean, Column, ForeignKey, Index, Integer, String, Text
 from sqlalchemy.orm import backref, joinedload, relationship
 from sqlalchemy.orm.session import Session
 
+import airflow.templates
 from airflow import settings, utils
 from airflow.configuration import conf
 from airflow.exceptions import AirflowException, DuplicateTaskIdFound, TaskNotFound
@@ -997,7 +998,7 @@ class DAG(LoggingMixin):
         if self.render_template_as_native_obj:
             env = NativeEnvironment(**jinja_env_options)
         else:
-            env = jinja2.Environment(**jinja_env_options)  # type: ignore
+            env = airflow.templates.SandboxedEnvironment(**jinja_env_options)  # type: ignore
 
         # Add any user defined items. Safe to edit globals as long as no templates are rendered yet.
         # http://jinja.pocoo.org/docs/2.10/api/#jinja2.Environment.globals
diff --git a/airflow/models/taskinstance.py b/airflow/models/taskinstance.py
index a377964..a7e94bd 100644
--- a/airflow/models/taskinstance.py
+++ b/airflow/models/taskinstance.py
@@ -1837,14 +1837,9 @@ class TaskInstance(Base, LoggingMixin):  # pylint: disable=R0902,R0904
                     max_tries=self.max_tries,
                 )
             )
-            if self.dag.render_template_as_native_obj:
-                jinja_env = jinja2.nativetypes.NativeEnvironment(
-                    loader=jinja2.FileSystemLoader(os.path.dirname(__file__)), autoescape=True
-                )
-            else:
-                jinja_env = jinja2.Environment(
-                    loader=jinja2.FileSystemLoader(os.path.dirname(__file__)), autoescape=True
-                )
+            jinja_env = jinja2.Environment(
+                loader=jinja2.FileSystemLoader(os.path.dirname(__file__)), autoescape=True
+            )
             subject = jinja_env.from_string(default_subject).render(**jinja_context)
             html_content = jinja_env.from_string(default_html_content).render(**jinja_context)
             html_content_err = jinja_env.from_string(default_html_content_err).render(**jinja_context)
diff --git a/airflow/templates.py b/airflow/templates.py
new file mode 100644
index 0000000..e475c86
--- /dev/null
+++ b/airflow/templates.py
@@ -0,0 +1,32 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import jinja2.sandbox
+
+
+class SandboxedEnvironment(jinja2.sandbox.SandboxedEnvironment):
+    """SandboxedEnvironment for Airflow task templates."""
+
+    def is_safe_attribute(self, obj, attr, value):
+        """
+        Allow access to ``_`` prefix vars (but not ``__``).
+
+        Unlike the stock SandboxedEnvironment, we allow access to "private" attributes (ones starting with
+        ``_``) whilst still blocking internal or truely private attributes (``__`` prefixed ones).
+        """
+        return not jinja2.sandbox.is_internal_attribute(obj, attr)
diff --git a/tests/core/test_templates.py b/tests/core/test_templates.py
new file mode 100644
index 0000000..160fea4
--- /dev/null
+++ b/tests/core/test_templates.py
@@ -0,0 +1,39 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import jinja2
+import jinja2.exceptions
+import pytest
+
+import airflow.templates
+
+
+@pytest.fixture
+def env():
+    return airflow.templates.SandboxedEnvironment(undefined=jinja2.StrictUndefined, cache_size=0)
+
+
+def test_protected_access(env):
+    class Test:
+        _protected = 123
+
+    assert env.from_string(r'{{ obj._protected }}').render(obj=Test) == "123"
+
+
+def test_private_access(env):
+    with pytest.raises(jinja2.exceptions.SecurityError):
+        env.from_string(r'{{ func.__code__ }}').render(func=test_private_access)
diff --git a/tests/models/test_baseoperator.py b/tests/models/test_baseoperator.py
index dace668..e030c4e 100644
--- a/tests/models/test_baseoperator.py
+++ b/tests/models/test_baseoperator.py
@@ -309,7 +309,7 @@ class TestBaseOperator(unittest.TestCase):
         with pytest.raises(jinja2.exceptions.TemplateSyntaxError):
             task.render_template("{{ invalid expression }}", {})
 
-    @mock.patch("jinja2.Environment", autospec=True)
+    @mock.patch("airflow.templates.SandboxedEnvironment", autospec=True)
     def test_jinja_env_creation(self, mock_jinja_env):
         """Verify if a Jinja environment is created only once when templating."""
         with DAG("test-dag", start_date=DEFAULT_DATE):