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 2022/03/16 14:19:00 UTC

[GitHub] [airflow] mnojek opened a new pull request #22311: New design of system tests

mnojek opened a new pull request #22311:
URL: https://github.com/apache/airflow/pull/22311


   Hello everyone,
   
   I am really excited to show you the collective work of the last few weeks!
   This is the initial PR that prepares Airflow to be compliant with
   [AIP-47](https://cwiki.apache.org/confluence/display/AIRFLOW/AIP-47+New+design+of+Airflow+System+Tests) which is related to the new design of system tests in Airflow. This is the first step to automated system tests in our CI. When it is merged, it will enable other providers to start migration of their system tests to the new design.
   
   Basically, the main advantage of this design is that we will have self-contained tests encapsulated in DAGs that can be run with pytest as well as just imported into Airflow and be executed there. Requirements for the environment are kept at minimum and are mostly related to specific test needs. By using built-in features of Airflow, each system test is a DAG that is also an example of usage for tested Operator. All in one, in a simple file :)
   
   It is a minimal PR that contains a bunch of tests for only 1 Google service (BigQuery) and everything else that is required by the tests to be working properly. It also consists of all necessary documentation, new pre-commit hook (`check-watcher-in-examples`), updated checks (in `tests/always`) and some other things. More tests will be migrated in a separate PRs, because we wanted this one to be as small as possible.
   
   Please familiarize with the changes and also read the AIP-47 to fully understand why they are needed.
   We really want to hear your feedback. In case of any questions, concerns or comments - we are here to answer them and explain things. The whole work was overseen by @potiuk and done together with @bhirsz and @kosteev and me (@mnojek).


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] kosteev commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
kosteev commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r831016711



##########
File path: tests/system/conftest.py
##########
@@ -0,0 +1,61 @@
+# 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 os
+import re
+from itertools import chain
+from pathlib import Path
+
+import pytest
+
+OLD_EXECUTOR = os.environ.get("AIRFLOW__CORE__EXECUTOR", default=None)
+REQUIRED_ENV_VARS = ("SYSTEM_TESTS_ENV_ID",)
+
+
+def pytest_configure(config):
+    os.environ["AIRFLOW__CORE__EXECUTOR"] = "DebugExecutor"
+
+
+def pytest_unconfigure(config):
+    if OLD_EXECUTOR is not None:
+        os.environ["AIRFLOW__CORE__EXECUTOR"] = OLD_EXECUTOR

Review comment:
       My concern was about the case when "AIRFLOW__CORE__EXECUTOR" environment variable is not set before (and it means prev_executor will be None), and then "AIRFLOW__CORE__EXECUTOR" will not be unset because of this condition "if prev_executor is not None:".
   I do not know the details of execution, maybe this will not happen or not an issue, but technically speaking looks wrong and asks for question. WDYT?
   
   I was thinking about this or something similar.
   <pre>
   with mock.patch.dict("os.environ", AIRFLOW__CORE__EXECUTOR="DebugExecutor"):
       yield
   </pre>




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] potiuk commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r830482863



##########
File path: tests/always/test_example_dags.py
##########
@@ -29,33 +30,38 @@
 NO_DB_QUERY_EXCEPTION = ["/airflow/example_dags/example_subdag_operator.py"]
 
 
-class TestExampleDags(unittest.TestCase):
-    def test_should_be_importable(self):
-        example_dags = list(glob(f"{ROOT_FOLDER}/airflow/**/example_dags/example_*.py", recursive=True))
-        assert 0 != len(example_dags)
-        for filepath in example_dags:
-            relative_filepath = os.path.relpath(filepath, ROOT_FOLDER)
-            with self.subTest(f"File {relative_filepath} should contain dags"):
-                dagbag = DagBag(
-                    dag_folder=filepath,
-                    include_examples=False,
-                )
-                assert 0 == len(dagbag.import_errors), f"import_errors={str(dagbag.import_errors)}"
-                assert len(dagbag.dag_ids) >= 1
-
-    def test_should_not_do_database_queries(self):
-        example_dags = glob(f"{ROOT_FOLDER}/airflow/**/example_dags/example_*.py", recursive=True)
-        example_dags = [
-            dag_file
-            for dag_file in example_dags
-            if any(not dag_file.endswith(e) for e in NO_DB_QUERY_EXCEPTION)
-        ]
-        assert 0 != len(example_dags)
-        for filepath in example_dags:
-            relative_filepath = os.path.relpath(filepath, ROOT_FOLDER)
-            with self.subTest(f"File {relative_filepath} shouldn't do database queries"):
-                with assert_queries_count(0):
-                    DagBag(
-                        dag_folder=filepath,
-                        include_examples=False,
-                    )
+def example_dags():
+    example_dirs = ["airflow/**/example_dags/example_*.py", "tests/system/providers/**/example_*.py"]
+    for example_dir in example_dirs:
+        yield from glob(f"{ROOT_FOLDER}/{example_dir}", recursive=True)
+
+
+def example_dags_except_db_exception():
+    return [
+        dag_file
+        for dag_file in example_dags()
+        if any(not dag_file.endswith(e) for e in NO_DB_QUERY_EXCEPTION)
+    ]
+
+
+def relative_path(path):
+    return os.path.relpath(path, ROOT_FOLDER)
+
+
+@pytest.mark.parametrize("example", list(example_dags()), ids=relative_path)
+def test_should_be_importable(example):
+    dagbag = DagBag(
+        dag_folder=example,
+        include_examples=False,
+    )
+    assert 0 == len(dagbag.import_errors), f"import_errors={str(dagbag.import_errors)}"

Review comment:
       Nope. Pytest (thank you Pytest maintainers!) reverted the expected/actual vs. assertsEqual (which follows JUnit's practice of expected first):  https://stackoverflow.com/questions/53651193/what-is-the-correct-order-for-actual-and-expected-in-pytest

##########
File path: tests/always/test_example_dags.py
##########
@@ -29,33 +30,38 @@
 NO_DB_QUERY_EXCEPTION = ["/airflow/example_dags/example_subdag_operator.py"]
 
 
-class TestExampleDags(unittest.TestCase):
-    def test_should_be_importable(self):
-        example_dags = list(glob(f"{ROOT_FOLDER}/airflow/**/example_dags/example_*.py", recursive=True))
-        assert 0 != len(example_dags)
-        for filepath in example_dags:
-            relative_filepath = os.path.relpath(filepath, ROOT_FOLDER)
-            with self.subTest(f"File {relative_filepath} should contain dags"):
-                dagbag = DagBag(
-                    dag_folder=filepath,
-                    include_examples=False,
-                )
-                assert 0 == len(dagbag.import_errors), f"import_errors={str(dagbag.import_errors)}"
-                assert len(dagbag.dag_ids) >= 1
-
-    def test_should_not_do_database_queries(self):
-        example_dags = glob(f"{ROOT_FOLDER}/airflow/**/example_dags/example_*.py", recursive=True)
-        example_dags = [
-            dag_file
-            for dag_file in example_dags
-            if any(not dag_file.endswith(e) for e in NO_DB_QUERY_EXCEPTION)
-        ]
-        assert 0 != len(example_dags)
-        for filepath in example_dags:
-            relative_filepath = os.path.relpath(filepath, ROOT_FOLDER)
-            with self.subTest(f"File {relative_filepath} shouldn't do database queries"):
-                with assert_queries_count(0):
-                    DagBag(
-                        dag_folder=filepath,
-                        include_examples=False,
-                    )
+def example_dags():
+    example_dirs = ["airflow/**/example_dags/example_*.py", "tests/system/providers/**/example_*.py"]
+    for example_dir in example_dirs:
+        yield from glob(f"{ROOT_FOLDER}/{example_dir}", recursive=True)
+
+
+def example_dags_except_db_exception():
+    return [
+        dag_file
+        for dag_file in example_dags()
+        if any(not dag_file.endswith(e) for e in NO_DB_QUERY_EXCEPTION)
+    ]
+
+
+def relative_path(path):
+    return os.path.relpath(path, ROOT_FOLDER)
+
+
+@pytest.mark.parametrize("example", list(example_dags()), ids=relative_path)
+def test_should_be_importable(example):
+    dagbag = DagBag(
+        dag_folder=example,
+        include_examples=False,
+    )
+    assert 0 == len(dagbag.import_errors), f"import_errors={str(dagbag.import_errors)}"

Review comment:
       True. Pytest (thank you Pytest maintainers!) reverted the expected/actual vs. assertsEqual (which follows JUnit's practice of expected first):  https://stackoverflow.com/questions/53651193/what-is-the-correct-order-for-actual-and-expected-in-pytest




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] potiuk commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r830482394



##########
File path: scripts/ci/pre_commit/pre_commit_check_watcher_in_examples.py
##########
@@ -0,0 +1,86 @@
+#!/usr/bin/env python3
+# 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 sys
+from pathlib import Path
+from typing import List
+
+from rich.console import Console
+
+if __name__ not in ("__main__", "__mp_main__"):
+    raise SystemExit(
+        "This file is intended to be executed as an executable program. You cannot use it as a module."
+        f"To run this script, run the ./{__file__} command [FILE] ..."
+    )
+
+
+console = Console(color_system="standard", width=200)
+
+errors: List[str] = []
+
+WATCHER_APPEND_INSTRUCTION = "list(dag.tasks) >> watcher()"
+
+PYTEST_FUNCTION = """
+def test_run():
+    from airflow.utils.state import State
+
+    dag.clear(dag_run_state=State.NONE)
+    dag.run()
+"""
+
+
+def _check_file(file: Path):
+    content = file.read_text()
+    if "from tests.system.utils.watcher import watcher" in content:
+        index = content.find(WATCHER_APPEND_INSTRUCTION)
+        if index == -1:
+            errors.append(
+                f"[red]The example {file} imports tests.system.utils.watcher "
+                f"but does not use it properly![/]\n\n"
+                "[yellow]Make sure you have:[/]\n\n"
+                f"        {WATCHER_APPEND_INSTRUCTION}\n\n"
+                "[yellow]as last instruction in your example DAG.[/]\n"
+            )
+        else:
+            operator_leftshift_index = content.find("<<", index + len(WATCHER_APPEND_INSTRUCTION))
+            operator_rightshift_index = content.find(">>", index + len(WATCHER_APPEND_INSTRUCTION))
+            if operator_leftshift_index != -1 or operator_rightshift_index != -1:
+                errors.append(
+                    f"[red]In the  example {file} "
+                    f"watcher is not last instruction in your DAG (there are << "
+                    f"or >> operators after it)![/]\n\n"
+                    "[yellow]Make sure you have:[/]\n"
+                    f"        {WATCHER_APPEND_INSTRUCTION}\n\n"
+                    "[yellow]as last instruction in your example DAG.[/]\n"
+                )
+        if PYTEST_FUNCTION not in content:
+            errors.append(
+                f"[yellow]The  example {file} missed the pytest function at the end.[/]\n\n"
+                "All example tests should have this function added:\n\n" + PYTEST_FUNCTION + "\n\n"
+                "[yellow]Automatically adding it now!\n"
+            )
+            file.write_text(content + "\n" + PYTEST_FUNCTION)

Review comment:
       Yep. It was only watcher initially :). But not any more.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] mnojek commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
mnojek commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r833989813



##########
File path: tests/system/utils/__init__.py
##########
@@ -0,0 +1,25 @@
+# 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.
+from airflow.utils.state import State
+
+
+def get_test_run(dag):
+    def test_run():
+        dag.clear(dag_run_state=State.NONE)
+        dag.run()
+
+    return test_run

Review comment:
       I was thinking the same thing (like @uranusjr) yesterday when I was looking at the whole solution (after the proposed changes) from the higher perspective. Our goal was to have self-contained tests that are just DAGs, and by exporting more and more things outside, we lost part of this feature.
   
   On the other hand, I agree with @potiuk that thanks to this 'export', we can easily change or update some behavior in one place. I haven't even thought about changing test configuration this way (log level etc.) but, yeah, this will be quite useful.
   It's a trade-off between fully self-contained and easily editable. We can't have both at the same time.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] mnojek commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
mnojek commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r833473087



##########
File path: scripts/ci/pre_commit/pre_commit_check_watcher_in_examples.py
##########
@@ -0,0 +1,86 @@
+#!/usr/bin/env python3
+# 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 sys
+from pathlib import Path
+from typing import List
+
+from rich.console import Console
+
+if __name__ not in ("__main__", "__mp_main__"):
+    raise SystemExit(
+        "This file is intended to be executed as an executable program. You cannot use it as a module."
+        f"To run this script, run the ./{__file__} command [FILE] ..."
+    )
+
+
+console = Console(color_system="standard", width=200)
+
+errors: List[str] = []
+
+WATCHER_APPEND_INSTRUCTION = "list(dag.tasks) >> watcher()"
+
+PYTEST_FUNCTION = """
+def test_run():

Review comment:
       Updated the check with a regex pattern, so that it accepts whether there is a comment in between lines in "pytest function" or not.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] potiuk commented on pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
potiuk commented on pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#issuecomment-1078332255


   Fixed Python 3.6 (needed for 2.2.5 release) in main. I hope it will succeed this time :)


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] mnojek commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
mnojek commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r829466203



##########
File path: scripts/ci/pre_commit/pre_commit_check_watcher_in_examples.py
##########
@@ -0,0 +1,86 @@
+#!/usr/bin/env python3
+# 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 sys
+from pathlib import Path
+from typing import List
+
+from rich.console import Console
+
+if __name__ not in ("__main__", "__mp_main__"):
+    raise SystemExit(
+        "This file is intended to be executed as an executable program. You cannot use it as a module."
+        f"To run this script, run the ./{__file__} command [FILE] ..."
+    )
+
+
+console = Console(color_system="standard", width=200)
+
+errors: List[str] = []
+
+WATCHER_APPEND_INSTRUCTION = "list(dag.tasks) >> watcher()"
+
+PYTEST_FUNCTION = """
+def test_run():
+    from airflow.utils.state import State
+
+    dag.clear(dag_run_state=State.NONE)
+    dag.run()
+"""
+
+
+def _check_file(file: Path):
+    content = file.read_text()
+    if "from tests.system.utils.watcher import watcher" in content:
+        index = content.find(WATCHER_APPEND_INSTRUCTION)
+        if index == -1:
+            errors.append(
+                f"[red]The example {file} imports tests.system.utils.watcher "
+                f"but does not use it properly![/]\n\n"
+                "[yellow]Make sure you have:[/]\n\n"
+                f"        {WATCHER_APPEND_INSTRUCTION}\n\n"
+                "[yellow]as last instruction in your example DAG.[/]\n"
+            )
+        else:
+            operator_leftshift_index = content.find("<<", index + len(WATCHER_APPEND_INSTRUCTION))
+            operator_rightshift_index = content.find(">>", index + len(WATCHER_APPEND_INSTRUCTION))
+            if operator_leftshift_index != -1 or operator_rightshift_index != -1:
+                errors.append(
+                    f"[red]In the  example {file} "
+                    f"watcher is not last instruction in your DAG (there are << "
+                    f"or >> operators after it)![/]\n\n"
+                    "[yellow]Make sure you have:[/]\n"
+                    f"        {WATCHER_APPEND_INSTRUCTION}\n\n"
+                    "[yellow]as last instruction in your example DAG.[/]\n"
+                )
+        if PYTEST_FUNCTION not in content:
+            errors.append(
+                f"[yellow]The  example {file} missed the pytest function at the end.[/]\n\n"
+                "All example tests should have this function added:\n\n" + PYTEST_FUNCTION + "\n\n"
+                "[yellow]Automatically adding it now!\n"

Review comment:
       Credit goes to @potiuk!

##########
File path: scripts/ci/pre_commit/pre_commit_check_watcher_in_examples.py
##########
@@ -0,0 +1,86 @@
+#!/usr/bin/env python3
+# 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 sys
+from pathlib import Path
+from typing import List
+
+from rich.console import Console
+
+if __name__ not in ("__main__", "__mp_main__"):
+    raise SystemExit(
+        "This file is intended to be executed as an executable program. You cannot use it as a module."
+        f"To run this script, run the ./{__file__} command [FILE] ..."
+    )
+
+
+console = Console(color_system="standard", width=200)
+
+errors: List[str] = []
+
+WATCHER_APPEND_INSTRUCTION = "list(dag.tasks) >> watcher()"
+
+PYTEST_FUNCTION = """
+def test_run():
+    from airflow.utils.state import State
+
+    dag.clear(dag_run_state=State.NONE)
+    dag.run()
+"""
+
+
+def _check_file(file: Path):
+    content = file.read_text()
+    if "from tests.system.utils.watcher import watcher" in content:

Review comment:
       Yes, that's correct. We can still improve this check.

##########
File path: tests/system/providers/google/bigquery/example_bigquery_operations.py
##########
@@ -0,0 +1,106 @@
+#
+# 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.
+
+"""
+Example Airflow DAG for Google BigQuery service local file upload and external table creation.
+"""
+import os
+from datetime import datetime
+from pathlib import Path
+
+from airflow import models
+from airflow.providers.google.cloud.operators.bigquery import (
+    BigQueryCreateEmptyDatasetOperator,
+    BigQueryCreateExternalTableOperator,
+    BigQueryDeleteDatasetOperator,
+)
+from airflow.providers.google.cloud.operators.gcs import GCSCreateBucketOperator, GCSDeleteBucketOperator
+from airflow.providers.google.cloud.transfers.local_to_gcs import LocalFilesystemToGCSOperator
+from airflow.utils.trigger_rule import TriggerRule
+from tests.system.utils.watcher import watcher
+
+ENV_ID = os.environ.get("SYSTEM_TESTS_ENV_ID")
+DAG_ID = "bigquery_operations"
+
+DATASET_NAME = f"dataset_{DAG_ID}_{ENV_ID}"
+DATA_SAMPLE_GCS_BUCKET_NAME = f"bucket_{DAG_ID}_{ENV_ID}"
+DATA_SAMPLE_GCS_OBJECT_NAME = "bigquery/us-states/us-states.csv"
+CSV_FILE_LOCAL_PATH = str(Path(__file__).parent / "resources" / "us-states.csv")
+
+
+with models.DAG(
+    DAG_ID,
+    schedule_interval="@once",
+    start_date=datetime(2021, 1, 1),
+    catchup=False,
+    tags=["example", "bigquery"],
+) as dag:
+    create_bucket = GCSCreateBucketOperator(task_id="create_bucket", bucket_name=DATA_SAMPLE_GCS_BUCKET_NAME)
+
+    create_dataset = BigQueryCreateEmptyDatasetOperator(task_id="create_dataset", dataset_id=DATASET_NAME)
+
+    upload_file = LocalFilesystemToGCSOperator(
+        task_id="upload_file_to_bucket",
+        src=CSV_FILE_LOCAL_PATH,
+        dst=DATA_SAMPLE_GCS_OBJECT_NAME,
+        bucket=DATA_SAMPLE_GCS_BUCKET_NAME,
+    )
+
+    # [START howto_operator_bigquery_create_external_table]
+    create_external_table = BigQueryCreateExternalTableOperator(
+        task_id="create_external_table",
+        destination_project_dataset_table=f"{DATASET_NAME}.external_table",
+        bucket=DATA_SAMPLE_GCS_BUCKET_NAME,
+        source_objects=[DATA_SAMPLE_GCS_OBJECT_NAME],
+        schema_fields=[
+            {"name": "emp_name", "type": "STRING", "mode": "REQUIRED"},
+            {"name": "salary", "type": "INTEGER", "mode": "NULLABLE"},
+        ],
+    )
+    # [END howto_operator_bigquery_create_external_table]
+
+    delete_dataset = BigQueryDeleteDatasetOperator(
+        task_id="delete_dataset",
+        dataset_id=DATASET_NAME,
+        delete_contents=True,
+        trigger_rule=TriggerRule.ALL_DONE,
+    )
+
+    delete_bucket = GCSDeleteBucketOperator(
+        task_id="delete_bucket", bucket_name=DATA_SAMPLE_GCS_BUCKET_NAME, trigger_rule=TriggerRule.ALL_DONE
+    )
+
+    (
+        # TEST SETUP
+        [create_bucket, create_dataset]
+        # TEST BODY
+        >> upload_file
+        >> create_external_table
+        # TEST TEARDOWN
+        >> delete_dataset
+        >> delete_bucket
+    )
+
+    list(dag.tasks) >> watcher()

Review comment:
       I don't have a better idea how this can be achieved without explicitly defining this task dependency. This is actually quite clever because in the previous version of this design we were listing all the tasks again and then defining `>>` operator with the watcher. Now it's just this one line that has a reference to all tasks in a local dag and then appends the watcher to all of them. And it is of course only needed when there is a teardown task in the dag.
   But of course I am open to any specific suggestions that may improve the design.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] o-nikolas commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
o-nikolas commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r829316088



##########
File path: scripts/ci/pre_commit/pre_commit_check_watcher_in_examples.py
##########
@@ -0,0 +1,86 @@
+#!/usr/bin/env python3
+# 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 sys
+from pathlib import Path
+from typing import List
+
+from rich.console import Console
+
+if __name__ not in ("__main__", "__mp_main__"):
+    raise SystemExit(
+        "This file is intended to be executed as an executable program. You cannot use it as a module."
+        f"To run this script, run the ./{__file__} command [FILE] ..."
+    )
+
+
+console = Console(color_system="standard", width=200)
+
+errors: List[str] = []
+
+WATCHER_APPEND_INSTRUCTION = "list(dag.tasks) >> watcher()"
+
+PYTEST_FUNCTION = """
+def test_run():

Review comment:
       I see that below this is being copy/pasted into each example dag file. What if this function needs to be modified or fixed in the future? Would that involve patching every single example dag file? 

##########
File path: tests/system/providers/google/README.md
##########
@@ -0,0 +1,99 @@
+<!--
+ 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.
+-->
+
+# Google provider system tests
+
+## Tests structure
+
+All Google-related system tests are located inside this subdirectory of system tests which is
+`tests/system/providers/google/`. They are grouped in directories by the related service name, e.g. all BigQuery
+tests are stored inside `tests/system/providers/google/bigquery/` directory. In each directory you will find test files
+as self-contained DAGs (one DAG per file). Each test may require some additional resources which should be placed in
+`resources` directory found on the same level as tests. Each test file should start with prefix `example_*`. If there
+is anything more needed for the test to be executed, it should be documented in the docstrings.
+
+Example files structure:
+
+```
+tests/system/providers/google
+├── bigquery
+│   ├── resources
+│   │   ├── example_bigquery_query.sql
+│   │   └── us-states.csv
+│   ├── example_bigquery_queries.py
+│   ├── example_bigquery_operations.py
+.   .
+│   └── example_bigquery_*.py
+├── dataflow
+├── gcs
+.
+└── *
+```
+
+## Initial configuration
+
+Each test requires some environment variables. Check how to set them up on your operating system, but on UNIX-based
+OSes this should work:
+
+```commandline
+export NAME_OF_ENV_VAR=value
+```
+
+To confirm that it is set up correctly, run `echo $NAME_OF_ENV_VAR` which will display its value.
+
+### Required environment variables
+
+- `SYSTEM_TESTS_GCP_PROJECT` - GCP project name that will be used to run system tests (this can be checked on the UI
+  dashboard of the GCP or by running `gcloud config list`).
+
+- `SYSTEM_TESTS_ENV_ID` - environment ID that is unique across between different executions of system tests (if they

Review comment:
       Nit:
   
   "`unique across between ...`"
   
   Use either "across" or "between", having both makes the sentence a bit odd.

##########
File path: tests/system/providers/google/bigquery/example_bigquery_dataset.py
##########
@@ -0,0 +1,94 @@
+#
+# 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.
+
+"""
+Example Airflow DAG for Google BigQuery service testing dataset operations.
+"""
+import os
+from datetime import datetime
+
+from airflow import models
+from airflow.operators.bash import BashOperator
+from airflow.providers.google.cloud.operators.bigquery import (
+    BigQueryCreateEmptyDatasetOperator,
+    BigQueryDeleteDatasetOperator,
+    BigQueryGetDatasetOperator,
+    BigQueryUpdateDatasetOperator,
+)
+from airflow.utils.trigger_rule import TriggerRule

Review comment:
       Is this a required component that authors must remember to include for system tests to work correctly? If so the pre-commit check should verify this is present as well and used below correctly.

##########
File path: scripts/ci/pre_commit/pre_commit_check_watcher_in_examples.py
##########
@@ -0,0 +1,86 @@
+#!/usr/bin/env python3
+# 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 sys
+from pathlib import Path
+from typing import List
+
+from rich.console import Console
+
+if __name__ not in ("__main__", "__mp_main__"):
+    raise SystemExit(
+        "This file is intended to be executed as an executable program. You cannot use it as a module."
+        f"To run this script, run the ./{__file__} command [FILE] ..."
+    )
+
+
+console = Console(color_system="standard", width=200)
+
+errors: List[str] = []
+
+WATCHER_APPEND_INSTRUCTION = "list(dag.tasks) >> watcher()"
+
+PYTEST_FUNCTION = """
+def test_run():
+    from airflow.utils.state import State
+
+    dag.clear(dag_run_state=State.NONE)
+    dag.run()
+"""
+
+
+def _check_file(file: Path):
+    content = file.read_text()
+    if "from tests.system.utils.watcher import watcher" in content:

Review comment:
       Someone could import the whole `watcher` module then just use `watcher.watcher()` which would evade this check right?




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] potiuk commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r829580834



##########
File path: tests/system/providers/google/bigquery/example_bigquery_operations.py
##########
@@ -0,0 +1,106 @@
+#
+# 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.
+
+"""
+Example Airflow DAG for Google BigQuery service local file upload and external table creation.
+"""
+import os
+from datetime import datetime
+from pathlib import Path
+
+from airflow import models
+from airflow.providers.google.cloud.operators.bigquery import (
+    BigQueryCreateEmptyDatasetOperator,
+    BigQueryCreateExternalTableOperator,
+    BigQueryDeleteDatasetOperator,
+)
+from airflow.providers.google.cloud.operators.gcs import GCSCreateBucketOperator, GCSDeleteBucketOperator
+from airflow.providers.google.cloud.transfers.local_to_gcs import LocalFilesystemToGCSOperator
+from airflow.utils.trigger_rule import TriggerRule
+from tests.system.utils.watcher import watcher
+
+ENV_ID = os.environ.get("SYSTEM_TESTS_ENV_ID")
+DAG_ID = "bigquery_operations"
+
+DATASET_NAME = f"dataset_{DAG_ID}_{ENV_ID}"
+DATA_SAMPLE_GCS_BUCKET_NAME = f"bucket_{DAG_ID}_{ENV_ID}"
+DATA_SAMPLE_GCS_OBJECT_NAME = "bigquery/us-states/us-states.csv"
+CSV_FILE_LOCAL_PATH = str(Path(__file__).parent / "resources" / "us-states.csv")
+
+
+with models.DAG(
+    DAG_ID,
+    schedule_interval="@once",
+    start_date=datetime(2021, 1, 1),
+    catchup=False,
+    tags=["example", "bigquery"],
+) as dag:
+    create_bucket = GCSCreateBucketOperator(task_id="create_bucket", bucket_name=DATA_SAMPLE_GCS_BUCKET_NAME)
+
+    create_dataset = BigQueryCreateEmptyDatasetOperator(task_id="create_dataset", dataset_id=DATASET_NAME)
+
+    upload_file = LocalFilesystemToGCSOperator(
+        task_id="upload_file_to_bucket",
+        src=CSV_FILE_LOCAL_PATH,
+        dst=DATA_SAMPLE_GCS_OBJECT_NAME,
+        bucket=DATA_SAMPLE_GCS_BUCKET_NAME,
+    )
+
+    # [START howto_operator_bigquery_create_external_table]
+    create_external_table = BigQueryCreateExternalTableOperator(
+        task_id="create_external_table",
+        destination_project_dataset_table=f"{DATASET_NAME}.external_table",
+        bucket=DATA_SAMPLE_GCS_BUCKET_NAME,
+        source_objects=[DATA_SAMPLE_GCS_OBJECT_NAME],
+        schema_fields=[
+            {"name": "emp_name", "type": "STRING", "mode": "REQUIRED"},
+            {"name": "salary", "type": "INTEGER", "mode": "NULLABLE"},
+        ],
+    )
+    # [END howto_operator_bigquery_create_external_table]
+
+    delete_dataset = BigQueryDeleteDatasetOperator(
+        task_id="delete_dataset",
+        dataset_id=DATASET_NAME,
+        delete_contents=True,
+        trigger_rule=TriggerRule.ALL_DONE,
+    )
+
+    delete_bucket = GCSDeleteBucketOperator(
+        task_id="delete_bucket", bucket_name=DATA_SAMPLE_GCS_BUCKET_NAME, trigger_rule=TriggerRule.ALL_DONE
+    )
+
+    (
+        # TEST SETUP
+        [create_bucket, create_dataset]
+        # TEST BODY
+        >> upload_file
+        >> create_external_table
+        # TEST TEARDOWN
+        >> delete_dataset
+        >> delete_bucket
+    )
+
+    list(dag.tasks) >> watcher()

Review comment:
       We thought about a special "util" function for that.
   
   I do not think we discussed about "magical" adding it. this is a bit of a hassle, I agreee but adding it would likely require something on the DAG level - for example a special parameter or decorator. And the problem with that is that it would have to be added somewhere at the beginning of the DAG - where you define the DAG most likely. And the problem with that is that you also use the DAG as examples in our documentation. And we extracts parts of the examples into the documentaiton and we should not pollute those examples with things that are not really good "examples" on how you should add your DAGs. - and those example usually show the DAG definition/default_args as part of the example - by having the special decorator, or parameter on the DAG to indicate that DAG shoudl have "watcher" added migth be too easily copied from those extracted examples.
   
   Having explicit watcher makes it so much easier - it is at the and and it is explicit (which means there is no magic)
   
   But actually what made me think now - we should actually make it even more separated an explicit.
   
   @mnojek why don't we change the the watcher's local import and some comment there to make it even more separated and "explicit" for example:
   
   ```
            >> delete_bucket
       ) 
       
       from tests.system.utils.watcher import watcher
       # This test run as a system test needs watcher in order to mark success/failure
       # where "tearDown" task is part of the DAG
       list(dag.tasks) >> watcher()
       
   ```

##########
File path: tests/system/providers/google/bigquery/example_bigquery_operations.py
##########
@@ -0,0 +1,106 @@
+#
+# 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.
+
+"""
+Example Airflow DAG for Google BigQuery service local file upload and external table creation.
+"""
+import os
+from datetime import datetime
+from pathlib import Path
+
+from airflow import models
+from airflow.providers.google.cloud.operators.bigquery import (
+    BigQueryCreateEmptyDatasetOperator,
+    BigQueryCreateExternalTableOperator,
+    BigQueryDeleteDatasetOperator,
+)
+from airflow.providers.google.cloud.operators.gcs import GCSCreateBucketOperator, GCSDeleteBucketOperator
+from airflow.providers.google.cloud.transfers.local_to_gcs import LocalFilesystemToGCSOperator
+from airflow.utils.trigger_rule import TriggerRule
+from tests.system.utils.watcher import watcher
+
+ENV_ID = os.environ.get("SYSTEM_TESTS_ENV_ID")
+DAG_ID = "bigquery_operations"
+
+DATASET_NAME = f"dataset_{DAG_ID}_{ENV_ID}"
+DATA_SAMPLE_GCS_BUCKET_NAME = f"bucket_{DAG_ID}_{ENV_ID}"
+DATA_SAMPLE_GCS_OBJECT_NAME = "bigquery/us-states/us-states.csv"
+CSV_FILE_LOCAL_PATH = str(Path(__file__).parent / "resources" / "us-states.csv")
+
+
+with models.DAG(
+    DAG_ID,
+    schedule_interval="@once",
+    start_date=datetime(2021, 1, 1),
+    catchup=False,
+    tags=["example", "bigquery"],
+) as dag:
+    create_bucket = GCSCreateBucketOperator(task_id="create_bucket", bucket_name=DATA_SAMPLE_GCS_BUCKET_NAME)
+
+    create_dataset = BigQueryCreateEmptyDatasetOperator(task_id="create_dataset", dataset_id=DATASET_NAME)
+
+    upload_file = LocalFilesystemToGCSOperator(
+        task_id="upload_file_to_bucket",
+        src=CSV_FILE_LOCAL_PATH,
+        dst=DATA_SAMPLE_GCS_OBJECT_NAME,
+        bucket=DATA_SAMPLE_GCS_BUCKET_NAME,
+    )
+
+    # [START howto_operator_bigquery_create_external_table]
+    create_external_table = BigQueryCreateExternalTableOperator(
+        task_id="create_external_table",
+        destination_project_dataset_table=f"{DATASET_NAME}.external_table",
+        bucket=DATA_SAMPLE_GCS_BUCKET_NAME,
+        source_objects=[DATA_SAMPLE_GCS_OBJECT_NAME],
+        schema_fields=[
+            {"name": "emp_name", "type": "STRING", "mode": "REQUIRED"},
+            {"name": "salary", "type": "INTEGER", "mode": "NULLABLE"},
+        ],
+    )
+    # [END howto_operator_bigquery_create_external_table]
+
+    delete_dataset = BigQueryDeleteDatasetOperator(
+        task_id="delete_dataset",
+        dataset_id=DATASET_NAME,
+        delete_contents=True,
+        trigger_rule=TriggerRule.ALL_DONE,
+    )
+
+    delete_bucket = GCSDeleteBucketOperator(
+        task_id="delete_bucket", bucket_name=DATA_SAMPLE_GCS_BUCKET_NAME, trigger_rule=TriggerRule.ALL_DONE
+    )
+
+    (
+        # TEST SETUP
+        [create_bucket, create_dataset]
+        # TEST BODY
+        >> upload_file
+        >> create_external_table
+        # TEST TEARDOWN
+        >> delete_dataset
+        >> delete_bucket
+    )
+
+    list(dag.tasks) >> watcher()

Review comment:
       We thought about a special "util" function for that.
   
   I do not think we discussed about "magical" adding it. this is a bit of a hassle, I agreee but adding it would likely require something on the DAG level - for example a special parameter or decorator. And the problem with that is that it would have to be added somewhere at the beginning of the DAG - where you define the DAG most likely. And the problem with that is that you also use the DAG as examples in our documentation. And we extracts parts of the examples into the documentaiton and we should not pollute those examples with things that are not really good "examples" on how you should add your DAGs. - and those example usually show the DAG definition/default_args as part of the example - by having the special decorator, or parameter on the DAG to indicate that DAG shoudl have "watcher" added migth be too easily copied from those extracted examples.
   
   Having explicit watcher makes it so much easier - it is at the and and it is explicit (which means there is no magic)
   
   But actually what made me think now - we should actually make it even more separated an explicit.
   
   @mnojek why don't we change the the watcher's local import and some comment there to make it even more separated and "explicit" for example:
   
   ```python
            >> delete_bucket
       ) 
       
       from tests.system.utils.watcher import watcher
       # This test run as a system test needs watcher in order to mark success/failure
       # where "tearDown" task is part of the DAG
       list(dag.tasks) >> watcher()
       
   ```

##########
File path: scripts/ci/pre_commit/pre_commit_check_watcher_in_examples.py
##########
@@ -0,0 +1,86 @@
+#!/usr/bin/env python3
+# 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 sys
+from pathlib import Path
+from typing import List
+
+from rich.console import Console
+
+if __name__ not in ("__main__", "__mp_main__"):
+    raise SystemExit(
+        "This file is intended to be executed as an executable program. You cannot use it as a module."
+        f"To run this script, run the ./{__file__} command [FILE] ..."
+    )
+
+
+console = Console(color_system="standard", width=200)
+
+errors: List[str] = []
+
+WATCHER_APPEND_INSTRUCTION = "list(dag.tasks) >> watcher()"
+
+PYTEST_FUNCTION = """
+def test_run():
+    from airflow.utils.state import State
+
+    dag.clear(dag_run_state=State.NONE)
+    dag.run()
+"""
+
+
+def _check_file(file: Path):
+    content = file.read_text()
+    if "from tests.system.utils.watcher import watcher" in content:

Review comment:
       Yeah. That check is "best effort" we could have import the AST and analyze it, but I think this is the 20/80 Pareto's rule in full swing. Even if we miss it once or twice, if somoene actually made an effort to import watcher they will likely not forget to add it as dependencies :) 

##########
File path: tests/system/providers/google/bigquery/example_bigquery_dataset.py
##########
@@ -0,0 +1,94 @@
+#
+# 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.
+
+"""
+Example Airflow DAG for Google BigQuery service testing dataset operations.
+"""
+import os
+from datetime import datetime
+
+from airflow import models
+from airflow.operators.bash import BashOperator
+from airflow.providers.google.cloud.operators.bigquery import (
+    BigQueryCreateEmptyDatasetOperator,
+    BigQueryDeleteDatasetOperator,
+    BigQueryGetDatasetOperator,
+    BigQueryUpdateDatasetOperator,
+)
+from airflow.utils.trigger_rule import TriggerRule

Review comment:
       Yeah - see the comments I added above.

##########
File path: scripts/ci/pre_commit/pre_commit_check_watcher_in_examples.py
##########
@@ -0,0 +1,86 @@
+#!/usr/bin/env python3
+# 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 sys
+from pathlib import Path
+from typing import List
+
+from rich.console import Console
+
+if __name__ not in ("__main__", "__mp_main__"):
+    raise SystemExit(
+        "This file is intended to be executed as an executable program. You cannot use it as a module."
+        f"To run this script, run the ./{__file__} command [FILE] ..."
+    )
+
+
+console = Console(color_system="standard", width=200)
+
+errors: List[str] = []
+
+WATCHER_APPEND_INSTRUCTION = "list(dag.tasks) >> watcher()"
+
+PYTEST_FUNCTION = """
+def test_run():

Review comment:
       That's a nice idea. If pytest still discovers it this way, I lke it a lot better. also you can even connect it with local import to make it even more localized (same reason as in the other comment , where we care about "example_dag" being still an example dag despite also being a self-contained test. Also it nicely solves the problem which I thought about - what happens if we have two dags in an example. They would have to have slightiy different method ( dag1.clear() dag1.run() vs dag2.clear(() dag2.run()). Passing dag as parameter solves it nicely.
   
   I'd be for something like this:
   
   ```
        # Needed to run the example dag as system test <link to the docs>
        from tests.airflow.proivders.util import get_test_run
        test_run = get_test_run(dag)
   ```
   
   

##########
File path: scripts/ci/pre_commit/pre_commit_check_watcher_in_examples.py
##########
@@ -0,0 +1,86 @@
+#!/usr/bin/env python3
+# 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 sys
+from pathlib import Path
+from typing import List
+
+from rich.console import Console
+
+if __name__ not in ("__main__", "__mp_main__"):
+    raise SystemExit(
+        "This file is intended to be executed as an executable program. You cannot use it as a module."
+        f"To run this script, run the ./{__file__} command [FILE] ..."
+    )
+
+
+console = Console(color_system="standard", width=200)
+
+errors: List[str] = []
+
+WATCHER_APPEND_INSTRUCTION = "list(dag.tasks) >> watcher()"
+
+PYTEST_FUNCTION = """
+def test_run():

Review comment:
       That's a nice idea. If pytest still discovers it this way, I lke it a lot better. also you can even connect it with local import to make it even more localized (same reason as in the other comment , where we care about "example_dag" being still an example dag despite also being a self-contained test. Also it nicely solves the problem which I thought about - what happens if we have two dags in an example. They would have to have slightiy different method ( dag1.clear() dag1.run() vs dag2.clear(() dag2.run()). Passing dag as parameter solves it nicely.
   
   I'd be for something like this:
   
   ```python
        # Needed to run the example dag as system test <link to the docs>
        from tests.airflow.proivders.util import get_test_run
        test_run = get_test_run(dag)
   ```
   
   

##########
File path: scripts/ci/pre_commit/pre_commit_check_watcher_in_examples.py
##########
@@ -0,0 +1,86 @@
+#!/usr/bin/env python3
+# 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 sys
+from pathlib import Path
+from typing import List
+
+from rich.console import Console
+
+if __name__ not in ("__main__", "__mp_main__"):
+    raise SystemExit(
+        "This file is intended to be executed as an executable program. You cannot use it as a module."
+        f"To run this script, run the ./{__file__} command [FILE] ..."
+    )
+
+
+console = Console(color_system="standard", width=200)
+
+errors: List[str] = []
+
+WATCHER_APPEND_INSTRUCTION = "list(dag.tasks) >> watcher()"
+
+PYTEST_FUNCTION = """
+def test_run():

Review comment:
       That's a nice idea. If pytest still discovers it this way, I lke it a lot better. also you can even connect it with local import to make it even more localized (same reason as in the other comment , where we care about "example_dag" being still an example dag despite also being a self-contained test. Also it nicely solves the problem which I thought about - what happens if we have two dags in an example. They would have to have slightiy different method ( dag1.clear() dag1.run() vs dag2.clear(() dag2.run()). Passing dag as parameter solves it nicely.
   
   I'd be for something like this:
   
   ```python
   # Needed to run the example dag as system test <link to the docs>
   from tests.airflow.proivders.util import get_test_run
   test_run = get_test_run(dag)
   ```
   
   




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] potiuk commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r833718762



##########
File path: tests/system/providers/google/bigquery/example_bigquery_dataset.py
##########
@@ -0,0 +1,95 @@
+#
+# 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.
+
+"""
+Example Airflow DAG for Google BigQuery service testing dataset operations.
+"""
+import os
+from datetime import datetime
+
+from airflow import models
+from airflow.operators.bash import BashOperator
+from airflow.providers.google.cloud.operators.bigquery import (
+    BigQueryCreateEmptyDatasetOperator,
+    BigQueryDeleteDatasetOperator,
+    BigQueryGetDatasetOperator,
+    BigQueryUpdateDatasetOperator,
+)
+from airflow.utils.trigger_rule import TriggerRule
+
+ENV_ID = os.environ.get("SYSTEM_TESTS_ENV_ID")
+DAG_ID = "bigquery_dataset"
+
+DATASET_NAME = f"dataset_{DAG_ID}_{ENV_ID}"
+
+
+with models.DAG(
+    DAG_ID,
+    schedule_interval="@once",
+    start_date=datetime(2021, 1, 1),
+    catchup=False,
+    tags=["example", "bigquery"],
+) as dag:
+    # [START howto_operator_bigquery_create_dataset]
+    create_dataset = BigQueryCreateEmptyDatasetOperator(task_id="create_dataset", dataset_id=DATASET_NAME)
+    # [END howto_operator_bigquery_create_dataset]
+
+    # [START howto_operator_bigquery_update_dataset]
+    update_dataset = BigQueryUpdateDatasetOperator(
+        task_id="update_dataset",
+        dataset_id=DATASET_NAME,
+        dataset_resource={"description": "Updated dataset"},
+    )
+    # [END howto_operator_bigquery_update_dataset]
+
+    # [START howto_operator_bigquery_get_dataset]
+    get_dataset = BigQueryGetDatasetOperator(task_id="get-dataset", dataset_id=DATASET_NAME)
+    # [END howto_operator_bigquery_get_dataset]
+
+    get_dataset_result = BashOperator(
+        task_id="get_dataset_result",
+        bash_command="echo \"{{ task_instance.xcom_pull('get-dataset')['id'] }}\"",
+    )
+
+    # [START howto_operator_bigquery_delete_dataset]
+    delete_dataset = BigQueryDeleteDatasetOperator(
+        task_id="delete_dataset", dataset_id=DATASET_NAME, delete_contents=True
+    )
+    # [END howto_operator_bigquery_delete_dataset]
+    delete_dataset.trigger_rule = TriggerRule.ALL_DONE
+
+    (
+        # TEST BODY
+        create_dataset
+        >> update_dataset
+        >> get_dataset
+        >> get_dataset_result
+        # TEST TEARDOWN
+        >> delete_dataset
+    )
+
+    from tests.system.utils.watcher import watcher

Review comment:
       I see. Right. Good reason. It was too beautiful. I though about addint a "get_watcher" method:
   
   ```
   with DAG as dag:
         ....
         from utiils import add_watcher
         add_watcher(dag)
   ```
   
   But apart of a litlle less code it does not change much.... so  guess we should stay with the original idea.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] kosteev commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
kosteev commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r833247929



##########
File path: docs/exts/docs_build/lint_checks.py
##########
@@ -230,13 +230,13 @@ def find_modules(deprecated_only: bool = False) -> Set[str]:
 
 
 def check_exampleinclude_for_example_dags() -> List[DocBuildError]:
-    """Checks all exampleincludes for  example dags."""
-    all_docs_files = glob(f"${DOCS_DIR}/**/*.rst", recursive=True)
+    """Checks all exampleincludes for example dags."""

Review comment:
       "exampleincludes"?, missed space?




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] bhirsz commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
bhirsz commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r832923867



##########
File path: docs/apache-airflow-providers-google/operators/cloud/bigquery.rst
##########
@@ -42,7 +42,7 @@ Create dataset
 To create an empty dataset in a BigQuery database you can use
 :class:`~airflow.providers.google.cloud.operators.bigquery.BigQueryCreateEmptyDatasetOperator`.
 
-.. exampleinclude:: /../../airflow/providers/google/cloud/example_dags/example_bigquery_operations.py
+.. exampleinclude:: /../../tests/system/providers/google/bigquery/example_bigquery_dataset.py

Review comment:
       This lint check actually doesn't work at all :) For example:
   ```
   f"${DOCS_DIR}/**/*.rst"
   ```
   ``$`` looks like a typo - after removing it and running the check I'm getting the errors (for files I didn't modify - this check was silently ignoring them). I will fix the check.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] mnojek commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
mnojek commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r832983255



##########
File path: docs/exts/docs_build/lint_checks.py
##########
@@ -231,12 +231,12 @@ def find_modules(deprecated_only: bool = False) -> Set[str]:
 
 def check_exampleinclude_for_example_dags() -> List[DocBuildError]:
     """Checks all exampleincludes for  example dags."""

Review comment:
       ```suggestion
       """Checks all exampleincludes for example dags."""
   ```




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] potiuk commented on pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
potiuk commented on pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#issuecomment-1077436333


   Any more comments ?


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] o-nikolas commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
o-nikolas commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r829316088



##########
File path: scripts/ci/pre_commit/pre_commit_check_watcher_in_examples.py
##########
@@ -0,0 +1,86 @@
+#!/usr/bin/env python3
+# 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 sys
+from pathlib import Path
+from typing import List
+
+from rich.console import Console
+
+if __name__ not in ("__main__", "__mp_main__"):
+    raise SystemExit(
+        "This file is intended to be executed as an executable program. You cannot use it as a module."
+        f"To run this script, run the ./{__file__} command [FILE] ..."
+    )
+
+
+console = Console(color_system="standard", width=200)
+
+errors: List[str] = []
+
+WATCHER_APPEND_INSTRUCTION = "list(dag.tasks) >> watcher()"
+
+PYTEST_FUNCTION = """
+def test_run():

Review comment:
       I see that below this is being copy/pasted into each example dag file. What if this function needs to be modified or fixed in the future? Would that involve patching every single example dag file? 

##########
File path: tests/system/providers/google/README.md
##########
@@ -0,0 +1,99 @@
+<!--
+ 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.
+-->
+
+# Google provider system tests
+
+## Tests structure
+
+All Google-related system tests are located inside this subdirectory of system tests which is
+`tests/system/providers/google/`. They are grouped in directories by the related service name, e.g. all BigQuery
+tests are stored inside `tests/system/providers/google/bigquery/` directory. In each directory you will find test files
+as self-contained DAGs (one DAG per file). Each test may require some additional resources which should be placed in
+`resources` directory found on the same level as tests. Each test file should start with prefix `example_*`. If there
+is anything more needed for the test to be executed, it should be documented in the docstrings.
+
+Example files structure:
+
+```
+tests/system/providers/google
+├── bigquery
+│   ├── resources
+│   │   ├── example_bigquery_query.sql
+│   │   └── us-states.csv
+│   ├── example_bigquery_queries.py
+│   ├── example_bigquery_operations.py
+.   .
+│   └── example_bigquery_*.py
+├── dataflow
+├── gcs
+.
+└── *
+```
+
+## Initial configuration
+
+Each test requires some environment variables. Check how to set them up on your operating system, but on UNIX-based
+OSes this should work:
+
+```commandline
+export NAME_OF_ENV_VAR=value
+```
+
+To confirm that it is set up correctly, run `echo $NAME_OF_ENV_VAR` which will display its value.
+
+### Required environment variables
+
+- `SYSTEM_TESTS_GCP_PROJECT` - GCP project name that will be used to run system tests (this can be checked on the UI
+  dashboard of the GCP or by running `gcloud config list`).
+
+- `SYSTEM_TESTS_ENV_ID` - environment ID that is unique across between different executions of system tests (if they

Review comment:
       Nit:
   
   "`unique across between ...`"
   
   Use either "across" or "between", having both makes the sentence a bit odd.

##########
File path: tests/system/providers/google/bigquery/example_bigquery_dataset.py
##########
@@ -0,0 +1,94 @@
+#
+# 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.
+
+"""
+Example Airflow DAG for Google BigQuery service testing dataset operations.
+"""
+import os
+from datetime import datetime
+
+from airflow import models
+from airflow.operators.bash import BashOperator
+from airflow.providers.google.cloud.operators.bigquery import (
+    BigQueryCreateEmptyDatasetOperator,
+    BigQueryDeleteDatasetOperator,
+    BigQueryGetDatasetOperator,
+    BigQueryUpdateDatasetOperator,
+)
+from airflow.utils.trigger_rule import TriggerRule

Review comment:
       Is this a required component that authors must remember to include for system tests to work correctly? If so the pre-commit check should verify this is present as well and used below correctly.

##########
File path: scripts/ci/pre_commit/pre_commit_check_watcher_in_examples.py
##########
@@ -0,0 +1,86 @@
+#!/usr/bin/env python3
+# 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 sys
+from pathlib import Path
+from typing import List
+
+from rich.console import Console
+
+if __name__ not in ("__main__", "__mp_main__"):
+    raise SystemExit(
+        "This file is intended to be executed as an executable program. You cannot use it as a module."
+        f"To run this script, run the ./{__file__} command [FILE] ..."
+    )
+
+
+console = Console(color_system="standard", width=200)
+
+errors: List[str] = []
+
+WATCHER_APPEND_INSTRUCTION = "list(dag.tasks) >> watcher()"
+
+PYTEST_FUNCTION = """
+def test_run():
+    from airflow.utils.state import State
+
+    dag.clear(dag_run_state=State.NONE)
+    dag.run()
+"""
+
+
+def _check_file(file: Path):
+    content = file.read_text()
+    if "from tests.system.utils.watcher import watcher" in content:

Review comment:
       Someone could import the whole `watcher` module then just use `watcher.watcher()` which would evade this check right?

##########
File path: scripts/ci/pre_commit/pre_commit_check_watcher_in_examples.py
##########
@@ -0,0 +1,86 @@
+#!/usr/bin/env python3
+# 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 sys
+from pathlib import Path
+from typing import List
+
+from rich.console import Console
+
+if __name__ not in ("__main__", "__mp_main__"):
+    raise SystemExit(
+        "This file is intended to be executed as an executable program. You cannot use it as a module."
+        f"To run this script, run the ./{__file__} command [FILE] ..."
+    )
+
+
+console = Console(color_system="standard", width=200)
+
+errors: List[str] = []
+
+WATCHER_APPEND_INSTRUCTION = "list(dag.tasks) >> watcher()"
+
+PYTEST_FUNCTION = """
+def test_run():

Review comment:
       > one file or 100 it doesn't matter for your awk/sed script
   
   Hmm, this is not the kind of precedent I want to be setting :smile: It's still quite finicky and error prone.
   
   You can clean this up quite a bit using python itself. The `test_run` method can be defined in one location in a separate module and then just imported into each example dag file.
   
   I've mocked up an example, see the attached image below:
   
   ![Screenshot from 2022-03-17 11-54-21](https://user-images.githubusercontent.com/65743084/158875769-241827a0-f4e1-4bd1-8fdd-969db8ea55ab.png)




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] ferruzzi commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
ferruzzi commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r829287507



##########
File path: scripts/ci/pre_commit/pre_commit_check_watcher_in_examples.py
##########
@@ -0,0 +1,86 @@
+#!/usr/bin/env python3
+# 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 sys
+from pathlib import Path
+from typing import List
+
+from rich.console import Console
+
+if __name__ not in ("__main__", "__mp_main__"):
+    raise SystemExit(
+        "This file is intended to be executed as an executable program. You cannot use it as a module."
+        f"To run this script, run the ./{__file__} command [FILE] ..."
+    )
+
+
+console = Console(color_system="standard", width=200)
+
+errors: List[str] = []
+
+WATCHER_APPEND_INSTRUCTION = "list(dag.tasks) >> watcher()"

Review comment:
       Apologies if I missed this in the email discussions.  If I am reading this and the code below correctly, then the `chain()` method of declaring the taskflow is not really supported anymore?
   
   Related, there is nothing more frustrating to me than a precommit failing because of something it could easily fix.  Could this append the ">> watcher()" if it is not found?

##########
File path: scripts/ci/pre_commit/pre_commit_check_watcher_in_examples.py
##########
@@ -0,0 +1,86 @@
+#!/usr/bin/env python3
+# 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 sys
+from pathlib import Path
+from typing import List
+
+from rich.console import Console
+
+if __name__ not in ("__main__", "__mp_main__"):
+    raise SystemExit(
+        "This file is intended to be executed as an executable program. You cannot use it as a module."
+        f"To run this script, run the ./{__file__} command [FILE] ..."
+    )
+
+
+console = Console(color_system="standard", width=200)
+
+errors: List[str] = []
+
+WATCHER_APPEND_INSTRUCTION = "list(dag.tasks) >> watcher()"

Review comment:
       Apologies if I missed this in the email discussions.  If I am reading this and the code below correctly, then the `chain()` method of declaring the taskflow is not really supported anymore?

##########
File path: scripts/ci/pre_commit/pre_commit_check_watcher_in_examples.py
##########
@@ -0,0 +1,86 @@
+#!/usr/bin/env python3
+# 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 sys
+from pathlib import Path
+from typing import List
+
+from rich.console import Console
+
+if __name__ not in ("__main__", "__mp_main__"):
+    raise SystemExit(
+        "This file is intended to be executed as an executable program. You cannot use it as a module."
+        f"To run this script, run the ./{__file__} command [FILE] ..."
+    )
+
+
+console = Console(color_system="standard", width=200)
+
+errors: List[str] = []
+
+WATCHER_APPEND_INSTRUCTION = "list(dag.tasks) >> watcher()"
+
+PYTEST_FUNCTION = """
+def test_run():
+    from airflow.utils.state import State
+
+    dag.clear(dag_run_state=State.NONE)
+    dag.run()
+"""
+
+
+def _check_file(file: Path):
+    content = file.read_text()
+    if "from tests.system.utils.watcher import watcher" in content:
+        index = content.find(WATCHER_APPEND_INSTRUCTION)
+        if index == -1:
+            errors.append(
+                f"[red]The example {file} imports tests.system.utils.watcher "
+                f"but does not use it properly![/]\n\n"
+                "[yellow]Make sure you have:[/]\n\n"
+                f"        {WATCHER_APPEND_INSTRUCTION}\n\n"
+                "[yellow]as last instruction in your example DAG.[/]\n"
+            )
+        else:
+            operator_leftshift_index = content.find("<<", index + len(WATCHER_APPEND_INSTRUCTION))
+            operator_rightshift_index = content.find(">>", index + len(WATCHER_APPEND_INSTRUCTION))
+            if operator_leftshift_index != -1 or operator_rightshift_index != -1:
+                errors.append(
+                    f"[red]In the  example {file} "
+                    f"watcher is not last instruction in your DAG (there are << "
+                    f"or >> operators after it)![/]\n\n"
+                    "[yellow]Make sure you have:[/]\n"
+                    f"        {WATCHER_APPEND_INSTRUCTION}\n\n"
+                    "[yellow]as last instruction in your example DAG.[/]\n"
+                )
+        if PYTEST_FUNCTION not in content:
+            errors.append(
+                f"[yellow]The  example {file} missed the pytest function at the end.[/]\n\n"
+                "All example tests should have this function added:\n\n" + PYTEST_FUNCTION + "\n\n"
+                "[yellow]Automatically adding it now!\n"

Review comment:
       Thank you for this.  There is nothing more frustrating to me than a precommit failing because of something it could easily fix magically.

##########
File path: tests/system/providers/google/bigquery/example_bigquery_operations.py
##########
@@ -0,0 +1,106 @@
+#
+# 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.
+
+"""
+Example Airflow DAG for Google BigQuery service local file upload and external table creation.
+"""
+import os
+from datetime import datetime
+from pathlib import Path
+
+from airflow import models
+from airflow.providers.google.cloud.operators.bigquery import (
+    BigQueryCreateEmptyDatasetOperator,
+    BigQueryCreateExternalTableOperator,
+    BigQueryDeleteDatasetOperator,
+)
+from airflow.providers.google.cloud.operators.gcs import GCSCreateBucketOperator, GCSDeleteBucketOperator
+from airflow.providers.google.cloud.transfers.local_to_gcs import LocalFilesystemToGCSOperator
+from airflow.utils.trigger_rule import TriggerRule
+from tests.system.utils.watcher import watcher
+
+ENV_ID = os.environ.get("SYSTEM_TESTS_ENV_ID")
+DAG_ID = "bigquery_operations"
+
+DATASET_NAME = f"dataset_{DAG_ID}_{ENV_ID}"
+DATA_SAMPLE_GCS_BUCKET_NAME = f"bucket_{DAG_ID}_{ENV_ID}"
+DATA_SAMPLE_GCS_OBJECT_NAME = "bigquery/us-states/us-states.csv"
+CSV_FILE_LOCAL_PATH = str(Path(__file__).parent / "resources" / "us-states.csv")
+
+
+with models.DAG(
+    DAG_ID,
+    schedule_interval="@once",
+    start_date=datetime(2021, 1, 1),
+    catchup=False,
+    tags=["example", "bigquery"],
+) as dag:
+    create_bucket = GCSCreateBucketOperator(task_id="create_bucket", bucket_name=DATA_SAMPLE_GCS_BUCKET_NAME)
+
+    create_dataset = BigQueryCreateEmptyDatasetOperator(task_id="create_dataset", dataset_id=DATASET_NAME)
+
+    upload_file = LocalFilesystemToGCSOperator(
+        task_id="upload_file_to_bucket",
+        src=CSV_FILE_LOCAL_PATH,
+        dst=DATA_SAMPLE_GCS_OBJECT_NAME,
+        bucket=DATA_SAMPLE_GCS_BUCKET_NAME,
+    )
+
+    # [START howto_operator_bigquery_create_external_table]
+    create_external_table = BigQueryCreateExternalTableOperator(
+        task_id="create_external_table",
+        destination_project_dataset_table=f"{DATASET_NAME}.external_table",
+        bucket=DATA_SAMPLE_GCS_BUCKET_NAME,
+        source_objects=[DATA_SAMPLE_GCS_OBJECT_NAME],
+        schema_fields=[
+            {"name": "emp_name", "type": "STRING", "mode": "REQUIRED"},
+            {"name": "salary", "type": "INTEGER", "mode": "NULLABLE"},
+        ],
+    )
+    # [END howto_operator_bigquery_create_external_table]
+
+    delete_dataset = BigQueryDeleteDatasetOperator(
+        task_id="delete_dataset",
+        dataset_id=DATASET_NAME,
+        delete_contents=True,
+        trigger_rule=TriggerRule.ALL_DONE,
+    )
+
+    delete_bucket = GCSDeleteBucketOperator(
+        task_id="delete_bucket", bucket_name=DATA_SAMPLE_GCS_BUCKET_NAME, trigger_rule=TriggerRule.ALL_DONE
+    )
+
+    (
+        # TEST SETUP
+        [create_bucket, create_dataset]
+        # TEST BODY
+        >> upload_file
+        >> create_external_table
+        # TEST TEARDOWN
+        >> delete_dataset
+        >> delete_bucket
+    )
+
+    list(dag.tasks) >> watcher()

Review comment:
       I know this was discussed int he email thread, but I'll raise the concern one last time because this example really drives it home, then I promise I'll drop it.  We are adding the exact same line of code to every Example DAG (now System Test), shouldn't that really be handled behind the scenes when the test DAG is being parsed instead of adding this copy/pasted line to literally hundreds of files?

##########
File path: scripts/ci/pre_commit/pre_commit_check_watcher_in_examples.py
##########
@@ -0,0 +1,86 @@
+#!/usr/bin/env python3
+# 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 sys
+from pathlib import Path
+from typing import List
+
+from rich.console import Console
+
+if __name__ not in ("__main__", "__mp_main__"):
+    raise SystemExit(
+        "This file is intended to be executed as an executable program. You cannot use it as a module."
+        f"To run this script, run the ./{__file__} command [FILE] ..."
+    )
+
+
+console = Console(color_system="standard", width=200)
+
+errors: List[str] = []
+
+WATCHER_APPEND_INSTRUCTION = "list(dag.tasks) >> watcher()"

Review comment:
       ~Apologies if I missed this in the email discussions.  If I am reading this and the code below correctly, then the `chain()` method of declaring the taskflow is not really supported anymore?~
   
   Totally did miss it, very sorry about that.

##########
File path: tests/system/providers/google/bigquery/example_bigquery_operations.py
##########
@@ -0,0 +1,106 @@
+#
+# 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.
+
+"""
+Example Airflow DAG for Google BigQuery service local file upload and external table creation.
+"""
+import os
+from datetime import datetime
+from pathlib import Path
+
+from airflow import models
+from airflow.providers.google.cloud.operators.bigquery import (
+    BigQueryCreateEmptyDatasetOperator,
+    BigQueryCreateExternalTableOperator,
+    BigQueryDeleteDatasetOperator,
+)
+from airflow.providers.google.cloud.operators.gcs import GCSCreateBucketOperator, GCSDeleteBucketOperator
+from airflow.providers.google.cloud.transfers.local_to_gcs import LocalFilesystemToGCSOperator
+from airflow.utils.trigger_rule import TriggerRule
+from tests.system.utils.watcher import watcher
+
+ENV_ID = os.environ.get("SYSTEM_TESTS_ENV_ID")
+DAG_ID = "bigquery_operations"
+
+DATASET_NAME = f"dataset_{DAG_ID}_{ENV_ID}"
+DATA_SAMPLE_GCS_BUCKET_NAME = f"bucket_{DAG_ID}_{ENV_ID}"
+DATA_SAMPLE_GCS_OBJECT_NAME = "bigquery/us-states/us-states.csv"
+CSV_FILE_LOCAL_PATH = str(Path(__file__).parent / "resources" / "us-states.csv")
+
+
+with models.DAG(
+    DAG_ID,
+    schedule_interval="@once",
+    start_date=datetime(2021, 1, 1),
+    catchup=False,
+    tags=["example", "bigquery"],
+) as dag:
+    create_bucket = GCSCreateBucketOperator(task_id="create_bucket", bucket_name=DATA_SAMPLE_GCS_BUCKET_NAME)
+
+    create_dataset = BigQueryCreateEmptyDatasetOperator(task_id="create_dataset", dataset_id=DATASET_NAME)
+
+    upload_file = LocalFilesystemToGCSOperator(
+        task_id="upload_file_to_bucket",
+        src=CSV_FILE_LOCAL_PATH,
+        dst=DATA_SAMPLE_GCS_OBJECT_NAME,
+        bucket=DATA_SAMPLE_GCS_BUCKET_NAME,
+    )
+
+    # [START howto_operator_bigquery_create_external_table]
+    create_external_table = BigQueryCreateExternalTableOperator(
+        task_id="create_external_table",
+        destination_project_dataset_table=f"{DATASET_NAME}.external_table",
+        bucket=DATA_SAMPLE_GCS_BUCKET_NAME,
+        source_objects=[DATA_SAMPLE_GCS_OBJECT_NAME],
+        schema_fields=[
+            {"name": "emp_name", "type": "STRING", "mode": "REQUIRED"},
+            {"name": "salary", "type": "INTEGER", "mode": "NULLABLE"},
+        ],
+    )
+    # [END howto_operator_bigquery_create_external_table]
+
+    delete_dataset = BigQueryDeleteDatasetOperator(
+        task_id="delete_dataset",
+        dataset_id=DATASET_NAME,
+        delete_contents=True,
+        trigger_rule=TriggerRule.ALL_DONE,
+    )
+
+    delete_bucket = GCSDeleteBucketOperator(
+        task_id="delete_bucket", bucket_name=DATA_SAMPLE_GCS_BUCKET_NAME, trigger_rule=TriggerRule.ALL_DONE
+    )
+
+    (
+        # TEST SETUP
+        [create_bucket, create_dataset]
+        # TEST BODY
+        >> upload_file
+        >> create_external_table
+        # TEST TEARDOWN
+        >> delete_dataset
+        >> delete_bucket
+    )
+
+    list(dag.tasks) >> watcher()

Review comment:
       I know this was discussed in the email thread, but I'll raise the concern one last time because this example really drives it home, then I promise I'll drop it.  We are adding the exact same line of code to every Example DAG (now System Test), shouldn't that really be handled behind the scenes when the test DAG is being parsed instead of adding this copy/pasted line to literally hundreds of files?

##########
File path: tests/system/providers/google/bigquery/example_bigquery_operations.py
##########
@@ -0,0 +1,106 @@
+#
+# 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.
+
+"""
+Example Airflow DAG for Google BigQuery service local file upload and external table creation.
+"""
+import os
+from datetime import datetime
+from pathlib import Path
+
+from airflow import models
+from airflow.providers.google.cloud.operators.bigquery import (
+    BigQueryCreateEmptyDatasetOperator,
+    BigQueryCreateExternalTableOperator,
+    BigQueryDeleteDatasetOperator,
+)
+from airflow.providers.google.cloud.operators.gcs import GCSCreateBucketOperator, GCSDeleteBucketOperator
+from airflow.providers.google.cloud.transfers.local_to_gcs import LocalFilesystemToGCSOperator
+from airflow.utils.trigger_rule import TriggerRule
+from tests.system.utils.watcher import watcher
+
+ENV_ID = os.environ.get("SYSTEM_TESTS_ENV_ID")
+DAG_ID = "bigquery_operations"
+
+DATASET_NAME = f"dataset_{DAG_ID}_{ENV_ID}"
+DATA_SAMPLE_GCS_BUCKET_NAME = f"bucket_{DAG_ID}_{ENV_ID}"
+DATA_SAMPLE_GCS_OBJECT_NAME = "bigquery/us-states/us-states.csv"
+CSV_FILE_LOCAL_PATH = str(Path(__file__).parent / "resources" / "us-states.csv")
+
+
+with models.DAG(
+    DAG_ID,
+    schedule_interval="@once",
+    start_date=datetime(2021, 1, 1),
+    catchup=False,
+    tags=["example", "bigquery"],
+) as dag:
+    create_bucket = GCSCreateBucketOperator(task_id="create_bucket", bucket_name=DATA_SAMPLE_GCS_BUCKET_NAME)
+
+    create_dataset = BigQueryCreateEmptyDatasetOperator(task_id="create_dataset", dataset_id=DATASET_NAME)
+
+    upload_file = LocalFilesystemToGCSOperator(
+        task_id="upload_file_to_bucket",
+        src=CSV_FILE_LOCAL_PATH,
+        dst=DATA_SAMPLE_GCS_OBJECT_NAME,
+        bucket=DATA_SAMPLE_GCS_BUCKET_NAME,
+    )
+
+    # [START howto_operator_bigquery_create_external_table]
+    create_external_table = BigQueryCreateExternalTableOperator(
+        task_id="create_external_table",
+        destination_project_dataset_table=f"{DATASET_NAME}.external_table",
+        bucket=DATA_SAMPLE_GCS_BUCKET_NAME,
+        source_objects=[DATA_SAMPLE_GCS_OBJECT_NAME],
+        schema_fields=[
+            {"name": "emp_name", "type": "STRING", "mode": "REQUIRED"},
+            {"name": "salary", "type": "INTEGER", "mode": "NULLABLE"},
+        ],
+    )
+    # [END howto_operator_bigquery_create_external_table]
+
+    delete_dataset = BigQueryDeleteDatasetOperator(
+        task_id="delete_dataset",
+        dataset_id=DATASET_NAME,
+        delete_contents=True,
+        trigger_rule=TriggerRule.ALL_DONE,
+    )
+
+    delete_bucket = GCSDeleteBucketOperator(
+        task_id="delete_bucket", bucket_name=DATA_SAMPLE_GCS_BUCKET_NAME, trigger_rule=TriggerRule.ALL_DONE
+    )
+
+    (
+        # TEST SETUP
+        [create_bucket, create_dataset]
+        # TEST BODY
+        >> upload_file
+        >> create_external_table
+        # TEST TEARDOWN
+        >> delete_dataset
+        >> delete_bucket
+    )
+
+    list(dag.tasks) >> watcher()

Review comment:
       > I am open to any specific suggestions
   
   I get that.  Unfortunately I only have half-baked suggestions.  the only thing I can really think of would be if we had something like `tests.system.DAG` which inherits `airflow.DAG` and adds that watcher to the task list, but that's only a half-baked idea and I haven't really thought it through all the way.  Writing the same line in hundreds of files just feels wrong, but without a more solid alternative, I'll stop beating that dead horse.  Thanks for the effort you've put into this, and I hope I'm not coming off as argumentative or anything.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] potiuk commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r831038630



##########
File path: tests/system/conftest.py
##########
@@ -0,0 +1,61 @@
+# 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 os
+import re
+from itertools import chain
+from pathlib import Path
+
+import pytest
+
+OLD_EXECUTOR = os.environ.get("AIRFLOW__CORE__EXECUTOR", default=None)
+REQUIRED_ENV_VARS = ("SYSTEM_TESTS_ENV_ID",)
+
+
+def pytest_configure(config):
+    os.environ["AIRFLOW__CORE__EXECUTOR"] = "DebugExecutor"
+
+
+def pytest_unconfigure(config):
+    if OLD_EXECUTOR is not None:
+        os.environ["AIRFLOW__CORE__EXECUTOR"] = OLD_EXECUTOR

Review comment:
       > Ah you're right, in this case if the env var was not set we should unset it. We will try with mock patching (like in your example).
   
   Yep. this is what I also had in mind with yield/patch for autofixture.
   




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] mik-laj commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r832166656



##########
File path: docs/apache-airflow-providers-google/operators/cloud/bigquery.rst
##########
@@ -42,7 +42,7 @@ Create dataset
 To create an empty dataset in a BigQuery database you can use
 :class:`~airflow.providers.google.cloud.operators.bigquery.BigQueryCreateEmptyDatasetOperator`.
 
-.. exampleinclude:: /../../airflow/providers/google/cloud/example_dags/example_bigquery_operations.py
+.. exampleinclude:: /../../tests/system/providers/google/bigquery/example_bigquery_dataset.py

Review comment:
       We have to check if the example include directive is correctly used. I think you might need to update it.
   https://github.com/apache/airflow/blob/main/docs/exts/docs_build/lint_checks.py#L232-L247




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] mnojek commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
mnojek commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r830825647



##########
File path: tests/always/test_example_dags.py
##########
@@ -29,33 +30,38 @@
 NO_DB_QUERY_EXCEPTION = ["/airflow/example_dags/example_subdag_operator.py"]
 
 
-class TestExampleDags(unittest.TestCase):
-    def test_should_be_importable(self):
-        example_dags = list(glob(f"{ROOT_FOLDER}/airflow/**/example_dags/example_*.py", recursive=True))
-        assert 0 != len(example_dags)
-        for filepath in example_dags:
-            relative_filepath = os.path.relpath(filepath, ROOT_FOLDER)
-            with self.subTest(f"File {relative_filepath} should contain dags"):
-                dagbag = DagBag(
-                    dag_folder=filepath,
-                    include_examples=False,
-                )
-                assert 0 == len(dagbag.import_errors), f"import_errors={str(dagbag.import_errors)}"
-                assert len(dagbag.dag_ids) >= 1
-
-    def test_should_not_do_database_queries(self):
-        example_dags = glob(f"{ROOT_FOLDER}/airflow/**/example_dags/example_*.py", recursive=True)
-        example_dags = [
-            dag_file
-            for dag_file in example_dags
-            if any(not dag_file.endswith(e) for e in NO_DB_QUERY_EXCEPTION)
-        ]
-        assert 0 != len(example_dags)
-        for filepath in example_dags:
-            relative_filepath = os.path.relpath(filepath, ROOT_FOLDER)
-            with self.subTest(f"File {relative_filepath} shouldn't do database queries"):
-                with assert_queries_count(0):
-                    DagBag(
-                        dag_folder=filepath,
-                        include_examples=False,
-                    )
+def example_dags():
+    example_dirs = ["airflow/**/example_dags/example_*.py", "tests/system/providers/**/example_*.py"]
+    for example_dir in example_dirs:
+        yield from glob(f"{ROOT_FOLDER}/{example_dir}", recursive=True)
+
+
+def example_dags_except_db_exception():
+    return [
+        dag_file
+        for dag_file in example_dags()
+        if any(not dag_file.endswith(e) for e in NO_DB_QUERY_EXCEPTION)
+    ]
+
+
+def relative_path(path):
+    return os.path.relpath(path, ROOT_FOLDER)
+
+
+@pytest.mark.parametrize("example", list(example_dags()), ids=relative_path)
+def test_should_be_importable(example):
+    dagbag = DagBag(
+        dag_folder=example,
+        include_examples=False,
+    )
+    assert 0 == len(dagbag.import_errors), f"import_errors={str(dagbag.import_errors)}"

Review comment:
       The code is updated




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] mnojek commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
mnojek commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r830825511



##########
File path: tests/always/test_example_dags.py
##########
@@ -29,33 +30,38 @@
 NO_DB_QUERY_EXCEPTION = ["/airflow/example_dags/example_subdag_operator.py"]
 
 
-class TestExampleDags(unittest.TestCase):
-    def test_should_be_importable(self):
-        example_dags = list(glob(f"{ROOT_FOLDER}/airflow/**/example_dags/example_*.py", recursive=True))
-        assert 0 != len(example_dags)
-        for filepath in example_dags:
-            relative_filepath = os.path.relpath(filepath, ROOT_FOLDER)
-            with self.subTest(f"File {relative_filepath} should contain dags"):
-                dagbag = DagBag(
-                    dag_folder=filepath,
-                    include_examples=False,
-                )
-                assert 0 == len(dagbag.import_errors), f"import_errors={str(dagbag.import_errors)}"
-                assert len(dagbag.dag_ids) >= 1
-
-    def test_should_not_do_database_queries(self):
-        example_dags = glob(f"{ROOT_FOLDER}/airflow/**/example_dags/example_*.py", recursive=True)
-        example_dags = [
-            dag_file
-            for dag_file in example_dags
-            if any(not dag_file.endswith(e) for e in NO_DB_QUERY_EXCEPTION)
-        ]
-        assert 0 != len(example_dags)
-        for filepath in example_dags:
-            relative_filepath = os.path.relpath(filepath, ROOT_FOLDER)
-            with self.subTest(f"File {relative_filepath} shouldn't do database queries"):
-                with assert_queries_count(0):
-                    DagBag(
-                        dag_folder=filepath,
-                        include_examples=False,
-                    )
+def example_dags():
+    example_dirs = ["airflow/**/example_dags/example_*.py", "tests/system/providers/**/example_*.py"]
+    for example_dir in example_dirs:
+        yield from glob(f"{ROOT_FOLDER}/{example_dir}", recursive=True)
+
+
+def example_dags_except_db_exception():
+    return [
+        dag_file
+        for dag_file in example_dags()
+        if any(not dag_file.endswith(e) for e in NO_DB_QUERY_EXCEPTION)
+    ]
+
+
+def relative_path(path):
+    return os.path.relpath(path, ROOT_FOLDER)
+
+
+@pytest.mark.parametrize("example", list(example_dags()), ids=relative_path)

Review comment:
       Changed




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] bhirsz commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
bhirsz commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r829950413



##########
File path: docs/apache-airflow-providers-google/operators/cloud/bigquery.rst
##########
@@ -29,7 +29,7 @@ data.
 Prerequisite Tasks
 ^^^^^^^^^^^^^^^^^^
 
-.. include::/operators/_partials/prerequisite_tasks.rst
+.. include:: ../_partials/prerequisite_tasks.rst

Review comment:
       It's not although it's common practice, see sphinx docs: https://docutils.sourceforge.io/docs/ref/rst/directives.html#include . This line is actually not needed for system tests - it's fix because I've noticed that Google provider docs doesn't contain prerequisite_tasks section content (due to invalid include - for at least few versions). It's very minor so I think it's fine to also include this change in this PR.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] ferruzzi commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
ferruzzi commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r829287507



##########
File path: scripts/ci/pre_commit/pre_commit_check_watcher_in_examples.py
##########
@@ -0,0 +1,86 @@
+#!/usr/bin/env python3
+# 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 sys
+from pathlib import Path
+from typing import List
+
+from rich.console import Console
+
+if __name__ not in ("__main__", "__mp_main__"):
+    raise SystemExit(
+        "This file is intended to be executed as an executable program. You cannot use it as a module."
+        f"To run this script, run the ./{__file__} command [FILE] ..."
+    )
+
+
+console = Console(color_system="standard", width=200)
+
+errors: List[str] = []
+
+WATCHER_APPEND_INSTRUCTION = "list(dag.tasks) >> watcher()"

Review comment:
       ~Apologies if I missed this in the email discussions.  If I am reading this and the code below correctly, then the `chain()` method of declaring the taskflow is not really supported anymore?~
   
   Totally did miss it, very sorry about that.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] potiuk commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r833970996



##########
File path: tests/system/utils/__init__.py
##########
@@ -0,0 +1,25 @@
+# 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.
+from airflow.utils.state import State
+
+
+def get_test_run(dag):
+    def test_run():
+        dag.clear(dag_run_state=State.NONE)
+        dag.run()
+
+    return test_run

Review comment:
       Good simplification. It is needed, because we want to be able to change it's behaviour globally (for example during manual testing). You might want to - for example - change log level or record additional information or start debugger. or start monitoring memory at the "run time" or do other things. This is purely to avoid having to do it manually in all tests that use it.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] ferruzzi commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
ferruzzi commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r834798110



##########
File path: tests/system/providers/google/bigquery/example_bigquery_dataset.py
##########
@@ -0,0 +1,95 @@
+#
+# 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.
+
+"""
+Example Airflow DAG for Google BigQuery service testing dataset operations.
+"""
+import os
+from datetime import datetime
+
+from airflow import models
+from airflow.operators.bash import BashOperator
+from airflow.providers.google.cloud.operators.bigquery import (
+    BigQueryCreateEmptyDatasetOperator,
+    BigQueryDeleteDatasetOperator,
+    BigQueryGetDatasetOperator,
+    BigQueryUpdateDatasetOperator,
+)
+from airflow.utils.trigger_rule import TriggerRule
+
+ENV_ID = os.environ.get("SYSTEM_TESTS_ENV_ID")
+DAG_ID = "bigquery_dataset"
+
+DATASET_NAME = f"dataset_{DAG_ID}_{ENV_ID}"
+
+
+with models.DAG(
+    DAG_ID,
+    schedule_interval="@once",
+    start_date=datetime(2021, 1, 1),
+    catchup=False,
+    tags=["example", "bigquery"],
+) as dag:
+    # [START howto_operator_bigquery_create_dataset]
+    create_dataset = BigQueryCreateEmptyDatasetOperator(task_id="create_dataset", dataset_id=DATASET_NAME)
+    # [END howto_operator_bigquery_create_dataset]
+
+    # [START howto_operator_bigquery_update_dataset]
+    update_dataset = BigQueryUpdateDatasetOperator(
+        task_id="update_dataset",
+        dataset_id=DATASET_NAME,
+        dataset_resource={"description": "Updated dataset"},
+    )
+    # [END howto_operator_bigquery_update_dataset]
+
+    # [START howto_operator_bigquery_get_dataset]
+    get_dataset = BigQueryGetDatasetOperator(task_id="get-dataset", dataset_id=DATASET_NAME)
+    # [END howto_operator_bigquery_get_dataset]
+
+    get_dataset_result = BashOperator(
+        task_id="get_dataset_result",
+        bash_command="echo \"{{ task_instance.xcom_pull('get-dataset')['id'] }}\"",
+    )
+
+    # [START howto_operator_bigquery_delete_dataset]
+    delete_dataset = BigQueryDeleteDatasetOperator(
+        task_id="delete_dataset", dataset_id=DATASET_NAME, delete_contents=True
+    )
+    # [END howto_operator_bigquery_delete_dataset]
+    delete_dataset.trigger_rule = TriggerRule.ALL_DONE
+
+    (
+        # TEST BODY
+        create_dataset
+        >> update_dataset
+        >> get_dataset
+        >> get_dataset_result
+        # TEST TEARDOWN
+        >> delete_dataset
+    )
+
+    from tests.system.utils.watcher import watcher

Review comment:
       This thread is a rollercoaster of "I DID GOOD! oh wait..no...  "  :stuck_out_tongue: 
   
   > but when running them through the Airflow UI - it doesn't.
   
   Am I understanding that to mean you are manually running tests from the Airflow UI?   I actually never realized that was a thing.   Or did you add that to a DAG which should have worked and it killed the DAG?




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] Bowrna commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
Bowrna commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r839484636



##########
File path: docs/apache-airflow/best-practices.rst
##########
@@ -247,18 +247,26 @@ each parameter by following the links):
 Example of watcher pattern with trigger rules
 ---------------------------------------------
 
-The watcher pattern is how we call a DAG with a task that is "watching" the states of the other tasks. It's primary purpose is to fail a DAG Run when any other task fail.
+The watcher pattern is how we call a DAG with a task that is "watching" the states of the other tasks.
+It's primary purpose is to fail a DAG Run when any other task fail.
 The need came from the Airflow system tests that are DAGs with different tasks (similarly like a test containing steps).
 
-Normally, when any task fails, all other tasks are not executed and the whole DAG Run gets failed status too. But when we use trigger rules, we can disrupt the normal flow of running tasks and the whole DAG may represent different status that we expect.
-For example, we can have a teardown task (with trigger rule set to ``"all_done"``) that will be executed regardless of the state of the other tasks (e.g. to clean up the resources). In such situation, the DAG would always run this task and the DAG Run will get the status of this particular task, so we can potentially lose the information about failing tasks.
-If we want to ensure that the DAG with teardown task would fail if any task fails, we need to use the watcher pattern.
-The watcher task is a task that will always fail if triggered, but it needs to be triggered only if any other task fails. It needs to have a trigger rule set to ``"one_failed"`` and it needs also to be a downstream task for all other tasks in the DAG.
-Thanks to this, if every other task will pass, the watcher will be skipped, but when something fails, the watcher task will be executed and fail making the DAG Run fail too.
+Normally, when any task fails, all other tasks are not executed and the whole DAG Run gets failed status too. But
+when we use trigger rules, we can disrupt the normal flow of running tasks and the whole DAG may represent different
+status that we expect. For example, we can have a teardown task (with trigger rule set to ``TriggerRule.ALL_DONE``)
+that will be executed regardless of the state of the other tasks (e.g. to clean up the resources). In such
+situation, the DAG would always run this task and the DAG Run will get the status of this particular task, so we can
+potentially lose the information about failing tasks. If we want to ensure that the DAG with teardown task would fail
+if any task fails, we need to  use the watcher pattern. The watcher task is a task that will always fail if
+triggered, but it needs to be triggered only if any other task fails. It needs to have a trigger rule set to
+``TriggerRule.ONE_FAILED`` and it needs also to be a  downstream task for all other tasks in the DAG. Thanks to
+this, if every other task will pass, the watcher will be skipped, but when something fails, the watcher task will be

Review comment:
       If every task will pass, then watcher will be skipped. Does it mean the teardown task(e.g. to clean up the resources) will be skipped? @mnojek 




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] potiuk commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r829580834



##########
File path: tests/system/providers/google/bigquery/example_bigquery_operations.py
##########
@@ -0,0 +1,106 @@
+#
+# 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.
+
+"""
+Example Airflow DAG for Google BigQuery service local file upload and external table creation.
+"""
+import os
+from datetime import datetime
+from pathlib import Path
+
+from airflow import models
+from airflow.providers.google.cloud.operators.bigquery import (
+    BigQueryCreateEmptyDatasetOperator,
+    BigQueryCreateExternalTableOperator,
+    BigQueryDeleteDatasetOperator,
+)
+from airflow.providers.google.cloud.operators.gcs import GCSCreateBucketOperator, GCSDeleteBucketOperator
+from airflow.providers.google.cloud.transfers.local_to_gcs import LocalFilesystemToGCSOperator
+from airflow.utils.trigger_rule import TriggerRule
+from tests.system.utils.watcher import watcher
+
+ENV_ID = os.environ.get("SYSTEM_TESTS_ENV_ID")
+DAG_ID = "bigquery_operations"
+
+DATASET_NAME = f"dataset_{DAG_ID}_{ENV_ID}"
+DATA_SAMPLE_GCS_BUCKET_NAME = f"bucket_{DAG_ID}_{ENV_ID}"
+DATA_SAMPLE_GCS_OBJECT_NAME = "bigquery/us-states/us-states.csv"
+CSV_FILE_LOCAL_PATH = str(Path(__file__).parent / "resources" / "us-states.csv")
+
+
+with models.DAG(
+    DAG_ID,
+    schedule_interval="@once",
+    start_date=datetime(2021, 1, 1),
+    catchup=False,
+    tags=["example", "bigquery"],
+) as dag:
+    create_bucket = GCSCreateBucketOperator(task_id="create_bucket", bucket_name=DATA_SAMPLE_GCS_BUCKET_NAME)
+
+    create_dataset = BigQueryCreateEmptyDatasetOperator(task_id="create_dataset", dataset_id=DATASET_NAME)
+
+    upload_file = LocalFilesystemToGCSOperator(
+        task_id="upload_file_to_bucket",
+        src=CSV_FILE_LOCAL_PATH,
+        dst=DATA_SAMPLE_GCS_OBJECT_NAME,
+        bucket=DATA_SAMPLE_GCS_BUCKET_NAME,
+    )
+
+    # [START howto_operator_bigquery_create_external_table]
+    create_external_table = BigQueryCreateExternalTableOperator(
+        task_id="create_external_table",
+        destination_project_dataset_table=f"{DATASET_NAME}.external_table",
+        bucket=DATA_SAMPLE_GCS_BUCKET_NAME,
+        source_objects=[DATA_SAMPLE_GCS_OBJECT_NAME],
+        schema_fields=[
+            {"name": "emp_name", "type": "STRING", "mode": "REQUIRED"},
+            {"name": "salary", "type": "INTEGER", "mode": "NULLABLE"},
+        ],
+    )
+    # [END howto_operator_bigquery_create_external_table]
+
+    delete_dataset = BigQueryDeleteDatasetOperator(
+        task_id="delete_dataset",
+        dataset_id=DATASET_NAME,
+        delete_contents=True,
+        trigger_rule=TriggerRule.ALL_DONE,
+    )
+
+    delete_bucket = GCSDeleteBucketOperator(
+        task_id="delete_bucket", bucket_name=DATA_SAMPLE_GCS_BUCKET_NAME, trigger_rule=TriggerRule.ALL_DONE
+    )
+
+    (
+        # TEST SETUP
+        [create_bucket, create_dataset]
+        # TEST BODY
+        >> upload_file
+        >> create_external_table
+        # TEST TEARDOWN
+        >> delete_dataset
+        >> delete_bucket
+    )
+
+    list(dag.tasks) >> watcher()

Review comment:
       We thought about a special "util" function for that.
   
   I do not think we discussed about "magical" adding it. this is a bit of a hassle, I agreee but adding it would likely require something on the DAG level - for example a special parameter or decorator. And the problem with that is that it would have to be added somewhere at the beginning of the DAG - where you define the DAG most likely. And the problem with that is that you also use the DAG as examples in our documentation. And we extracts parts of the examples into the documentaiton and we should not pollute those examples with things that are not really good "examples" on how you should add your DAGs. - and those example usually show the DAG definition/default_args as part of the example - by having the special decorator, or parameter on the DAG to indicate that DAG shoudl have "watcher" added migth be too easily copied from those extracted examples.
   
   Having explicit watcher makes it so much easier - it is at the and and it is explicit (which means there is no magic)
   
   But actually what made me think now - we should actually make it even more separated an explicit.
   
   @mnojek why don't we change the the watcher's local import and some comment there to make it even more separated and "explicit" for example:
   
   ```
            >> delete_bucket
       ) 
       
       from tests.system.utils.watcher import watcher
       # This test run as a system test needs watcher in order to mark success/failure
       # where "tearDown" task is part of the DAG
       list(dag.tasks) >> watcher()
       
   ```




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] mnojek commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
mnojek commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r830827694



##########
File path: tests/system/conftest.py
##########
@@ -0,0 +1,61 @@
+# 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 os
+import re
+from itertools import chain
+from pathlib import Path
+
+import pytest
+
+OLD_EXECUTOR = os.environ.get("AIRFLOW__CORE__EXECUTOR", default=None)
+REQUIRED_ENV_VARS = ("SYSTEM_TESTS_ENV_ID",)
+
+
+def pytest_configure(config):
+    os.environ["AIRFLOW__CORE__EXECUTOR"] = "DebugExecutor"
+
+
+def pytest_unconfigure(config):
+    if OLD_EXECUTOR is not None:
+        os.environ["AIRFLOW__CORE__EXECUTOR"] = OLD_EXECUTOR

Review comment:
       @potiuk @kosteev 
   Can I resolve this conversation now? Please see the recent changes




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] bhirsz commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
bhirsz commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r830826624



##########
File path: tests/system/conftest.py
##########
@@ -0,0 +1,61 @@
+# 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 os
+import re
+from itertools import chain
+from pathlib import Path
+
+import pytest
+
+OLD_EXECUTOR = os.environ.get("AIRFLOW__CORE__EXECUTOR", default=None)
+REQUIRED_ENV_VARS = ("SYSTEM_TESTS_ENV_ID",)
+
+
+def pytest_configure(config):
+    os.environ["AIRFLOW__CORE__EXECUTOR"] = "DebugExecutor"
+
+
+def pytest_unconfigure(config):
+    if OLD_EXECUTOR is not None:
+        os.environ["AIRFLOW__CORE__EXECUTOR"] = OLD_EXECUTOR

Review comment:
       Changed from pytest_configure to auto-fixture




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] kosteev commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
kosteev commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r831162317



##########
File path: tests/system/conftest.py
##########
@@ -0,0 +1,61 @@
+# 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 os
+import re
+from itertools import chain
+from pathlib import Path
+
+import pytest
+
+OLD_EXECUTOR = os.environ.get("AIRFLOW__CORE__EXECUTOR", default=None)
+REQUIRED_ENV_VARS = ("SYSTEM_TESTS_ENV_ID",)
+
+
+def pytest_configure(config):
+    os.environ["AIRFLOW__CORE__EXECUTOR"] = "DebugExecutor"
+
+
+def pytest_unconfigure(config):
+    if OLD_EXECUTOR is not None:
+        os.environ["AIRFLOW__CORE__EXECUTOR"] = OLD_EXECUTOR

Review comment:
       Thx




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] mnojek commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
mnojek commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r831111690



##########
File path: tests/system/providers/google/bigquery/example_bigquery_operations.py
##########
@@ -0,0 +1,106 @@
+#
+# 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.
+
+"""
+Example Airflow DAG for Google BigQuery service local file upload and external table creation.
+"""
+import os
+from datetime import datetime
+from pathlib import Path
+
+from airflow import models
+from airflow.providers.google.cloud.operators.bigquery import (
+    BigQueryCreateEmptyDatasetOperator,
+    BigQueryCreateExternalTableOperator,
+    BigQueryDeleteDatasetOperator,
+)
+from airflow.providers.google.cloud.operators.gcs import GCSCreateBucketOperator, GCSDeleteBucketOperator
+from airflow.providers.google.cloud.transfers.local_to_gcs import LocalFilesystemToGCSOperator
+from airflow.utils.trigger_rule import TriggerRule
+from tests.system.utils.watcher import watcher
+
+ENV_ID = os.environ.get("SYSTEM_TESTS_ENV_ID")
+DAG_ID = "bigquery_operations"
+
+DATASET_NAME = f"dataset_{DAG_ID}_{ENV_ID}"
+DATA_SAMPLE_GCS_BUCKET_NAME = f"bucket_{DAG_ID}_{ENV_ID}"
+DATA_SAMPLE_GCS_OBJECT_NAME = "bigquery/us-states/us-states.csv"
+CSV_FILE_LOCAL_PATH = str(Path(__file__).parent / "resources" / "us-states.csv")
+
+
+with models.DAG(
+    DAG_ID,
+    schedule_interval="@once",
+    start_date=datetime(2021, 1, 1),
+    catchup=False,
+    tags=["example", "bigquery"],
+) as dag:
+    create_bucket = GCSCreateBucketOperator(task_id="create_bucket", bucket_name=DATA_SAMPLE_GCS_BUCKET_NAME)
+
+    create_dataset = BigQueryCreateEmptyDatasetOperator(task_id="create_dataset", dataset_id=DATASET_NAME)
+
+    upload_file = LocalFilesystemToGCSOperator(
+        task_id="upload_file_to_bucket",
+        src=CSV_FILE_LOCAL_PATH,
+        dst=DATA_SAMPLE_GCS_OBJECT_NAME,
+        bucket=DATA_SAMPLE_GCS_BUCKET_NAME,
+    )
+
+    # [START howto_operator_bigquery_create_external_table]
+    create_external_table = BigQueryCreateExternalTableOperator(
+        task_id="create_external_table",
+        destination_project_dataset_table=f"{DATASET_NAME}.external_table",
+        bucket=DATA_SAMPLE_GCS_BUCKET_NAME,
+        source_objects=[DATA_SAMPLE_GCS_OBJECT_NAME],
+        schema_fields=[
+            {"name": "emp_name", "type": "STRING", "mode": "REQUIRED"},
+            {"name": "salary", "type": "INTEGER", "mode": "NULLABLE"},
+        ],
+    )
+    # [END howto_operator_bigquery_create_external_table]
+
+    delete_dataset = BigQueryDeleteDatasetOperator(
+        task_id="delete_dataset",
+        dataset_id=DATASET_NAME,
+        delete_contents=True,
+        trigger_rule=TriggerRule.ALL_DONE,
+    )
+
+    delete_bucket = GCSDeleteBucketOperator(
+        task_id="delete_bucket", bucket_name=DATA_SAMPLE_GCS_BUCKET_NAME, trigger_rule=TriggerRule.ALL_DONE
+    )
+
+    (
+        # TEST SETUP
+        [create_bucket, create_dataset]
+        # TEST BODY
+        >> upload_file
+        >> create_external_table
+        # TEST TEARDOWN
+        >> delete_dataset
+        >> delete_bucket
+    )
+
+    list(dag.tasks) >> watcher()

Review comment:
       Updated the code and moved the watcher import closer its usage.
   Also added the comment to demystify this piece of code.
   Please check and confirm if it's fine now 😃 




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] kosteev commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
kosteev commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r829863869



##########
File path: docs/apache-airflow-providers-google/operators/cloud/bigquery.rst
##########
@@ -29,7 +29,7 @@ data.
 Prerequisite Tasks
 ^^^^^^^^^^^^^^^^^^
 
-.. include::/operators/_partials/prerequisite_tasks.rst
+.. include:: ../_partials/prerequisite_tasks.rst

Review comment:
       Is space needed before ".."?

##########
File path: tests/always/test_example_dags.py
##########
@@ -29,33 +30,38 @@
 NO_DB_QUERY_EXCEPTION = ["/airflow/example_dags/example_subdag_operator.py"]
 
 
-class TestExampleDags(unittest.TestCase):
-    def test_should_be_importable(self):
-        example_dags = list(glob(f"{ROOT_FOLDER}/airflow/**/example_dags/example_*.py", recursive=True))
-        assert 0 != len(example_dags)
-        for filepath in example_dags:
-            relative_filepath = os.path.relpath(filepath, ROOT_FOLDER)
-            with self.subTest(f"File {relative_filepath} should contain dags"):
-                dagbag = DagBag(
-                    dag_folder=filepath,
-                    include_examples=False,
-                )
-                assert 0 == len(dagbag.import_errors), f"import_errors={str(dagbag.import_errors)}"
-                assert len(dagbag.dag_ids) >= 1
-
-    def test_should_not_do_database_queries(self):
-        example_dags = glob(f"{ROOT_FOLDER}/airflow/**/example_dags/example_*.py", recursive=True)
-        example_dags = [
-            dag_file
-            for dag_file in example_dags
-            if any(not dag_file.endswith(e) for e in NO_DB_QUERY_EXCEPTION)
-        ]
-        assert 0 != len(example_dags)
-        for filepath in example_dags:
-            relative_filepath = os.path.relpath(filepath, ROOT_FOLDER)
-            with self.subTest(f"File {relative_filepath} shouldn't do database queries"):
-                with assert_queries_count(0):
-                    DagBag(
-                        dag_folder=filepath,
-                        include_examples=False,
-                    )
+def example_dags():
+    example_dirs = ["airflow/**/example_dags/example_*.py", "tests/system/providers/**/example_*.py"]
+    for example_dir in example_dirs:
+        yield from glob(f"{ROOT_FOLDER}/{example_dir}", recursive=True)
+
+
+def example_dags_except_db_exception():
+    return [
+        dag_file
+        for dag_file in example_dags()
+        if any(not dag_file.endswith(e) for e in NO_DB_QUERY_EXCEPTION)
+    ]
+
+
+def relative_path(path):
+    return os.path.relpath(path, ROOT_FOLDER)
+
+
+@pytest.mark.parametrize("example", list(example_dags()), ids=relative_path)
+def test_should_be_importable(example):
+    dagbag = DagBag(
+        dag_folder=example,
+        include_examples=False,
+    )
+    assert 0 == len(dagbag.import_errors), f"import_errors={str(dagbag.import_errors)}"

Review comment:
       Nit: I am not sure about this, but I believe the general guide is to use expected value on the right side of assert statement.
   It is not different at all from the test status perspective, but may be more clear in the error message in case assert fails that `len(dagbag.import_errors)` actually has to be qual to zero.

##########
File path: scripts/ci/pre_commit/pre_commit_check_watcher_in_examples.py
##########
@@ -0,0 +1,86 @@
+#!/usr/bin/env python3
+# 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 sys
+from pathlib import Path
+from typing import List
+
+from rich.console import Console
+
+if __name__ not in ("__main__", "__mp_main__"):
+    raise SystemExit(
+        "This file is intended to be executed as an executable program. You cannot use it as a module."
+        f"To run this script, run the ./{__file__} command [FILE] ..."
+    )
+
+
+console = Console(color_system="standard", width=200)
+
+errors: List[str] = []
+
+WATCHER_APPEND_INSTRUCTION = "list(dag.tasks) >> watcher()"
+
+PYTEST_FUNCTION = """
+def test_run():
+    from airflow.utils.state import State
+
+    dag.clear(dag_run_state=State.NONE)
+    dag.run()
+"""
+
+
+def _check_file(file: Path):
+    content = file.read_text()
+    if "from tests.system.utils.watcher import watcher" in content:
+        index = content.find(WATCHER_APPEND_INSTRUCTION)
+        if index == -1:
+            errors.append(
+                f"[red]The example {file} imports tests.system.utils.watcher "
+                f"but does not use it properly![/]\n\n"
+                "[yellow]Make sure you have:[/]\n\n"
+                f"        {WATCHER_APPEND_INSTRUCTION}\n\n"
+                "[yellow]as last instruction in your example DAG.[/]\n"
+            )
+        else:
+            operator_leftshift_index = content.find("<<", index + len(WATCHER_APPEND_INSTRUCTION))
+            operator_rightshift_index = content.find(">>", index + len(WATCHER_APPEND_INSTRUCTION))
+            if operator_leftshift_index != -1 or operator_rightshift_index != -1:
+                errors.append(
+                    f"[red]In the  example {file} "
+                    f"watcher is not last instruction in your DAG (there are << "
+                    f"or >> operators after it)![/]\n\n"
+                    "[yellow]Make sure you have:[/]\n"
+                    f"        {WATCHER_APPEND_INSTRUCTION}\n\n"
+                    "[yellow]as last instruction in your example DAG.[/]\n"
+                )
+        if PYTEST_FUNCTION not in content:
+            errors.append(
+                f"[yellow]The  example {file} missed the pytest function at the end.[/]\n\n"
+                "All example tests should have this function added:\n\n" + PYTEST_FUNCTION + "\n\n"
+                "[yellow]Automatically adding it now!\n"
+            )
+            file.write_text(content + "\n" + PYTEST_FUNCTION)

Review comment:
       The name of the script does not completely correspond to what it does.
   Maybe this is not a big deal. Just pointing this out as it may confuse developer.

##########
File path: tests/always/test_example_dags.py
##########
@@ -29,33 +30,38 @@
 NO_DB_QUERY_EXCEPTION = ["/airflow/example_dags/example_subdag_operator.py"]
 
 
-class TestExampleDags(unittest.TestCase):
-    def test_should_be_importable(self):
-        example_dags = list(glob(f"{ROOT_FOLDER}/airflow/**/example_dags/example_*.py", recursive=True))
-        assert 0 != len(example_dags)
-        for filepath in example_dags:
-            relative_filepath = os.path.relpath(filepath, ROOT_FOLDER)
-            with self.subTest(f"File {relative_filepath} should contain dags"):
-                dagbag = DagBag(
-                    dag_folder=filepath,
-                    include_examples=False,
-                )
-                assert 0 == len(dagbag.import_errors), f"import_errors={str(dagbag.import_errors)}"
-                assert len(dagbag.dag_ids) >= 1
-
-    def test_should_not_do_database_queries(self):
-        example_dags = glob(f"{ROOT_FOLDER}/airflow/**/example_dags/example_*.py", recursive=True)
-        example_dags = [
-            dag_file
-            for dag_file in example_dags
-            if any(not dag_file.endswith(e) for e in NO_DB_QUERY_EXCEPTION)
-        ]
-        assert 0 != len(example_dags)
-        for filepath in example_dags:
-            relative_filepath = os.path.relpath(filepath, ROOT_FOLDER)
-            with self.subTest(f"File {relative_filepath} shouldn't do database queries"):
-                with assert_queries_count(0):
-                    DagBag(
-                        dag_folder=filepath,
-                        include_examples=False,
-                    )
+def example_dags():
+    example_dirs = ["airflow/**/example_dags/example_*.py", "tests/system/providers/**/example_*.py"]
+    for example_dir in example_dirs:
+        yield from glob(f"{ROOT_FOLDER}/{example_dir}", recursive=True)
+
+
+def example_dags_except_db_exception():
+    return [
+        dag_file
+        for dag_file in example_dags()
+        if any(not dag_file.endswith(e) for e in NO_DB_QUERY_EXCEPTION)
+    ]
+
+
+def relative_path(path):
+    return os.path.relpath(path, ROOT_FOLDER)
+
+
+@pytest.mark.parametrize("example", list(example_dags()), ids=relative_path)

Review comment:
       Nit: `list(example_dags())` converting to list is not necessary here as `parametrize` accepts iterable, right?

##########
File path: tests/system/conftest.py
##########
@@ -0,0 +1,61 @@
+# 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 os
+import re
+from itertools import chain
+from pathlib import Path
+
+import pytest
+
+OLD_EXECUTOR = os.environ.get("AIRFLOW__CORE__EXECUTOR", default=None)
+REQUIRED_ENV_VARS = ("SYSTEM_TESTS_ENV_ID",)
+
+
+def pytest_configure(config):
+    os.environ["AIRFLOW__CORE__EXECUTOR"] = "DebugExecutor"
+
+
+def pytest_unconfigure(config):
+    if OLD_EXECUTOR is not None:
+        os.environ["AIRFLOW__CORE__EXECUTOR"] = OLD_EXECUTOR

Review comment:
       Should unset "DebugExecutor" here also if the OLD_EXECUTOR is None?




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] mnojek commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
mnojek commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r833226218



##########
File path: tests/system/providers/google/bigquery/example_bigquery_dataset.py
##########
@@ -0,0 +1,95 @@
+#
+# 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.
+
+"""
+Example Airflow DAG for Google BigQuery service testing dataset operations.
+"""
+import os
+from datetime import datetime
+
+from airflow import models
+from airflow.operators.bash import BashOperator
+from airflow.providers.google.cloud.operators.bigquery import (
+    BigQueryCreateEmptyDatasetOperator,
+    BigQueryDeleteDatasetOperator,
+    BigQueryGetDatasetOperator,
+    BigQueryUpdateDatasetOperator,
+)
+from airflow.utils.trigger_rule import TriggerRule
+
+ENV_ID = os.environ.get("SYSTEM_TESTS_ENV_ID")
+DAG_ID = "bigquery_dataset"
+
+DATASET_NAME = f"dataset_{DAG_ID}_{ENV_ID}"
+
+
+with models.DAG(
+    DAG_ID,
+    schedule_interval="@once",
+    start_date=datetime(2021, 1, 1),
+    catchup=False,
+    tags=["example", "bigquery"],
+) as dag:
+    # [START howto_operator_bigquery_create_dataset]
+    create_dataset = BigQueryCreateEmptyDatasetOperator(task_id="create_dataset", dataset_id=DATASET_NAME)
+    # [END howto_operator_bigquery_create_dataset]
+
+    # [START howto_operator_bigquery_update_dataset]
+    update_dataset = BigQueryUpdateDatasetOperator(
+        task_id="update_dataset",
+        dataset_id=DATASET_NAME,
+        dataset_resource={"description": "Updated dataset"},
+    )
+    # [END howto_operator_bigquery_update_dataset]
+
+    # [START howto_operator_bigquery_get_dataset]
+    get_dataset = BigQueryGetDatasetOperator(task_id="get-dataset", dataset_id=DATASET_NAME)
+    # [END howto_operator_bigquery_get_dataset]
+
+    get_dataset_result = BashOperator(
+        task_id="get_dataset_result",
+        bash_command="echo \"{{ task_instance.xcom_pull('get-dataset')['id'] }}\"",
+    )
+
+    # [START howto_operator_bigquery_delete_dataset]
+    delete_dataset = BigQueryDeleteDatasetOperator(
+        task_id="delete_dataset", dataset_id=DATASET_NAME, delete_contents=True
+    )
+    # [END howto_operator_bigquery_delete_dataset]
+    delete_dataset.trigger_rule = TriggerRule.ALL_DONE
+
+    (
+        # TEST BODY
+        create_dataset
+        >> update_dataset
+        >> get_dataset
+        >> get_dataset_result
+        # TEST TEARDOWN
+        >> delete_dataset
+    )
+
+    from tests.system.utils.watcher import watcher

Review comment:
       Ok, it was too soon to get excited before trying it out. After some tests I must say that it doesn't work as expected. 
   When running tests using pytest, it reads the `get_test_run` function and it works fine, but when running them through the Airflow UI - it doesn't.
   AFAIK The problem is that this function is outside of the DAG and Airflow UI doesn't read it, so the watcher is never added to the DAG. While when it is run with pytest, the watcher is there.
   So I guess we are at the same spot where we were before this suggestion popped out.
   I don't see any better way to simplify it, so I just want a confirmation from others, that we stick to what we have now, unless there is another idea. @potiuk @bhirsz 




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] ferruzzi commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
ferruzzi commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r834798110



##########
File path: tests/system/providers/google/bigquery/example_bigquery_dataset.py
##########
@@ -0,0 +1,95 @@
+#
+# 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.
+
+"""
+Example Airflow DAG for Google BigQuery service testing dataset operations.
+"""
+import os
+from datetime import datetime
+
+from airflow import models
+from airflow.operators.bash import BashOperator
+from airflow.providers.google.cloud.operators.bigquery import (
+    BigQueryCreateEmptyDatasetOperator,
+    BigQueryDeleteDatasetOperator,
+    BigQueryGetDatasetOperator,
+    BigQueryUpdateDatasetOperator,
+)
+from airflow.utils.trigger_rule import TriggerRule
+
+ENV_ID = os.environ.get("SYSTEM_TESTS_ENV_ID")
+DAG_ID = "bigquery_dataset"
+
+DATASET_NAME = f"dataset_{DAG_ID}_{ENV_ID}"
+
+
+with models.DAG(
+    DAG_ID,
+    schedule_interval="@once",
+    start_date=datetime(2021, 1, 1),
+    catchup=False,
+    tags=["example", "bigquery"],
+) as dag:
+    # [START howto_operator_bigquery_create_dataset]
+    create_dataset = BigQueryCreateEmptyDatasetOperator(task_id="create_dataset", dataset_id=DATASET_NAME)
+    # [END howto_operator_bigquery_create_dataset]
+
+    # [START howto_operator_bigquery_update_dataset]
+    update_dataset = BigQueryUpdateDatasetOperator(
+        task_id="update_dataset",
+        dataset_id=DATASET_NAME,
+        dataset_resource={"description": "Updated dataset"},
+    )
+    # [END howto_operator_bigquery_update_dataset]
+
+    # [START howto_operator_bigquery_get_dataset]
+    get_dataset = BigQueryGetDatasetOperator(task_id="get-dataset", dataset_id=DATASET_NAME)
+    # [END howto_operator_bigquery_get_dataset]
+
+    get_dataset_result = BashOperator(
+        task_id="get_dataset_result",
+        bash_command="echo \"{{ task_instance.xcom_pull('get-dataset')['id'] }}\"",
+    )
+
+    # [START howto_operator_bigquery_delete_dataset]
+    delete_dataset = BigQueryDeleteDatasetOperator(
+        task_id="delete_dataset", dataset_id=DATASET_NAME, delete_contents=True
+    )
+    # [END howto_operator_bigquery_delete_dataset]
+    delete_dataset.trigger_rule = TriggerRule.ALL_DONE
+
+    (
+        # TEST BODY
+        create_dataset
+        >> update_dataset
+        >> get_dataset
+        >> get_dataset_result
+        # TEST TEARDOWN
+        >> delete_dataset
+    )
+
+    from tests.system.utils.watcher import watcher

Review comment:
       > but when running them through the Airflow UI - it doesn't.
   
   Am I understanding that to mean you are manually running tests from the Airflow UI?   I actually never realized that as a thing.   Or did you add that to a DASG which should have worked and it killed the DAG?




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] uranusjr commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
uranusjr commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r833932479



##########
File path: tests/system/utils/__init__.py
##########
@@ -0,0 +1,25 @@
+# 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.
+from airflow.utils.state import State
+
+
+def get_test_run(dag):
+    def test_run():
+        dag.clear(dag_run_state=State.NONE)
+        dag.run()
+
+    return test_run

Review comment:
       ```suggestion
   def get_test_run(dag):
       def test_run():
           dag.clear(dag_run_state=None)
           dag.run()
   
       return test_run
   ```
   
   Why is this needed though? It’s just two straightforward lines we can copy pretty much everywhere. Having to import this function mostly negates the benefits it brings.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] ferruzzi commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
ferruzzi commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r834799952



##########
File path: tests/system/providers/google/bigquery/example_bigquery_dataset.py
##########
@@ -0,0 +1,95 @@
+#
+# 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.
+
+"""
+Example Airflow DAG for Google BigQuery service testing dataset operations.
+"""
+import os
+from datetime import datetime
+
+from airflow import models
+from airflow.operators.bash import BashOperator
+from airflow.providers.google.cloud.operators.bigquery import (
+    BigQueryCreateEmptyDatasetOperator,
+    BigQueryDeleteDatasetOperator,
+    BigQueryGetDatasetOperator,
+    BigQueryUpdateDatasetOperator,
+)
+from airflow.utils.trigger_rule import TriggerRule
+
+ENV_ID = os.environ.get("SYSTEM_TESTS_ENV_ID")
+DAG_ID = "bigquery_dataset"
+
+DATASET_NAME = f"dataset_{DAG_ID}_{ENV_ID}"
+
+
+with models.DAG(
+    DAG_ID,
+    schedule_interval="@once",
+    start_date=datetime(2021, 1, 1),
+    catchup=False,
+    tags=["example", "bigquery"],
+) as dag:
+    # [START howto_operator_bigquery_create_dataset]
+    create_dataset = BigQueryCreateEmptyDatasetOperator(task_id="create_dataset", dataset_id=DATASET_NAME)
+    # [END howto_operator_bigquery_create_dataset]
+
+    # [START howto_operator_bigquery_update_dataset]
+    update_dataset = BigQueryUpdateDatasetOperator(
+        task_id="update_dataset",
+        dataset_id=DATASET_NAME,
+        dataset_resource={"description": "Updated dataset"},
+    )
+    # [END howto_operator_bigquery_update_dataset]
+
+    # [START howto_operator_bigquery_get_dataset]
+    get_dataset = BigQueryGetDatasetOperator(task_id="get-dataset", dataset_id=DATASET_NAME)
+    # [END howto_operator_bigquery_get_dataset]
+
+    get_dataset_result = BashOperator(
+        task_id="get_dataset_result",
+        bash_command="echo \"{{ task_instance.xcom_pull('get-dataset')['id'] }}\"",
+    )
+
+    # [START howto_operator_bigquery_delete_dataset]
+    delete_dataset = BigQueryDeleteDatasetOperator(
+        task_id="delete_dataset", dataset_id=DATASET_NAME, delete_contents=True
+    )
+    # [END howto_operator_bigquery_delete_dataset]
+    delete_dataset.trigger_rule = TriggerRule.ALL_DONE
+
+    (
+        # TEST BODY
+        create_dataset
+        >> update_dataset
+        >> get_dataset
+        >> get_dataset_result
+        # TEST TEARDOWN
+        >> delete_dataset
+    )
+
+    from tests.system.utils.watcher import watcher

Review comment:
       Maybe not worth holding up the merge, but I wonder if moving the "add watcher" flag to the DAG class itself may be a resolution?  then usage might look like:
   
   ```
   with DAG(
       dag_id='example_dag',
       schedule_interval=None,
       start_date=datetime(2021, 1, 1),
       tags=['example'],
       catchup=False,
       add_watcher=True,
   ) as dag:
   ```




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] potiuk commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r833998073



##########
File path: tests/system/utils/__init__.py
##########
@@ -0,0 +1,25 @@
+# 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.
+from airflow.utils.state import State
+
+
+def get_test_run(dag):
+    def test_run():
+        dag.clear(dag_run_state=State.NONE)
+        dag.run()
+
+    return test_run

Review comment:
       Yeah. This is really the balance between DRY and DAMP and while tests should be more DAMP than DRY than the actual code, some DRY-ness is also useful:
   
   https://stackoverflow.com/questions/6453235/what-does-damp-not-dry-mean-when-talking-about-unit-tests




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] bhirsz commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
bhirsz commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r829895775



##########
File path: scripts/ci/pre_commit/pre_commit_check_watcher_in_examples.py
##########
@@ -0,0 +1,86 @@
+#!/usr/bin/env python3
+# 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 sys
+from pathlib import Path
+from typing import List
+
+from rich.console import Console
+
+if __name__ not in ("__main__", "__mp_main__"):
+    raise SystemExit(
+        "This file is intended to be executed as an executable program. You cannot use it as a module."
+        f"To run this script, run the ./{__file__} command [FILE] ..."
+    )
+
+
+console = Console(color_system="standard", width=200)
+
+errors: List[str] = []
+
+WATCHER_APPEND_INSTRUCTION = "list(dag.tasks) >> watcher()"
+
+PYTEST_FUNCTION = """
+def test_run():

Review comment:
       I like the idea - it's also making it easier for temporary updates of the test method (ie adding verbose mode, overwriting executor etc). I've tested it and it appears to be working. Pytest recognizes it if I run it using pytest command however it's not detected by IDE as test method (but maybe I can workaround it somehow with pytest ini). I will discuss it with @mnojek and if we both agree we can update it
   




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] ferruzzi commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
ferruzzi commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r829287507



##########
File path: scripts/ci/pre_commit/pre_commit_check_watcher_in_examples.py
##########
@@ -0,0 +1,86 @@
+#!/usr/bin/env python3
+# 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 sys
+from pathlib import Path
+from typing import List
+
+from rich.console import Console
+
+if __name__ not in ("__main__", "__mp_main__"):
+    raise SystemExit(
+        "This file is intended to be executed as an executable program. You cannot use it as a module."
+        f"To run this script, run the ./{__file__} command [FILE] ..."
+    )
+
+
+console = Console(color_system="standard", width=200)
+
+errors: List[str] = []
+
+WATCHER_APPEND_INSTRUCTION = "list(dag.tasks) >> watcher()"

Review comment:
       Apologies if I missed this in the email discussions.  If I am reading this and the code below correctly, then the `chain()` method of declaring the taskflow is not really supported anymore?




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] ferruzzi commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
ferruzzi commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r829288965



##########
File path: scripts/ci/pre_commit/pre_commit_check_watcher_in_examples.py
##########
@@ -0,0 +1,86 @@
+#!/usr/bin/env python3
+# 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 sys
+from pathlib import Path
+from typing import List
+
+from rich.console import Console
+
+if __name__ not in ("__main__", "__mp_main__"):
+    raise SystemExit(
+        "This file is intended to be executed as an executable program. You cannot use it as a module."
+        f"To run this script, run the ./{__file__} command [FILE] ..."
+    )
+
+
+console = Console(color_system="standard", width=200)
+
+errors: List[str] = []
+
+WATCHER_APPEND_INSTRUCTION = "list(dag.tasks) >> watcher()"
+
+PYTEST_FUNCTION = """
+def test_run():
+    from airflow.utils.state import State
+
+    dag.clear(dag_run_state=State.NONE)
+    dag.run()
+"""
+
+
+def _check_file(file: Path):
+    content = file.read_text()
+    if "from tests.system.utils.watcher import watcher" in content:
+        index = content.find(WATCHER_APPEND_INSTRUCTION)
+        if index == -1:
+            errors.append(
+                f"[red]The example {file} imports tests.system.utils.watcher "
+                f"but does not use it properly![/]\n\n"
+                "[yellow]Make sure you have:[/]\n\n"
+                f"        {WATCHER_APPEND_INSTRUCTION}\n\n"
+                "[yellow]as last instruction in your example DAG.[/]\n"
+            )
+        else:
+            operator_leftshift_index = content.find("<<", index + len(WATCHER_APPEND_INSTRUCTION))
+            operator_rightshift_index = content.find(">>", index + len(WATCHER_APPEND_INSTRUCTION))
+            if operator_leftshift_index != -1 or operator_rightshift_index != -1:
+                errors.append(
+                    f"[red]In the  example {file} "
+                    f"watcher is not last instruction in your DAG (there are << "
+                    f"or >> operators after it)![/]\n\n"
+                    "[yellow]Make sure you have:[/]\n"
+                    f"        {WATCHER_APPEND_INSTRUCTION}\n\n"
+                    "[yellow]as last instruction in your example DAG.[/]\n"
+                )
+        if PYTEST_FUNCTION not in content:
+            errors.append(
+                f"[yellow]The  example {file} missed the pytest function at the end.[/]\n\n"
+                "All example tests should have this function added:\n\n" + PYTEST_FUNCTION + "\n\n"
+                "[yellow]Automatically adding it now!\n"

Review comment:
       Thank you for this.  There is nothing more frustrating to me than a precommit failing because of something it could easily fix magically.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] mnojek commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
mnojek commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r829466203



##########
File path: scripts/ci/pre_commit/pre_commit_check_watcher_in_examples.py
##########
@@ -0,0 +1,86 @@
+#!/usr/bin/env python3
+# 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 sys
+from pathlib import Path
+from typing import List
+
+from rich.console import Console
+
+if __name__ not in ("__main__", "__mp_main__"):
+    raise SystemExit(
+        "This file is intended to be executed as an executable program. You cannot use it as a module."
+        f"To run this script, run the ./{__file__} command [FILE] ..."
+    )
+
+
+console = Console(color_system="standard", width=200)
+
+errors: List[str] = []
+
+WATCHER_APPEND_INSTRUCTION = "list(dag.tasks) >> watcher()"
+
+PYTEST_FUNCTION = """
+def test_run():
+    from airflow.utils.state import State
+
+    dag.clear(dag_run_state=State.NONE)
+    dag.run()
+"""
+
+
+def _check_file(file: Path):
+    content = file.read_text()
+    if "from tests.system.utils.watcher import watcher" in content:
+        index = content.find(WATCHER_APPEND_INSTRUCTION)
+        if index == -1:
+            errors.append(
+                f"[red]The example {file} imports tests.system.utils.watcher "
+                f"but does not use it properly![/]\n\n"
+                "[yellow]Make sure you have:[/]\n\n"
+                f"        {WATCHER_APPEND_INSTRUCTION}\n\n"
+                "[yellow]as last instruction in your example DAG.[/]\n"
+            )
+        else:
+            operator_leftshift_index = content.find("<<", index + len(WATCHER_APPEND_INSTRUCTION))
+            operator_rightshift_index = content.find(">>", index + len(WATCHER_APPEND_INSTRUCTION))
+            if operator_leftshift_index != -1 or operator_rightshift_index != -1:
+                errors.append(
+                    f"[red]In the  example {file} "
+                    f"watcher is not last instruction in your DAG (there are << "
+                    f"or >> operators after it)![/]\n\n"
+                    "[yellow]Make sure you have:[/]\n"
+                    f"        {WATCHER_APPEND_INSTRUCTION}\n\n"
+                    "[yellow]as last instruction in your example DAG.[/]\n"
+                )
+        if PYTEST_FUNCTION not in content:
+            errors.append(
+                f"[yellow]The  example {file} missed the pytest function at the end.[/]\n\n"
+                "All example tests should have this function added:\n\n" + PYTEST_FUNCTION + "\n\n"
+                "[yellow]Automatically adding it now!\n"

Review comment:
       Credit goes to @potiuk!




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] bhirsz commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
bhirsz commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r831021723



##########
File path: tests/system/conftest.py
##########
@@ -0,0 +1,61 @@
+# 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 os
+import re
+from itertools import chain
+from pathlib import Path
+
+import pytest
+
+OLD_EXECUTOR = os.environ.get("AIRFLOW__CORE__EXECUTOR", default=None)
+REQUIRED_ENV_VARS = ("SYSTEM_TESTS_ENV_ID",)
+
+
+def pytest_configure(config):
+    os.environ["AIRFLOW__CORE__EXECUTOR"] = "DebugExecutor"
+
+
+def pytest_unconfigure(config):
+    if OLD_EXECUTOR is not None:
+        os.environ["AIRFLOW__CORE__EXECUTOR"] = OLD_EXECUTOR

Review comment:
       Ah you're right, in this case if the env var was not set we should unset it. We will try with mock patching (like in your example). 




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] potiuk commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r832093808



##########
File path: tests/system/conftest.py
##########
@@ -0,0 +1,58 @@
+# 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 os
+import re
+from itertools import chain
+from pathlib import Path
+from unittest import mock
+
+import pytest
+
+REQUIRED_ENV_VARS = ("SYSTEM_TESTS_ENV_ID",)
+
+
+@pytest.fixture(scope="package", autouse=True)
+def use_debug_executor():
+    with mock.patch.dict("os.environ", AIRFLOW__CORE__EXECUTOR="DebugExecutor"):
+        yield
+
+
+@pytest.fixture()
+def provider_env_vars():

Review comment:
       Good idea!




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] potiuk commented on pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
potiuk commented on pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#issuecomment-1078195770


   Ah.. need rebase 
   


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] ferruzzi commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
ferruzzi commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r834798110



##########
File path: tests/system/providers/google/bigquery/example_bigquery_dataset.py
##########
@@ -0,0 +1,95 @@
+#
+# 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.
+
+"""
+Example Airflow DAG for Google BigQuery service testing dataset operations.
+"""
+import os
+from datetime import datetime
+
+from airflow import models
+from airflow.operators.bash import BashOperator
+from airflow.providers.google.cloud.operators.bigquery import (
+    BigQueryCreateEmptyDatasetOperator,
+    BigQueryDeleteDatasetOperator,
+    BigQueryGetDatasetOperator,
+    BigQueryUpdateDatasetOperator,
+)
+from airflow.utils.trigger_rule import TriggerRule
+
+ENV_ID = os.environ.get("SYSTEM_TESTS_ENV_ID")
+DAG_ID = "bigquery_dataset"
+
+DATASET_NAME = f"dataset_{DAG_ID}_{ENV_ID}"
+
+
+with models.DAG(
+    DAG_ID,
+    schedule_interval="@once",
+    start_date=datetime(2021, 1, 1),
+    catchup=False,
+    tags=["example", "bigquery"],
+) as dag:
+    # [START howto_operator_bigquery_create_dataset]
+    create_dataset = BigQueryCreateEmptyDatasetOperator(task_id="create_dataset", dataset_id=DATASET_NAME)
+    # [END howto_operator_bigquery_create_dataset]
+
+    # [START howto_operator_bigquery_update_dataset]
+    update_dataset = BigQueryUpdateDatasetOperator(
+        task_id="update_dataset",
+        dataset_id=DATASET_NAME,
+        dataset_resource={"description": "Updated dataset"},
+    )
+    # [END howto_operator_bigquery_update_dataset]
+
+    # [START howto_operator_bigquery_get_dataset]
+    get_dataset = BigQueryGetDatasetOperator(task_id="get-dataset", dataset_id=DATASET_NAME)
+    # [END howto_operator_bigquery_get_dataset]
+
+    get_dataset_result = BashOperator(
+        task_id="get_dataset_result",
+        bash_command="echo \"{{ task_instance.xcom_pull('get-dataset')['id'] }}\"",
+    )
+
+    # [START howto_operator_bigquery_delete_dataset]
+    delete_dataset = BigQueryDeleteDatasetOperator(
+        task_id="delete_dataset", dataset_id=DATASET_NAME, delete_contents=True
+    )
+    # [END howto_operator_bigquery_delete_dataset]
+    delete_dataset.trigger_rule = TriggerRule.ALL_DONE
+
+    (
+        # TEST BODY
+        create_dataset
+        >> update_dataset
+        >> get_dataset
+        >> get_dataset_result
+        # TEST TEARDOWN
+        >> delete_dataset
+    )
+
+    from tests.system.utils.watcher import watcher

Review comment:
       This thread is a rollercoaster of "I DID GOOD! oh wait..no...  "  :stuck_out_tongue: 
   
   > but when running them through the Airflow UI - it doesn't.
   
   Am I understanding that to mean you are manually running tests from the Airflow UI?   I actually never realized that as a thing.   Or did you add that to a DASG which should have worked and it killed the DAG?




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] potiuk commented on pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
potiuk commented on pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#issuecomment-1078163698


   Seems like the precommit check did not include bigquery loop case :) 


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] bhirsz commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
bhirsz commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r829331339



##########
File path: scripts/ci/pre_commit/pre_commit_check_watcher_in_examples.py
##########
@@ -0,0 +1,86 @@
+#!/usr/bin/env python3
+# 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 sys
+from pathlib import Path
+from typing import List
+
+from rich.console import Console
+
+if __name__ not in ("__main__", "__mp_main__"):
+    raise SystemExit(
+        "This file is intended to be executed as an executable program. You cannot use it as a module."
+        f"To run this script, run the ./{__file__} command [FILE] ..."
+    )
+
+
+console = Console(color_system="standard", width=200)
+
+errors: List[str] = []
+
+WATCHER_APPEND_INSTRUCTION = "list(dag.tasks) >> watcher()"
+
+PYTEST_FUNCTION = """
+def test_run():

Review comment:
       Unfortunately yes. Although in this case it isn't complicated to replace few lines - one file or 100 it doesn't matter for your awk/sed script ;). It's bit of the trade off - we could also leave example dags without any pytest method and generate tests dynamically but by adding this method it's easier to run just one system tests.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] potiuk commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r829582290



##########
File path: tests/system/providers/google/bigquery/example_bigquery_dataset.py
##########
@@ -0,0 +1,94 @@
+#
+# 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.
+
+"""
+Example Airflow DAG for Google BigQuery service testing dataset operations.
+"""
+import os
+from datetime import datetime
+
+from airflow import models
+from airflow.operators.bash import BashOperator
+from airflow.providers.google.cloud.operators.bigquery import (
+    BigQueryCreateEmptyDatasetOperator,
+    BigQueryDeleteDatasetOperator,
+    BigQueryGetDatasetOperator,
+    BigQueryUpdateDatasetOperator,
+)
+from airflow.utils.trigger_rule import TriggerRule

Review comment:
       Yeah - see the comments I added above.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] potiuk commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r829585899



##########
File path: scripts/ci/pre_commit/pre_commit_check_watcher_in_examples.py
##########
@@ -0,0 +1,86 @@
+#!/usr/bin/env python3
+# 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 sys
+from pathlib import Path
+from typing import List
+
+from rich.console import Console
+
+if __name__ not in ("__main__", "__mp_main__"):
+    raise SystemExit(
+        "This file is intended to be executed as an executable program. You cannot use it as a module."
+        f"To run this script, run the ./{__file__} command [FILE] ..."
+    )
+
+
+console = Console(color_system="standard", width=200)
+
+errors: List[str] = []
+
+WATCHER_APPEND_INSTRUCTION = "list(dag.tasks) >> watcher()"
+
+PYTEST_FUNCTION = """
+def test_run():

Review comment:
       That's a nice idea. If pytest still discovers it this way, I lke it a lot better. also you can even connect it with local import to make it even more localized (same reason as in the other comment , where we care about "example_dag" being still an example dag despite also being a self-contained test. Also it nicely solves the problem which I thought about - what happens if we have two dags in an example. They would have to have slightiy different method ( dag1.clear() dag1.run() vs dag2.clear(() dag2.run()). Passing dag as parameter solves it nicely.
   
   I'd be for something like this:
   
   ```
        # Needed to run the example dag as system test <link to the docs>
        from tests.airflow.proivders.util import get_test_run
        test_run = get_test_run(dag)
   ```
   
   




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] bhirsz commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
bhirsz commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r829885457



##########
File path: tests/system/providers/google/bigquery/example_bigquery_operations.py
##########
@@ -0,0 +1,106 @@
+#
+# 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.
+
+"""
+Example Airflow DAG for Google BigQuery service local file upload and external table creation.
+"""
+import os
+from datetime import datetime
+from pathlib import Path
+
+from airflow import models
+from airflow.providers.google.cloud.operators.bigquery import (
+    BigQueryCreateEmptyDatasetOperator,
+    BigQueryCreateExternalTableOperator,
+    BigQueryDeleteDatasetOperator,
+)
+from airflow.providers.google.cloud.operators.gcs import GCSCreateBucketOperator, GCSDeleteBucketOperator
+from airflow.providers.google.cloud.transfers.local_to_gcs import LocalFilesystemToGCSOperator
+from airflow.utils.trigger_rule import TriggerRule
+from tests.system.utils.watcher import watcher
+
+ENV_ID = os.environ.get("SYSTEM_TESTS_ENV_ID")
+DAG_ID = "bigquery_operations"
+
+DATASET_NAME = f"dataset_{DAG_ID}_{ENV_ID}"
+DATA_SAMPLE_GCS_BUCKET_NAME = f"bucket_{DAG_ID}_{ENV_ID}"
+DATA_SAMPLE_GCS_OBJECT_NAME = "bigquery/us-states/us-states.csv"
+CSV_FILE_LOCAL_PATH = str(Path(__file__).parent / "resources" / "us-states.csv")
+
+
+with models.DAG(
+    DAG_ID,
+    schedule_interval="@once",
+    start_date=datetime(2021, 1, 1),
+    catchup=False,
+    tags=["example", "bigquery"],
+) as dag:
+    create_bucket = GCSCreateBucketOperator(task_id="create_bucket", bucket_name=DATA_SAMPLE_GCS_BUCKET_NAME)
+
+    create_dataset = BigQueryCreateEmptyDatasetOperator(task_id="create_dataset", dataset_id=DATASET_NAME)
+
+    upload_file = LocalFilesystemToGCSOperator(
+        task_id="upload_file_to_bucket",
+        src=CSV_FILE_LOCAL_PATH,
+        dst=DATA_SAMPLE_GCS_OBJECT_NAME,
+        bucket=DATA_SAMPLE_GCS_BUCKET_NAME,
+    )
+
+    # [START howto_operator_bigquery_create_external_table]
+    create_external_table = BigQueryCreateExternalTableOperator(
+        task_id="create_external_table",
+        destination_project_dataset_table=f"{DATASET_NAME}.external_table",
+        bucket=DATA_SAMPLE_GCS_BUCKET_NAME,
+        source_objects=[DATA_SAMPLE_GCS_OBJECT_NAME],
+        schema_fields=[
+            {"name": "emp_name", "type": "STRING", "mode": "REQUIRED"},
+            {"name": "salary", "type": "INTEGER", "mode": "NULLABLE"},
+        ],
+    )
+    # [END howto_operator_bigquery_create_external_table]
+
+    delete_dataset = BigQueryDeleteDatasetOperator(
+        task_id="delete_dataset",
+        dataset_id=DATASET_NAME,
+        delete_contents=True,
+        trigger_rule=TriggerRule.ALL_DONE,
+    )
+
+    delete_bucket = GCSDeleteBucketOperator(
+        task_id="delete_bucket", bucket_name=DATA_SAMPLE_GCS_BUCKET_NAME, trigger_rule=TriggerRule.ALL_DONE
+    )
+
+    (
+        # TEST SETUP
+        [create_bucket, create_dataset]
+        # TEST BODY
+        >> upload_file
+        >> create_external_table
+        # TEST TEARDOWN
+        >> delete_dataset
+        >> delete_bucket
+    )
+
+    list(dag.tasks) >> watcher()

Review comment:
       > Writing the same line in hundreds of files just feels wrong
   I fully agree with you and it was actually the reason for the redesign - where we had dozens of lines repeated for every test only to trigger other file with the DAG. And we always welcome any suggestions and discussion. We changed the design several times over the time thanks for the input from the others. 
   I agree with Jarek here - the goal is to have an example dag that can be used by other Airflow users as reference and it would be best to avoid any code specific only to system test. Proposed local import of the watcher looks like a good idea for me for this reason alone

##########
File path: tests/system/providers/google/bigquery/example_bigquery_operations.py
##########
@@ -0,0 +1,106 @@
+#
+# 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.
+
+"""
+Example Airflow DAG for Google BigQuery service local file upload and external table creation.
+"""
+import os
+from datetime import datetime
+from pathlib import Path
+
+from airflow import models
+from airflow.providers.google.cloud.operators.bigquery import (
+    BigQueryCreateEmptyDatasetOperator,
+    BigQueryCreateExternalTableOperator,
+    BigQueryDeleteDatasetOperator,
+)
+from airflow.providers.google.cloud.operators.gcs import GCSCreateBucketOperator, GCSDeleteBucketOperator
+from airflow.providers.google.cloud.transfers.local_to_gcs import LocalFilesystemToGCSOperator
+from airflow.utils.trigger_rule import TriggerRule
+from tests.system.utils.watcher import watcher
+
+ENV_ID = os.environ.get("SYSTEM_TESTS_ENV_ID")
+DAG_ID = "bigquery_operations"
+
+DATASET_NAME = f"dataset_{DAG_ID}_{ENV_ID}"
+DATA_SAMPLE_GCS_BUCKET_NAME = f"bucket_{DAG_ID}_{ENV_ID}"
+DATA_SAMPLE_GCS_OBJECT_NAME = "bigquery/us-states/us-states.csv"
+CSV_FILE_LOCAL_PATH = str(Path(__file__).parent / "resources" / "us-states.csv")
+
+
+with models.DAG(
+    DAG_ID,
+    schedule_interval="@once",
+    start_date=datetime(2021, 1, 1),
+    catchup=False,
+    tags=["example", "bigquery"],
+) as dag:
+    create_bucket = GCSCreateBucketOperator(task_id="create_bucket", bucket_name=DATA_SAMPLE_GCS_BUCKET_NAME)
+
+    create_dataset = BigQueryCreateEmptyDatasetOperator(task_id="create_dataset", dataset_id=DATASET_NAME)
+
+    upload_file = LocalFilesystemToGCSOperator(
+        task_id="upload_file_to_bucket",
+        src=CSV_FILE_LOCAL_PATH,
+        dst=DATA_SAMPLE_GCS_OBJECT_NAME,
+        bucket=DATA_SAMPLE_GCS_BUCKET_NAME,
+    )
+
+    # [START howto_operator_bigquery_create_external_table]
+    create_external_table = BigQueryCreateExternalTableOperator(
+        task_id="create_external_table",
+        destination_project_dataset_table=f"{DATASET_NAME}.external_table",
+        bucket=DATA_SAMPLE_GCS_BUCKET_NAME,
+        source_objects=[DATA_SAMPLE_GCS_OBJECT_NAME],
+        schema_fields=[
+            {"name": "emp_name", "type": "STRING", "mode": "REQUIRED"},
+            {"name": "salary", "type": "INTEGER", "mode": "NULLABLE"},
+        ],
+    )
+    # [END howto_operator_bigquery_create_external_table]
+
+    delete_dataset = BigQueryDeleteDatasetOperator(
+        task_id="delete_dataset",
+        dataset_id=DATASET_NAME,
+        delete_contents=True,
+        trigger_rule=TriggerRule.ALL_DONE,
+    )
+
+    delete_bucket = GCSDeleteBucketOperator(
+        task_id="delete_bucket", bucket_name=DATA_SAMPLE_GCS_BUCKET_NAME, trigger_rule=TriggerRule.ALL_DONE
+    )
+
+    (
+        # TEST SETUP
+        [create_bucket, create_dataset]
+        # TEST BODY
+        >> upload_file
+        >> create_external_table
+        # TEST TEARDOWN
+        >> delete_dataset
+        >> delete_bucket
+    )
+
+    list(dag.tasks) >> watcher()

Review comment:
       > Writing the same line in hundreds of files just feels wrong
   
   I fully agree with you and it was actually the reason for the redesign - where we had dozens of lines repeated for every test only to trigger other file with the DAG. And we always welcome any suggestions and discussion. We changed the design several times over the time thanks for the input from the others. 
   I agree with Jarek here - the goal is to have an example dag that can be used by other Airflow users as reference and it would be best to avoid any code specific only to system test. Proposed local import of the watcher looks like a good idea for me for this reason alone




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] bhirsz commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
bhirsz commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r830821117



##########
File path: scripts/ci/pre_commit/pre_commit_check_watcher_in_examples.py
##########
@@ -0,0 +1,86 @@
+#!/usr/bin/env python3
+# 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 sys
+from pathlib import Path
+from typing import List
+
+from rich.console import Console
+
+if __name__ not in ("__main__", "__mp_main__"):
+    raise SystemExit(
+        "This file is intended to be executed as an executable program. You cannot use it as a module."
+        f"To run this script, run the ./{__file__} command [FILE] ..."
+    )
+
+
+console = Console(color_system="standard", width=200)
+
+errors: List[str] = []
+
+WATCHER_APPEND_INSTRUCTION = "list(dag.tasks) >> watcher()"
+
+PYTEST_FUNCTION = """
+def test_run():

Review comment:
       Updated the code with the test method proposed by @o-nikolas (just added  # noqa: E402 since we're importing not on the top of the file)




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] mnojek commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
mnojek commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r832131776



##########
File path: tests/system/providers/google/bigquery/example_bigquery_dataset.py
##########
@@ -0,0 +1,95 @@
+#
+# 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.
+
+"""
+Example Airflow DAG for Google BigQuery service testing dataset operations.
+"""
+import os
+from datetime import datetime
+
+from airflow import models
+from airflow.operators.bash import BashOperator
+from airflow.providers.google.cloud.operators.bigquery import (
+    BigQueryCreateEmptyDatasetOperator,
+    BigQueryDeleteDatasetOperator,
+    BigQueryGetDatasetOperator,
+    BigQueryUpdateDatasetOperator,
+)
+from airflow.utils.trigger_rule import TriggerRule
+
+ENV_ID = os.environ.get("SYSTEM_TESTS_ENV_ID")
+DAG_ID = "bigquery_dataset"
+
+DATASET_NAME = f"dataset_{DAG_ID}_{ENV_ID}"
+
+
+with models.DAG(
+    DAG_ID,
+    schedule_interval="@once",
+    start_date=datetime(2021, 1, 1),
+    catchup=False,
+    tags=["example", "bigquery"],
+) as dag:
+    # [START howto_operator_bigquery_create_dataset]
+    create_dataset = BigQueryCreateEmptyDatasetOperator(task_id="create_dataset", dataset_id=DATASET_NAME)
+    # [END howto_operator_bigquery_create_dataset]
+
+    # [START howto_operator_bigquery_update_dataset]
+    update_dataset = BigQueryUpdateDatasetOperator(
+        task_id="update_dataset",
+        dataset_id=DATASET_NAME,
+        dataset_resource={"description": "Updated dataset"},
+    )
+    # [END howto_operator_bigquery_update_dataset]
+
+    # [START howto_operator_bigquery_get_dataset]
+    get_dataset = BigQueryGetDatasetOperator(task_id="get-dataset", dataset_id=DATASET_NAME)
+    # [END howto_operator_bigquery_get_dataset]
+
+    get_dataset_result = BashOperator(
+        task_id="get_dataset_result",
+        bash_command="echo \"{{ task_instance.xcom_pull('get-dataset')['id'] }}\"",
+    )
+
+    # [START howto_operator_bigquery_delete_dataset]
+    delete_dataset = BigQueryDeleteDatasetOperator(
+        task_id="delete_dataset", dataset_id=DATASET_NAME, delete_contents=True
+    )
+    # [END howto_operator_bigquery_delete_dataset]
+    delete_dataset.trigger_rule = TriggerRule.ALL_DONE
+
+    (
+        # TEST BODY
+        create_dataset
+        >> update_dataset
+        >> get_dataset
+        >> get_dataset_result
+        # TEST TEARDOWN
+        >> delete_dataset
+    )
+
+    from tests.system.utils.watcher import watcher

Review comment:
       I think this is brilliant 💎 
   We have a `dag` reference there, so encapsulating everything in one place seems like a great idea. This makes code cleaner, and adding a watcher now only requires changing one flag. Seriously, collaborative thinking is amazing - many brains can come up with great ideas 😃 
   I'm all for it.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] bhirsz commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
bhirsz commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r832909804



##########
File path: docs/apache-airflow-providers-google/operators/cloud/bigquery.rst
##########
@@ -42,7 +42,7 @@ Create dataset
 To create an empty dataset in a BigQuery database you can use
 :class:`~airflow.providers.google.cloud.operators.bigquery.BigQueryCreateEmptyDatasetOperator`.
 
-.. exampleinclude:: /../../airflow/providers/google/cloud/example_dags/example_bigquery_operations.py
+.. exampleinclude:: /../../tests/system/providers/google/bigquery/example_bigquery_dataset.py

Review comment:
       I see, we will need to update the path in the linked lint check to math two possible paths (example_dags and tests/system),




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] kosteev commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
kosteev commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r833327491



##########
File path: docs/exts/docs_build/lint_checks.py
##########
@@ -230,13 +230,13 @@ def find_modules(deprecated_only: bool = False) -> Set[str]:
 
 
 def check_exampleinclude_for_example_dags() -> List[DocBuildError]:
-    """Checks all exampleincludes for  example dags."""
-    all_docs_files = glob(f"${DOCS_DIR}/**/*.rst", recursive=True)
+    """Checks all exampleincludes for example dags."""

Review comment:
       ok, thx




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] potiuk merged pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
potiuk merged pull request #22311:
URL: https://github.com/apache/airflow/pull/22311


   


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] uranusjr commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
uranusjr commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r833932479



##########
File path: tests/system/utils/__init__.py
##########
@@ -0,0 +1,25 @@
+# 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.
+from airflow.utils.state import State
+
+
+def get_test_run(dag):
+    def test_run():
+        dag.clear(dag_run_state=State.NONE)
+        dag.run()
+
+    return test_run

Review comment:
       ```suggestion
   def get_test_run(dag):
       def test_run():
           dag.clear(dag_run_state=None)
           dag.run()
   
       return test_run
   ```




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] potiuk closed pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
potiuk closed pull request #22311:
URL: https://github.com/apache/airflow/pull/22311


   


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] bhirsz commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
bhirsz commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r832905948



##########
File path: tests/system/providers/google/bigquery/example_bigquery_operations.py
##########
@@ -0,0 +1,106 @@
+#
+# 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.
+
+"""
+Example Airflow DAG for Google BigQuery service local file upload and external table creation.
+"""
+import os
+from datetime import datetime
+from pathlib import Path
+
+from airflow import models
+from airflow.providers.google.cloud.operators.bigquery import (
+    BigQueryCreateEmptyDatasetOperator,
+    BigQueryCreateExternalTableOperator,
+    BigQueryDeleteDatasetOperator,
+)
+from airflow.providers.google.cloud.operators.gcs import GCSCreateBucketOperator, GCSDeleteBucketOperator
+from airflow.providers.google.cloud.transfers.local_to_gcs import LocalFilesystemToGCSOperator
+from airflow.utils.trigger_rule import TriggerRule
+from tests.system.utils.watcher import watcher
+
+ENV_ID = os.environ.get("SYSTEM_TESTS_ENV_ID")
+DAG_ID = "bigquery_operations"
+
+DATASET_NAME = f"dataset_{DAG_ID}_{ENV_ID}"
+DATA_SAMPLE_GCS_BUCKET_NAME = f"bucket_{DAG_ID}_{ENV_ID}"
+DATA_SAMPLE_GCS_OBJECT_NAME = "bigquery/us-states/us-states.csv"
+CSV_FILE_LOCAL_PATH = str(Path(__file__).parent / "resources" / "us-states.csv")
+
+
+with models.DAG(
+    DAG_ID,
+    schedule_interval="@once",
+    start_date=datetime(2021, 1, 1),
+    catchup=False,
+    tags=["example", "bigquery"],
+) as dag:
+    create_bucket = GCSCreateBucketOperator(task_id="create_bucket", bucket_name=DATA_SAMPLE_GCS_BUCKET_NAME)
+
+    create_dataset = BigQueryCreateEmptyDatasetOperator(task_id="create_dataset", dataset_id=DATASET_NAME)
+
+    upload_file = LocalFilesystemToGCSOperator(
+        task_id="upload_file_to_bucket",
+        src=CSV_FILE_LOCAL_PATH,
+        dst=DATA_SAMPLE_GCS_OBJECT_NAME,
+        bucket=DATA_SAMPLE_GCS_BUCKET_NAME,
+    )
+
+    # [START howto_operator_bigquery_create_external_table]
+    create_external_table = BigQueryCreateExternalTableOperator(
+        task_id="create_external_table",
+        destination_project_dataset_table=f"{DATASET_NAME}.external_table",
+        bucket=DATA_SAMPLE_GCS_BUCKET_NAME,
+        source_objects=[DATA_SAMPLE_GCS_OBJECT_NAME],
+        schema_fields=[
+            {"name": "emp_name", "type": "STRING", "mode": "REQUIRED"},
+            {"name": "salary", "type": "INTEGER", "mode": "NULLABLE"},
+        ],
+    )
+    # [END howto_operator_bigquery_create_external_table]
+
+    delete_dataset = BigQueryDeleteDatasetOperator(
+        task_id="delete_dataset",
+        dataset_id=DATASET_NAME,
+        delete_contents=True,
+        trigger_rule=TriggerRule.ALL_DONE,
+    )
+
+    delete_bucket = GCSDeleteBucketOperator(
+        task_id="delete_bucket", bucket_name=DATA_SAMPLE_GCS_BUCKET_NAME, trigger_rule=TriggerRule.ALL_DONE
+    )
+
+    (
+        # TEST SETUP
+        [create_bucket, create_dataset]
+        # TEST BODY
+        >> upload_file
+        >> create_external_table
+        # TEST TEARDOWN
+        >> delete_dataset
+        >> delete_bucket
+    )
+
+    list(dag.tasks) >> watcher()

Review comment:
       Cluster policy looks nice but it's overkill in our case when after discussion in this PR we managed to store the watcher together in the test method. It's worth to remember though in case we will need any extra steps.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] bhirsz commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
bhirsz commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r829940171



##########
File path: tests/system/conftest.py
##########
@@ -0,0 +1,61 @@
+# 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 os
+import re
+from itertools import chain
+from pathlib import Path
+
+import pytest
+
+OLD_EXECUTOR = os.environ.get("AIRFLOW__CORE__EXECUTOR", default=None)
+REQUIRED_ENV_VARS = ("SYSTEM_TESTS_ENV_ID",)
+
+
+def pytest_configure(config):
+    os.environ["AIRFLOW__CORE__EXECUTOR"] = "DebugExecutor"
+
+
+def pytest_unconfigure(config):
+    if OLD_EXECUTOR is not None:
+        os.environ["AIRFLOW__CORE__EXECUTOR"] = OLD_EXECUTOR

Review comment:
       I think that with the AIRFLOW__CORE__EXECUTOR set to None the execution will fail anyway. But I could introduce extra flag ie:
   ```
   IS_EXECUTOR_SET = "AIRFLOW__CORE__EXECUTOR" in os.environ
   OLD_EXECUTOR = os.environ.get("AIRFLOW__CORE__EXECUTOR", default=None)
   (...)
   def pytest_unconfigure(config):
       if IS_EXECUTOR_SET:
           os.environ["AIRFLOW__CORE__EXECUTOR"] = OLD_EXECUTOR
   ```




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] bhirsz commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
bhirsz commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r829326254



##########
File path: tests/system/providers/google/bigquery/example_bigquery_dataset.py
##########
@@ -0,0 +1,94 @@
+#
+# 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.
+
+"""
+Example Airflow DAG for Google BigQuery service testing dataset operations.
+"""
+import os
+from datetime import datetime
+
+from airflow import models
+from airflow.operators.bash import BashOperator
+from airflow.providers.google.cloud.operators.bigquery import (
+    BigQueryCreateEmptyDatasetOperator,
+    BigQueryDeleteDatasetOperator,
+    BigQueryGetDatasetOperator,
+    BigQueryUpdateDatasetOperator,
+)
+from airflow.utils.trigger_rule import TriggerRule

Review comment:
       It's not - we're using trigger rules only if there is teardown-like task. For example if you're creating resource using one task you want to make sure to run task that removes that resource even if some tasks inbetween fails. There may be system tests that don't need it at all.

##########
File path: scripts/ci/pre_commit/pre_commit_check_watcher_in_examples.py
##########
@@ -0,0 +1,86 @@
+#!/usr/bin/env python3
+# 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 sys
+from pathlib import Path
+from typing import List
+
+from rich.console import Console
+
+if __name__ not in ("__main__", "__mp_main__"):
+    raise SystemExit(
+        "This file is intended to be executed as an executable program. You cannot use it as a module."
+        f"To run this script, run the ./{__file__} command [FILE] ..."
+    )
+
+
+console = Console(color_system="standard", width=200)
+
+errors: List[str] = []
+
+WATCHER_APPEND_INSTRUCTION = "list(dag.tasks) >> watcher()"
+
+PYTEST_FUNCTION = """
+def test_run():

Review comment:
       Unfortunately yes. Although in this case it isn't complicated to replace few lines - one file or 100 it doesn't matter for your awk/sed script ;). It's bit of the trade off - we could also leave example dags without any pytest method and generate tests dynamically but by adding this method it's easier to run just one system tests.

##########
File path: tests/system/providers/google/bigquery/example_bigquery_operations.py
##########
@@ -0,0 +1,106 @@
+#
+# 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.
+
+"""
+Example Airflow DAG for Google BigQuery service local file upload and external table creation.
+"""
+import os
+from datetime import datetime
+from pathlib import Path
+
+from airflow import models
+from airflow.providers.google.cloud.operators.bigquery import (
+    BigQueryCreateEmptyDatasetOperator,
+    BigQueryCreateExternalTableOperator,
+    BigQueryDeleteDatasetOperator,
+)
+from airflow.providers.google.cloud.operators.gcs import GCSCreateBucketOperator, GCSDeleteBucketOperator
+from airflow.providers.google.cloud.transfers.local_to_gcs import LocalFilesystemToGCSOperator
+from airflow.utils.trigger_rule import TriggerRule
+from tests.system.utils.watcher import watcher
+
+ENV_ID = os.environ.get("SYSTEM_TESTS_ENV_ID")
+DAG_ID = "bigquery_operations"
+
+DATASET_NAME = f"dataset_{DAG_ID}_{ENV_ID}"
+DATA_SAMPLE_GCS_BUCKET_NAME = f"bucket_{DAG_ID}_{ENV_ID}"
+DATA_SAMPLE_GCS_OBJECT_NAME = "bigquery/us-states/us-states.csv"
+CSV_FILE_LOCAL_PATH = str(Path(__file__).parent / "resources" / "us-states.csv")
+
+
+with models.DAG(
+    DAG_ID,
+    schedule_interval="@once",
+    start_date=datetime(2021, 1, 1),
+    catchup=False,
+    tags=["example", "bigquery"],
+) as dag:
+    create_bucket = GCSCreateBucketOperator(task_id="create_bucket", bucket_name=DATA_SAMPLE_GCS_BUCKET_NAME)
+
+    create_dataset = BigQueryCreateEmptyDatasetOperator(task_id="create_dataset", dataset_id=DATASET_NAME)
+
+    upload_file = LocalFilesystemToGCSOperator(
+        task_id="upload_file_to_bucket",
+        src=CSV_FILE_LOCAL_PATH,
+        dst=DATA_SAMPLE_GCS_OBJECT_NAME,
+        bucket=DATA_SAMPLE_GCS_BUCKET_NAME,
+    )
+
+    # [START howto_operator_bigquery_create_external_table]
+    create_external_table = BigQueryCreateExternalTableOperator(
+        task_id="create_external_table",
+        destination_project_dataset_table=f"{DATASET_NAME}.external_table",
+        bucket=DATA_SAMPLE_GCS_BUCKET_NAME,
+        source_objects=[DATA_SAMPLE_GCS_OBJECT_NAME],
+        schema_fields=[
+            {"name": "emp_name", "type": "STRING", "mode": "REQUIRED"},
+            {"name": "salary", "type": "INTEGER", "mode": "NULLABLE"},
+        ],
+    )
+    # [END howto_operator_bigquery_create_external_table]
+
+    delete_dataset = BigQueryDeleteDatasetOperator(
+        task_id="delete_dataset",
+        dataset_id=DATASET_NAME,
+        delete_contents=True,
+        trigger_rule=TriggerRule.ALL_DONE,
+    )
+
+    delete_bucket = GCSDeleteBucketOperator(
+        task_id="delete_bucket", bucket_name=DATA_SAMPLE_GCS_BUCKET_NAME, trigger_rule=TriggerRule.ALL_DONE
+    )
+
+    (
+        # TEST SETUP
+        [create_bucket, create_dataset]
+        # TEST BODY
+        >> upload_file
+        >> create_external_table
+        # TEST TEARDOWN
+        >> delete_dataset
+        >> delete_bucket
+    )
+
+    list(dag.tasks) >> watcher()

Review comment:
       > Writing the same line in hundreds of files just feels wrong
   I fully agree with you and it was actually the reason for the redesign - where we had dozens of lines repeated for every test only to trigger other file with the DAG. And we always welcome any suggestions and discussion. We changed the design several times over the time thanks for the input from the others. 
   I agree with Jarek here - the goal is to have an example dag that can be used by other Airflow users as reference and it would be best to avoid any code specific only to system test. Proposed local import of the watcher looks like a good idea for me for this reason alone

##########
File path: tests/system/providers/google/bigquery/example_bigquery_operations.py
##########
@@ -0,0 +1,106 @@
+#
+# 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.
+
+"""
+Example Airflow DAG for Google BigQuery service local file upload and external table creation.
+"""
+import os
+from datetime import datetime
+from pathlib import Path
+
+from airflow import models
+from airflow.providers.google.cloud.operators.bigquery import (
+    BigQueryCreateEmptyDatasetOperator,
+    BigQueryCreateExternalTableOperator,
+    BigQueryDeleteDatasetOperator,
+)
+from airflow.providers.google.cloud.operators.gcs import GCSCreateBucketOperator, GCSDeleteBucketOperator
+from airflow.providers.google.cloud.transfers.local_to_gcs import LocalFilesystemToGCSOperator
+from airflow.utils.trigger_rule import TriggerRule
+from tests.system.utils.watcher import watcher
+
+ENV_ID = os.environ.get("SYSTEM_TESTS_ENV_ID")
+DAG_ID = "bigquery_operations"
+
+DATASET_NAME = f"dataset_{DAG_ID}_{ENV_ID}"
+DATA_SAMPLE_GCS_BUCKET_NAME = f"bucket_{DAG_ID}_{ENV_ID}"
+DATA_SAMPLE_GCS_OBJECT_NAME = "bigquery/us-states/us-states.csv"
+CSV_FILE_LOCAL_PATH = str(Path(__file__).parent / "resources" / "us-states.csv")
+
+
+with models.DAG(
+    DAG_ID,
+    schedule_interval="@once",
+    start_date=datetime(2021, 1, 1),
+    catchup=False,
+    tags=["example", "bigquery"],
+) as dag:
+    create_bucket = GCSCreateBucketOperator(task_id="create_bucket", bucket_name=DATA_SAMPLE_GCS_BUCKET_NAME)
+
+    create_dataset = BigQueryCreateEmptyDatasetOperator(task_id="create_dataset", dataset_id=DATASET_NAME)
+
+    upload_file = LocalFilesystemToGCSOperator(
+        task_id="upload_file_to_bucket",
+        src=CSV_FILE_LOCAL_PATH,
+        dst=DATA_SAMPLE_GCS_OBJECT_NAME,
+        bucket=DATA_SAMPLE_GCS_BUCKET_NAME,
+    )
+
+    # [START howto_operator_bigquery_create_external_table]
+    create_external_table = BigQueryCreateExternalTableOperator(
+        task_id="create_external_table",
+        destination_project_dataset_table=f"{DATASET_NAME}.external_table",
+        bucket=DATA_SAMPLE_GCS_BUCKET_NAME,
+        source_objects=[DATA_SAMPLE_GCS_OBJECT_NAME],
+        schema_fields=[
+            {"name": "emp_name", "type": "STRING", "mode": "REQUIRED"},
+            {"name": "salary", "type": "INTEGER", "mode": "NULLABLE"},
+        ],
+    )
+    # [END howto_operator_bigquery_create_external_table]
+
+    delete_dataset = BigQueryDeleteDatasetOperator(
+        task_id="delete_dataset",
+        dataset_id=DATASET_NAME,
+        delete_contents=True,
+        trigger_rule=TriggerRule.ALL_DONE,
+    )
+
+    delete_bucket = GCSDeleteBucketOperator(
+        task_id="delete_bucket", bucket_name=DATA_SAMPLE_GCS_BUCKET_NAME, trigger_rule=TriggerRule.ALL_DONE
+    )
+
+    (
+        # TEST SETUP
+        [create_bucket, create_dataset]
+        # TEST BODY
+        >> upload_file
+        >> create_external_table
+        # TEST TEARDOWN
+        >> delete_dataset
+        >> delete_bucket
+    )
+
+    list(dag.tasks) >> watcher()

Review comment:
       > Writing the same line in hundreds of files just feels wrong
   
   I fully agree with you and it was actually the reason for the redesign - where we had dozens of lines repeated for every test only to trigger other file with the DAG. And we always welcome any suggestions and discussion. We changed the design several times over the time thanks for the input from the others. 
   I agree with Jarek here - the goal is to have an example dag that can be used by other Airflow users as reference and it would be best to avoid any code specific only to system test. Proposed local import of the watcher looks like a good idea for me for this reason alone

##########
File path: scripts/ci/pre_commit/pre_commit_check_watcher_in_examples.py
##########
@@ -0,0 +1,86 @@
+#!/usr/bin/env python3
+# 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 sys
+from pathlib import Path
+from typing import List
+
+from rich.console import Console
+
+if __name__ not in ("__main__", "__mp_main__"):
+    raise SystemExit(
+        "This file is intended to be executed as an executable program. You cannot use it as a module."
+        f"To run this script, run the ./{__file__} command [FILE] ..."
+    )
+
+
+console = Console(color_system="standard", width=200)
+
+errors: List[str] = []
+
+WATCHER_APPEND_INSTRUCTION = "list(dag.tasks) >> watcher()"
+
+PYTEST_FUNCTION = """
+def test_run():

Review comment:
       I like the idea - it's also making it easier for temporary updates of the test method (ie adding verbose mode, overwriting executor etc). I've tested it and it appears to be working. Pytest recognizes it if I run it using pytest command however it's not detected by IDE as test method (but maybe I can workaround it somehow with pytest ini). I will discuss it with @mnojek and if we both agree we can update it
   

##########
File path: tests/system/conftest.py
##########
@@ -0,0 +1,61 @@
+# 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 os
+import re
+from itertools import chain
+from pathlib import Path
+
+import pytest
+
+OLD_EXECUTOR = os.environ.get("AIRFLOW__CORE__EXECUTOR", default=None)
+REQUIRED_ENV_VARS = ("SYSTEM_TESTS_ENV_ID",)
+
+
+def pytest_configure(config):
+    os.environ["AIRFLOW__CORE__EXECUTOR"] = "DebugExecutor"
+
+
+def pytest_unconfigure(config):
+    if OLD_EXECUTOR is not None:
+        os.environ["AIRFLOW__CORE__EXECUTOR"] = OLD_EXECUTOR

Review comment:
       I think that with the AIRFLOW__CORE__EXECUTOR set to None the execution will fail anyway. But I could introduce extra flag ie:
   ```
   IS_EXECUTOR_SET = "AIRFLOW__CORE__EXECUTOR" in os.environ
   OLD_EXECUTOR = os.environ.get("AIRFLOW__CORE__EXECUTOR", default=None)
   (...)
   def pytest_unconfigure(config):
       if IS_EXECUTOR_SET:
           os.environ["AIRFLOW__CORE__EXECUTOR"] = OLD_EXECUTOR
   ```

##########
File path: docs/apache-airflow-providers-google/operators/cloud/bigquery.rst
##########
@@ -29,7 +29,7 @@ data.
 Prerequisite Tasks
 ^^^^^^^^^^^^^^^^^^
 
-.. include::/operators/_partials/prerequisite_tasks.rst
+.. include:: ../_partials/prerequisite_tasks.rst

Review comment:
       It's not although it's common practice, see sphinx docs: https://docutils.sourceforge.io/docs/ref/rst/directives.html#include . This line is actually not needed for system tests - it's fix because I've noticed that Google provider docs doesn't contain prerequisite_tasks section content (due to invalid include - for at least few versions). It's very minor so I think it's fine to also include this change in this PR.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] o-nikolas commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
o-nikolas commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r829396861



##########
File path: scripts/ci/pre_commit/pre_commit_check_watcher_in_examples.py
##########
@@ -0,0 +1,86 @@
+#!/usr/bin/env python3
+# 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 sys
+from pathlib import Path
+from typing import List
+
+from rich.console import Console
+
+if __name__ not in ("__main__", "__mp_main__"):
+    raise SystemExit(
+        "This file is intended to be executed as an executable program. You cannot use it as a module."
+        f"To run this script, run the ./{__file__} command [FILE] ..."
+    )
+
+
+console = Console(color_system="standard", width=200)
+
+errors: List[str] = []
+
+WATCHER_APPEND_INSTRUCTION = "list(dag.tasks) >> watcher()"
+
+PYTEST_FUNCTION = """
+def test_run():

Review comment:
       > one file or 100 it doesn't matter for your awk/sed script
   
   Hmm, this is not the kind of precedent I want to be setting :smile: It's still quite finicky and error prone.
   
   You can clean this up quite a bit using python itself. The `test_run` method can be defined in one location in a separate module and then just imported into each example dag file.
   
   I've mocked up an example, see the attached image below:
   
   ![Screenshot from 2022-03-17 11-54-21](https://user-images.githubusercontent.com/65743084/158875769-241827a0-f4e1-4bd1-8fdd-969db8ea55ab.png)




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] mnojek commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
mnojek commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r830482091



##########
File path: tests/system/providers/google/bigquery/example_bigquery_operations.py
##########
@@ -0,0 +1,106 @@
+#
+# 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.
+
+"""
+Example Airflow DAG for Google BigQuery service local file upload and external table creation.
+"""
+import os
+from datetime import datetime
+from pathlib import Path
+
+from airflow import models
+from airflow.providers.google.cloud.operators.bigquery import (
+    BigQueryCreateEmptyDatasetOperator,
+    BigQueryCreateExternalTableOperator,
+    BigQueryDeleteDatasetOperator,
+)
+from airflow.providers.google.cloud.operators.gcs import GCSCreateBucketOperator, GCSDeleteBucketOperator
+from airflow.providers.google.cloud.transfers.local_to_gcs import LocalFilesystemToGCSOperator
+from airflow.utils.trigger_rule import TriggerRule
+from tests.system.utils.watcher import watcher
+
+ENV_ID = os.environ.get("SYSTEM_TESTS_ENV_ID")
+DAG_ID = "bigquery_operations"
+
+DATASET_NAME = f"dataset_{DAG_ID}_{ENV_ID}"
+DATA_SAMPLE_GCS_BUCKET_NAME = f"bucket_{DAG_ID}_{ENV_ID}"
+DATA_SAMPLE_GCS_OBJECT_NAME = "bigquery/us-states/us-states.csv"
+CSV_FILE_LOCAL_PATH = str(Path(__file__).parent / "resources" / "us-states.csv")
+
+
+with models.DAG(
+    DAG_ID,
+    schedule_interval="@once",
+    start_date=datetime(2021, 1, 1),
+    catchup=False,
+    tags=["example", "bigquery"],
+) as dag:
+    create_bucket = GCSCreateBucketOperator(task_id="create_bucket", bucket_name=DATA_SAMPLE_GCS_BUCKET_NAME)
+
+    create_dataset = BigQueryCreateEmptyDatasetOperator(task_id="create_dataset", dataset_id=DATASET_NAME)
+
+    upload_file = LocalFilesystemToGCSOperator(
+        task_id="upload_file_to_bucket",
+        src=CSV_FILE_LOCAL_PATH,
+        dst=DATA_SAMPLE_GCS_OBJECT_NAME,
+        bucket=DATA_SAMPLE_GCS_BUCKET_NAME,
+    )
+
+    # [START howto_operator_bigquery_create_external_table]
+    create_external_table = BigQueryCreateExternalTableOperator(
+        task_id="create_external_table",
+        destination_project_dataset_table=f"{DATASET_NAME}.external_table",
+        bucket=DATA_SAMPLE_GCS_BUCKET_NAME,
+        source_objects=[DATA_SAMPLE_GCS_OBJECT_NAME],
+        schema_fields=[
+            {"name": "emp_name", "type": "STRING", "mode": "REQUIRED"},
+            {"name": "salary", "type": "INTEGER", "mode": "NULLABLE"},
+        ],
+    )
+    # [END howto_operator_bigquery_create_external_table]
+
+    delete_dataset = BigQueryDeleteDatasetOperator(
+        task_id="delete_dataset",
+        dataset_id=DATASET_NAME,
+        delete_contents=True,
+        trigger_rule=TriggerRule.ALL_DONE,
+    )
+
+    delete_bucket = GCSDeleteBucketOperator(
+        task_id="delete_bucket", bucket_name=DATA_SAMPLE_GCS_BUCKET_NAME, trigger_rule=TriggerRule.ALL_DONE
+    )
+
+    (
+        # TEST SETUP
+        [create_bucket, create_dataset]
+        # TEST BODY
+        >> upload_file
+        >> create_external_table
+        # TEST TEARDOWN
+        >> delete_dataset
+        >> delete_bucket
+    )
+
+    list(dag.tasks) >> watcher()

Review comment:
       I support the idea about moving the import for the watcher to local scope. It will be more obvious why it is needed, and we can also add the comment that @potiuk suggested (or similar).
   And yeah, copy-pasting the same piece of code in many places is not the best, but at least here this "piece" is small and it's better to have it explicitly than doing some magic stuff that will only shadow the real implementation.
   I will do the changes next week.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] o-nikolas commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
o-nikolas commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r830235329



##########
File path: scripts/ci/pre_commit/pre_commit_check_watcher_in_examples.py
##########
@@ -0,0 +1,86 @@
+#!/usr/bin/env python3
+# 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 sys
+from pathlib import Path
+from typing import List
+
+from rich.console import Console
+
+if __name__ not in ("__main__", "__mp_main__"):
+    raise SystemExit(
+        "This file is intended to be executed as an executable program. You cannot use it as a module."
+        f"To run this script, run the ./{__file__} command [FILE] ..."
+    )
+
+
+console = Console(color_system="standard", width=200)
+
+errors: List[str] = []
+
+WATCHER_APPEND_INSTRUCTION = "list(dag.tasks) >> watcher()"
+
+PYTEST_FUNCTION = """
+def test_run():

Review comment:
       > Also it nicely solves the problem which I thought about - what happens if we have two dags in an example.
   
   Yupp, good catch! I meant to add this to my original post but totally forgot. It makes it much more configurable this way.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] potiuk commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r832092133



##########
File path: tests/always/test_example_dags.py
##########
@@ -29,33 +30,38 @@
 NO_DB_QUERY_EXCEPTION = ["/airflow/example_dags/example_subdag_operator.py"]
 
 
-class TestExampleDags(unittest.TestCase):
-    def test_should_be_importable(self):
-        example_dags = list(glob(f"{ROOT_FOLDER}/airflow/**/example_dags/example_*.py", recursive=True))
-        assert 0 != len(example_dags)
-        for filepath in example_dags:
-            relative_filepath = os.path.relpath(filepath, ROOT_FOLDER)
-            with self.subTest(f"File {relative_filepath} should contain dags"):
-                dagbag = DagBag(
-                    dag_folder=filepath,
-                    include_examples=False,
-                )
-                assert 0 == len(dagbag.import_errors), f"import_errors={str(dagbag.import_errors)}"
-                assert len(dagbag.dag_ids) >= 1
-
-    def test_should_not_do_database_queries(self):
-        example_dags = glob(f"{ROOT_FOLDER}/airflow/**/example_dags/example_*.py", recursive=True)
-        example_dags = [
-            dag_file
-            for dag_file in example_dags
-            if any(not dag_file.endswith(e) for e in NO_DB_QUERY_EXCEPTION)
-        ]
-        assert 0 != len(example_dags)
-        for filepath in example_dags:
-            relative_filepath = os.path.relpath(filepath, ROOT_FOLDER)
-            with self.subTest(f"File {relative_filepath} shouldn't do database queries"):
-                with assert_queries_count(0):
-                    DagBag(
-                        dag_folder=filepath,
-                        include_examples=False,
-                    )
+def example_dags():
+    example_dirs = ["airflow/**/example_dags/example_*.py", "tests/system/providers/**/example_*.py"]
+    for example_dir in example_dirs:
+        yield from glob(f"{ROOT_FOLDER}/{example_dir}", recursive=True)
+
+
+def example_dags_except_db_exception():
+    return [
+        dag_file
+        for dag_file in example_dags()
+        if any(not dag_file.endswith(e) for e in NO_DB_QUERY_EXCEPTION)
+    ]
+
+
+def relative_path(path):
+    return os.path.relpath(path, ROOT_FOLDER)
+
+
+@pytest.mark.parametrize("example", example_dags(), ids=relative_path)
+def test_should_be_importable(example):
+    dagbag = DagBag(
+        dag_folder=example,
+        include_examples=False,
+    )
+    assert len(dagbag.import_errors) == 0, f"import_errors={str(dagbag.import_errors)}"
+    assert len(dagbag.dag_ids) >= 1
+
+
+@pytest.mark.parametrize("example", example_dags_except_db_exception(), ids=relative_path)
+def test_should_not_do_database_queries(example):

Review comment:
       Nice one! 




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] potiuk commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r832093161



##########
File path: tests/system/conftest.py
##########
@@ -0,0 +1,58 @@
+# 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 os
+import re
+from itertools import chain
+from pathlib import Path
+from unittest import mock
+
+import pytest
+
+REQUIRED_ENV_VARS = ("SYSTEM_TESTS_ENV_ID",)
+
+
+@pytest.fixture(scope="package", autouse=True)
+def use_debug_executor():

Review comment:
       +1




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] potiuk commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r832094453



##########
File path: tests/system/conftest.py
##########
@@ -0,0 +1,58 @@
+# 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 os
+import re
+from itertools import chain
+from pathlib import Path
+from unittest import mock
+
+import pytest
+
+REQUIRED_ENV_VARS = ("SYSTEM_TESTS_ENV_ID",)
+
+
+@pytest.fixture(scope="package", autouse=True)
+def use_debug_executor():
+    with mock.patch.dict("os.environ", AIRFLOW__CORE__EXECUTOR="DebugExecutor"):
+        yield
+
+
+@pytest.fixture()
+def provider_env_vars():
+    """Override this fixture in provider's conftest.py"""
+    return ()
+
+
+@pytest.fixture(autouse=True)
+def skip_if_env_var_not_set(provider_env_vars):
+    for env in chain(REQUIRED_ENV_VARS, provider_env_vars):
+        if env not in os.environ:
+            pytest.skip(f"Missing required environment variable {env}")
+            return
+
+
+def pytest_collection_modifyitems(config, items):
+    """Add @pytest.mark.system(provider_name) for every system test."""

Review comment:
       Cool. I love pytest with it's versatility. 




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] mik-laj commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r832156404



##########
File path: tests/system/providers/google/bigquery/example_bigquery_operations.py
##########
@@ -0,0 +1,106 @@
+#
+# 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.
+
+"""
+Example Airflow DAG for Google BigQuery service local file upload and external table creation.
+"""
+import os
+from datetime import datetime
+from pathlib import Path
+
+from airflow import models
+from airflow.providers.google.cloud.operators.bigquery import (
+    BigQueryCreateEmptyDatasetOperator,
+    BigQueryCreateExternalTableOperator,
+    BigQueryDeleteDatasetOperator,
+)
+from airflow.providers.google.cloud.operators.gcs import GCSCreateBucketOperator, GCSDeleteBucketOperator
+from airflow.providers.google.cloud.transfers.local_to_gcs import LocalFilesystemToGCSOperator
+from airflow.utils.trigger_rule import TriggerRule
+from tests.system.utils.watcher import watcher
+
+ENV_ID = os.environ.get("SYSTEM_TESTS_ENV_ID")
+DAG_ID = "bigquery_operations"
+
+DATASET_NAME = f"dataset_{DAG_ID}_{ENV_ID}"
+DATA_SAMPLE_GCS_BUCKET_NAME = f"bucket_{DAG_ID}_{ENV_ID}"
+DATA_SAMPLE_GCS_OBJECT_NAME = "bigquery/us-states/us-states.csv"
+CSV_FILE_LOCAL_PATH = str(Path(__file__).parent / "resources" / "us-states.csv")
+
+
+with models.DAG(
+    DAG_ID,
+    schedule_interval="@once",
+    start_date=datetime(2021, 1, 1),
+    catchup=False,
+    tags=["example", "bigquery"],
+) as dag:
+    create_bucket = GCSCreateBucketOperator(task_id="create_bucket", bucket_name=DATA_SAMPLE_GCS_BUCKET_NAME)
+
+    create_dataset = BigQueryCreateEmptyDatasetOperator(task_id="create_dataset", dataset_id=DATASET_NAME)
+
+    upload_file = LocalFilesystemToGCSOperator(
+        task_id="upload_file_to_bucket",
+        src=CSV_FILE_LOCAL_PATH,
+        dst=DATA_SAMPLE_GCS_OBJECT_NAME,
+        bucket=DATA_SAMPLE_GCS_BUCKET_NAME,
+    )
+
+    # [START howto_operator_bigquery_create_external_table]
+    create_external_table = BigQueryCreateExternalTableOperator(
+        task_id="create_external_table",
+        destination_project_dataset_table=f"{DATASET_NAME}.external_table",
+        bucket=DATA_SAMPLE_GCS_BUCKET_NAME,
+        source_objects=[DATA_SAMPLE_GCS_OBJECT_NAME],
+        schema_fields=[
+            {"name": "emp_name", "type": "STRING", "mode": "REQUIRED"},
+            {"name": "salary", "type": "INTEGER", "mode": "NULLABLE"},
+        ],
+    )
+    # [END howto_operator_bigquery_create_external_table]
+
+    delete_dataset = BigQueryDeleteDatasetOperator(
+        task_id="delete_dataset",
+        dataset_id=DATASET_NAME,
+        delete_contents=True,
+        trigger_rule=TriggerRule.ALL_DONE,
+    )
+
+    delete_bucket = GCSDeleteBucketOperator(
+        task_id="delete_bucket", bucket_name=DATA_SAMPLE_GCS_BUCKET_NAME, trigger_rule=TriggerRule.ALL_DONE
+    )
+
+    (
+        # TEST SETUP
+        [create_bucket, create_dataset]
+        # TEST BODY
+        >> upload_file
+        >> create_external_table
+        # TEST TEARDOWN
+        >> delete_dataset
+        >> delete_bucket
+    )
+
+    list(dag.tasks) >> watcher()

Review comment:
       We can also configure a cluster policy that will perform the necessary steps only in tests.
   Here are docs about cluster policy: 
   https://airflow.apache.org/docs/apache-airflow/stable/concepts/cluster-policies.html?highlight=cluster%20policy#dag-policies




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] mnojek commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
mnojek commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r833251414



##########
File path: docs/exts/docs_build/lint_checks.py
##########
@@ -230,13 +230,13 @@ def find_modules(deprecated_only: bool = False) -> Set[str]:
 
 
 def check_exampleinclude_for_example_dags() -> List[DocBuildError]:
-    """Checks all exampleincludes for  example dags."""
-    all_docs_files = glob(f"${DOCS_DIR}/**/*.rst", recursive=True)
+    """Checks all exampleincludes for example dags."""

Review comment:
       I guess not, it is how it looks in the docs:
   `.. exampleinclude::`




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] potiuk commented on pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
potiuk commented on pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#issuecomment-1078337961


   Echch
   


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] mnojek commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
mnojek commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r833226218



##########
File path: tests/system/providers/google/bigquery/example_bigquery_dataset.py
##########
@@ -0,0 +1,95 @@
+#
+# 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.
+
+"""
+Example Airflow DAG for Google BigQuery service testing dataset operations.
+"""
+import os
+from datetime import datetime
+
+from airflow import models
+from airflow.operators.bash import BashOperator
+from airflow.providers.google.cloud.operators.bigquery import (
+    BigQueryCreateEmptyDatasetOperator,
+    BigQueryDeleteDatasetOperator,
+    BigQueryGetDatasetOperator,
+    BigQueryUpdateDatasetOperator,
+)
+from airflow.utils.trigger_rule import TriggerRule
+
+ENV_ID = os.environ.get("SYSTEM_TESTS_ENV_ID")
+DAG_ID = "bigquery_dataset"
+
+DATASET_NAME = f"dataset_{DAG_ID}_{ENV_ID}"
+
+
+with models.DAG(
+    DAG_ID,
+    schedule_interval="@once",
+    start_date=datetime(2021, 1, 1),
+    catchup=False,
+    tags=["example", "bigquery"],
+) as dag:
+    # [START howto_operator_bigquery_create_dataset]
+    create_dataset = BigQueryCreateEmptyDatasetOperator(task_id="create_dataset", dataset_id=DATASET_NAME)
+    # [END howto_operator_bigquery_create_dataset]
+
+    # [START howto_operator_bigquery_update_dataset]
+    update_dataset = BigQueryUpdateDatasetOperator(
+        task_id="update_dataset",
+        dataset_id=DATASET_NAME,
+        dataset_resource={"description": "Updated dataset"},
+    )
+    # [END howto_operator_bigquery_update_dataset]
+
+    # [START howto_operator_bigquery_get_dataset]
+    get_dataset = BigQueryGetDatasetOperator(task_id="get-dataset", dataset_id=DATASET_NAME)
+    # [END howto_operator_bigquery_get_dataset]
+
+    get_dataset_result = BashOperator(
+        task_id="get_dataset_result",
+        bash_command="echo \"{{ task_instance.xcom_pull('get-dataset')['id'] }}\"",
+    )
+
+    # [START howto_operator_bigquery_delete_dataset]
+    delete_dataset = BigQueryDeleteDatasetOperator(
+        task_id="delete_dataset", dataset_id=DATASET_NAME, delete_contents=True
+    )
+    # [END howto_operator_bigquery_delete_dataset]
+    delete_dataset.trigger_rule = TriggerRule.ALL_DONE
+
+    (
+        # TEST BODY
+        create_dataset
+        >> update_dataset
+        >> get_dataset
+        >> get_dataset_result
+        # TEST TEARDOWN
+        >> delete_dataset
+    )
+
+    from tests.system.utils.watcher import watcher

Review comment:
       Ok, it was too soon to get excited before trying it out. After some tests I must say that it doesn't work as expected. 
   When running tests using pytest, it reads the `get_test_run` function and it works fine, but when running them through the Airflow UI - it doesn't.
   AFAIK The problem is that this function is outside of the DAG and Airflow UI doesn't read it, so the watcher is never added to the DAG. While when it is run with pytest, the watcher is there.
   So I guess we are at the same spot where we were before this suggestion popped out.
   I don't see any better way to simplify it, so I just want a confirmation from others, that we stick to what we have now, unless there is another idea. @potiuk @bhirsz @kosteev 




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] mnojek commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
mnojek commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r831116012



##########
File path: scripts/ci/pre_commit/pre_commit_check_watcher_in_examples.py
##########
@@ -0,0 +1,86 @@
+#!/usr/bin/env python3
+# 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 sys
+from pathlib import Path
+from typing import List
+
+from rich.console import Console
+
+if __name__ not in ("__main__", "__mp_main__"):
+    raise SystemExit(
+        "This file is intended to be executed as an executable program. You cannot use it as a module."
+        f"To run this script, run the ./{__file__} command [FILE] ..."
+    )
+
+
+console = Console(color_system="standard", width=200)
+
+errors: List[str] = []
+
+WATCHER_APPEND_INSTRUCTION = "list(dag.tasks) >> watcher()"
+
+PYTEST_FUNCTION = """
+def test_run():
+    from airflow.utils.state import State
+
+    dag.clear(dag_run_state=State.NONE)
+    dag.run()
+"""
+
+
+def _check_file(file: Path):
+    content = file.read_text()
+    if "from tests.system.utils.watcher import watcher" in content:
+        index = content.find(WATCHER_APPEND_INSTRUCTION)
+        if index == -1:
+            errors.append(
+                f"[red]The example {file} imports tests.system.utils.watcher "
+                f"but does not use it properly![/]\n\n"
+                "[yellow]Make sure you have:[/]\n\n"
+                f"        {WATCHER_APPEND_INSTRUCTION}\n\n"
+                "[yellow]as last instruction in your example DAG.[/]\n"
+            )
+        else:
+            operator_leftshift_index = content.find("<<", index + len(WATCHER_APPEND_INSTRUCTION))
+            operator_rightshift_index = content.find(">>", index + len(WATCHER_APPEND_INSTRUCTION))
+            if operator_leftshift_index != -1 or operator_rightshift_index != -1:
+                errors.append(
+                    f"[red]In the  example {file} "
+                    f"watcher is not last instruction in your DAG (there are << "
+                    f"or >> operators after it)![/]\n\n"
+                    "[yellow]Make sure you have:[/]\n"
+                    f"        {WATCHER_APPEND_INSTRUCTION}\n\n"
+                    "[yellow]as last instruction in your example DAG.[/]\n"
+                )
+        if PYTEST_FUNCTION not in content:
+            errors.append(
+                f"[yellow]The  example {file} missed the pytest function at the end.[/]\n\n"
+                "All example tests should have this function added:\n\n" + PYTEST_FUNCTION + "\n\n"
+                "[yellow]Automatically adding it now!\n"
+            )
+            file.write_text(content + "\n" + PYTEST_FUNCTION)

Review comment:
       Changed the name to simple `pre_commit_check_system_tests.py` and updated relevant places where it was mentioned.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] kosteev commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
kosteev commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r832165512



##########
File path: tests/system/providers/google/bigquery/example_bigquery_dataset.py
##########
@@ -0,0 +1,95 @@
+#
+# 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.
+
+"""
+Example Airflow DAG for Google BigQuery service testing dataset operations.
+"""
+import os
+from datetime import datetime
+
+from airflow import models
+from airflow.operators.bash import BashOperator
+from airflow.providers.google.cloud.operators.bigquery import (
+    BigQueryCreateEmptyDatasetOperator,
+    BigQueryDeleteDatasetOperator,
+    BigQueryGetDatasetOperator,
+    BigQueryUpdateDatasetOperator,
+)
+from airflow.utils.trigger_rule import TriggerRule
+
+ENV_ID = os.environ.get("SYSTEM_TESTS_ENV_ID")
+DAG_ID = "bigquery_dataset"
+
+DATASET_NAME = f"dataset_{DAG_ID}_{ENV_ID}"
+
+
+with models.DAG(
+    DAG_ID,
+    schedule_interval="@once",
+    start_date=datetime(2021, 1, 1),
+    catchup=False,
+    tags=["example", "bigquery"],
+) as dag:
+    # [START howto_operator_bigquery_create_dataset]
+    create_dataset = BigQueryCreateEmptyDatasetOperator(task_id="create_dataset", dataset_id=DATASET_NAME)
+    # [END howto_operator_bigquery_create_dataset]
+
+    # [START howto_operator_bigquery_update_dataset]
+    update_dataset = BigQueryUpdateDatasetOperator(
+        task_id="update_dataset",
+        dataset_id=DATASET_NAME,
+        dataset_resource={"description": "Updated dataset"},
+    )
+    # [END howto_operator_bigquery_update_dataset]
+
+    # [START howto_operator_bigquery_get_dataset]
+    get_dataset = BigQueryGetDatasetOperator(task_id="get-dataset", dataset_id=DATASET_NAME)
+    # [END howto_operator_bigquery_get_dataset]
+
+    get_dataset_result = BashOperator(
+        task_id="get_dataset_result",
+        bash_command="echo \"{{ task_instance.xcom_pull('get-dataset')['id'] }}\"",
+    )
+
+    # [START howto_operator_bigquery_delete_dataset]
+    delete_dataset = BigQueryDeleteDatasetOperator(
+        task_id="delete_dataset", dataset_id=DATASET_NAME, delete_contents=True
+    )
+    # [END howto_operator_bigquery_delete_dataset]
+    delete_dataset.trigger_rule = TriggerRule.ALL_DONE
+
+    (
+        # TEST BODY
+        create_dataset
+        >> update_dataset
+        >> get_dataset
+        >> get_dataset_result
+        # TEST TEARDOWN
+        >> delete_dataset
+    )
+
+    from tests.system.utils.watcher import watcher

Review comment:
       +1




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] potiuk commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r833970996



##########
File path: tests/system/utils/__init__.py
##########
@@ -0,0 +1,25 @@
+# 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.
+from airflow.utils.state import State
+
+
+def get_test_run(dag):
+    def test_run():
+        dag.clear(dag_run_state=State.NONE)
+        dag.run()
+
+    return test_run

Review comment:
       Good simplification. It is needed, because we want to be able to change it's behaviour globally (for example during manual testing). You might want to - for example change log level or record additional information or start debugger. or start monitoring memory at the "run time" or do other things. This is purely to avoid having to do it manually in all tests that use it.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] github-actions[bot] commented on pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#issuecomment-1076816591


   The PR most likely needs to run full matrix of tests because it modifies parts of the core of Airflow. However, committers might decide to merge it quickly and take the risk. If they don't merge it quickly - please rebase it to the latest main at your convenience, or amend the last commit of the PR, and push it with --force-with-lease.


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] ferruzzi commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
ferruzzi commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r834798110



##########
File path: tests/system/providers/google/bigquery/example_bigquery_dataset.py
##########
@@ -0,0 +1,95 @@
+#
+# 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.
+
+"""
+Example Airflow DAG for Google BigQuery service testing dataset operations.
+"""
+import os
+from datetime import datetime
+
+from airflow import models
+from airflow.operators.bash import BashOperator
+from airflow.providers.google.cloud.operators.bigquery import (
+    BigQueryCreateEmptyDatasetOperator,
+    BigQueryDeleteDatasetOperator,
+    BigQueryGetDatasetOperator,
+    BigQueryUpdateDatasetOperator,
+)
+from airflow.utils.trigger_rule import TriggerRule
+
+ENV_ID = os.environ.get("SYSTEM_TESTS_ENV_ID")
+DAG_ID = "bigquery_dataset"
+
+DATASET_NAME = f"dataset_{DAG_ID}_{ENV_ID}"
+
+
+with models.DAG(
+    DAG_ID,
+    schedule_interval="@once",
+    start_date=datetime(2021, 1, 1),
+    catchup=False,
+    tags=["example", "bigquery"],
+) as dag:
+    # [START howto_operator_bigquery_create_dataset]
+    create_dataset = BigQueryCreateEmptyDatasetOperator(task_id="create_dataset", dataset_id=DATASET_NAME)
+    # [END howto_operator_bigquery_create_dataset]
+
+    # [START howto_operator_bigquery_update_dataset]
+    update_dataset = BigQueryUpdateDatasetOperator(
+        task_id="update_dataset",
+        dataset_id=DATASET_NAME,
+        dataset_resource={"description": "Updated dataset"},
+    )
+    # [END howto_operator_bigquery_update_dataset]
+
+    # [START howto_operator_bigquery_get_dataset]
+    get_dataset = BigQueryGetDatasetOperator(task_id="get-dataset", dataset_id=DATASET_NAME)
+    # [END howto_operator_bigquery_get_dataset]
+
+    get_dataset_result = BashOperator(
+        task_id="get_dataset_result",
+        bash_command="echo \"{{ task_instance.xcom_pull('get-dataset')['id'] }}\"",
+    )
+
+    # [START howto_operator_bigquery_delete_dataset]
+    delete_dataset = BigQueryDeleteDatasetOperator(
+        task_id="delete_dataset", dataset_id=DATASET_NAME, delete_contents=True
+    )
+    # [END howto_operator_bigquery_delete_dataset]
+    delete_dataset.trigger_rule = TriggerRule.ALL_DONE
+
+    (
+        # TEST BODY
+        create_dataset
+        >> update_dataset
+        >> get_dataset
+        >> get_dataset_result
+        # TEST TEARDOWN
+        >> delete_dataset
+    )
+
+    from tests.system.utils.watcher import watcher

Review comment:
       This thread is a rollercoaster of "I DID GOOD! oh wait..no...  "  :stuck_out_tongue: 
   
   > but when running them through the Airflow UI - it doesn't.
   
   Am I understanding that to mean you are manually running tests from the Airflow UI?   I actually never realized that as a thing.   Or did you add that to a DAG which should have worked and it killed the DAG?




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] potiuk closed pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
potiuk closed pull request #22311:
URL: https://github.com/apache/airflow/pull/22311


   


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] bhirsz commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
bhirsz commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r830795321



##########
File path: tests/system/conftest.py
##########
@@ -0,0 +1,61 @@
+# 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 os
+import re
+from itertools import chain
+from pathlib import Path
+
+import pytest
+
+OLD_EXECUTOR = os.environ.get("AIRFLOW__CORE__EXECUTOR", default=None)
+REQUIRED_ENV_VARS = ("SYSTEM_TESTS_ENV_ID",)
+
+
+def pytest_configure(config):
+    os.environ["AIRFLOW__CORE__EXECUTOR"] = "DebugExecutor"
+
+
+def pytest_unconfigure(config):
+    if OLD_EXECUTOR is not None:
+        os.environ["AIRFLOW__CORE__EXECUTOR"] = OLD_EXECUTOR

Review comment:
       It should - there was reason to use pytest_configure but it was lost in some of the recent designs so it's totally fine (and preffered now) to use auto-fixture. I will do it.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] mnojek commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
mnojek commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r841184322



##########
File path: docs/apache-airflow/best-practices.rst
##########
@@ -247,18 +247,26 @@ each parameter by following the links):
 Example of watcher pattern with trigger rules
 ---------------------------------------------
 
-The watcher pattern is how we call a DAG with a task that is "watching" the states of the other tasks. It's primary purpose is to fail a DAG Run when any other task fail.
+The watcher pattern is how we call a DAG with a task that is "watching" the states of the other tasks.
+It's primary purpose is to fail a DAG Run when any other task fail.
 The need came from the Airflow system tests that are DAGs with different tasks (similarly like a test containing steps).
 
-Normally, when any task fails, all other tasks are not executed and the whole DAG Run gets failed status too. But when we use trigger rules, we can disrupt the normal flow of running tasks and the whole DAG may represent different status that we expect.
-For example, we can have a teardown task (with trigger rule set to ``"all_done"``) that will be executed regardless of the state of the other tasks (e.g. to clean up the resources). In such situation, the DAG would always run this task and the DAG Run will get the status of this particular task, so we can potentially lose the information about failing tasks.
-If we want to ensure that the DAG with teardown task would fail if any task fails, we need to use the watcher pattern.
-The watcher task is a task that will always fail if triggered, but it needs to be triggered only if any other task fails. It needs to have a trigger rule set to ``"one_failed"`` and it needs also to be a downstream task for all other tasks in the DAG.
-Thanks to this, if every other task will pass, the watcher will be skipped, but when something fails, the watcher task will be executed and fail making the DAG Run fail too.
+Normally, when any task fails, all other tasks are not executed and the whole DAG Run gets failed status too. But
+when we use trigger rules, we can disrupt the normal flow of running tasks and the whole DAG may represent different
+status that we expect. For example, we can have a teardown task (with trigger rule set to ``TriggerRule.ALL_DONE``)
+that will be executed regardless of the state of the other tasks (e.g. to clean up the resources). In such
+situation, the DAG would always run this task and the DAG Run will get the status of this particular task, so we can
+potentially lose the information about failing tasks. If we want to ensure that the DAG with teardown task would fail
+if any task fails, we need to  use the watcher pattern. The watcher task is a task that will always fail if
+triggered, but it needs to be triggered only if any other task fails. It needs to have a trigger rule set to
+``TriggerRule.ONE_FAILED`` and it needs also to be a  downstream task for all other tasks in the DAG. Thanks to
+this, if every other task will pass, the watcher will be skipped, but when something fails, the watcher task will be

Review comment:
       No, the teardown task will be executed because by design it has a trigger rule set to "all done" which means it is executed always when all parent tasks are executed. The watcher is a child for every task so it doesn't count into the tasks that need to be executed to trigger the teardown. It is skipped only if everything passed.
   And the watcher is triggered only if any of its parent tasks failed, and because all tasks are parent for the watcher, if any of them fails, the watcher is triggered.
   Hope it answers your question. If you still have concerns, please comment and I will try to help.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] mnojek commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
mnojek commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r829474886



##########
File path: scripts/ci/pre_commit/pre_commit_check_watcher_in_examples.py
##########
@@ -0,0 +1,86 @@
+#!/usr/bin/env python3
+# 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 sys
+from pathlib import Path
+from typing import List
+
+from rich.console import Console
+
+if __name__ not in ("__main__", "__mp_main__"):
+    raise SystemExit(
+        "This file is intended to be executed as an executable program. You cannot use it as a module."
+        f"To run this script, run the ./{__file__} command [FILE] ..."
+    )
+
+
+console = Console(color_system="standard", width=200)
+
+errors: List[str] = []
+
+WATCHER_APPEND_INSTRUCTION = "list(dag.tasks) >> watcher()"
+
+PYTEST_FUNCTION = """
+def test_run():
+    from airflow.utils.state import State
+
+    dag.clear(dag_run_state=State.NONE)
+    dag.run()
+"""
+
+
+def _check_file(file: Path):
+    content = file.read_text()
+    if "from tests.system.utils.watcher import watcher" in content:

Review comment:
       Yes, that's correct. We can still improve this check.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] potiuk commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r829580834



##########
File path: tests/system/providers/google/bigquery/example_bigquery_operations.py
##########
@@ -0,0 +1,106 @@
+#
+# 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.
+
+"""
+Example Airflow DAG for Google BigQuery service local file upload and external table creation.
+"""
+import os
+from datetime import datetime
+from pathlib import Path
+
+from airflow import models
+from airflow.providers.google.cloud.operators.bigquery import (
+    BigQueryCreateEmptyDatasetOperator,
+    BigQueryCreateExternalTableOperator,
+    BigQueryDeleteDatasetOperator,
+)
+from airflow.providers.google.cloud.operators.gcs import GCSCreateBucketOperator, GCSDeleteBucketOperator
+from airflow.providers.google.cloud.transfers.local_to_gcs import LocalFilesystemToGCSOperator
+from airflow.utils.trigger_rule import TriggerRule
+from tests.system.utils.watcher import watcher
+
+ENV_ID = os.environ.get("SYSTEM_TESTS_ENV_ID")
+DAG_ID = "bigquery_operations"
+
+DATASET_NAME = f"dataset_{DAG_ID}_{ENV_ID}"
+DATA_SAMPLE_GCS_BUCKET_NAME = f"bucket_{DAG_ID}_{ENV_ID}"
+DATA_SAMPLE_GCS_OBJECT_NAME = "bigquery/us-states/us-states.csv"
+CSV_FILE_LOCAL_PATH = str(Path(__file__).parent / "resources" / "us-states.csv")
+
+
+with models.DAG(
+    DAG_ID,
+    schedule_interval="@once",
+    start_date=datetime(2021, 1, 1),
+    catchup=False,
+    tags=["example", "bigquery"],
+) as dag:
+    create_bucket = GCSCreateBucketOperator(task_id="create_bucket", bucket_name=DATA_SAMPLE_GCS_BUCKET_NAME)
+
+    create_dataset = BigQueryCreateEmptyDatasetOperator(task_id="create_dataset", dataset_id=DATASET_NAME)
+
+    upload_file = LocalFilesystemToGCSOperator(
+        task_id="upload_file_to_bucket",
+        src=CSV_FILE_LOCAL_PATH,
+        dst=DATA_SAMPLE_GCS_OBJECT_NAME,
+        bucket=DATA_SAMPLE_GCS_BUCKET_NAME,
+    )
+
+    # [START howto_operator_bigquery_create_external_table]
+    create_external_table = BigQueryCreateExternalTableOperator(
+        task_id="create_external_table",
+        destination_project_dataset_table=f"{DATASET_NAME}.external_table",
+        bucket=DATA_SAMPLE_GCS_BUCKET_NAME,
+        source_objects=[DATA_SAMPLE_GCS_OBJECT_NAME],
+        schema_fields=[
+            {"name": "emp_name", "type": "STRING", "mode": "REQUIRED"},
+            {"name": "salary", "type": "INTEGER", "mode": "NULLABLE"},
+        ],
+    )
+    # [END howto_operator_bigquery_create_external_table]
+
+    delete_dataset = BigQueryDeleteDatasetOperator(
+        task_id="delete_dataset",
+        dataset_id=DATASET_NAME,
+        delete_contents=True,
+        trigger_rule=TriggerRule.ALL_DONE,
+    )
+
+    delete_bucket = GCSDeleteBucketOperator(
+        task_id="delete_bucket", bucket_name=DATA_SAMPLE_GCS_BUCKET_NAME, trigger_rule=TriggerRule.ALL_DONE
+    )
+
+    (
+        # TEST SETUP
+        [create_bucket, create_dataset]
+        # TEST BODY
+        >> upload_file
+        >> create_external_table
+        # TEST TEARDOWN
+        >> delete_dataset
+        >> delete_bucket
+    )
+
+    list(dag.tasks) >> watcher()

Review comment:
       We thought about a special "util" function for that.
   
   I do not think we discussed about "magical" adding it. this is a bit of a hassle, I agreee but adding it would likely require something on the DAG level - for example a special parameter or decorator. And the problem with that is that it would have to be added somewhere at the beginning of the DAG - where you define the DAG most likely. And the problem with that is that you also use the DAG as examples in our documentation. And we extracts parts of the examples into the documentaiton and we should not pollute those examples with things that are not really good "examples" on how you should add your DAGs. - and those example usually show the DAG definition/default_args as part of the example - by having the special decorator, or parameter on the DAG to indicate that DAG shoudl have "watcher" added migth be too easily copied from those extracted examples.
   
   Having explicit watcher makes it so much easier - it is at the and and it is explicit (which means there is no magic)
   
   But actually what made me think now - we should actually make it even more separated an explicit.
   
   @mnojek why don't we change the the watcher's local import and some comment there to make it even more separated and "explicit" for example:
   
   ```python
            >> delete_bucket
       ) 
       
       from tests.system.utils.watcher import watcher
       # This test run as a system test needs watcher in order to mark success/failure
       # where "tearDown" task is part of the DAG
       list(dag.tasks) >> watcher()
       
   ```




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] potiuk commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r829585899



##########
File path: scripts/ci/pre_commit/pre_commit_check_watcher_in_examples.py
##########
@@ -0,0 +1,86 @@
+#!/usr/bin/env python3
+# 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 sys
+from pathlib import Path
+from typing import List
+
+from rich.console import Console
+
+if __name__ not in ("__main__", "__mp_main__"):
+    raise SystemExit(
+        "This file is intended to be executed as an executable program. You cannot use it as a module."
+        f"To run this script, run the ./{__file__} command [FILE] ..."
+    )
+
+
+console = Console(color_system="standard", width=200)
+
+errors: List[str] = []
+
+WATCHER_APPEND_INSTRUCTION = "list(dag.tasks) >> watcher()"
+
+PYTEST_FUNCTION = """
+def test_run():

Review comment:
       That's a nice idea. If pytest still discovers it this way, I lke it a lot better. also you can even connect it with local import to make it even more localized (same reason as in the other comment , where we care about "example_dag" being still an example dag despite also being a self-contained test. Also it nicely solves the problem which I thought about - what happens if we have two dags in an example. They would have to have slightiy different method ( dag1.clear() dag1.run() vs dag2.clear(() dag2.run()). Passing dag as parameter solves it nicely.
   
   I'd be for something like this:
   
   ```python
        # Needed to run the example dag as system test <link to the docs>
        from tests.airflow.proivders.util import get_test_run
        test_run = get_test_run(dag)
   ```
   
   




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] kosteev commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
kosteev commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r829863869



##########
File path: docs/apache-airflow-providers-google/operators/cloud/bigquery.rst
##########
@@ -29,7 +29,7 @@ data.
 Prerequisite Tasks
 ^^^^^^^^^^^^^^^^^^
 
-.. include::/operators/_partials/prerequisite_tasks.rst
+.. include:: ../_partials/prerequisite_tasks.rst

Review comment:
       Is space needed before ".."?

##########
File path: tests/always/test_example_dags.py
##########
@@ -29,33 +30,38 @@
 NO_DB_QUERY_EXCEPTION = ["/airflow/example_dags/example_subdag_operator.py"]
 
 
-class TestExampleDags(unittest.TestCase):
-    def test_should_be_importable(self):
-        example_dags = list(glob(f"{ROOT_FOLDER}/airflow/**/example_dags/example_*.py", recursive=True))
-        assert 0 != len(example_dags)
-        for filepath in example_dags:
-            relative_filepath = os.path.relpath(filepath, ROOT_FOLDER)
-            with self.subTest(f"File {relative_filepath} should contain dags"):
-                dagbag = DagBag(
-                    dag_folder=filepath,
-                    include_examples=False,
-                )
-                assert 0 == len(dagbag.import_errors), f"import_errors={str(dagbag.import_errors)}"
-                assert len(dagbag.dag_ids) >= 1
-
-    def test_should_not_do_database_queries(self):
-        example_dags = glob(f"{ROOT_FOLDER}/airflow/**/example_dags/example_*.py", recursive=True)
-        example_dags = [
-            dag_file
-            for dag_file in example_dags
-            if any(not dag_file.endswith(e) for e in NO_DB_QUERY_EXCEPTION)
-        ]
-        assert 0 != len(example_dags)
-        for filepath in example_dags:
-            relative_filepath = os.path.relpath(filepath, ROOT_FOLDER)
-            with self.subTest(f"File {relative_filepath} shouldn't do database queries"):
-                with assert_queries_count(0):
-                    DagBag(
-                        dag_folder=filepath,
-                        include_examples=False,
-                    )
+def example_dags():
+    example_dirs = ["airflow/**/example_dags/example_*.py", "tests/system/providers/**/example_*.py"]
+    for example_dir in example_dirs:
+        yield from glob(f"{ROOT_FOLDER}/{example_dir}", recursive=True)
+
+
+def example_dags_except_db_exception():
+    return [
+        dag_file
+        for dag_file in example_dags()
+        if any(not dag_file.endswith(e) for e in NO_DB_QUERY_EXCEPTION)
+    ]
+
+
+def relative_path(path):
+    return os.path.relpath(path, ROOT_FOLDER)
+
+
+@pytest.mark.parametrize("example", list(example_dags()), ids=relative_path)
+def test_should_be_importable(example):
+    dagbag = DagBag(
+        dag_folder=example,
+        include_examples=False,
+    )
+    assert 0 == len(dagbag.import_errors), f"import_errors={str(dagbag.import_errors)}"

Review comment:
       Nit: I am not sure about this, but I believe the general guide is to use expected value on the right side of assert statement.
   It is not different at all from the test status perspective, but may be more clear in the error message in case assert fails that `len(dagbag.import_errors)` actually has to be qual to zero.

##########
File path: scripts/ci/pre_commit/pre_commit_check_watcher_in_examples.py
##########
@@ -0,0 +1,86 @@
+#!/usr/bin/env python3
+# 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 sys
+from pathlib import Path
+from typing import List
+
+from rich.console import Console
+
+if __name__ not in ("__main__", "__mp_main__"):
+    raise SystemExit(
+        "This file is intended to be executed as an executable program. You cannot use it as a module."
+        f"To run this script, run the ./{__file__} command [FILE] ..."
+    )
+
+
+console = Console(color_system="standard", width=200)
+
+errors: List[str] = []
+
+WATCHER_APPEND_INSTRUCTION = "list(dag.tasks) >> watcher()"
+
+PYTEST_FUNCTION = """
+def test_run():
+    from airflow.utils.state import State
+
+    dag.clear(dag_run_state=State.NONE)
+    dag.run()
+"""
+
+
+def _check_file(file: Path):
+    content = file.read_text()
+    if "from tests.system.utils.watcher import watcher" in content:
+        index = content.find(WATCHER_APPEND_INSTRUCTION)
+        if index == -1:
+            errors.append(
+                f"[red]The example {file} imports tests.system.utils.watcher "
+                f"but does not use it properly![/]\n\n"
+                "[yellow]Make sure you have:[/]\n\n"
+                f"        {WATCHER_APPEND_INSTRUCTION}\n\n"
+                "[yellow]as last instruction in your example DAG.[/]\n"
+            )
+        else:
+            operator_leftshift_index = content.find("<<", index + len(WATCHER_APPEND_INSTRUCTION))
+            operator_rightshift_index = content.find(">>", index + len(WATCHER_APPEND_INSTRUCTION))
+            if operator_leftshift_index != -1 or operator_rightshift_index != -1:
+                errors.append(
+                    f"[red]In the  example {file} "
+                    f"watcher is not last instruction in your DAG (there are << "
+                    f"or >> operators after it)![/]\n\n"
+                    "[yellow]Make sure you have:[/]\n"
+                    f"        {WATCHER_APPEND_INSTRUCTION}\n\n"
+                    "[yellow]as last instruction in your example DAG.[/]\n"
+                )
+        if PYTEST_FUNCTION not in content:
+            errors.append(
+                f"[yellow]The  example {file} missed the pytest function at the end.[/]\n\n"
+                "All example tests should have this function added:\n\n" + PYTEST_FUNCTION + "\n\n"
+                "[yellow]Automatically adding it now!\n"
+            )
+            file.write_text(content + "\n" + PYTEST_FUNCTION)

Review comment:
       The name of the script does not completely correspond to what it does.
   Maybe this is not a big deal. Just pointing this out as it may confuse developer.

##########
File path: tests/always/test_example_dags.py
##########
@@ -29,33 +30,38 @@
 NO_DB_QUERY_EXCEPTION = ["/airflow/example_dags/example_subdag_operator.py"]
 
 
-class TestExampleDags(unittest.TestCase):
-    def test_should_be_importable(self):
-        example_dags = list(glob(f"{ROOT_FOLDER}/airflow/**/example_dags/example_*.py", recursive=True))
-        assert 0 != len(example_dags)
-        for filepath in example_dags:
-            relative_filepath = os.path.relpath(filepath, ROOT_FOLDER)
-            with self.subTest(f"File {relative_filepath} should contain dags"):
-                dagbag = DagBag(
-                    dag_folder=filepath,
-                    include_examples=False,
-                )
-                assert 0 == len(dagbag.import_errors), f"import_errors={str(dagbag.import_errors)}"
-                assert len(dagbag.dag_ids) >= 1
-
-    def test_should_not_do_database_queries(self):
-        example_dags = glob(f"{ROOT_FOLDER}/airflow/**/example_dags/example_*.py", recursive=True)
-        example_dags = [
-            dag_file
-            for dag_file in example_dags
-            if any(not dag_file.endswith(e) for e in NO_DB_QUERY_EXCEPTION)
-        ]
-        assert 0 != len(example_dags)
-        for filepath in example_dags:
-            relative_filepath = os.path.relpath(filepath, ROOT_FOLDER)
-            with self.subTest(f"File {relative_filepath} shouldn't do database queries"):
-                with assert_queries_count(0):
-                    DagBag(
-                        dag_folder=filepath,
-                        include_examples=False,
-                    )
+def example_dags():
+    example_dirs = ["airflow/**/example_dags/example_*.py", "tests/system/providers/**/example_*.py"]
+    for example_dir in example_dirs:
+        yield from glob(f"{ROOT_FOLDER}/{example_dir}", recursive=True)
+
+
+def example_dags_except_db_exception():
+    return [
+        dag_file
+        for dag_file in example_dags()
+        if any(not dag_file.endswith(e) for e in NO_DB_QUERY_EXCEPTION)
+    ]
+
+
+def relative_path(path):
+    return os.path.relpath(path, ROOT_FOLDER)
+
+
+@pytest.mark.parametrize("example", list(example_dags()), ids=relative_path)

Review comment:
       Nit: `list(example_dags())` converting to list is not necessary here as `parametrize` accepts iterable, right?

##########
File path: tests/system/conftest.py
##########
@@ -0,0 +1,61 @@
+# 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 os
+import re
+from itertools import chain
+from pathlib import Path
+
+import pytest
+
+OLD_EXECUTOR = os.environ.get("AIRFLOW__CORE__EXECUTOR", default=None)
+REQUIRED_ENV_VARS = ("SYSTEM_TESTS_ENV_ID",)
+
+
+def pytest_configure(config):
+    os.environ["AIRFLOW__CORE__EXECUTOR"] = "DebugExecutor"
+
+
+def pytest_unconfigure(config):
+    if OLD_EXECUTOR is not None:
+        os.environ["AIRFLOW__CORE__EXECUTOR"] = OLD_EXECUTOR

Review comment:
       Should unset "DebugExecutor" here also if the OLD_EXECUTOR is None?




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] bhirsz commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
bhirsz commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r829326254



##########
File path: tests/system/providers/google/bigquery/example_bigquery_dataset.py
##########
@@ -0,0 +1,94 @@
+#
+# 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.
+
+"""
+Example Airflow DAG for Google BigQuery service testing dataset operations.
+"""
+import os
+from datetime import datetime
+
+from airflow import models
+from airflow.operators.bash import BashOperator
+from airflow.providers.google.cloud.operators.bigquery import (
+    BigQueryCreateEmptyDatasetOperator,
+    BigQueryDeleteDatasetOperator,
+    BigQueryGetDatasetOperator,
+    BigQueryUpdateDatasetOperator,
+)
+from airflow.utils.trigger_rule import TriggerRule

Review comment:
       It's not - we're using trigger rules only if there is teardown-like task. For example if you're creating resource using one task you want to make sure to run task that removes that resource even if some tasks inbetween fails. There may be system tests that don't need it at all.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] mnojek commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
mnojek commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r829488627



##########
File path: tests/system/providers/google/bigquery/example_bigquery_operations.py
##########
@@ -0,0 +1,106 @@
+#
+# 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.
+
+"""
+Example Airflow DAG for Google BigQuery service local file upload and external table creation.
+"""
+import os
+from datetime import datetime
+from pathlib import Path
+
+from airflow import models
+from airflow.providers.google.cloud.operators.bigquery import (
+    BigQueryCreateEmptyDatasetOperator,
+    BigQueryCreateExternalTableOperator,
+    BigQueryDeleteDatasetOperator,
+)
+from airflow.providers.google.cloud.operators.gcs import GCSCreateBucketOperator, GCSDeleteBucketOperator
+from airflow.providers.google.cloud.transfers.local_to_gcs import LocalFilesystemToGCSOperator
+from airflow.utils.trigger_rule import TriggerRule
+from tests.system.utils.watcher import watcher
+
+ENV_ID = os.environ.get("SYSTEM_TESTS_ENV_ID")
+DAG_ID = "bigquery_operations"
+
+DATASET_NAME = f"dataset_{DAG_ID}_{ENV_ID}"
+DATA_SAMPLE_GCS_BUCKET_NAME = f"bucket_{DAG_ID}_{ENV_ID}"
+DATA_SAMPLE_GCS_OBJECT_NAME = "bigquery/us-states/us-states.csv"
+CSV_FILE_LOCAL_PATH = str(Path(__file__).parent / "resources" / "us-states.csv")
+
+
+with models.DAG(
+    DAG_ID,
+    schedule_interval="@once",
+    start_date=datetime(2021, 1, 1),
+    catchup=False,
+    tags=["example", "bigquery"],
+) as dag:
+    create_bucket = GCSCreateBucketOperator(task_id="create_bucket", bucket_name=DATA_SAMPLE_GCS_BUCKET_NAME)
+
+    create_dataset = BigQueryCreateEmptyDatasetOperator(task_id="create_dataset", dataset_id=DATASET_NAME)
+
+    upload_file = LocalFilesystemToGCSOperator(
+        task_id="upload_file_to_bucket",
+        src=CSV_FILE_LOCAL_PATH,
+        dst=DATA_SAMPLE_GCS_OBJECT_NAME,
+        bucket=DATA_SAMPLE_GCS_BUCKET_NAME,
+    )
+
+    # [START howto_operator_bigquery_create_external_table]
+    create_external_table = BigQueryCreateExternalTableOperator(
+        task_id="create_external_table",
+        destination_project_dataset_table=f"{DATASET_NAME}.external_table",
+        bucket=DATA_SAMPLE_GCS_BUCKET_NAME,
+        source_objects=[DATA_SAMPLE_GCS_OBJECT_NAME],
+        schema_fields=[
+            {"name": "emp_name", "type": "STRING", "mode": "REQUIRED"},
+            {"name": "salary", "type": "INTEGER", "mode": "NULLABLE"},
+        ],
+    )
+    # [END howto_operator_bigquery_create_external_table]
+
+    delete_dataset = BigQueryDeleteDatasetOperator(
+        task_id="delete_dataset",
+        dataset_id=DATASET_NAME,
+        delete_contents=True,
+        trigger_rule=TriggerRule.ALL_DONE,
+    )
+
+    delete_bucket = GCSDeleteBucketOperator(
+        task_id="delete_bucket", bucket_name=DATA_SAMPLE_GCS_BUCKET_NAME, trigger_rule=TriggerRule.ALL_DONE
+    )
+
+    (
+        # TEST SETUP
+        [create_bucket, create_dataset]
+        # TEST BODY
+        >> upload_file
+        >> create_external_table
+        # TEST TEARDOWN
+        >> delete_dataset
+        >> delete_bucket
+    )
+
+    list(dag.tasks) >> watcher()

Review comment:
       I don't have a better idea how this can be achieved without explicitly defining this task dependency. This is actually quite clever because in the previous version of this design we were listing all the tasks again and then defining `>>` operator with the watcher. Now it's just this one line that has a reference to all tasks in a local dag and then appends the watcher to all of them. And it is of course only needed when there is a teardown task in the dag.
   But of course I am open to any specific suggestions that may improve the design.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] ferruzzi commented on pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
ferruzzi commented on pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#issuecomment-1079222017


   Nice.  Congrats on the merge.


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] potiuk commented on pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
potiuk commented on pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#issuecomment-1078149904


   AAAAH. Static checks!


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] potiuk commented on pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
potiuk commented on pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#issuecomment-1078415415


   Rebuilding (aftermath of preparing 2.2.5 release)


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] ferruzzi commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
ferruzzi commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r834799952



##########
File path: tests/system/providers/google/bigquery/example_bigquery_dataset.py
##########
@@ -0,0 +1,95 @@
+#
+# 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.
+
+"""
+Example Airflow DAG for Google BigQuery service testing dataset operations.
+"""
+import os
+from datetime import datetime
+
+from airflow import models
+from airflow.operators.bash import BashOperator
+from airflow.providers.google.cloud.operators.bigquery import (
+    BigQueryCreateEmptyDatasetOperator,
+    BigQueryDeleteDatasetOperator,
+    BigQueryGetDatasetOperator,
+    BigQueryUpdateDatasetOperator,
+)
+from airflow.utils.trigger_rule import TriggerRule
+
+ENV_ID = os.environ.get("SYSTEM_TESTS_ENV_ID")
+DAG_ID = "bigquery_dataset"
+
+DATASET_NAME = f"dataset_{DAG_ID}_{ENV_ID}"
+
+
+with models.DAG(
+    DAG_ID,
+    schedule_interval="@once",
+    start_date=datetime(2021, 1, 1),
+    catchup=False,
+    tags=["example", "bigquery"],
+) as dag:
+    # [START howto_operator_bigquery_create_dataset]
+    create_dataset = BigQueryCreateEmptyDatasetOperator(task_id="create_dataset", dataset_id=DATASET_NAME)
+    # [END howto_operator_bigquery_create_dataset]
+
+    # [START howto_operator_bigquery_update_dataset]
+    update_dataset = BigQueryUpdateDatasetOperator(
+        task_id="update_dataset",
+        dataset_id=DATASET_NAME,
+        dataset_resource={"description": "Updated dataset"},
+    )
+    # [END howto_operator_bigquery_update_dataset]
+
+    # [START howto_operator_bigquery_get_dataset]
+    get_dataset = BigQueryGetDatasetOperator(task_id="get-dataset", dataset_id=DATASET_NAME)
+    # [END howto_operator_bigquery_get_dataset]
+
+    get_dataset_result = BashOperator(
+        task_id="get_dataset_result",
+        bash_command="echo \"{{ task_instance.xcom_pull('get-dataset')['id'] }}\"",
+    )
+
+    # [START howto_operator_bigquery_delete_dataset]
+    delete_dataset = BigQueryDeleteDatasetOperator(
+        task_id="delete_dataset", dataset_id=DATASET_NAME, delete_contents=True
+    )
+    # [END howto_operator_bigquery_delete_dataset]
+    delete_dataset.trigger_rule = TriggerRule.ALL_DONE
+
+    (
+        # TEST BODY
+        create_dataset
+        >> update_dataset
+        >> get_dataset
+        >> get_dataset_result
+        # TEST TEARDOWN
+        >> delete_dataset
+    )
+
+    from tests.system.utils.watcher import watcher

Review comment:
       Maybe not worth holding up the merge, but I wonder if moving the "add watcher" flag to the DAG class itself may be a resolution?  then usage might look like:
   
   ```
   with DAG(
       dag_id='example_dag',
       schedule_interval=None,
       start_date=datetime(2021, 1, 1),
       tags=['example'],
       catchup=False,
       add_watcher=True,
   ) as dag:
   ```




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] uranusjr commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
uranusjr commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r833932479



##########
File path: tests/system/utils/__init__.py
##########
@@ -0,0 +1,25 @@
+# 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.
+from airflow.utils.state import State
+
+
+def get_test_run(dag):
+    def test_run():
+        dag.clear(dag_run_state=State.NONE)
+        dag.run()
+
+    return test_run

Review comment:
       ```suggestion
   def get_test_run(dag):
       def test_run():
           dag.clear(dag_run_state=None)
           dag.run()
   
       return test_run
   ```
   
   Why is this needed though? It’s just three straightforward lines we can copy pretty much everywhere. Having to import this function mostly negates the benefits it brings.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] potiuk closed pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
potiuk closed pull request #22311:
URL: https://github.com/apache/airflow/pull/22311


   


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] potiuk commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r832105831



##########
File path: tests/system/providers/google/bigquery/example_bigquery_dataset.py
##########
@@ -0,0 +1,95 @@
+#
+# 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.
+
+"""
+Example Airflow DAG for Google BigQuery service testing dataset operations.
+"""
+import os
+from datetime import datetime
+
+from airflow import models
+from airflow.operators.bash import BashOperator
+from airflow.providers.google.cloud.operators.bigquery import (
+    BigQueryCreateEmptyDatasetOperator,
+    BigQueryDeleteDatasetOperator,
+    BigQueryGetDatasetOperator,
+    BigQueryUpdateDatasetOperator,
+)
+from airflow.utils.trigger_rule import TriggerRule
+
+ENV_ID = os.environ.get("SYSTEM_TESTS_ENV_ID")
+DAG_ID = "bigquery_dataset"
+
+DATASET_NAME = f"dataset_{DAG_ID}_{ENV_ID}"
+
+
+with models.DAG(
+    DAG_ID,
+    schedule_interval="@once",
+    start_date=datetime(2021, 1, 1),
+    catchup=False,
+    tags=["example", "bigquery"],
+) as dag:
+    # [START howto_operator_bigquery_create_dataset]
+    create_dataset = BigQueryCreateEmptyDatasetOperator(task_id="create_dataset", dataset_id=DATASET_NAME)
+    # [END howto_operator_bigquery_create_dataset]
+
+    # [START howto_operator_bigquery_update_dataset]
+    update_dataset = BigQueryUpdateDatasetOperator(
+        task_id="update_dataset",
+        dataset_id=DATASET_NAME,
+        dataset_resource={"description": "Updated dataset"},
+    )
+    # [END howto_operator_bigquery_update_dataset]
+
+    # [START howto_operator_bigquery_get_dataset]
+    get_dataset = BigQueryGetDatasetOperator(task_id="get-dataset", dataset_id=DATASET_NAME)
+    # [END howto_operator_bigquery_get_dataset]
+
+    get_dataset_result = BashOperator(
+        task_id="get_dataset_result",
+        bash_command="echo \"{{ task_instance.xcom_pull('get-dataset')['id'] }}\"",
+    )
+
+    # [START howto_operator_bigquery_delete_dataset]
+    delete_dataset = BigQueryDeleteDatasetOperator(
+        task_id="delete_dataset", dataset_id=DATASET_NAME, delete_contents=True
+    )
+    # [END howto_operator_bigquery_delete_dataset]
+    delete_dataset.trigger_rule = TriggerRule.ALL_DONE
+
+    (
+        # TEST BODY
+        create_dataset
+        >> update_dataset
+        >> get_dataset
+        >> get_dataset_result
+        # TEST TEARDOWN
+        >> delete_dataset
+    )
+
+    from tests.system.utils.watcher import watcher

Review comment:
       Hmm. I thought a bit about our earlier discussion - https://github.com/apache/airflow/pull/22311#discussion_r829687607 with @ferruzzi and hmm. I changed my mind I think. 
   
    I think we have an easy way to decrease the amount of code here indeed in almost the way suggested :). And remove one of the "pre-commit checks".
   
   ```
   from tests.system.utils.watcher import watcher
   
   def get_test_run(dag, *, add_watcher: bool = False):
       def test_run():
           if add_watcher:
                # This test needs watcher in order to properly mark success/failure
                # when "tearDown" task with trigger rule is part of the DAG
                 list(dag.tasks) >> watcher()
           dag.clear(dag_run_state=State.NONE)
           dag.run()
   
       return test_run
   ```
   
   And then in the DAG:
   
   ```
   test_run = get_test_run(dag, add_watcher=True)
   ```
   
   This way we;
   
   * do not need local import in the DAG
   * do not need the pre-commit to add watcher code
   * avoid code duplication
   
   WDYT?
   




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] potiuk commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r830483564



##########
File path: tests/system/conftest.py
##########
@@ -0,0 +1,61 @@
+# 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 os
+import re
+from itertools import chain
+from pathlib import Path
+
+import pytest
+
+OLD_EXECUTOR = os.environ.get("AIRFLOW__CORE__EXECUTOR", default=None)
+REQUIRED_ENV_VARS = ("SYSTEM_TESTS_ENV_ID",)
+
+
+def pytest_configure(config):
+    os.environ["AIRFLOW__CORE__EXECUTOR"] = "DebugExecutor"
+
+
+def pytest_unconfigure(config):
+    if OLD_EXECUTOR is not None:
+        os.environ["AIRFLOW__CORE__EXECUTOR"] = OLD_EXECUTOR

Review comment:
       Actually - I think pytest_configure is not the best idea - it will set the executor for the whole pytest execution. I know it would not be likely to have "system" and "non-system" tests - but should not be setting that it in auto-fixture with "package" scope  ?




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] bhirsz commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
bhirsz commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r830800139



##########
File path: tests/system/conftest.py
##########
@@ -0,0 +1,61 @@
+# 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 os
+import re
+from itertools import chain
+from pathlib import Path
+
+import pytest
+
+OLD_EXECUTOR = os.environ.get("AIRFLOW__CORE__EXECUTOR", default=None)
+REQUIRED_ENV_VARS = ("SYSTEM_TESTS_ENV_ID",)
+
+
+def pytest_configure(config):
+    os.environ["AIRFLOW__CORE__EXECUTOR"] = "DebugExecutor"
+
+
+def pytest_unconfigure(config):
+    if OLD_EXECUTOR is not None:
+        os.environ["AIRFLOW__CORE__EXECUTOR"] = OLD_EXECUTOR

Review comment:
       And I've checked and actually env variables can be only be strings so OLD_EXECUTOR will not be None as long as the variable existed before (referring to original question by @kosteev)




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] potiuk edited a comment on pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
potiuk edited a comment on pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#issuecomment-1078192924


   I pushed a fixup @bhirsz  - > just ignoring this file in pre-commit (as an exception)


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] bhirsz commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
bhirsz commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r832978462



##########
File path: docs/apache-airflow-providers-google/operators/cloud/bigquery.rst
##########
@@ -42,7 +42,7 @@ Create dataset
 To create an empty dataset in a BigQuery database you can use
 :class:`~airflow.providers.google.cloud.operators.bigquery.BigQueryCreateEmptyDatasetOperator`.
 
-.. exampleinclude:: /../../airflow/providers/google/cloud/example_dags/example_bigquery_operations.py
+.. exampleinclude:: /../../tests/system/providers/google/bigquery/example_bigquery_dataset.py

Review comment:
       Done




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] mnojek commented on pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
mnojek commented on pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#issuecomment-1076545681


   Thanks everyone involved in checking this PR!
   We want to slowly go towards merging it with the main branch and **ask you to take a look at it one more time** & respond in relevant threads if the problem/concern has been resolved/answered.
   From my point of view, the solution is now complete, but of course something may still be missing. It doesn't need to be perfect - we can improve it along with the next PRs and issues. By now, I think that it looks really fine and your feedback was very helpful. Share your thoughts and let's make it work!


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] potiuk commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r829582024



##########
File path: scripts/ci/pre_commit/pre_commit_check_watcher_in_examples.py
##########
@@ -0,0 +1,86 @@
+#!/usr/bin/env python3
+# 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 sys
+from pathlib import Path
+from typing import List
+
+from rich.console import Console
+
+if __name__ not in ("__main__", "__mp_main__"):
+    raise SystemExit(
+        "This file is intended to be executed as an executable program. You cannot use it as a module."
+        f"To run this script, run the ./{__file__} command [FILE] ..."
+    )
+
+
+console = Console(color_system="standard", width=200)
+
+errors: List[str] = []
+
+WATCHER_APPEND_INSTRUCTION = "list(dag.tasks) >> watcher()"
+
+PYTEST_FUNCTION = """
+def test_run():
+    from airflow.utils.state import State
+
+    dag.clear(dag_run_state=State.NONE)
+    dag.run()
+"""
+
+
+def _check_file(file: Path):
+    content = file.read_text()
+    if "from tests.system.utils.watcher import watcher" in content:

Review comment:
       Yeah. That check is "best effort" we could have import the AST and analyze it, but I think this is the 20/80 Pareto's rule in full swing. Even if we miss it once or twice, if somoene actually made an effort to import watcher they will likely not forget to add it as dependencies :) 




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] ferruzzi commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
ferruzzi commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r829287507



##########
File path: scripts/ci/pre_commit/pre_commit_check_watcher_in_examples.py
##########
@@ -0,0 +1,86 @@
+#!/usr/bin/env python3
+# 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 sys
+from pathlib import Path
+from typing import List
+
+from rich.console import Console
+
+if __name__ not in ("__main__", "__mp_main__"):
+    raise SystemExit(
+        "This file is intended to be executed as an executable program. You cannot use it as a module."
+        f"To run this script, run the ./{__file__} command [FILE] ..."
+    )
+
+
+console = Console(color_system="standard", width=200)
+
+errors: List[str] = []
+
+WATCHER_APPEND_INSTRUCTION = "list(dag.tasks) >> watcher()"

Review comment:
       Apologies if I missed this in the email discussions.  If I am reading this and the code below correctly, then the `chain()` method of declaring the taskflow is not really supported anymore?
   
   Related, there is nothing more frustrating to me than a precommit failing because of something it could easily fix.  Could this append the ">> watcher()" if it is not found?




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] ferruzzi commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
ferruzzi commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r829310567



##########
File path: tests/system/providers/google/bigquery/example_bigquery_operations.py
##########
@@ -0,0 +1,106 @@
+#
+# 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.
+
+"""
+Example Airflow DAG for Google BigQuery service local file upload and external table creation.
+"""
+import os
+from datetime import datetime
+from pathlib import Path
+
+from airflow import models
+from airflow.providers.google.cloud.operators.bigquery import (
+    BigQueryCreateEmptyDatasetOperator,
+    BigQueryCreateExternalTableOperator,
+    BigQueryDeleteDatasetOperator,
+)
+from airflow.providers.google.cloud.operators.gcs import GCSCreateBucketOperator, GCSDeleteBucketOperator
+from airflow.providers.google.cloud.transfers.local_to_gcs import LocalFilesystemToGCSOperator
+from airflow.utils.trigger_rule import TriggerRule
+from tests.system.utils.watcher import watcher
+
+ENV_ID = os.environ.get("SYSTEM_TESTS_ENV_ID")
+DAG_ID = "bigquery_operations"
+
+DATASET_NAME = f"dataset_{DAG_ID}_{ENV_ID}"
+DATA_SAMPLE_GCS_BUCKET_NAME = f"bucket_{DAG_ID}_{ENV_ID}"
+DATA_SAMPLE_GCS_OBJECT_NAME = "bigquery/us-states/us-states.csv"
+CSV_FILE_LOCAL_PATH = str(Path(__file__).parent / "resources" / "us-states.csv")
+
+
+with models.DAG(
+    DAG_ID,
+    schedule_interval="@once",
+    start_date=datetime(2021, 1, 1),
+    catchup=False,
+    tags=["example", "bigquery"],
+) as dag:
+    create_bucket = GCSCreateBucketOperator(task_id="create_bucket", bucket_name=DATA_SAMPLE_GCS_BUCKET_NAME)
+
+    create_dataset = BigQueryCreateEmptyDatasetOperator(task_id="create_dataset", dataset_id=DATASET_NAME)
+
+    upload_file = LocalFilesystemToGCSOperator(
+        task_id="upload_file_to_bucket",
+        src=CSV_FILE_LOCAL_PATH,
+        dst=DATA_SAMPLE_GCS_OBJECT_NAME,
+        bucket=DATA_SAMPLE_GCS_BUCKET_NAME,
+    )
+
+    # [START howto_operator_bigquery_create_external_table]
+    create_external_table = BigQueryCreateExternalTableOperator(
+        task_id="create_external_table",
+        destination_project_dataset_table=f"{DATASET_NAME}.external_table",
+        bucket=DATA_SAMPLE_GCS_BUCKET_NAME,
+        source_objects=[DATA_SAMPLE_GCS_OBJECT_NAME],
+        schema_fields=[
+            {"name": "emp_name", "type": "STRING", "mode": "REQUIRED"},
+            {"name": "salary", "type": "INTEGER", "mode": "NULLABLE"},
+        ],
+    )
+    # [END howto_operator_bigquery_create_external_table]
+
+    delete_dataset = BigQueryDeleteDatasetOperator(
+        task_id="delete_dataset",
+        dataset_id=DATASET_NAME,
+        delete_contents=True,
+        trigger_rule=TriggerRule.ALL_DONE,
+    )
+
+    delete_bucket = GCSDeleteBucketOperator(
+        task_id="delete_bucket", bucket_name=DATA_SAMPLE_GCS_BUCKET_NAME, trigger_rule=TriggerRule.ALL_DONE
+    )
+
+    (
+        # TEST SETUP
+        [create_bucket, create_dataset]
+        # TEST BODY
+        >> upload_file
+        >> create_external_table
+        # TEST TEARDOWN
+        >> delete_dataset
+        >> delete_bucket
+    )
+
+    list(dag.tasks) >> watcher()

Review comment:
       I know this was discussed int he email thread, but I'll raise the concern one last time because this example really drives it home, then I promise I'll drop it.  We are adding the exact same line of code to every Example DAG (now System Test), shouldn't that really be handled behind the scenes when the test DAG is being parsed instead of adding this copy/pasted line to literally hundreds of files?




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] ferruzzi commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
ferruzzi commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r829310567



##########
File path: tests/system/providers/google/bigquery/example_bigquery_operations.py
##########
@@ -0,0 +1,106 @@
+#
+# 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.
+
+"""
+Example Airflow DAG for Google BigQuery service local file upload and external table creation.
+"""
+import os
+from datetime import datetime
+from pathlib import Path
+
+from airflow import models
+from airflow.providers.google.cloud.operators.bigquery import (
+    BigQueryCreateEmptyDatasetOperator,
+    BigQueryCreateExternalTableOperator,
+    BigQueryDeleteDatasetOperator,
+)
+from airflow.providers.google.cloud.operators.gcs import GCSCreateBucketOperator, GCSDeleteBucketOperator
+from airflow.providers.google.cloud.transfers.local_to_gcs import LocalFilesystemToGCSOperator
+from airflow.utils.trigger_rule import TriggerRule
+from tests.system.utils.watcher import watcher
+
+ENV_ID = os.environ.get("SYSTEM_TESTS_ENV_ID")
+DAG_ID = "bigquery_operations"
+
+DATASET_NAME = f"dataset_{DAG_ID}_{ENV_ID}"
+DATA_SAMPLE_GCS_BUCKET_NAME = f"bucket_{DAG_ID}_{ENV_ID}"
+DATA_SAMPLE_GCS_OBJECT_NAME = "bigquery/us-states/us-states.csv"
+CSV_FILE_LOCAL_PATH = str(Path(__file__).parent / "resources" / "us-states.csv")
+
+
+with models.DAG(
+    DAG_ID,
+    schedule_interval="@once",
+    start_date=datetime(2021, 1, 1),
+    catchup=False,
+    tags=["example", "bigquery"],
+) as dag:
+    create_bucket = GCSCreateBucketOperator(task_id="create_bucket", bucket_name=DATA_SAMPLE_GCS_BUCKET_NAME)
+
+    create_dataset = BigQueryCreateEmptyDatasetOperator(task_id="create_dataset", dataset_id=DATASET_NAME)
+
+    upload_file = LocalFilesystemToGCSOperator(
+        task_id="upload_file_to_bucket",
+        src=CSV_FILE_LOCAL_PATH,
+        dst=DATA_SAMPLE_GCS_OBJECT_NAME,
+        bucket=DATA_SAMPLE_GCS_BUCKET_NAME,
+    )
+
+    # [START howto_operator_bigquery_create_external_table]
+    create_external_table = BigQueryCreateExternalTableOperator(
+        task_id="create_external_table",
+        destination_project_dataset_table=f"{DATASET_NAME}.external_table",
+        bucket=DATA_SAMPLE_GCS_BUCKET_NAME,
+        source_objects=[DATA_SAMPLE_GCS_OBJECT_NAME],
+        schema_fields=[
+            {"name": "emp_name", "type": "STRING", "mode": "REQUIRED"},
+            {"name": "salary", "type": "INTEGER", "mode": "NULLABLE"},
+        ],
+    )
+    # [END howto_operator_bigquery_create_external_table]
+
+    delete_dataset = BigQueryDeleteDatasetOperator(
+        task_id="delete_dataset",
+        dataset_id=DATASET_NAME,
+        delete_contents=True,
+        trigger_rule=TriggerRule.ALL_DONE,
+    )
+
+    delete_bucket = GCSDeleteBucketOperator(
+        task_id="delete_bucket", bucket_name=DATA_SAMPLE_GCS_BUCKET_NAME, trigger_rule=TriggerRule.ALL_DONE
+    )
+
+    (
+        # TEST SETUP
+        [create_bucket, create_dataset]
+        # TEST BODY
+        >> upload_file
+        >> create_external_table
+        # TEST TEARDOWN
+        >> delete_dataset
+        >> delete_bucket
+    )
+
+    list(dag.tasks) >> watcher()

Review comment:
       I know this was discussed in the email thread, but I'll raise the concern one last time because this example really drives it home, then I promise I'll drop it.  We are adding the exact same line of code to every Example DAG (now System Test), shouldn't that really be handled behind the scenes when the test DAG is being parsed instead of adding this copy/pasted line to literally hundreds of files?




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] potiuk commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r829585899



##########
File path: scripts/ci/pre_commit/pre_commit_check_watcher_in_examples.py
##########
@@ -0,0 +1,86 @@
+#!/usr/bin/env python3
+# 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 sys
+from pathlib import Path
+from typing import List
+
+from rich.console import Console
+
+if __name__ not in ("__main__", "__mp_main__"):
+    raise SystemExit(
+        "This file is intended to be executed as an executable program. You cannot use it as a module."
+        f"To run this script, run the ./{__file__} command [FILE] ..."
+    )
+
+
+console = Console(color_system="standard", width=200)
+
+errors: List[str] = []
+
+WATCHER_APPEND_INSTRUCTION = "list(dag.tasks) >> watcher()"
+
+PYTEST_FUNCTION = """
+def test_run():

Review comment:
       That's a nice idea. If pytest still discovers it this way, I lke it a lot better. also you can even connect it with local import to make it even more localized (same reason as in the other comment , where we care about "example_dag" being still an example dag despite also being a self-contained test. Also it nicely solves the problem which I thought about - what happens if we have two dags in an example. They would have to have slightiy different method ( dag1.clear() dag1.run() vs dag2.clear(() dag2.run()). Passing dag as parameter solves it nicely.
   
   I'd be for something like this:
   
   ```python
   # Needed to run the example dag as system test <link to the docs>
   from tests.airflow.proivders.util import get_test_run
   test_run = get_test_run(dag)
   ```
   
   




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] ferruzzi commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
ferruzzi commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r829687607



##########
File path: tests/system/providers/google/bigquery/example_bigquery_operations.py
##########
@@ -0,0 +1,106 @@
+#
+# 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.
+
+"""
+Example Airflow DAG for Google BigQuery service local file upload and external table creation.
+"""
+import os
+from datetime import datetime
+from pathlib import Path
+
+from airflow import models
+from airflow.providers.google.cloud.operators.bigquery import (
+    BigQueryCreateEmptyDatasetOperator,
+    BigQueryCreateExternalTableOperator,
+    BigQueryDeleteDatasetOperator,
+)
+from airflow.providers.google.cloud.operators.gcs import GCSCreateBucketOperator, GCSDeleteBucketOperator
+from airflow.providers.google.cloud.transfers.local_to_gcs import LocalFilesystemToGCSOperator
+from airflow.utils.trigger_rule import TriggerRule
+from tests.system.utils.watcher import watcher
+
+ENV_ID = os.environ.get("SYSTEM_TESTS_ENV_ID")
+DAG_ID = "bigquery_operations"
+
+DATASET_NAME = f"dataset_{DAG_ID}_{ENV_ID}"
+DATA_SAMPLE_GCS_BUCKET_NAME = f"bucket_{DAG_ID}_{ENV_ID}"
+DATA_SAMPLE_GCS_OBJECT_NAME = "bigquery/us-states/us-states.csv"
+CSV_FILE_LOCAL_PATH = str(Path(__file__).parent / "resources" / "us-states.csv")
+
+
+with models.DAG(
+    DAG_ID,
+    schedule_interval="@once",
+    start_date=datetime(2021, 1, 1),
+    catchup=False,
+    tags=["example", "bigquery"],
+) as dag:
+    create_bucket = GCSCreateBucketOperator(task_id="create_bucket", bucket_name=DATA_SAMPLE_GCS_BUCKET_NAME)
+
+    create_dataset = BigQueryCreateEmptyDatasetOperator(task_id="create_dataset", dataset_id=DATASET_NAME)
+
+    upload_file = LocalFilesystemToGCSOperator(
+        task_id="upload_file_to_bucket",
+        src=CSV_FILE_LOCAL_PATH,
+        dst=DATA_SAMPLE_GCS_OBJECT_NAME,
+        bucket=DATA_SAMPLE_GCS_BUCKET_NAME,
+    )
+
+    # [START howto_operator_bigquery_create_external_table]
+    create_external_table = BigQueryCreateExternalTableOperator(
+        task_id="create_external_table",
+        destination_project_dataset_table=f"{DATASET_NAME}.external_table",
+        bucket=DATA_SAMPLE_GCS_BUCKET_NAME,
+        source_objects=[DATA_SAMPLE_GCS_OBJECT_NAME],
+        schema_fields=[
+            {"name": "emp_name", "type": "STRING", "mode": "REQUIRED"},
+            {"name": "salary", "type": "INTEGER", "mode": "NULLABLE"},
+        ],
+    )
+    # [END howto_operator_bigquery_create_external_table]
+
+    delete_dataset = BigQueryDeleteDatasetOperator(
+        task_id="delete_dataset",
+        dataset_id=DATASET_NAME,
+        delete_contents=True,
+        trigger_rule=TriggerRule.ALL_DONE,
+    )
+
+    delete_bucket = GCSDeleteBucketOperator(
+        task_id="delete_bucket", bucket_name=DATA_SAMPLE_GCS_BUCKET_NAME, trigger_rule=TriggerRule.ALL_DONE
+    )
+
+    (
+        # TEST SETUP
+        [create_bucket, create_dataset]
+        # TEST BODY
+        >> upload_file
+        >> create_external_table
+        # TEST TEARDOWN
+        >> delete_dataset
+        >> delete_bucket
+    )
+
+    list(dag.tasks) >> watcher()

Review comment:
       > I am open to any specific suggestions
   
   I get that.  Unfortunately I only have half-baked suggestions.  the only thing I can really think of would be if we had something like `tests.system.DAG` which inherits `airflow.DAG` and adds that watcher to the task list, but that's only a half-baked idea and I haven't really thought it through all the way.  Writing the same line in hundreds of files just feels wrong, but without a more solid alternative, I'll stop beating that dead horse.  Thanks for the effort you've put into this, and I hope I'm not coming off as argumentative or anything.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] o-nikolas commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
o-nikolas commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r830236599



##########
File path: scripts/ci/pre_commit/pre_commit_check_watcher_in_examples.py
##########
@@ -0,0 +1,86 @@
+#!/usr/bin/env python3
+# 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 sys
+from pathlib import Path
+from typing import List
+
+from rich.console import Console
+
+if __name__ not in ("__main__", "__mp_main__"):
+    raise SystemExit(
+        "This file is intended to be executed as an executable program. You cannot use it as a module."
+        f"To run this script, run the ./{__file__} command [FILE] ..."
+    )
+
+
+console = Console(color_system="standard", width=200)
+
+errors: List[str] = []
+
+WATCHER_APPEND_INSTRUCTION = "list(dag.tasks) >> watcher()"
+
+PYTEST_FUNCTION = """
+def test_run():
+    from airflow.utils.state import State
+
+    dag.clear(dag_run_state=State.NONE)
+    dag.run()
+"""
+
+
+def _check_file(file: Path):
+    content = file.read_text()
+    if "from tests.system.utils.watcher import watcher" in content:
+        index = content.find(WATCHER_APPEND_INSTRUCTION)
+        if index == -1:
+            errors.append(
+                f"[red]The example {file} imports tests.system.utils.watcher "
+                f"but does not use it properly![/]\n\n"
+                "[yellow]Make sure you have:[/]\n\n"
+                f"        {WATCHER_APPEND_INSTRUCTION}\n\n"
+                "[yellow]as last instruction in your example DAG.[/]\n"
+            )
+        else:
+            operator_leftshift_index = content.find("<<", index + len(WATCHER_APPEND_INSTRUCTION))
+            operator_rightshift_index = content.find(">>", index + len(WATCHER_APPEND_INSTRUCTION))
+            if operator_leftshift_index != -1 or operator_rightshift_index != -1:
+                errors.append(
+                    f"[red]In the  example {file} "
+                    f"watcher is not last instruction in your DAG (there are << "
+                    f"or >> operators after it)![/]\n\n"
+                    "[yellow]Make sure you have:[/]\n"
+                    f"        {WATCHER_APPEND_INSTRUCTION}\n\n"
+                    "[yellow]as last instruction in your example DAG.[/]\n"
+                )
+        if PYTEST_FUNCTION not in content:
+            errors.append(
+                f"[yellow]The  example {file} missed the pytest function at the end.[/]\n\n"
+                "All example tests should have this function added:\n\n" + PYTEST_FUNCTION + "\n\n"
+                "[yellow]Automatically adding it now!\n"
+            )
+            file.write_text(content + "\n" + PYTEST_FUNCTION)

Review comment:
       I noticed this as well while reviewing but convinced myself not to comment as it might be too much of a nit. But if two of us have now noticed this, the name should probably be updated :+1:




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] mnojek commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
mnojek commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r830482886



##########
File path: scripts/ci/pre_commit/pre_commit_check_watcher_in_examples.py
##########
@@ -0,0 +1,86 @@
+#!/usr/bin/env python3
+# 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 sys
+from pathlib import Path
+from typing import List
+
+from rich.console import Console
+
+if __name__ not in ("__main__", "__mp_main__"):
+    raise SystemExit(
+        "This file is intended to be executed as an executable program. You cannot use it as a module."
+        f"To run this script, run the ./{__file__} command [FILE] ..."
+    )
+
+
+console = Console(color_system="standard", width=200)
+
+errors: List[str] = []
+
+WATCHER_APPEND_INSTRUCTION = "list(dag.tasks) >> watcher()"
+
+PYTEST_FUNCTION = """
+def test_run():
+    from airflow.utils.state import State
+
+    dag.clear(dag_run_state=State.NONE)
+    dag.run()
+"""
+
+
+def _check_file(file: Path):
+    content = file.read_text()
+    if "from tests.system.utils.watcher import watcher" in content:
+        index = content.find(WATCHER_APPEND_INSTRUCTION)
+        if index == -1:
+            errors.append(
+                f"[red]The example {file} imports tests.system.utils.watcher "
+                f"but does not use it properly![/]\n\n"
+                "[yellow]Make sure you have:[/]\n\n"
+                f"        {WATCHER_APPEND_INSTRUCTION}\n\n"
+                "[yellow]as last instruction in your example DAG.[/]\n"
+            )
+        else:
+            operator_leftshift_index = content.find("<<", index + len(WATCHER_APPEND_INSTRUCTION))
+            operator_rightshift_index = content.find(">>", index + len(WATCHER_APPEND_INSTRUCTION))
+            if operator_leftshift_index != -1 or operator_rightshift_index != -1:
+                errors.append(
+                    f"[red]In the  example {file} "
+                    f"watcher is not last instruction in your DAG (there are << "
+                    f"or >> operators after it)![/]\n\n"
+                    "[yellow]Make sure you have:[/]\n"
+                    f"        {WATCHER_APPEND_INSTRUCTION}\n\n"
+                    "[yellow]as last instruction in your example DAG.[/]\n"
+                )
+        if PYTEST_FUNCTION not in content:
+            errors.append(
+                f"[yellow]The  example {file} missed the pytest function at the end.[/]\n\n"
+                "All example tests should have this function added:\n\n" + PYTEST_FUNCTION + "\n\n"
+                "[yellow]Automatically adding it now!\n"
+            )
+            file.write_text(content + "\n" + PYTEST_FUNCTION)

Review comment:
       I must admit that I was also thinking about it 😄 We will change it to something that better explains the action executed by the check.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] bhirsz commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
bhirsz commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r831044898



##########
File path: tests/system/conftest.py
##########
@@ -0,0 +1,61 @@
+# 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 os
+import re
+from itertools import chain
+from pathlib import Path
+
+import pytest
+
+OLD_EXECUTOR = os.environ.get("AIRFLOW__CORE__EXECUTOR", default=None)
+REQUIRED_ENV_VARS = ("SYSTEM_TESTS_ENV_ID",)
+
+
+def pytest_configure(config):
+    os.environ["AIRFLOW__CORE__EXECUTOR"] = "DebugExecutor"
+
+
+def pytest_unconfigure(config):
+    if OLD_EXECUTOR is not None:
+        os.environ["AIRFLOW__CORE__EXECUTOR"] = OLD_EXECUTOR

Review comment:
       Updated the code




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] mnojek commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
mnojek commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r831115087



##########
File path: scripts/ci/pre_commit/pre_commit_check_system_tests.py
##########
@@ -0,0 +1,84 @@
+#!/usr/bin/env python3
+# 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 sys
+from pathlib import Path
+from typing import List
+
+from rich.console import Console
+
+if __name__ not in ("__main__", "__mp_main__"):
+    raise SystemExit(
+        "This file is intended to be executed as an executable program. You cannot use it as a module."
+        f"To execute this script, run ./{__file__} [FILE] ..."
+    )
+
+
+console = Console(color_system="standard", width=200)
+
+errors: List[str] = []
+
+WATCHER_APPEND_INSTRUCTION = "list(dag.tasks) >> watcher()"
+
+PYTEST_FUNCTION = """
+from tests.system.utils import get_test_run  # noqa: E402
+
+test_run = get_test_run(dag)
+"""
+
+
+def _check_file(file: Path):
+    content = file.read_text()
+    if "from tests.system.utils.watcher import watcher" in content:
+        index = content.find(WATCHER_APPEND_INSTRUCTION)
+        if index == -1:
+            errors.append(
+                f"[red]The example {file} imports tests.system.utils.watcher "
+                f"but does not use it properly![/]\n\n"
+                "[yellow]Make sure you have:[/]\n\n"
+                f"        {WATCHER_APPEND_INSTRUCTION}\n\n"
+                "[yellow]as the last instruction in your example DAG.[/]\n"
+            )
+        else:
+            operator_leftshift_index = content.find("<<", index + len(WATCHER_APPEND_INSTRUCTION))
+            operator_rightshift_index = content.find(">>", index + len(WATCHER_APPEND_INSTRUCTION))
+            if operator_leftshift_index != -1 or operator_rightshift_index != -1:
+                errors.append(
+                    f"[red]In the example {file} "
+                    f"watcher is not the last instruction in your DAG "
+                    f"(there are << or >> operators after it)![/]\n\n"
+                    "[yellow]Make sure you have:[/]\n"
+                    f"        {WATCHER_APPEND_INSTRUCTION}\n\n"
+                    "[yellow]as the last instruction in your example DAG.[/]\n"
+                )
+    if PYTEST_FUNCTION not in content:
+        errors.append(
+            f"[yellow]The example {file} missed the pytest function at the end.[/]\n\n"
+            "All example tests should have this function added:\n\n" + PYTEST_FUNCTION + "\n\n"
+            "[yellow]Automatically adding it now![/]\n"
+        )
+        file.write_text(content + "\n" + PYTEST_FUNCTION)

Review comment:
       Dedentet this part so it is checked regardlessly if the watcher import is there. I believe it was done like this by mistake.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] potiuk commented on a change in pull request #22311: New design of system tests

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #22311:
URL: https://github.com/apache/airflow/pull/22311#discussion_r833972541



##########
File path: tests/system/utils/__init__.py
##########
@@ -0,0 +1,25 @@
+# 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.
+from airflow.utils.state import State
+
+
+def get_test_run(dag):
+    def test_run():
+        dag.clear(dag_run_state=State.NONE)
+        dag.run()
+
+    return test_run

Review comment:
       Basically it's the same reason why we wanted to extract "list(dag.task) >> watcher()" to separate function (but it ain't so easy and there it isn't worth it indeed.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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