From a6704d02ef13d934e7075390b16b3037edaab4d0 Mon Sep 17 00:00:00 2001 From: Ephraim Anierobi Date: Sat, 3 Dec 2022 10:47:16 +0100 Subject: [PATCH 001/158] Update 2.5.0 release date (#28081) (cherry picked from commit 24745c71f246b48b7981d7f4af93f097080c8afd) --- RELEASE_NOTES.rst | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/RELEASE_NOTES.rst b/RELEASE_NOTES.rst index 7d860a218c3da..f124693fb41b8 100644 --- a/RELEASE_NOTES.rst +++ b/RELEASE_NOTES.rst @@ -21,7 +21,7 @@ .. towncrier release notes start -Airflow 2.5.0 (2022-11-30) +Airflow 2.5.0 (2022-12-02) -------------------------- Significant Changes From 5012093007e3f3a89e394c8d48d07c0cceddf246 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Fri, 30 Dec 2022 00:29:19 -0800 Subject: [PATCH 002/158] Allow generation of connection URI to work when no conn type (#26765) Previously if get_uri was called it would fail with `NoneType not iterable`, because of the check `if '-' in conn_type`. (cherry picked from commit b124d6cac1180e21d21408f74729e77f75a46cb9) --- airflow/models/connection.py | 7 +++++-- tests/always/test_connection.py | 8 ++++++++ 2 files changed, 13 insertions(+), 2 deletions(-) diff --git a/airflow/models/connection.py b/airflow/models/connection.py index 73a284be9e5ef..8ed46e325d150 100644 --- a/airflow/models/connection.py +++ b/airflow/models/connection.py @@ -206,13 +206,16 @@ def _parse_from_uri(self, uri: str): def get_uri(self) -> str: """Return connection in URI format""" - if "_" in self.conn_type: + if self.conn_type and "_" in self.conn_type: self.log.warning( "Connection schemes (type: %s) shall not contain '_' according to RFC3986.", self.conn_type, ) - uri = f"{str(self.conn_type).lower().replace('_', '-')}://" + if self.conn_type: + uri = f"{self.conn_type.lower().replace('_', '-')}://" + else: + uri = "//" authority_block = "" if self.login is not None: diff --git a/tests/always/test_connection.py b/tests/always/test_connection.py index dac7bb110458f..750b36ef53cfc 100644 --- a/tests/always/test_connection.py +++ b/tests/always/test_connection.py @@ -735,3 +735,11 @@ def test_extra_warnings_non_json(self): def test_extra_warnings_non_dict_json(self): with pytest.warns(DeprecationWarning, match="not parse as a dictionary"): Connection(conn_id="test_extra", conn_type="none", extra='"hi"') + + def test_get_uri_no_conn_type(self): + # no conn type --> scheme-relative URI + assert Connection().get_uri() == "//" + # with host, still works + assert Connection(host="abc").get_uri() == "//abc" + # parsing back as conn still works + assert Connection(uri="//abc").host == "abc" From 39791b6b2dd41d6fa827f8b35ecdf8ac8690bca6 Mon Sep 17 00:00:00 2001 From: doiken <6147573+doiken@users.noreply.github.com> Date: Wed, 14 Dec 2022 17:51:23 +0900 Subject: [PATCH 003/158] fix: continue checking sla (#26968) Co-authored-by: doiken (cherry picked from commit e350b6d44d6d34a1759b9c2d925b7d27532436ab) --- airflow/dag_processing/processor.py | 2 +- tests/dag_processing/test_processor.py | 43 ++++++++++++++++++++++++++ 2 files changed, 44 insertions(+), 1 deletion(-) diff --git a/airflow/dag_processing/processor.py b/airflow/dag_processing/processor.py index 3580b0429f37d..02bb5eeebf399 100644 --- a/airflow/dag_processing/processor.py +++ b/airflow/dag_processing/processor.py @@ -425,7 +425,7 @@ def manage_slas(self, dag: DAG, session: Session = None) -> None: if next_info is None: break if (ti.dag_id, ti.task_id, next_info.logical_date) in recorded_slas_query: - break + continue if next_info.logical_date + task.sla < ts: sla_miss = SlaMiss( diff --git a/tests/dag_processing/test_processor.py b/tests/dag_processing/test_processor.py index c38b88d6e69ef..3601dd9a9a191 100644 --- a/tests/dag_processing/test_processor.py +++ b/tests/dag_processing/test_processor.py @@ -249,6 +249,49 @@ def test_dag_file_processor_sla_miss_doesnot_raise_integrity_error(self, mock_st # ti is successful thereby trying to insert a duplicate record. dag_file_processor.manage_slas(dag=dag, session=session) + @mock.patch("airflow.dag_processing.processor.Stats.incr") + def test_dag_file_processor_sla_miss_continue_checking_the_task_instances_after_recording_missing_sla( + self, mock_stats_incr, dag_maker + ): + """ + Test that the dag file processor continue checking subsequent task instances + even if the preceding task instance misses the sla ahead + """ + session = settings.Session() + + # Create a dag with a start of 3 days ago and sla of 1 day, + # so we have 2 missing slas + now = timezone.utcnow() + test_start_date = now - datetime.timedelta(days=3) + with dag_maker( + dag_id="test_sla_miss", + default_args={"start_date": test_start_date, "sla": datetime.timedelta(days=1)}, + ) as dag: + task = EmptyOperator(task_id="dummy") + + dag_maker.create_dagrun(execution_date=test_start_date, state=State.SUCCESS) + + session.merge(TaskInstance(task=task, execution_date=test_start_date, state="success")) + session.merge( + SlaMiss(task_id=task.task_id, dag_id=dag.dag_id, execution_date=now - datetime.timedelta(days=2)) + ) + session.flush() + + dag_file_processor = DagFileProcessor( + dag_ids=[], dag_directory=TEST_DAGS_FOLDER, log=mock.MagicMock() + ) + dag_file_processor.manage_slas(dag=dag, session=session) + sla_miss_count = ( + session.query(SlaMiss) + .filter( + SlaMiss.dag_id == dag.dag_id, + SlaMiss.task_id == task.task_id, + ) + .count() + ) + assert sla_miss_count == 2 + mock_stats_incr.assert_called_with("sla_missed") + @mock.patch("airflow.dag_processing.processor.Stats.incr") def test_dag_file_processor_sla_miss_callback_exception(self, mock_stats_incr, create_dummy_dag): """ From 5a565c44e924b68d16b1141f7b359013767a3f00 Mon Sep 17 00:00:00 2001 From: Andrey Anshin Date: Mon, 5 Dec 2022 01:22:51 +0300 Subject: [PATCH 004/158] Additional info about Segmentation Fault in LocalTaskJob (#27381) (cherry picked from commit 76f81cd4a7433b7eeddb863b2ae6ee59176cf816) --- airflow/jobs/local_task_job.py | 38 +++++++++++++++++++++++++++++++ tests/jobs/test_local_task_job.py | 37 +++++++++++++++++++++++++++++- 2 files changed, 74 insertions(+), 1 deletion(-) diff --git a/airflow/jobs/local_task_job.py b/airflow/jobs/local_task_job.py index fe7fc4a5613a7..07541ff3ca8f3 100644 --- a/airflow/jobs/local_task_job.py +++ b/airflow/jobs/local_task_job.py @@ -34,6 +34,36 @@ from airflow.utils.session import provide_session from airflow.utils.state import State +SIGSEGV_MESSAGE = """ +******************************************* Received SIGSEGV ******************************************* +SIGSEGV (Segmentation Violation) signal indicates Segmentation Fault error which refers to +an attempt by a program/library to write or read outside its allocated memory. + +In Python environment usually this signal refers to libraries which use low level C API. +Make sure that you use use right libraries/Docker Images +for your architecture (Intel/ARM) and/or Operational System (Linux/macOS). + +Suggested way to debug +====================== + - Set environment variable 'PYTHONFAULTHANDLER' to 'true'. + - Start airflow services. + - Restart failed airflow task. + - Check 'scheduler' and 'worker' services logs for additional traceback + which might contain information about module/library where actual error happen. + +Known Issues +============ + +Note: Only Linux-based distros supported as "Production" execution environment for Airflow. + +macOS +----- + 1. Due to limitations in Apple's libraries not every process might 'fork' safe. + One of the general error is unable to query the macOS system configuration for network proxies. + If your are not using a proxy you could disable it by set environment variable 'no_proxy' to '*'. + See: https://github.com/python/cpython/issues/58037 and https://bugs.python.org/issue30385#msg293958 +********************************************************************************************************""" + class LocalTaskJob(BaseJob): """LocalTaskJob runs a single task instance.""" @@ -83,6 +113,14 @@ def signal_handler(signum, frame): self.task_runner.terminate() self.handle_task_exit(128 + signum) + def segfault_signal_handler(signum, frame): + """Setting sigmentation violation signal handler""" + self.log.critical(SIGSEGV_MESSAGE) + self.task_runner.terminate() + self.handle_task_exit(128 + signum) + raise AirflowException("Segmentation Fault detected.") + + signal.signal(signal.SIGSEGV, segfault_signal_handler) signal.signal(signal.SIGTERM, signal_handler) if not self.task_instance.check_and_change_state_before_execution( diff --git a/tests/jobs/test_local_task_job.py b/tests/jobs/test_local_task_job.py index ecfb750fd12f8..f9661d869a2a6 100644 --- a/tests/jobs/test_local_task_job.py +++ b/tests/jobs/test_local_task_job.py @@ -18,6 +18,7 @@ from __future__ import annotations import datetime +import logging import os import re import signal @@ -34,7 +35,7 @@ from airflow import settings from airflow.exceptions import AirflowException from airflow.executors.sequential_executor import SequentialExecutor -from airflow.jobs.local_task_job import LocalTaskJob +from airflow.jobs.local_task_job import SIGSEGV_MESSAGE, LocalTaskJob from airflow.jobs.scheduler_job import SchedulerJob from airflow.models.dagbag import DagBag from airflow.models.serialized_dag import SerializedDagModel @@ -805,6 +806,40 @@ def task_function(ti): assert "Received SIGTERM. Terminating subprocesses" in caplog.text assert "Task exited with return code 143" in caplog.text + def test_process_sigsegv_error_message(self, caplog, dag_maker): + """Test that shows error if process failed with segmentation fault.""" + caplog.set_level(logging.CRITICAL, logger="local_task_job.py") + + def task_function(ti): + # pytest enable faulthandler by default unless `-p no:faulthandler` is given. + # It can not be disabled on the test level out of the box and + # that mean debug traceback would show in pytest output. + # For avoid this we disable it within the task which run in separate process. + import faulthandler + + if faulthandler.is_enabled(): + faulthandler.disable() + + while not ti.pid: + time.sleep(0.1) + + os.kill(psutil.Process(os.getpid()).ppid(), signal.SIGSEGV) + + with dag_maker(dag_id="test_segmentation_fault"): + task = PythonOperator( + task_id="test_sigsegv", + python_callable=task_function, + ) + dag_run = dag_maker.create_dagrun() + ti = TaskInstance(task=task, run_id=dag_run.run_id) + ti.refresh_from_db() + job = LocalTaskJob(task_instance=ti, ignore_ti_state=True, executor=SequentialExecutor()) + settings.engine.dispose() + with timeout(10): + with pytest.raises(AirflowException, match=r"Segmentation Fault detected"): + job.run() + assert SIGSEGV_MESSAGE in caplog.messages + @pytest.fixture() def clean_db_helper(): From f8db720253c14b5630218a1f18ccf1d80a40395c Mon Sep 17 00:00:00 2001 From: Jarek Potiuk Date: Sun, 27 Nov 2022 11:51:09 +0100 Subject: [PATCH 005/158] Handle DAG disappearing mid-flight when dag verification happens (#27720) When scheduler schedules a DAG and it disappears mid-flight by DagFileProcessor, it might lead to scheduler crashing in the verify_integrity method. This PR simply skips scheduling the DAG in such case rather than attempting to schedule it. Fixes: #27622 (cherry picked from commit 15e842da56d9b3a1c2f47f9dec7682a4230dbc41) --- airflow/jobs/scheduler_job.py | 17 +++++++++--- tests/jobs/test_scheduler_job.py | 44 ++++++++++++++++++++++++++++++++ 2 files changed, 57 insertions(+), 4 deletions(-) diff --git a/airflow/jobs/scheduler_job.py b/airflow/jobs/scheduler_job.py index 623bb948dc8c3..64bc9008f5702 100644 --- a/airflow/jobs/scheduler_job.py +++ b/airflow/jobs/scheduler_job.py @@ -1315,7 +1315,9 @@ def _schedule_dag_run( self.log.error("Execution date is in future: %s", dag_run.execution_date) return callback - self._verify_integrity_if_dag_changed(dag_run=dag_run, session=session) + if not self._verify_integrity_if_dag_changed(dag_run=dag_run, session=session): + self.log.warning("The DAG disappeared before verifying integrity: %s. Skipping.", dag_run.dag_id) + return callback # TODO[HA]: Rename update_state -> schedule_dag_run, ?? something else? schedulable_tis, callback_to_run = dag_run.update_state(session=session, execute_callbacks=False) if dag_run.state in State.finished: @@ -1331,20 +1333,27 @@ def _schedule_dag_run( return callback_to_run - def _verify_integrity_if_dag_changed(self, dag_run: DagRun, session: Session) -> None: - """Only run DagRun.verify integrity if Serialized DAG has changed since it is slow""" + def _verify_integrity_if_dag_changed(self, dag_run: DagRun, session: Session) -> bool: + """ + Only run DagRun.verify integrity if Serialized DAG has changed since it is slow. + + Return True if we determine that DAG still exists. + """ latest_version = SerializedDagModel.get_latest_version_hash(dag_run.dag_id, session=session) if dag_run.dag_hash == latest_version: self.log.debug("DAG %s not changed structure, skipping dagrun.verify_integrity", dag_run.dag_id) - return + return True dag_run.dag_hash = latest_version # Refresh the DAG dag_run.dag = self.dagbag.get_dag(dag_id=dag_run.dag_id, session=session) + if not dag_run.dag: + return False # Verify integrity also takes care of session.flush dag_run.verify_integrity(session=session) + return True def _send_dag_callbacks_to_processor(self, dag: DAG, callback: DagCallbackRequest | None = None) -> None: self._send_sla_callbacks_to_processor(dag) diff --git a/tests/jobs/test_scheduler_job.py b/tests/jobs/test_scheduler_job.py index a1d7fb31e0712..b0c8e016c8321 100644 --- a/tests/jobs/test_scheduler_job.py +++ b/tests/jobs/test_scheduler_job.py @@ -2699,6 +2699,50 @@ def test_verify_integrity_if_dag_changed(self, dag_maker): session.rollback() session.close() + def test_verify_integrity_if_dag_disappeared(self, dag_maker, caplog): + # CleanUp + with create_session() as session: + session.query(SerializedDagModel).filter( + SerializedDagModel.dag_id == "test_verify_integrity_if_dag_disappeared" + ).delete(synchronize_session=False) + + with dag_maker(dag_id="test_verify_integrity_if_dag_disappeared") as dag: + BashOperator(task_id="dummy", bash_command="echo hi") + + self.scheduler_job = SchedulerJob(subdir=os.devnull) + + session = settings.Session() + orm_dag = dag_maker.dag_model + assert orm_dag is not None + + self.scheduler_job = SchedulerJob(subdir=os.devnull) + self.scheduler_job.processor_agent = mock.MagicMock() + dag = self.scheduler_job.dagbag.get_dag("test_verify_integrity_if_dag_disappeared", session=session) + self.scheduler_job._create_dag_runs([orm_dag], session) + dag_id = dag.dag_id + drs = DagRun.find(dag_id=dag_id, session=session) + assert len(drs) == 1 + dr = drs[0] + + dag_version_1 = SerializedDagModel.get_latest_version_hash(dag_id, session=session) + assert dr.dag_hash == dag_version_1 + assert self.scheduler_job.dagbag.dags == {"test_verify_integrity_if_dag_disappeared": dag} + assert len(self.scheduler_job.dagbag.dags.get("test_verify_integrity_if_dag_disappeared").tasks) == 1 + + SerializedDagModel.remove_dag(dag_id=dag_id) + dag = self.scheduler_job.dagbag.dags[dag_id] + self.scheduler_job.dagbag.dags = MagicMock() + self.scheduler_job.dagbag.dags.get.side_effect = [dag, None] + session.flush() + with caplog.at_level(logging.WARNING): + callback = self.scheduler_job._schedule_dag_run(dr, session) + assert "The DAG disappeared before verifying integrity" in caplog.text + + assert callback is None + + session.rollback() + session.close() + @pytest.mark.need_serialized_dag def test_retry_still_in_executor(self, dag_maker): """ From 7a03761c821df9372fcb0cafe3054e32fa0006de Mon Sep 17 00:00:00 2001 From: Jeremy Monat Date: Sun, 27 Nov 2022 08:21:47 -0500 Subject: [PATCH 006/158] Note which versions of Python are supported (#27798) Copied from Prerequisites page which versions of Python that Airflow is tested with. Noted explicitly that 3.11 is not yet supported. As suggested in #27676. (cherry picked from commit 017ed9ac662d50b6e2767f297f36cb01bf79d825) --- docs/apache-airflow/start.rst | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docs/apache-airflow/start.rst b/docs/apache-airflow/start.rst index b164859d009c1..3e6a25e01c9d7 100644 --- a/docs/apache-airflow/start.rst +++ b/docs/apache-airflow/start.rst @@ -24,7 +24,8 @@ This quick start guide will help you bootstrap an Airflow standalone instance on .. note:: - Successful installation requires a Python 3 environment. + Successful installation requires a Python 3 environment. Starting with Airflow 2.3.0, Airflow is tested with Python 3.7, 3.8, 3.9, 3.10. Note that + Python 3.11 is not yet supported. Only ``pip`` installation is currently officially supported. From d174ef1de715225f4ff2e3d4e53a281d62adc478 Mon Sep 17 00:00:00 2001 From: Michael Petro <40223998+michaelmicheal@users.noreply.github.com> Date: Fri, 25 Nov 2022 16:11:23 -0500 Subject: [PATCH 007/158] dagrun, next_dagruns_to_examine, add MySQL index hint (#27821) * dagrun, next_dagruns_to_examine, add MySQL index hint * resolve static check errors (cherry picked from commit 5e45cb019995e8b80104b33da1c93eefae12d161) --- airflow/models/dagrun.py | 1 + 1 file changed, 1 insertion(+) diff --git a/airflow/models/dagrun.py b/airflow/models/dagrun.py index 205f2b82fe043..9e02f4775f549 100644 --- a/airflow/models/dagrun.py +++ b/airflow/models/dagrun.py @@ -305,6 +305,7 @@ def next_dagruns_to_examine( # TODO: Bake this query, it is run _A lot_ query = ( session.query(cls) + .with_hint(cls, "USE INDEX (idx_dag_run_running_dags)", dialect_name="mysql") .filter(cls.state == state, cls.run_type != DagRunType.BACKFILL_JOB) .join(DagModel, DagModel.dag_id == cls.dag_id) .filter(DagModel.is_paused == false(), DagModel.is_active == true()) From d34bd50683a91ab8fe5f64845752a429c45f44fa Mon Sep 17 00:00:00 2001 From: Stijn De Haes Date: Mon, 5 Dec 2022 02:12:36 +0100 Subject: [PATCH 008/158] Make sure we can get out of a faulty scheduler state (#27834) * Make sure we can get out of a faulty scheduler state This PR fixed the case where we have a faulty state in the database. The state that is fixed is that both the unmapped task instance and mapped task instances exist at the same time. So we have instances with map_index [-1, 0, 1]. The -1 task instances should be removed in this case. (cherry picked from commit 73d9352225bcc1f086b63f1c767d25b2d7c4c221) --- airflow/models/abstractoperator.py | 36 +++++++++++++------- airflow/models/dagrun.py | 9 ++--- tests/models/test_dagrun.py | 43 +++++++++++++++++++++++- tests/models/test_mappedoperator.py | 51 +++++++++++++++++++++++++++++ 4 files changed, 123 insertions(+), 16 deletions(-) diff --git a/airflow/models/abstractoperator.py b/airflow/models/abstractoperator.py index d5d6ad082f3d8..ba0a8954ae183 100644 --- a/airflow/models/abstractoperator.py +++ b/airflow/models/abstractoperator.py @@ -484,7 +484,6 @@ def expand_mapped_task(self, run_id: str, *, session: Session) -> tuple[Sequence # are not done yet, so the task can't fail yet. if not self.dag or not self.dag.partial: unmapped_ti.state = TaskInstanceState.UPSTREAM_FAILED - indexes_to_map: Iterable[int] = () elif total_length < 1: # If the upstream maps this to a zero-length value, simply mark # the unmapped task instance as SKIPPED (if needed). @@ -494,18 +493,33 @@ def expand_mapped_task(self, run_id: str, *, session: Session) -> tuple[Sequence total_length, ) unmapped_ti.state = TaskInstanceState.SKIPPED - indexes_to_map = () else: - # Otherwise convert this into the first mapped index, and create - # TaskInstance for other indexes. - unmapped_ti.map_index = 0 - self.log.debug("Updated in place to become %s", unmapped_ti) - all_expanded_tis.append(unmapped_ti) - indexes_to_map = range(1, total_length) - state = unmapped_ti.state - elif not total_length: + zero_index_ti_exists = ( + session.query(TaskInstance) + .filter( + TaskInstance.dag_id == self.dag_id, + TaskInstance.task_id == self.task_id, + TaskInstance.run_id == run_id, + TaskInstance.map_index == 0, + ) + .count() + > 0 + ) + if not zero_index_ti_exists: + # Otherwise convert this into the first mapped index, and create + # TaskInstance for other indexes. + unmapped_ti.map_index = 0 + self.log.debug("Updated in place to become %s", unmapped_ti) + all_expanded_tis.append(unmapped_ti) + session.flush() + else: + self.log.debug("Deleting the original task instance: %s", unmapped_ti) + session.delete(unmapped_ti) + state = unmapped_ti.state + + if total_length is None or total_length < 1: # Nothing to fixup. - indexes_to_map = () + indexes_to_map: Iterable[int] = () else: # Only create "missing" ones. current_max_mapping = ( diff --git a/airflow/models/dagrun.py b/airflow/models/dagrun.py index 9e02f4775f549..ae3a390653e01 100644 --- a/airflow/models/dagrun.py +++ b/airflow/models/dagrun.py @@ -768,7 +768,8 @@ def _expand_mapped_task_if_needed(ti: TI) -> Iterable[TI] | None: """Try to expand the ti, if needed. If the ti needs expansion, newly created task instances are - returned. The original ti is modified in-place and assigned the + returned as well as the original ti. + The original ti is also modified in-place and assigned the ``map_index`` of 0. If the ti does not need expansion, either because the task is not @@ -781,8 +782,7 @@ def _expand_mapped_task_if_needed(ti: TI) -> Iterable[TI] | None: except NotMapped: # Not a mapped task, nothing needed. return None if expanded_tis: - assert expanded_tis[0] is ti - return expanded_tis[1:] + return expanded_tis return () # Check dependencies. @@ -798,12 +798,13 @@ def _expand_mapped_task_if_needed(ti: TI) -> Iterable[TI] | None: # in the scheduler to ensure that the mapped task is correctly # expanded before executed. Also see _revise_map_indexes_if_mapped # docstring for additional information. + new_tis = None if schedulable.map_index < 0: new_tis = _expand_mapped_task_if_needed(schedulable) if new_tis is not None: additional_tis.extend(new_tis) expansion_happened = True - if schedulable.state in SCHEDULEABLE_STATES: + if new_tis is None and schedulable.state in SCHEDULEABLE_STATES: ready_tis.extend(self._revise_map_indexes_if_mapped(schedulable.task, session=session)) ready_tis.append(schedulable) diff --git a/tests/models/test_dagrun.py b/tests/models/test_dagrun.py index ffee25f5e89a9..34b67ba543eaf 100644 --- a/tests/models/test_dagrun.py +++ b/tests/models/test_dagrun.py @@ -29,7 +29,15 @@ from airflow import settings from airflow.callbacks.callback_requests import DagCallbackRequest from airflow.decorators import task, task_group -from airflow.models import DAG, DagBag, DagModel, DagRun, TaskInstance as TI, clear_task_instances +from airflow.models import ( + DAG, + DagBag, + DagModel, + DagRun, + TaskInstance, + TaskInstance as TI, + clear_task_instances, +) from airflow.models.baseoperator import BaseOperator from airflow.models.taskmap import TaskMap from airflow.operators.empty import EmptyOperator @@ -1285,6 +1293,39 @@ def task_2(arg2): ] +def test_mapped_literal_faulty_state_in_db(dag_maker, session): + """ + This test tries to recreate a faulty state in the database and checks if we can recover from it. + The state that happens is that there exists mapped task instances and the unmapped task instance. + So we have instances with map_index [-1, 0, 1]. The -1 task instances should be removed in this case. + """ + + with dag_maker(session=session) as dag: + + @task + def task_1(): + return [1, 2] + + @task + def task_2(arg2): + ... + + task_2.expand(arg2=task_1()) + + dr = dag_maker.create_dagrun() + ti = dr.get_task_instance(task_id="task_1") + ti.run() + decision = dr.task_instance_scheduling_decisions() + assert len(decision.schedulable_tis) == 2 + + # We insert a faulty record + session.add(TaskInstance(dag.get_task("task_2"), dr.execution_date, dr.run_id)) + session.flush() + + decision = dr.task_instance_scheduling_decisions() + assert len(decision.schedulable_tis) == 2 + + def test_mapped_literal_length_with_no_change_at_runtime_doesnt_call_verify_integrity(dag_maker, session): """ Test that when there's no change to mapped task indexes at runtime, the dagrun.verify_integrity diff --git a/tests/models/test_mappedoperator.py b/tests/models/test_mappedoperator.py index 1998563d70433..036a12fac4c2a 100644 --- a/tests/models/test_mappedoperator.py +++ b/tests/models/test_mappedoperator.py @@ -228,6 +228,57 @@ def test_expand_mapped_task_instance(dag_maker, session, num_existing_tis, expec assert indices == expected +def test_expand_mapped_task_failed_state_in_db(dag_maker, session): + """ + This test tries to recreate a faulty state in the database and checks if we can recover from it. + The state that happens is that there exists mapped task instances and the unmapped task instance. + So we have instances with map_index [-1, 0, 1]. The -1 task instances should be removed in this case. + """ + literal = [1, 2] + with dag_maker(session=session): + task1 = BaseOperator(task_id="op1") + mapped = MockOperator.partial(task_id="task_2").expand(arg2=task1.output) + + dr = dag_maker.create_dagrun() + + session.add( + TaskMap( + dag_id=dr.dag_id, + task_id=task1.task_id, + run_id=dr.run_id, + map_index=-1, + length=len(literal), + keys=None, + ) + ) + + for index in range(2): + # Give the existing TIs a state to make sure we don't change them + ti = TaskInstance(mapped, run_id=dr.run_id, map_index=index, state=TaskInstanceState.SUCCESS) + session.add(ti) + session.flush() + + indices = ( + session.query(TaskInstance.map_index, TaskInstance.state) + .filter_by(task_id=mapped.task_id, dag_id=mapped.dag_id, run_id=dr.run_id) + .order_by(TaskInstance.map_index) + .all() + ) + # Make sure we have the faulty state in the database + assert indices == [(-1, None), (0, "success"), (1, "success")] + + mapped.expand_mapped_task(dr.run_id, session=session) + + indices = ( + session.query(TaskInstance.map_index, TaskInstance.state) + .filter_by(task_id=mapped.task_id, dag_id=mapped.dag_id, run_id=dr.run_id) + .order_by(TaskInstance.map_index) + .all() + ) + # The -1 index should be cleaned up + assert indices == [(0, "success"), (1, "success")] + + def test_expand_mapped_task_instance_skipped_on_zero(dag_maker, session): with dag_maker(session=session): task1 = BaseOperator(task_id="op1") From dcb9d4797dd062072890a26c5b5065bc848dfcc6 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Fri, 2 Dec 2022 22:32:28 -0800 Subject: [PATCH 009/158] Don't log CLI actions if db not initialized (#27851) (cherry picked from commit d97a249404e2fa6854e182965274df83aa35bdb9) --- airflow/utils/cli_action_loggers.py | 16 ++++++++++++++-- 1 file changed, 14 insertions(+), 2 deletions(-) diff --git a/airflow/utils/cli_action_loggers.py b/airflow/utils/cli_action_loggers.py index 17a4c4f890b22..0c93148308fbc 100644 --- a/airflow/utils/cli_action_loggers.py +++ b/airflow/utils/cli_action_loggers.py @@ -97,6 +97,8 @@ def default_action_log(sub_command, user, task_id, dag_id, execution_date, host_ :param **_: other keyword arguments that is not being used by this function :return: None """ + from sqlalchemy.exc import OperationalError, ProgrammingError + from airflow.models.log import Log from airflow.utils import timezone from airflow.utils.session import create_session @@ -121,8 +123,18 @@ def default_action_log(sub_command, user, task_id, dag_id, execution_date, host_ } ], ) - except Exception as error: - logging.warning("Failed to log action with %s", error) + except (OperationalError, ProgrammingError) as e: + expected = [ + '"log" does not exist', # postgres + "no such table", # sqlite + "log' doesn't exist", # mysql + "Invalid object name 'log'", # mssql + ] + error_is_ok = e.args and any(x in e.args[0] for x in expected) + if not error_is_ok: + logging.warning("Failed to log action %s", e) + except Exception as e: + logging.warning("Failed to log action %s", e) __pre_exec_callbacks: list[Callable] = [] From cc02bdc63b001d94a81788c5c1592c0bdf37ddb7 Mon Sep 17 00:00:00 2001 From: jingkai Date: Sat, 3 Dec 2022 16:08:33 +0000 Subject: [PATCH 010/158] fix: current_state method on TaskInstance doesn't filter by map_index (#27898) (cherry picked from commit 51c70a5d6990a6af1188aab080ae2cbe7b935eb2) --- airflow/models/taskinstance.py | 14 +++++--------- tests/models/test_taskinstance.py | 23 +++++++++++++++++++++++ 2 files changed, 28 insertions(+), 9 deletions(-) diff --git a/airflow/models/taskinstance.py b/airflow/models/taskinstance.py index 8f08f5105b45e..1be51834120e8 100644 --- a/airflow/models/taskinstance.py +++ b/airflow/models/taskinstance.py @@ -736,17 +736,13 @@ def current_state(self, session: Session = NEW_SESSION) -> str: we use and looking up the state becomes part of the session, otherwise a new session is used. + sqlalchemy.inspect is used here to get the primary keys ensuring that if they change + it will not regress + :param session: SQLAlchemy ORM Session """ - return ( - session.query(TaskInstance.state) - .filter( - TaskInstance.dag_id == self.dag_id, - TaskInstance.task_id == self.task_id, - TaskInstance.run_id == self.run_id, - ) - .scalar() - ) + filters = (col == getattr(self, col.name) for col in inspect(TaskInstance).primary_key) + return session.query(TaskInstance.state).filter(*filters).scalar() @provide_session def error(self, session: Session = NEW_SESSION) -> None: diff --git a/tests/models/test_taskinstance.py b/tests/models/test_taskinstance.py index dd542666cdb75..a7ce20c87c12c 100644 --- a/tests/models/test_taskinstance.py +++ b/tests/models/test_taskinstance.py @@ -1863,6 +1863,29 @@ def test_outlet_datasets_failed(self, create_task_instance): # check that no dataset events were generated assert session.query(DatasetEvent).count() == 0 + def test_mapped_current_state(self, dag_maker): + with dag_maker(dag_id="test_mapped_current_state") as _: + from airflow.decorators import task + + @task() + def raise_an_exception(placeholder: int): + if placeholder == 0: + raise AirflowFailException("failing task") + else: + pass + + _ = raise_an_exception.expand(placeholder=[0, 1]) + + tis = dag_maker.create_dagrun(execution_date=timezone.utcnow()).task_instances + for task_instance in tis: + if task_instance.map_index == 0: + with pytest.raises(AirflowFailException): + task_instance.run() + assert task_instance.current_state() == TaskInstanceState.FAILED + else: + task_instance.run() + assert task_instance.current_state() == TaskInstanceState.SUCCESS + def test_outlet_datasets_skipped(self, create_task_instance): """ Verify that when we have an outlet dataset on a task, and the task From 5302cc25affad35f3379d270a57dd295cd28b3ba Mon Sep 17 00:00:00 2001 From: Tanel Kiis Date: Mon, 5 Dec 2022 03:17:09 +0200 Subject: [PATCH 011/158] Order TIs by map_index (#27904) (cherry picked from commit 820c5bbad9e3fb2c6eb19a51eafc800267746eae) --- airflow/jobs/scheduler_job.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airflow/jobs/scheduler_job.py b/airflow/jobs/scheduler_job.py index 64bc9008f5702..6fe8d3d710cf9 100644 --- a/airflow/jobs/scheduler_job.py +++ b/airflow/jobs/scheduler_job.py @@ -318,7 +318,7 @@ def _executable_task_instances_to_queued(self, max_tis: int, session: Session) - .filter(not_(DM.is_paused)) .filter(TI.state == TaskInstanceState.SCHEDULED) .options(selectinload("dag_model")) - .order_by(-TI.priority_weight, DR.execution_date) + .order_by(-TI.priority_weight, DR.execution_date, TI.map_index) ) if starved_pools: From 9733de8d3874b6eef2aa9ae4fed060f3c3c293da Mon Sep 17 00:00:00 2001 From: Ephraim Anierobi Date: Tue, 29 Nov 2022 08:34:12 +0100 Subject: [PATCH 012/158] Fix deadlock when chaining multiple empty mapped tasks (#27964) The fix here was to set changed_tis to True if there was an expansion. (cherry picked from commit f89ca94c3e60bfae888dfac60c7472d207f60f22) --- airflow/models/dagrun.py | 1 + tests/models/test_dagrun.py | 49 ++++++++++++++++++++++++++++++++++++- 2 files changed, 49 insertions(+), 1 deletion(-) diff --git a/airflow/models/dagrun.py b/airflow/models/dagrun.py index ae3a390653e01..c601193b272d9 100644 --- a/airflow/models/dagrun.py +++ b/airflow/models/dagrun.py @@ -716,6 +716,7 @@ def _filter_tis_and_exclude_removed(dag: DAG, tis: list[TI]) -> Iterable[TI]: # During expansion we may change some tis into non-schedulable # states, so we need to re-compute. if expansion_happened: + changed_tis = True new_unfinished_tis = [t for t in unfinished_tis if t.state in State.unfinished] finished_tis.extend(t for t in unfinished_tis if t.state in State.finished) unfinished_tis = new_unfinished_tis diff --git a/tests/models/test_dagrun.py b/tests/models/test_dagrun.py index 34b67ba543eaf..c08b548870add 100644 --- a/tests/models/test_dagrun.py +++ b/tests/models/test_dagrun.py @@ -1984,7 +1984,9 @@ def say_hi(): tis["say_hi"].state = TaskInstanceState.SUCCESS session.flush() - dr.update_state(session=session) + dr.update_state(session=session) # expands the mapped tasks + dr.update_state(session=session) # marks the task as skipped + dr.update_state(session=session) # marks dagrun as success assert dr.state == DagRunState.SUCCESS assert tis["add_one__1"].state == TaskInstanceState.SKIPPED @@ -2140,3 +2142,48 @@ def tg(va): ti.run() assert len(results) == 1 assert list(results[("t3", -1)]) == [["a", "b"], [4], ["z"]] + + +def test_mapping_against_empty_list(dag_maker, session): + with dag_maker(session=session): + + @task + def add_one(x: int): + return x + 1 + + @task + def say_hi(): + print("Hi") + + @task + def say_bye(): + print("Bye") + + added_values = add_one.expand(x=[]) + added_more_values = add_one.expand(x=[]) + added_more_more_values = add_one.expand(x=[]) + say_hi() >> say_bye() >> added_values + added_values >> added_more_values >> added_more_more_values + + dr: DagRun = dag_maker.create_dagrun() + + tis = {ti.task_id: ti for ti in dr.get_task_instances(session=session)} + say_hi_ti = tis["say_hi"] + say_bye_ti = tis["say_bye"] + say_hi_ti.state = TaskInstanceState.SUCCESS + say_bye_ti.state = TaskInstanceState.SUCCESS + session.merge(say_hi_ti) + session.merge(say_bye_ti) + session.flush() + + dr.update_state(session=session) + dr.update_state(session=session) # marks first empty mapped task as skipped + dr.update_state(session=session) # marks second empty mapped task as skipped + dr.update_state(session=session) # marks the third empty mapped task as skipped and dagrun as success + tis = {ti.task_id: ti.state for ti in dr.get_task_instances(session=session)} + assert tis["say_hi"] == TaskInstanceState.SUCCESS + assert tis["say_bye"] == TaskInstanceState.SUCCESS + assert tis["add_one"] == TaskInstanceState.SKIPPED + assert tis["add_one__1"] == TaskInstanceState.SKIPPED + assert tis["add_one__2"] == TaskInstanceState.SKIPPED + assert dr.state == State.SUCCESS From 5bb64304631bd89de3ceb79e4dabd84bba54b171 Mon Sep 17 00:00:00 2001 From: Igor Kholopov Date: Mon, 28 Nov 2022 22:56:20 +0100 Subject: [PATCH 013/158] Documentation for the LocalTaskJob return code counter (#27972) Co-authored-by: Igor Kholopov (cherry picked from commit 4a391150aae346d011f5016e2bcea0ed2f44d23b) --- .../logging-monitoring/metrics.rst | 88 ++++++++++--------- 1 file changed, 45 insertions(+), 43 deletions(-) diff --git a/docs/apache-airflow/logging-monitoring/metrics.rst b/docs/apache-airflow/logging-monitoring/metrics.rst index f97682bf8c97b..62410338d06e2 100644 --- a/docs/apache-airflow/logging-monitoring/metrics.rst +++ b/docs/apache-airflow/logging-monitoring/metrics.rst @@ -77,49 +77,51 @@ See :doc:`../modules_management` for details on how Python and Airflow manage mo Counters -------- -=========================================== ================================================================ -Name Description -=========================================== ================================================================ -``_start`` Number of started ```` job, ex. ``SchedulerJob``, ``LocalTaskJob`` -``_end`` Number of ended ```` job, ex. ``SchedulerJob``, ``LocalTaskJob`` -``_heartbeat_failure`` Number of failed Heartbeats for a ```` job, ex. ``SchedulerJob``, - ``LocalTaskJob`` -``operator_failures_`` Operator ```` failures -``operator_successes_`` Operator ```` successes -``ti_failures`` Overall task instances failures -``ti_successes`` Overall task instances successes -``previously_succeeded`` Number of previously succeeded task instances -``zombies_killed`` Zombie tasks killed -``scheduler_heartbeat`` Scheduler heartbeats -``dag_processing.processes`` Number of currently running DAG parsing processes -``dag_processing.processor_timeouts`` Number of file processors that have been killed due to taking too long -``dag_file_processor_timeouts`` (DEPRECATED) same behavior as ``dag_processing.processor_timeouts`` -``dag_processing.manager_stalls`` Number of stalled ``DagFileProcessorManager`` -``dag_file_refresh_error`` Number of failures loading any DAG files -``scheduler.tasks.killed_externally`` Number of tasks killed externally -``scheduler.orphaned_tasks.cleared`` Number of Orphaned tasks cleared by the Scheduler -``scheduler.orphaned_tasks.adopted`` Number of Orphaned tasks adopted by the Scheduler -``scheduler.critical_section_busy`` Count of times a scheduler process tried to get a lock on the critical - section (needed to send tasks to the executor) and found it locked by - another process. -``sla_missed`` Number of SLA misses -``sla_callback_notification_failure`` Number of failed SLA miss callback notification attempts -``sla_email_notification_failure`` Number of failed SLA miss email notification attempts -``ti.start..`` Number of started task in a given dag. Similar to _start but for task -``ti.finish...`` Number of completed task in a given dag. Similar to _end but for task -``dag.callback_exceptions`` Number of exceptions raised from DAG callbacks. When this happens, it - means DAG callback is not working. -``celery.task_timeout_error`` Number of ``AirflowTaskTimeout`` errors raised when publishing Task to Celery Broker. -``celery.execute_command.failure`` Number of non-zero exit code from Celery task. -``task_removed_from_dag.`` Number of tasks removed for a given dag (i.e. task no longer exists in DAG) -``task_restored_to_dag.`` Number of tasks restored for a given dag (i.e. task instance which was - previously in REMOVED state in the DB is added to DAG file) -``task_instance_created-`` Number of tasks instances created for a given Operator -``triggers.blocked_main_thread`` Number of triggers that blocked the main thread (likely due to not being - fully asynchronous) -``triggers.failed`` Number of triggers that errored before they could fire an event -``triggers.succeeded`` Number of triggers that have fired at least one event -=========================================== ================================================================ +====================================================================== ================================================================ +Name Description +====================================================================== ================================================================ +``_start`` Number of started ```` job, ex. ``SchedulerJob``, ``LocalTaskJob`` +``_end`` Number of ended ```` job, ex. ``SchedulerJob``, ``LocalTaskJob`` +``_heartbeat_failure`` Number of failed Heartbeats for a ```` job, ex. ``SchedulerJob``, + ``LocalTaskJob`` +``local_task_job.task_exit....`` Number of ``LocalTaskJob`` terminations with a ```` + while running a task ```` of a DAG ````. +``operator_failures_`` Operator ```` failures +``operator_successes_`` Operator ```` successes +``ti_failures`` Overall task instances failures +``ti_successes`` Overall task instances successes +``previously_succeeded`` Number of previously succeeded task instances +``zombies_killed`` Zombie tasks killed +``scheduler_heartbeat`` Scheduler heartbeats +``dag_processing.processes`` Number of currently running DAG parsing processes +``dag_processing.processor_timeouts`` Number of file processors that have been killed due to taking too long +``dag_file_processor_timeouts`` (DEPRECATED) same behavior as ``dag_processing.processor_timeouts`` +``dag_processing.manager_stalls`` Number of stalled ``DagFileProcessorManager`` +``dag_file_refresh_error`` Number of failures loading any DAG files +``scheduler.tasks.killed_externally`` Number of tasks killed externally +``scheduler.orphaned_tasks.cleared`` Number of Orphaned tasks cleared by the Scheduler +``scheduler.orphaned_tasks.adopted`` Number of Orphaned tasks adopted by the Scheduler +``scheduler.critical_section_busy`` Count of times a scheduler process tried to get a lock on the critical + section (needed to send tasks to the executor) and found it locked by + another process. +``sla_missed`` Number of SLA misses +``sla_callback_notification_failure`` Number of failed SLA miss callback notification attempts +``sla_email_notification_failure`` Number of failed SLA miss email notification attempts +``ti.start..`` Number of started task in a given dag. Similar to _start but for task +``ti.finish...`` Number of completed task in a given dag. Similar to _end but for task +``dag.callback_exceptions`` Number of exceptions raised from DAG callbacks. When this happens, it + means DAG callback is not working. +``celery.task_timeout_error`` Number of ``AirflowTaskTimeout`` errors raised when publishing Task to Celery Broker. +``celery.execute_command.failure`` Number of non-zero exit code from Celery task. +``task_removed_from_dag.`` Number of tasks removed for a given dag (i.e. task no longer exists in DAG) +``task_restored_to_dag.`` Number of tasks restored for a given dag (i.e. task instance which was + previously in REMOVED state in the DB is added to DAG file) +``task_instance_created-`` Number of tasks instances created for a given Operator +``triggers.blocked_main_thread`` Number of triggers that blocked the main thread (likely due to not being + fully asynchronous) +``triggers.failed`` Number of triggers that errored before they could fire an event +``triggers.succeeded`` Number of triggers that have fired at least one event +====================================================================== ================================================================ Gauges ------ From 9aa72e43b4b50fd572332d5a12eee8e9d68b9122 Mon Sep 17 00:00:00 2001 From: Tzu-ping Chung Date: Tue, 29 Nov 2022 16:23:15 +0800 Subject: [PATCH 014/158] Ignore Blackification commit from Git Blame (#27981) (cherry picked from commit 405087df3db8ca7c7333def64f8e96209117066a) --- .git-blame-ignore-revs | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/.git-blame-ignore-revs b/.git-blame-ignore-revs index 3073b9b98ea41..3cbfd2db16eb2 100644 --- a/.git-blame-ignore-revs +++ b/.git-blame-ignore-revs @@ -1,4 +1,5 @@ -# Black enabled on providers. +# Black enabled. +4e8f9cc8d02b29c325b8a5a76b4837671bdf5f68 fdd9b6f65b608c516b8a062b058972d9a45ec9e3 # PEP-563 (Postponed Evaluation of Annotations). From 2016d4a604335b2eba517b248560947487166c2e Mon Sep 17 00:00:00 2001 From: Brent Bovenzi Date: Wed, 30 Nov 2022 16:35:06 -0600 Subject: [PATCH 015/158] Simplify dataset subgraph logic (#27987) * fix merging connected dataset graphs * refactor graph calculation (cherry picked from commit f1c4c27e4aed79eef01f2873fab3a66af2aa3fa0) --- .../static/js/api/useDatasetDependencies.ts | 165 ++++++++---------- 1 file changed, 75 insertions(+), 90 deletions(-) diff --git a/airflow/www/static/js/api/useDatasetDependencies.ts b/airflow/www/static/js/api/useDatasetDependencies.ts index bd61ee9087a06..a4167c6f8fbdb 100644 --- a/airflow/www/static/js/api/useDatasetDependencies.ts +++ b/airflow/www/static/js/api/useDatasetDependencies.ts @@ -24,13 +24,16 @@ import ELK, { ElkShape, ElkExtendedEdge } from 'elkjs'; import { getMetaValue } from 'src/utils'; import type { DepEdge, DepNode } from 'src/types'; import type { NodeType } from 'src/datasets/Graph/Node'; -import { unionBy } from 'lodash'; interface DatasetDependencies { edges: DepEdge[]; nodes: DepNode[]; } +interface EdgeGroup { + edges: DepEdge[]; +} + interface GenerateProps { nodes: DepNode[]; edges: DepEdge[]; @@ -82,111 +85,92 @@ const generateGraph = ({ nodes, edges, font }: GenerateProps) => ({ edges: edges.map((e) => ({ id: `${e.source}-${e.target}`, sources: [e.source], targets: [e.target] })), }); -interface SeparateGraphsProps extends DatasetDependencies { - graphs: DatasetDependencies[]; +interface SeparateGraphsProps { + edges: DepEdge[]; + graphs: EdgeGroup[]; } -const graphIndicesToMerge: Record = {}; -const indicesToRemove: number[] = []; - // find the downstream graph of each upstream edge const findDownstreamGraph = ( - { edges, nodes, graphs = [] }: SeparateGraphsProps, -): DatasetDependencies[] => { - const newGraphs = [...graphs]; - let filteredEdges = [...edges]; - - graphs.forEach((g, i) => { - // find downstream edges - const downstreamEdges = edges.filter((e) => g.edges.some((ge) => ge.target === e.source)); - const downstreamNodes: DepNode[] = []; - - downstreamEdges.forEach((e) => { - const newNode = nodes.find((n) => n.id === e.target); - if (newNode) { - downstreamNodes.push(newNode); - - // check if the node already exists in a different graph - const existingGraphIndex = newGraphs - .findIndex(((ng) => ng.nodes.some((n) => n.id === newNode.id))); - - // mark if the graph needs to merge with another - if (existingGraphIndex > -1) { - indicesToRemove.push(existingGraphIndex); - graphIndicesToMerge[i] = [...(graphIndicesToMerge[i] || []), existingGraphIndex]; + { edges, graphs = [] }: SeparateGraphsProps, +): EdgeGroup[] => { + let unassignedEdges = [...edges]; + + const mergedGraphs = graphs + .reduce( + (newGraphs, graph) => { + const otherGroupIndex = newGraphs.findIndex( + (otherGroup) => otherGroup.edges.some( + (otherEdge) => graph.edges.some( + (edge) => edge.target === otherEdge.target, + ), + ), + ); + if (otherGroupIndex === -1) { + return [...newGraphs, graph]; } - // add node and edge to the graph - newGraphs[i] = { - nodes: [...newGraphs[i].nodes, newNode], - edges: [...newGraphs[i].edges, e], - }; - - // remove edge from edge list - filteredEdges = filteredEdges - .filter((fe) => !(fe.source === e.source && fe.target === e.target)); - } - }); - }); - - // once there are no more filtered edges left, merge relevant graphs - // we merge afterwards to make sure we captured all nodes + edges - if (!filteredEdges.length) { - Object.keys(graphIndicesToMerge).forEach((key) => { - const realKey = key as unknown as number; - const values = graphIndicesToMerge[realKey]; - values.forEach((v) => { - newGraphs[realKey] = { - nodes: unionBy(newGraphs[realKey].nodes, newGraphs[v].nodes, 'id'), - edges: [...newGraphs[realKey].edges, ...newGraphs[v].edges] - .filter((e, i, s) => ( - i === s.findIndex((t) => t.source === e.source && t.target === e.target) - )), - }; + const mergedEdges = [...newGraphs[otherGroupIndex].edges, ...graph.edges] + .filter((edge, edgeIndex, otherEdges) => ( + edgeIndex === otherEdges.findIndex( + (otherEdge) => otherEdge.source === edge.source && otherEdge.target === edge.target, + ) + )); + return [ + ...newGraphs.filter((_, newGraphIndex) => newGraphIndex !== otherGroupIndex), + { edges: mergedEdges }, + ]; + }, + [] as EdgeGroup[], + ) + .map((graph) => { + // find the next set of downstream edges and filter them out of the unassigned edges list + const downstreamEdges: DepEdge[] = []; + unassignedEdges = unassignedEdges.filter((edge) => { + const isDownstream = graph.edges.some((graphEdge) => graphEdge.target === edge.source); + if (isDownstream) downstreamEdges.push(edge); + return !isDownstream; }); + + return { + edges: [...graph.edges, ...downstreamEdges], + }; }); - return newGraphs.filter((g, i) => !indicesToRemove.some((j) => i === j)); - } - return findDownstreamGraph({ edges: filteredEdges, nodes, graphs: newGraphs }); + // recursively find downstream edges until there are no unassigned edges + return unassignedEdges.length + ? findDownstreamGraph({ edges: unassignedEdges, graphs: mergedGraphs }) + : mergedGraphs; }; // separate the list of nodes/edges into distinct dataset pipeline graphs const separateGraphs = ({ edges, nodes }: DatasetDependencies): DatasetDependencies[] => { - const separatedGraphs: DatasetDependencies[] = []; - let remainingEdges = [...edges]; - let remainingNodes = [...nodes]; - - edges.forEach((edge) => { - const isDownstream = edges.some((e) => e.target === edge.source); - - // if the edge is not downstream of anything, then start building the graph - if (!isDownstream) { - const connectedNodes = nodes.filter((n) => n.id === edge.source || n.id === edge.target); - - // check if one of the nodes is already connected to a separated graph - const nodesInUse = separatedGraphs - .findIndex((g) => g.nodes.some((n) => connectedNodes.some((nn) => nn.id === n.id))); - - if (nodesInUse > -1) { - // if one of the nodes is already in use, merge the graphs - const { nodes: existingNodes, edges: existingEdges } = separatedGraphs[nodesInUse]; - separatedGraphs[nodesInUse] = { nodes: unionBy(existingNodes, connectedNodes, 'id'), edges: [...existingEdges, edge] }; - } else { - // else just add the new separated graph - separatedGraphs.push({ nodes: connectedNodes, edges: [edge] }); - } - - // filter out used nodes and edges - remainingEdges = remainingEdges.filter((e) => e.source !== edge.source); - remainingNodes = remainingNodes.filter((n) => !connectedNodes.some((nn) => nn.id === n.id)); + const separatedGraphs: EdgeGroup[] = []; + const remainingEdges: DepEdge[] = []; + + edges.forEach((e) => { + // add a separate graph for each edge without an upstream + if (!edges.some((ee) => e.source === ee.target)) { + separatedGraphs.push({ edges: [e] }); + } else { + remainingEdges.push(e); } }); - if (remainingEdges.length) { - return findDownstreamGraph({ edges: remainingEdges, nodes, graphs: separatedGraphs }); - } - return separatedGraphs; + const edgeGraphs = findDownstreamGraph({ edges: remainingEdges, graphs: separatedGraphs }); + + // once all the edges are found, add the nodes + return edgeGraphs.map((eg) => { + const graphNodes = nodes.filter( + (n) => eg.edges.some( + (e) => e.target === n.id || e.source === n.id, + ), + ); + return ({ + edges: eg.edges, + nodes: graphNodes, + }); + }); }; const formatDependencies = async ({ edges, nodes }: DatasetDependencies) => { @@ -202,6 +186,7 @@ const formatDependencies = async ({ edges, nodes }: DatasetDependencies) => { elk.layout(generateGraph({ nodes: g.nodes, edges: g.edges, font })) ))); const fullGraph = await elk.layout(generateGraph({ nodes, edges, font })); + return { fullGraph, subGraphs, From 7646bf65d5a88717905f9c18e9910b379f946fd7 Mon Sep 17 00:00:00 2001 From: Ash Berlin-Taylor Date: Tue, 29 Nov 2022 18:23:12 +0000 Subject: [PATCH 016/158] Prevent double loading of providers from local paths (#27988) I noticed a case where the same local providers were loaded more than once, and it turned out to be caused by having `.` in the python search path. The fix for this is to canonicalize the path before looking for providers in it, and not searching in a path more than once. (cherry picked from commit 1a02ad9e1d73e4c33d48b25ec9781c54af91f748) --- airflow/providers_manager.py | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/airflow/providers_manager.py b/airflow/providers_manager.py index 51c5a632c5710..6088e3b37347e 100644 --- a/airflow/providers_manager.py +++ b/airflow/providers_manager.py @@ -491,7 +491,14 @@ def _discover_all_airflow_builtin_providers_from_local_sources(self) -> None: log.info("You have no providers installed.") return try: + seen = set() for path in airflow.providers.__path__: # type: ignore[attr-defined] + # The same path can appear in the __path__ twice, under non-normalized paths (ie. + # /path/to/repo/airflow/providers and /path/to/repo/./airflow/providers) + path = os.path.realpath(path) + if path in seen: + continue + seen.add(path) self._add_provider_info_from_local_source_files_on_path(path) except Exception as e: log.warning("Error when loading 'provider.yaml' files from airflow sources: %s", e) From be004cb2cafdfd34f06219b9035473e18cee575d Mon Sep 17 00:00:00 2001 From: Charles Machalow Date: Tue, 29 Nov 2022 20:47:57 -0800 Subject: [PATCH 017/158] Add documentation for [core] mp_start_method config (#27993) (cherry picked from commit 56b5f3f4eed6a48180e9d15ba9bb9664656077b1) --- airflow/config_templates/config.yml | 11 +++++++++++ airflow/config_templates/default_airflow.cfg | 8 ++++++++ 2 files changed, 19 insertions(+) diff --git a/airflow/config_templates/config.yml b/airflow/config_templates/config.yml index f3ae101785b92..009f6a48466c2 100644 --- a/airflow/config_templates/config.yml +++ b/airflow/config_templates/config.yml @@ -99,6 +99,17 @@ type: string example: ~ default: "16" + - name: mp_start_method + description: | + The name of the method used in order to start Python processes via the multiprocessing module. + This corresponds directly with the options available in the Python docs: + https://docs.python.org/3/library/multiprocessing.html#multiprocessing.set_start_method. + Must be one of the values returned by: + https://docs.python.org/3/library/multiprocessing.html#multiprocessing.get_all_start_methods. + version_added: "2.0.0" + type: string + default: ~ + example: "fork" - name: load_examples description: | Whether to load the DAG examples that ship with Airflow. It's good to diff --git a/airflow/config_templates/default_airflow.cfg b/airflow/config_templates/default_airflow.cfg index 277109452bb02..1cff5f8dbe75f 100644 --- a/airflow/config_templates/default_airflow.cfg +++ b/airflow/config_templates/default_airflow.cfg @@ -75,6 +75,14 @@ dags_are_paused_at_creation = True # which is defaulted as ``max_active_runs_per_dag``. max_active_runs_per_dag = 16 +# The name of the method used in order to start Python processes via the multiprocessing module. +# This corresponds directly with the options available in the Python docs: +# https://docs.python.org/3/library/multiprocessing.html#multiprocessing.set_start_method. +# Must be one of the values returned by: +# https://docs.python.org/3/library/multiprocessing.html#multiprocessing.get_all_start_methods. +# Example: mp_start_method = fork +# mp_start_method = + # Whether to load the DAG examples that ship with Airflow. It's good to # get started, but you probably want to set this to ``False`` in a production # environment From e573389fcdd1ff1d41afea35d5510e3d54ff25ca Mon Sep 17 00:00:00 2001 From: Dan Dascalescu Date: Tue, 29 Nov 2022 19:31:34 -0800 Subject: [PATCH 018/158] docs: copyedit DAG (#27995) (cherry picked from commit 893253a4c36634c17810dd6fd0a44fb1fc174939) --- docs/apache-airflow/concepts/dags.rst | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/apache-airflow/concepts/dags.rst b/docs/apache-airflow/concepts/dags.rst index 85aaf8229eb85..a7e0a75956b07 100644 --- a/docs/apache-airflow/concepts/dags.rst +++ b/docs/apache-airflow/concepts/dags.rst @@ -59,7 +59,7 @@ Or, you can use the ``@dag`` decorator to :ref:`turn a function into a DAG gener dag = generate_dag() -DAGs are nothing without :doc:`tasks` to run, and those will usually either come in the form of either :doc:`operators`, :doc:`sensors` or :doc:`taskflow`. +DAGs are nothing without :doc:`tasks` to run, and those will usually come in the form of either :doc:`operators`, :doc:`sensors` or :doc:`taskflow`. Task Dependencies @@ -96,7 +96,7 @@ And if you want to chain together dependencies, you can use ``chain``:: # You can also do it dynamically chain(*[EmptyOperator(task_id='op' + i) for i in range(1, 6)]) -Chain can also do *pairwise* dependencies for lists the same size (this is different to the *cross dependencies* done by ``cross_downstream``!):: +Chain can also do *pairwise* dependencies for lists the same size (this is different from the *cross dependencies* created by ``cross_downstream``!):: from airflow.models.baseoperator import chain From 3f3fd0f55aa0d29c11d2750015856a3a793298ca Mon Sep 17 00:00:00 2001 From: Jarek Potiuk Date: Wed, 30 Nov 2022 16:57:07 +0100 Subject: [PATCH 019/158] Handle bad zip files nicely when parsing DAGs. (#28011) (cherry picked from commit 8924cf1751e5190a1a7b4e33bb40de604b8b76b2) --- airflow/dag_processing/manager.py | 19 +++++++++++-------- 1 file changed, 11 insertions(+), 8 deletions(-) diff --git a/airflow/dag_processing/manager.py b/airflow/dag_processing/manager.py index d1450b7ed92e7..a41b737837226 100644 --- a/airflow/dag_processing/manager.py +++ b/airflow/dag_processing/manager.py @@ -741,14 +741,17 @@ def _refresh_dag_dir(self): dag_filelocs = [] for fileloc in self._file_paths: if not fileloc.endswith(".py") and zipfile.is_zipfile(fileloc): - with zipfile.ZipFile(fileloc) as z: - dag_filelocs.extend( - [ - os.path.join(fileloc, info.filename) - for info in z.infolist() - if might_contain_dag(info.filename, True, z) - ] - ) + try: + with zipfile.ZipFile(fileloc) as z: + dag_filelocs.extend( + [ + os.path.join(fileloc, info.filename) + for info in z.infolist() + if might_contain_dag(info.filename, True, z) + ] + ) + except zipfile.BadZipFile as err: + self.log.error("There was an err accessing %s, %s", fileloc, err) else: dag_filelocs.append(fileloc) From c5a0a6b179bf5b71e93c8d050b5907c25a73c5e8 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Fri, 2 Dec 2022 15:27:46 -0800 Subject: [PATCH 020/158] Use asserts instead of exceptions for executor not started (#28019) (cherry picked from commit abe3b318b525cca703cd6c0cda25af87cdf19b1b) --- airflow/executors/base_executor.py | 2 - airflow/executors/dask_executor.py | 32 +++++----- airflow/executors/kubernetes_executor.py | 81 ++++++++++++------------ airflow/executors/local_executor.py | 39 ++++++------ 4 files changed, 80 insertions(+), 74 deletions(-) diff --git a/airflow/executors/base_executor.py b/airflow/executors/base_executor.py index f9fac2fd3d117..0c9af11864f99 100644 --- a/airflow/executors/base_executor.py +++ b/airflow/executors/base_executor.py @@ -33,8 +33,6 @@ PARALLELISM: int = conf.getint("core", "PARALLELISM") -NOT_STARTED_MESSAGE = "The executor should be started first!" - QUEUEING_ATTEMPTS = 5 # Command to execute - list of strings diff --git a/airflow/executors/dask_executor.py b/airflow/executors/dask_executor.py index 41a560ddc8cda..a2c2c571630a6 100644 --- a/airflow/executors/dask_executor.py +++ b/airflow/executors/dask_executor.py @@ -25,14 +25,14 @@ from __future__ import annotations import subprocess -from typing import Any +from typing import TYPE_CHECKING, Any from distributed import Client, Future, as_completed from distributed.security import Security from airflow.configuration import conf from airflow.exceptions import AirflowException -from airflow.executors.base_executor import NOT_STARTED_MESSAGE, BaseExecutor, CommandType +from airflow.executors.base_executor import BaseExecutor, CommandType from airflow.models.taskinstance import TaskInstanceKey # queue="default" is a special case since this is the base config default queue name, @@ -78,15 +78,14 @@ def execute_async( queue: str | None = None, executor_config: Any | None = None, ) -> None: + if TYPE_CHECKING: + assert self.client self.validate_airflow_tasks_run_command(command) def airflow_run(): return subprocess.check_call(command, close_fds=True) - if not self.client: - raise AirflowException(NOT_STARTED_MESSAGE) - resources = None if queue not in _UNDEFINED_QUEUES: scheduler_info = self.client.scheduler_info() @@ -102,8 +101,9 @@ def airflow_run(): self.futures[future] = key # type: ignore def _process_future(self, future: Future) -> None: - if not self.futures: - raise AirflowException(NOT_STARTED_MESSAGE) + if TYPE_CHECKING: + assert self.futures + if future.done(): key = self.futures[future] if future.exception(): @@ -117,23 +117,25 @@ def _process_future(self, future: Future) -> None: self.futures.pop(future) def sync(self) -> None: - if self.futures is None: - raise AirflowException(NOT_STARTED_MESSAGE) + if TYPE_CHECKING: + assert self.futures + # make a copy so futures can be popped during iteration for future in self.futures.copy(): self._process_future(future) def end(self) -> None: - if not self.client: - raise AirflowException(NOT_STARTED_MESSAGE) - if self.futures is None: - raise AirflowException(NOT_STARTED_MESSAGE) + if TYPE_CHECKING: + assert self.client + assert self.futures + self.client.cancel(list(self.futures.keys())) for future in as_completed(self.futures.copy()): self._process_future(future) def terminate(self): - if self.futures is None: - raise AirflowException(NOT_STARTED_MESSAGE) + if TYPE_CHECKING: + assert self.futures + self.client.cancel(self.futures.keys()) self.end() diff --git a/airflow/executors/kubernetes_executor.py b/airflow/executors/kubernetes_executor.py index 72de3e050e956..16cf1b282f19a 100644 --- a/airflow/executors/kubernetes_executor.py +++ b/airflow/executors/kubernetes_executor.py @@ -30,7 +30,7 @@ import time from datetime import timedelta from queue import Empty, Queue -from typing import Any, Dict, Optional, Sequence, Tuple +from typing import TYPE_CHECKING, Any, Dict, Optional, Sequence, Tuple from kubernetes import client, watch from kubernetes.client import Configuration, models as k8s @@ -38,7 +38,7 @@ from urllib3.exceptions import ReadTimeoutError from airflow.exceptions import AirflowException, PodMutationHookException, PodReconciliationError -from airflow.executors.base_executor import NOT_STARTED_MESSAGE, BaseExecutor, CommandType +from airflow.executors.base_executor import BaseExecutor, CommandType from airflow.kubernetes import pod_generator from airflow.kubernetes.kube_client import get_kube_client from airflow.kubernetes.kube_config import KubeConfig @@ -96,9 +96,10 @@ def __init__( def run(self) -> None: """Performs watching""" + if TYPE_CHECKING: + assert self.scheduler_job_id + kube_client: client.CoreV1Api = get_kube_client() - if not self.scheduler_job_id: - raise AirflowException(NOT_STARTED_MESSAGE) while True: try: self.resource_version = self._run( @@ -463,10 +464,10 @@ def clear_not_launched_queued_tasks(self, session=None) -> None: is around, and if not, and there's no matching entry in our own task_queue, marks it for re-execution. """ - self.log.debug("Clearing tasks that have not been launched") - if not self.kube_client: - raise AirflowException(NOT_STARTED_MESSAGE) + if TYPE_CHECKING: + assert self.kube_client + self.log.debug("Clearing tasks that have not been launched") query = session.query(TaskInstance).filter( TaskInstance.state == State.QUEUED, TaskInstance.queued_by_job_id == self.job_id ) @@ -555,6 +556,9 @@ def execute_async( executor_config: Any | None = None, ) -> None: """Executes task asynchronously""" + if TYPE_CHECKING: + assert self.task_queue + if self.log.isEnabledFor(logging.DEBUG): self.log.debug("Add task %s with command %s, executor_config %s", key, command, executor_config) else: @@ -571,8 +575,6 @@ def execute_async( pod_template_file = executor_config.get("pod_template_file", None) else: pod_template_file = None - if not self.task_queue: - raise AirflowException(NOT_STARTED_MESSAGE) self.event_buffer[key] = (State.QUEUED, self.scheduler_job_id) self.task_queue.put((key, command, kube_executor_config, pod_template_file)) # We keep a temporary local record that we've handled this so we don't @@ -581,22 +583,18 @@ def execute_async( def sync(self) -> None: """Synchronize task state.""" + if TYPE_CHECKING: + assert self.scheduler_job_id + assert self.kube_scheduler + assert self.kube_config + assert self.result_queue + assert self.task_queue + assert self.event_scheduler + if self.running: self.log.debug("self.running: %s", self.running) if self.queued_tasks: self.log.debug("self.queued: %s", self.queued_tasks) - if not self.scheduler_job_id: - raise AirflowException(NOT_STARTED_MESSAGE) - if not self.kube_scheduler: - raise AirflowException(NOT_STARTED_MESSAGE) - if not self.kube_config: - raise AirflowException(NOT_STARTED_MESSAGE) - if not self.result_queue: - raise AirflowException(NOT_STARTED_MESSAGE) - if not self.task_queue: - raise AirflowException(NOT_STARTED_MESSAGE) - if not self.event_scheduler: - raise AirflowException(NOT_STARTED_MESSAGE) self.kube_scheduler.sync() last_resource_version = None @@ -671,8 +669,9 @@ def sync(self) -> None: def _check_worker_pods_pending_timeout(self): """Check if any pending worker pods have timed out""" - if not self.scheduler_job_id: - raise AirflowException(NOT_STARTED_MESSAGE) + if TYPE_CHECKING: + assert self.scheduler_job_id + timeout = self.kube_config.worker_pods_pending_timeout self.log.debug("Looking for pending worker pods older than %d seconds", timeout) @@ -706,10 +705,11 @@ def _check_worker_pods_pending_timeout(self): self.kube_scheduler.delete_pod(pod.metadata.name, pod.metadata.namespace) def _change_state(self, key: TaskInstanceKey, state: str | None, pod_id: str, namespace: str) -> None: + if TYPE_CHECKING: + assert self.kube_scheduler + if state != State.RUNNING: if self.kube_config.delete_worker_pods: - if not self.kube_scheduler: - raise AirflowException(NOT_STARTED_MESSAGE) if state != State.FAILED or self.kube_config.delete_worker_pods_on_failure: self.kube_scheduler.delete_pod(pod_id, namespace) self.log.info("Deleted pod: %s in namespace %s", str(key), str(namespace)) @@ -744,8 +744,9 @@ def adopt_launched_task( :param pod: V1Pod spec that we will patch with new label :param pod_ids: pod_ids we expect to patch. """ - if not self.scheduler_job_id: - raise AirflowException(NOT_STARTED_MESSAGE) + if TYPE_CHECKING: + assert self.scheduler_job_id + self.log.info("attempting to adopt pod %s", pod.metadata.name) pod.metadata.labels["airflow-worker"] = pod_generator.make_safe_label_value(self.scheduler_job_id) pod_id = annotations_to_key(pod.metadata.annotations) @@ -771,8 +772,9 @@ def _adopt_completed_pods(self, kube_client: client.CoreV1Api) -> None: :param kube_client: kubernetes client for speaking to kube API """ - if not self.scheduler_job_id: - raise AirflowException(NOT_STARTED_MESSAGE) + if TYPE_CHECKING: + assert self.scheduler_job_id + new_worker_id_label = pod_generator.make_safe_label_value(self.scheduler_job_id) kwargs = { "field_selector": "status.phase=Succeeded", @@ -792,8 +794,9 @@ def _adopt_completed_pods(self, kube_client: client.CoreV1Api) -> None: self.log.info("Failed to adopt pod %s. Reason: %s", pod.metadata.name, e) def _flush_task_queue(self) -> None: - if not self.task_queue: - raise AirflowException(NOT_STARTED_MESSAGE) + if TYPE_CHECKING: + assert self.task_queue + self.log.debug("Executor shutting down, task_queue approximate size=%d", self.task_queue.qsize()) while True: try: @@ -805,8 +808,9 @@ def _flush_task_queue(self) -> None: break def _flush_result_queue(self) -> None: - if not self.result_queue: - raise AirflowException(NOT_STARTED_MESSAGE) + if TYPE_CHECKING: + assert self.result_queue + self.log.debug("Executor shutting down, result_queue approximate size=%d", self.result_queue.qsize()) while True: try: @@ -833,12 +837,11 @@ def _flush_result_queue(self) -> None: def end(self) -> None: """Called when the executor shuts down""" - if not self.task_queue: - raise AirflowException(NOT_STARTED_MESSAGE) - if not self.result_queue: - raise AirflowException(NOT_STARTED_MESSAGE) - if not self.kube_scheduler: - raise AirflowException(NOT_STARTED_MESSAGE) + if TYPE_CHECKING: + assert self.task_queue + assert self.result_queue + assert self.kube_scheduler + self.log.info("Shutting down Kubernetes executor") self.log.debug("Flushing task_queue...") self._flush_task_queue() diff --git a/airflow/executors/local_executor.py b/airflow/executors/local_executor.py index 4fae990bc3d22..c2c82d863944a 100644 --- a/airflow/executors/local_executor.py +++ b/airflow/executors/local_executor.py @@ -31,13 +31,13 @@ from multiprocessing import Manager, Process from multiprocessing.managers import SyncManager from queue import Empty, Queue -from typing import Any, Optional, Tuple +from typing import TYPE_CHECKING, Any, Optional, Tuple from setproctitle import getproctitle, setproctitle from airflow import settings from airflow.exceptions import AirflowException -from airflow.executors.base_executor import NOT_STARTED_MESSAGE, PARALLELISM, BaseExecutor, CommandType +from airflow.executors.base_executor import PARALLELISM, BaseExecutor, CommandType from airflow.models.taskinstance import TaskInstanceKey, TaskInstanceStateType from airflow.utils.log.logging_mixin import LoggingMixin from airflow.utils.state import State @@ -245,8 +245,9 @@ def execute_async( :param queue: Name of the queue :param executor_config: configuration for the executor """ - if not self.executor.result_queue: - raise AirflowException(NOT_STARTED_MESSAGE) + if TYPE_CHECKING: + assert self.executor.result_queue + local_worker = LocalWorker(self.executor.result_queue, key=key, command=command) self.executor.workers_used += 1 self.executor.workers_active += 1 @@ -284,11 +285,11 @@ def __init__(self, executor: LocalExecutor): def start(self) -> None: """Starts limited parallelism implementation.""" - if not self.executor.manager: - raise AirflowException(NOT_STARTED_MESSAGE) + if TYPE_CHECKING: + assert self.executor.manager + assert self.executor.result_queue + self.queue = self.executor.manager.Queue() - if not self.executor.result_queue: - raise AirflowException(NOT_STARTED_MESSAGE) self.executor.workers = [ QueuedLocalWorker(self.queue, self.executor.result_queue) for _ in range(self.executor.parallelism) @@ -314,8 +315,9 @@ def execute_async( :param queue: name of the queue :param executor_config: configuration for the executor """ - if not self.queue: - raise AirflowException(NOT_STARTED_MESSAGE) + if TYPE_CHECKING: + assert self.queue + self.queue.put((key, command)) def sync(self): @@ -365,8 +367,8 @@ def execute_async( executor_config: Any | None = None, ) -> None: """Execute asynchronously.""" - if not self.impl: - raise AirflowException(NOT_STARTED_MESSAGE) + if TYPE_CHECKING: + assert self.impl self.validate_airflow_tasks_run_command(command) @@ -374,8 +376,9 @@ def execute_async( def sync(self) -> None: """Sync will get called periodically by the heartbeat method.""" - if not self.impl: - raise AirflowException(NOT_STARTED_MESSAGE) + if TYPE_CHECKING: + assert self.impl + self.impl.sync() def end(self) -> None: @@ -383,10 +386,10 @@ def end(self) -> None: Ends the executor. :return: """ - if not self.impl: - raise AirflowException(NOT_STARTED_MESSAGE) - if not self.manager: - raise AirflowException(NOT_STARTED_MESSAGE) + if TYPE_CHECKING: + assert self.impl + assert self.manager + self.log.info( "Shutting down LocalExecutor" "; waiting for running tasks to finish. Signal again if you don't want to wait." From b322572c6e7402b45bdd03e2a5d6ffecbf5eaa1a Mon Sep 17 00:00:00 2001 From: Jarek Potiuk Date: Fri, 2 Dec 2022 10:05:03 +0100 Subject: [PATCH 021/158] Add Andrey as committer (#28050) (cherry picked from commit ada91b686508218752fee176d29d63334364a7f2) --- docs/apache-airflow/project.rst | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/apache-airflow/project.rst b/docs/apache-airflow/project.rst index 4a3df1b7636d9..9c6fe09e250aa 100644 --- a/docs/apache-airflow/project.rst +++ b/docs/apache-airflow/project.rst @@ -39,6 +39,7 @@ Committers - Aizhamal Nurmamat kyzy (@aijamalnk) - Alex Guziel (@saguziel) - Alex Van Boxel (@alexvanboxel) +- Andrey Anshin (@taragolis) - Aneesh Joseph (@aneesh-joseph) - Arthur Wiedmer (@artwr) - Ash Berlin-Taylor (@ashb) From 51791a8cf49f1c359fbeda4f70f54fcc0ddb41da Mon Sep 17 00:00:00 2001 From: atrbgithub <14765982+atrbgithub@users.noreply.github.com> Date: Mon, 5 Dec 2022 18:51:57 +0000 Subject: [PATCH 022/158] Return list of tasks that will be changed (#28066) Ensure that when a user clicks on 'Queue up new tasks' a list of tasks that will be run is returned. (cherry picked from commit af29ff0a8aa133f0476bf6662e6c06c67de21dd5) --- airflow/www/views.py | 17 +++++++++--- tests/www/views/test_views_dagrun.py | 39 +++++++++++++++++++++++++++- 2 files changed, 52 insertions(+), 4 deletions(-) diff --git a/airflow/www/views.py b/airflow/www/views.py index a4126535cad64..a22ca795695fd 100644 --- a/airflow/www/views.py +++ b/airflow/www/views.py @@ -2269,7 +2269,8 @@ def _mark_dagrun_state_as_success(self, dag_id, dag_run_id, confirmed): return htmlsafe_json_dumps(details, separators=(",", ":")) - def _mark_dagrun_state_as_queued(self, dag_id: str, dag_run_id: str, confirmed: bool): + @provide_session + def _mark_dagrun_state_as_queued(self, dag_id: str, dag_run_id: str, confirmed: bool, session=None): if not dag_run_id: return {"status": "error", "message": "Invalid dag_run_id"} @@ -2278,13 +2279,23 @@ def _mark_dagrun_state_as_queued(self, dag_id: str, dag_run_id: str, confirmed: if not dag: return {"status": "error", "message": f"Cannot find DAG: {dag_id}"} - new_dag_state = set_dag_run_state_to_queued(dag=dag, run_id=dag_run_id, commit=confirmed) + set_dag_run_state_to_queued(dag=dag, run_id=dag_run_id, commit=confirmed) if confirmed: return {"status": "success", "message": "Marked the DagRun as queued."} else: - details = [str(t) for t in new_dag_state] + # Identify tasks that will be queued up to run when confirmed + all_task_ids = [task.task_id for task in dag.tasks] + + existing_tis = session.query(TaskInstance.task_id).filter( + TaskInstance.dag_id == dag.dag_id, + TaskInstance.run_id == dag_run_id, + ) + + completed_tis_ids = [task_id for task_id, in existing_tis] + tasks_with_no_state = list(set(all_task_ids) - set(completed_tis_ids)) + details = [str(t) for t in tasks_with_no_state] return htmlsafe_json_dumps(details, separators=(",", ":")) diff --git a/tests/www/views/test_views_dagrun.py b/tests/www/views/test_views_dagrun.py index 504ee7b09b956..e647bf23098ad 100644 --- a/tests/www/views/test_views_dagrun.py +++ b/tests/www/views/test_views_dagrun.py @@ -28,7 +28,7 @@ from airflow.utils.session import create_session from airflow.www.views import DagRunModelView from tests.test_utils.api_connexion_utils import create_user, delete_roles, delete_user -from tests.test_utils.www import check_content_in_response, client_with_login +from tests.test_utils.www import check_content_in_response, check_content_not_in_response, client_with_login from tests.www.views.test_views_tasks import _get_appbuilder_pk_string @@ -126,6 +126,31 @@ def running_dag_run(session): return dr +@pytest.fixture() +def completed_dag_run_with_missing_task(session): + dag = DagBag().get_dag("example_bash_operator") + execution_date = timezone.datetime(2016, 1, 9) + dr = dag.create_dagrun( + state="success", + execution_date=execution_date, + data_interval=(execution_date, execution_date), + run_id="test_dag_runs_action", + session=session, + ) + session.add(dr) + tis = [ + TaskInstance(dag.get_task("runme_0"), run_id=dr.run_id, state="success"), + TaskInstance(dag.get_task("runme_1"), run_id=dr.run_id, state="success"), + TaskInstance(dag.get_task("also_run_this"), run_id=dr.run_id, state="success"), + TaskInstance(dag.get_task("run_after_loop"), run_id=dr.run_id, state="success"), + TaskInstance(dag.get_task("this_will_skip"), run_id=dr.run_id, state="success"), + TaskInstance(dag.get_task("run_this_last"), run_id=dr.run_id, state="success"), + ] + session.bulk_save_objects(tis) + session.commit() + return dag, dr + + def test_delete_dagrun(session, admin_client, running_dag_run): composite_key = _get_appbuilder_pk_string(DagRunModelView, running_dag_run) assert session.query(DagRun).filter(DagRun.dag_id == running_dag_run.dag_id).count() == 1 @@ -235,3 +260,15 @@ def test_set_dag_runs_action_permission_denied(client_dr_without_dag_edit, runni follow_redirects=True, ) check_content_in_response(f"Access denied for dag_id {running_dag_run.dag_id}", resp) + + +def test_dag_runs_queue_new_tasks_action(session, admin_client, completed_dag_run_with_missing_task): + dag, dag_run = completed_dag_run_with_missing_task + resp = admin_client.post( + "/dagrun_queued", + data={"dag_id": dag.dag_id, "dag_run_id": dag_run.run_id, "confirmed": False}, + ) + + check_content_in_response("runme_2", resp) + check_content_not_in_response("runme_1", resp) + assert resp.status_code == 200 From dacc8fc2d9392271417b75cbec63c7be64d6539b Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Fri, 2 Dec 2022 13:33:05 -0800 Subject: [PATCH 023/158] Resolve false warning about calling conf.get on moved item (#28075) Calling `items` on config has the effect of calling `get` on each item. If we call `get` on a moved item, we will falsely get a warning letting us know to update our code. So, we suppress such warnings when iterating the config. (cherry picked from commit 2de613121b0d882bcf26ea944c91f3e915e3ae3f) --- airflow/configuration.py | 9 ++++++++- tests/core/test_configuration.py | 10 ++++++++++ 2 files changed, 18 insertions(+), 1 deletion(-) diff --git a/airflow/configuration.py b/airflow/configuration.py index 0bbaac485cf89..e6d3512c5bbd7 100644 --- a/airflow/configuration.py +++ b/airflow/configuration.py @@ -1267,7 +1267,14 @@ def _replace_section_config_with_display_sources( include_secret: bool, ): sect = config_sources.setdefault(section, OrderedDict()) - for (k, val) in config.items(section=section, raw=raw): + with warnings.catch_warnings(): + # calling `items` on config has the effect of calling `get` on each item + # if we call `get` on a moved item, we will falsely get a warning + # letting us know to update our code + # so we suppress such warnings here + warnings.simplefilter("ignore", category=FutureWarning) + items = config.items(section=section, raw=raw) + for (k, val) in items: deprecated_section, deprecated_key, _ = deprecated_options.get((section, k), (None, None, None)) if deprecated_section and deprecated_key: if source_name == "default": diff --git a/tests/core/test_configuration.py b/tests/core/test_configuration.py index 281aceaea4776..ca14fe14cb1c1 100644 --- a/tests/core/test_configuration.py +++ b/tests/core/test_configuration.py @@ -1369,3 +1369,13 @@ def test_conf_as_dict_when_deprecated_value_in_secrets_disabled_config( conf.read_dict(dictionary=cfg_dict) os.environ.clear() assert conf.get("database", "sql_alchemy_conn") == f"sqlite:///{HOME_DIR}/airflow/airflow.db" + + def test_should_not_falsely_emit_future_warning(self): + from airflow.configuration import AirflowConfigParser + + test_conf = AirflowConfigParser() + test_conf.read_dict({"scheduler": {"deactivate_stale_dags_interval": 60}}) + + with warnings.catch_warnings(record=True) as captured: + test_conf.as_dict() + assert captured == [] From 7c916b7ba74e4edd28b1b7cc0c0d73160953dbf2 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 5 Dec 2022 03:43:43 +0100 Subject: [PATCH 024/158] Bump decode-uri-component from 0.2.0 to 0.2.2 in /airflow/www (#28080) Bumps [decode-uri-component](https://github.com/SamVerschueren/decode-uri-component) from 0.2.0 to 0.2.2. - [Release notes](https://github.com/SamVerschueren/decode-uri-component/releases) - [Commits](https://github.com/SamVerschueren/decode-uri-component/compare/v0.2.0...v0.2.2) --- updated-dependencies: - dependency-name: decode-uri-component dependency-type: indirect ... Signed-off-by: dependabot[bot] Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> (cherry picked from commit c4cb5b3d35f1982df2144d645ef1d191b4b1df85) --- airflow/www/yarn.lock | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/airflow/www/yarn.lock b/airflow/www/yarn.lock index bafd63a368550..a30b381fe4945 100644 --- a/airflow/www/yarn.lock +++ b/airflow/www/yarn.lock @@ -4905,9 +4905,9 @@ decko@^1.2.0: integrity sha1-/UPHNelnuAEzBohKVvvmZZlraBc= decode-uri-component@^0.2.0: - version "0.2.0" - resolved "https://registry.yarnpkg.com/decode-uri-component/-/decode-uri-component-0.2.0.tgz#eb3913333458775cb84cd1a1fae062106bb87545" - integrity sha1-6zkTMzRYd1y4TNGh+uBiEGu4dUU= + version "0.2.2" + resolved "https://registry.yarnpkg.com/decode-uri-component/-/decode-uri-component-0.2.2.tgz#e69dbe25d37941171dd540e024c444cd5188e1e9" + integrity sha512-FqUYQ+8o158GyGTrMFJms9qh3CqTKvAqgqsTnkLI8sKu0028orqBhxNMFkFen0zGyg6epACD32pjVk58ngIErQ== dedent@^0.7.0: version "0.7.0" From 780a02838d8f379980520dee9aa89d2999498fa9 Mon Sep 17 00:00:00 2001 From: Jarek Potiuk Date: Sun, 4 Dec 2022 20:07:07 +0100 Subject: [PATCH 025/158] Add airflow.api.auth.backend.session to backend sessions in compose (#28094) The default backend seting in Docker compose needs to be updated to get rid of the warning introduced in #21640 Fixes: #28091 (cherry picked from commit 9d73830209aa1de03f2de6e6461b8416011c6ba6) --- docs/apache-airflow/howto/docker-compose/docker-compose.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/apache-airflow/howto/docker-compose/docker-compose.yaml b/docs/apache-airflow/howto/docker-compose/docker-compose.yaml index 30a1454edd638..99e675a42ca73 100644 --- a/docs/apache-airflow/howto/docker-compose/docker-compose.yaml +++ b/docs/apache-airflow/howto/docker-compose/docker-compose.yaml @@ -57,7 +57,7 @@ x-airflow-common: AIRFLOW__CORE__FERNET_KEY: '' AIRFLOW__CORE__DAGS_ARE_PAUSED_AT_CREATION: 'true' AIRFLOW__CORE__LOAD_EXAMPLES: 'true' - AIRFLOW__API__AUTH_BACKENDS: 'airflow.api.auth.backend.basic_auth' + AIRFLOW__API__AUTH_BACKENDS: 'airflow.api.auth.backend.basic_auth,airflow.api.auth.backend.session' _PIP_ADDITIONAL_REQUIREMENTS: ${_PIP_ADDITIONAL_REQUIREMENTS:-} volumes: - ./dags:/opt/airflow/dags From 4c15384e9e208fedabc39adb8654fddace0ec53e Mon Sep 17 00:00:00 2001 From: Jarek Potiuk Date: Sun, 4 Dec 2022 22:05:32 +0100 Subject: [PATCH 026/158] Improve handling of warnings in CI (#28096) Warnings printed in CI have been making it difficult to see what is going on (they were taking far too much space after the test results and GitHub CI UI rendered those multi-line warnings slowly. Also we did not have the right tools to capture the number and list of warnings that we should deal with. We are usign pytest-capture-warnings plugin now that improves the situation twofold: * warning summary printed by the plugin in the output is shorter - each warning is at most one line * the warning text files are uploaded as artifacts which make them usable in any kind of approach where we want to attempt to start an effort to remove all warnings (cherry picked from commit 16fddbae83d03c9b3e2d249cc8852fb006c65c3b) --- .github/actions/post_tests/action.yml | 6 ++++ .gitignore | 2 +- .rat-excludes | 3 ++ Dockerfile.ci | 3 ++ TESTING.rst | 3 ++ scripts/docker/entrypoint_ci.sh | 3 ++ scripts/in_container/filter_out_warnings.py | 31 +++++++++++++++++++ scripts/in_container/run_ci_tests.sh | 8 +++-- setup.py | 1 + .../slack/hooks/test_slack_webhook.py | 6 ++-- 10 files changed, 60 insertions(+), 6 deletions(-) create mode 100644 scripts/in_container/filter_out_warnings.py diff --git a/.github/actions/post_tests/action.yml b/.github/actions/post_tests/action.yml index 2e15b01ba8bdb..96bed9211bb5b 100644 --- a/.github/actions/post_tests/action.yml +++ b/.github/actions/post_tests/action.yml @@ -42,6 +42,12 @@ runs: name: coverage-${{env.JOB_ID}} path: ./files/coverage*.xml retention-days: 7 + - name: "Upload artifact for warnings" + uses: actions/upload-artifact@v3 + with: + name: test-warnings-${{env.JOB_ID}} + path: ./files/warnings-*.txt + retention-days: 7 - name: "Fix ownership" shell: bash run: breeze ci fix-ownership diff --git a/.gitignore b/.gitignore index 98c9dc2b239b2..edd1362f96c3e 100644 --- a/.gitignore +++ b/.gitignore @@ -14,10 +14,10 @@ unittests.db airflow/git_version airflow/www/static/coverage/ airflow/www/*.log - /logs/ airflow-webserver.pid standalone_admin_password.txt +warnings.txt # Byte-compiled / optimized / DLL files __pycache__/ diff --git a/.rat-excludes b/.rat-excludes index 56c81748371c5..1e16d61a67f5d 100644 --- a/.rat-excludes +++ b/.rat-excludes @@ -121,6 +121,9 @@ chart/values_schema.schema.json # Newsfragments are snippets that will be, eventually, consumed into RELEASE_NOTES newsfragments/* +# Warning file generated +warnings.txt + # Dev stuff tests/* scripts/* diff --git a/Dockerfile.ci b/Dockerfile.ci index 149bfc17f44d6..f2e64830fbbd6 100644 --- a/Dockerfile.ci +++ b/Dockerfile.ci @@ -802,6 +802,7 @@ fi set -u export RESULT_LOG_FILE="/files/test_result-${TEST_TYPE/\[*\]/}-${BACKEND}.xml" +export WARNINGS_FILE="/files/warnings-${TEST_TYPE/\[*\]/}-${BACKEND}.txt" EXTRA_PYTEST_ARGS=( "--verbosity=0" @@ -816,6 +817,8 @@ EXTRA_PYTEST_ARGS=( "--setup-timeout=${TEST_TIMEOUT}" "--execution-timeout=${TEST_TIMEOUT}" "--teardown-timeout=${TEST_TIMEOUT}" + "--output=${WARNINGS_FILE}" + "--disable-warnings" # Only display summary for non-expected case # f - failed # E - error diff --git a/TESTING.rst b/TESTING.rst index 26b95eec7d669..7520d58811618 100644 --- a/TESTING.rst +++ b/TESTING.rst @@ -51,6 +51,9 @@ Follow the guidelines when writing unit tests: * For new tests, use standard "asserts" of Python and ``pytest`` decorators/context managers for testing rather than ``unittest`` ones. See `pytest docs `_ for details. * Use a parameterized framework for tests that have variations in parameters. +* Use with ``pytest.warn`` to capture warnings rather than ``recwarn`` fixture. We are aiming for 0-warning in our + tests, so we run Pytest with ``--disable-warnings`` but instead we have ``pytest-capture-warnings`` plugin that + overrides ``recwarn`` fixture behaviour. **NOTE:** We plan to convert all unit tests to standard "asserts" semi-automatically, but this will be done later in Airflow 2.0 development phase. That will include setUp/tearDown/context managers and decorators. diff --git a/scripts/docker/entrypoint_ci.sh b/scripts/docker/entrypoint_ci.sh index 87bce601160b1..ca147c61c031d 100755 --- a/scripts/docker/entrypoint_ci.sh +++ b/scripts/docker/entrypoint_ci.sh @@ -245,6 +245,7 @@ fi set -u export RESULT_LOG_FILE="/files/test_result-${TEST_TYPE/\[*\]/}-${BACKEND}.xml" +export WARNINGS_FILE="/files/warnings-${TEST_TYPE/\[*\]/}-${BACKEND}.txt" EXTRA_PYTEST_ARGS=( "--verbosity=0" @@ -259,6 +260,8 @@ EXTRA_PYTEST_ARGS=( "--setup-timeout=${TEST_TIMEOUT}" "--execution-timeout=${TEST_TIMEOUT}" "--teardown-timeout=${TEST_TIMEOUT}" + "--output=${WARNINGS_FILE}" + "--disable-warnings" # Only display summary for non-expected case # f - failed # E - error diff --git a/scripts/in_container/filter_out_warnings.py b/scripts/in_container/filter_out_warnings.py new file mode 100644 index 0000000000000..df27eda7e6876 --- /dev/null +++ b/scripts/in_container/filter_out_warnings.py @@ -0,0 +1,31 @@ +#!/usr/bin/env python +# 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 __future__ import annotations + +import fileinput + +suppress = False + +for line in fileinput.input(): + if line.startswith("warnings summary:"): + suppress = True + if line.startswith("All Warning errors can be found in"): + suppress = False + if not suppress: + print(line, end="") diff --git a/scripts/in_container/run_ci_tests.sh b/scripts/in_container/run_ci_tests.sh index efab241fb7711..963375d77645d 100755 --- a/scripts/in_container/run_ci_tests.sh +++ b/scripts/in_container/run_ci_tests.sh @@ -23,10 +23,14 @@ echo "Starting the tests with those pytest arguments:" "${@}" echo set +e -pytest "${@}" - +pytest "${@}" | python "$( dirname "${BASH_SOURCE[0]}" )/filter_out_warnings.py" RES=$? +if [[ -f ${WARNINGS_FILE} ]]; then + echo "Number of warnings: $(wc -l "${WARNINGS_FILE}")" +fi + + if [[ ${RES} == "139" ]]; then echo "${COLOR_YELLOW}Sometimes Pytest fails at exiting with segfault, but all tests actually passed${COLOR_RESET}" echo "${COLOR_YELLOW}We should ignore such case. Checking if junitxml file ${RESULT_LOG_FILE} is there with 0 errors and failures${COLOR_RESET}" diff --git a/setup.py b/setup.py index e347380aa362f..1f0e8e39feaa3 100644 --- a/setup.py +++ b/setup.py @@ -391,6 +391,7 @@ def write_version(filename: str = str(AIRFLOW_SOURCES_ROOT / "airflow" / "git_ve # TODO: upgrade it and remove the limit "pytest~=6.0", "pytest-asyncio", + "pytest-capture-warnings", "pytest-cov", "pytest-instafail", # We should attempt to remove the limit when we upgrade Pytest diff --git a/tests/providers/slack/hooks/test_slack_webhook.py b/tests/providers/slack/hooks/test_slack_webhook.py index f687675891e7d..0c3527902f86f 100644 --- a/tests/providers/slack/hooks/test_slack_webhook.py +++ b/tests/providers/slack/hooks/test_slack_webhook.py @@ -526,13 +526,13 @@ def test_hook_send_by_hook_attributes(self, mock_hook_send_dict, deprecated_hook mock_hook_send_dict.assert_called_once_with(body=expected_body, headers=None) @mock.patch("airflow.providers.slack.hooks.slack_webhook.WebhookClient") - def test_hook_ignored_attributes(self, mock_webhook_client_cls, recwarn): + def test_hook_ignored_attributes(self, mock_webhook_client_cls): """Test hook constructor warn users about ignored attributes.""" mock_webhook_client = mock_webhook_client_cls.return_value mock_webhook_client_send_dict = mock_webhook_client.send_dict mock_webhook_client_send_dict.return_value = MOCK_WEBHOOK_RESPONSE - - hook = SlackWebhookHook(slack_webhook_conn_id=TEST_CONN_ID, link_names="test-value") + with pytest.warns(UserWarning) as recwarn: + hook = SlackWebhookHook(slack_webhook_conn_id=TEST_CONN_ID, link_names="test-value") assert len(recwarn) == 2 assert str(recwarn.pop(UserWarning).message).startswith( "`link_names` has no affect, if you want to mention user see:" From b4efb3086fcb02fbb813d372f3b9afca0a315c9c Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Tue, 6 Dec 2022 00:09:48 -0800 Subject: [PATCH 027/158] Don't emit FutureWarning when code not calling old key (#28109) Tried fixing this before using simplefilter but it doesn't work when application threaded. See here https://docs.python.org/3/library/warnings.html#temporarily-suppressing-warnings. It was tricky to solve. When "actually" reading the values we call super().get. You'd think this would not invoke airflow config parser `get` right? But because of config parser interpolation, ultimately, it invokes `get` again on airflow config parser. So, we can manipulate an attr to signal when we are _accessing_ a deprecated key but only because we're retrieving the backcompat val (not because the user requested it). Additionally we have to handle the case where `items` is called (e.g. from within as_dict) which calls `get` for every option you have in your config. (cherry picked from commit 27a84637b3ab9b5f7d0e93252ef93656bc6907ea) --- airflow/configuration.py | 64 ++++++++++++++++++-------------- tests/core/test_configuration.py | 56 +++++++++++++++++++++++++++- 2 files changed, 91 insertions(+), 29 deletions(-) diff --git a/airflow/configuration.py b/airflow/configuration.py index e6d3512c5bbd7..2fd96d1710738 100644 --- a/airflow/configuration.py +++ b/airflow/configuration.py @@ -33,7 +33,7 @@ # Ignored Mypy on configparser because it thinks the configparser module has no _UNSET attribute from configparser import _UNSET, ConfigParser, NoOptionError, NoSectionError # type: ignore -from contextlib import suppress +from contextlib import contextmanager, suppress from json.decoder import JSONDecodeError from re import Pattern from typing import IO, Any, Dict, Iterable, Tuple, Union @@ -334,6 +334,7 @@ def __init__(self, default_config: str | None = None, *args, **kwargs): del self.deprecated_values["logging"]["log_filename_template"] self.is_validated = False + self._suppress_future_warnings = False def validate(self): self._validate_config_dependencies() @@ -560,8 +561,7 @@ def get(self, section: str, key: str, **kwargs) -> str | None: # type: ignore[o def get(self, section: str, key: str, **kwargs) -> str | None: # type: ignore[override, misc] section = str(section).lower() key = str(key).lower() - - issue_warning = True + warning_emitted = False deprecated_section: str | None deprecated_key: str | None @@ -569,18 +569,19 @@ def get(self, section: str, key: str, **kwargs) -> str | None: # type: ignore[o if section in self.inversed_deprecated_sections: deprecated_section, deprecated_key = (section, key) section = self.inversed_deprecated_sections[section] - warnings.warn( - f"The config section [{deprecated_section}] has been renamed to " - f"[{section}]. Please update your `conf.get*` call to use the new name", - FutureWarning, - stacklevel=2, - ) + if not self._suppress_future_warnings: + warnings.warn( + f"The config section [{deprecated_section}] has been renamed to " + f"[{section}]. Please update your `conf.get*` call to use the new name", + FutureWarning, + stacklevel=2, + ) # Don't warn about individual rename if the whole section is renamed - issue_warning = False + warning_emitted = True elif (section, key) in self.inversed_deprecated_options: # Handle using deprecated section/key instead of the new section/key new_section, new_key = self.inversed_deprecated_options[(section, key)] - if issue_warning: + if not self._suppress_future_warnings and not warning_emitted: warnings.warn( f"section/key [{section}/{key}] has been deprecated, you should use" f"[{new_section}/{new_key}] instead. Please update your `conf.get*` call to use the " @@ -588,7 +589,7 @@ def get(self, section: str, key: str, **kwargs) -> str | None: # type: ignore[o FutureWarning, stacklevel=2, ) - issue_warning = False + warning_emitted = True deprecated_section, deprecated_key = section, key section, key = (new_section, new_key) elif section in self.deprecated_sections: @@ -602,28 +603,28 @@ def get(self, section: str, key: str, **kwargs) -> str | None: # type: ignore[o # first check environment variables option = self._get_environment_variables( - deprecated_key, deprecated_section, key, section, issue_warning=issue_warning + deprecated_key, deprecated_section, key, section, issue_warning=not warning_emitted ) if option is not None: return option # ...then the config file option = self._get_option_from_config_file( - deprecated_key, deprecated_section, key, kwargs, section, issue_warning=issue_warning + deprecated_key, deprecated_section, key, kwargs, section, issue_warning=not warning_emitted ) if option is not None: return option # ...then commands option = self._get_option_from_commands( - deprecated_key, deprecated_section, key, section, issue_warning=issue_warning + deprecated_key, deprecated_section, key, section, issue_warning=not warning_emitted ) if option is not None: return option # ...then from secret backends option = self._get_option_from_secrets( - deprecated_key, deprecated_section, key, section, issue_warning=issue_warning + deprecated_key, deprecated_section, key, section, issue_warning=not warning_emitted ) if option is not None: return option @@ -648,7 +649,8 @@ def _get_option_from_secrets( if option: return option if deprecated_section and deprecated_key: - option = self._get_secret_option(deprecated_section, deprecated_key) + with self.suppress_future_warnings(): + option = self._get_secret_option(deprecated_section, deprecated_key) if option: if issue_warning: self._warn_deprecate(section, key, deprecated_section, deprecated_key) @@ -667,7 +669,8 @@ def _get_option_from_commands( if option: return option if deprecated_section and deprecated_key: - option = self._get_cmd_option(deprecated_section, deprecated_key) + with self.suppress_future_warnings(): + option = self._get_cmd_option(deprecated_section, deprecated_key) if option: if issue_warning: self._warn_deprecate(section, key, deprecated_section, deprecated_key) @@ -691,7 +694,8 @@ def _get_option_from_config_file( if super().has_option(deprecated_section, deprecated_key): if issue_warning: self._warn_deprecate(section, key, deprecated_section, deprecated_key) - return expand_env_var(super().get(deprecated_section, deprecated_key, **kwargs)) + with self.suppress_future_warnings(): + return expand_env_var(super().get(deprecated_section, deprecated_key, **kwargs)) return None def _get_environment_variables( @@ -706,7 +710,8 @@ def _get_environment_variables( if option is not None: return option if deprecated_section and deprecated_key: - option = self._get_env_var_option(deprecated_section, deprecated_key) + with self.suppress_future_warnings(): + option = self._get_env_var_option(deprecated_section, deprecated_key) if option is not None: if issue_warning: self._warn_deprecate(section, key, deprecated_section, deprecated_key) @@ -1252,6 +1257,13 @@ def _deprecated_variable_secret_is_set(deprecated_section: str, deprecated_key: is not None ) + @contextmanager + def suppress_future_warnings(self): + suppress_future_warnings = self._suppress_future_warnings + self._suppress_future_warnings = True + yield self + self._suppress_future_warnings = suppress_future_warnings + @staticmethod def _replace_section_config_with_display_sources( config: ConfigParser, @@ -1267,14 +1279,12 @@ def _replace_section_config_with_display_sources( include_secret: bool, ): sect = config_sources.setdefault(section, OrderedDict()) - with warnings.catch_warnings(): - # calling `items` on config has the effect of calling `get` on each item - # if we call `get` on a moved item, we will falsely get a warning - # letting us know to update our code - # so we suppress such warnings here - warnings.simplefilter("ignore", category=FutureWarning) + if isinstance(config, AirflowConfigParser): + with config.suppress_future_warnings(): + items = config.items(section=section, raw=raw) + else: items = config.items(section=section, raw=raw) - for (k, val) in items: + for k, val in items: deprecated_section, deprecated_key, _ = deprecated_options.get((section, k), (None, None, None)) if deprecated_section and deprecated_key: if source_name == "default": diff --git a/tests/core/test_configuration.py b/tests/core/test_configuration.py index ca14fe14cb1c1..6cbe0e9076988 100644 --- a/tests/core/test_configuration.py +++ b/tests/core/test_configuration.py @@ -30,6 +30,7 @@ from unittest import mock import pytest +from pytest import param from airflow import configuration from airflow.configuration import ( @@ -1370,7 +1371,7 @@ def test_conf_as_dict_when_deprecated_value_in_secrets_disabled_config( os.environ.clear() assert conf.get("database", "sql_alchemy_conn") == f"sqlite:///{HOME_DIR}/airflow/airflow.db" - def test_should_not_falsely_emit_future_warning(self): + def test_as_dict_should_not_falsely_emit_future_warning(self): from airflow.configuration import AirflowConfigParser test_conf = AirflowConfigParser() @@ -1378,4 +1379,55 @@ def test_should_not_falsely_emit_future_warning(self): with warnings.catch_warnings(record=True) as captured: test_conf.as_dict() - assert captured == [] + for w in captured: # only one expected + assert "deactivate_stale_dags_interval option in [scheduler] has been renamed" in str(w.message) + + def test_suppress_future_warnings_no_future_warning(self): + from airflow.configuration import AirflowConfigParser + + test_conf = AirflowConfigParser() + test_conf.read_dict({"scheduler": {"deactivate_stale_dags_interval": 60}}) + with warnings.catch_warnings(record=True) as captured: + test_conf.items("scheduler") + assert len(captured) == 1 + c = captured[0] + assert c.category == FutureWarning + assert ( + "you should use[scheduler/parsing_cleanup_interval] " + "instead. Please update your `conf.get*`" in str(c.message) + ) + with warnings.catch_warnings(record=True) as captured: + with test_conf.suppress_future_warnings(): + test_conf.items("scheduler") + assert len(captured) == 1 + c = captured[0] + assert c.category == DeprecationWarning + assert ( + "deactivate_stale_dags_interval option in [scheduler] " + "has been renamed to parsing_cleanup_interval" in str(c.message) + ) + + @pytest.mark.parametrize( + "key", + [ + param("deactivate_stale_dags_interval", id="old"), + param("parsing_cleanup_interval", id="new"), + ], + ) + def test_future_warning_only_for_code_ref(self, key): + from airflow.configuration import AirflowConfigParser + + old_val = "deactivate_stale_dags_interval" + test_conf = AirflowConfigParser() + test_conf.read_dict({"scheduler": {old_val: 60}}) # config has old value + with warnings.catch_warnings(record=True) as captured: + test_conf.get("scheduler", str(key)) # could be old or new value + + w = captured.pop() + assert "the old setting has been used, but please update" in str(w.message) + assert w.category == DeprecationWarning + # only if we use old value, do we also get a warning about code update + if key == old_val: + w = captured.pop() + assert "your `conf.get*` call to use the new name" in str(w.message) + assert w.category == FutureWarning From 2bc0a0da4326647b24b5cda4e4811a14c7ce65db Mon Sep 17 00:00:00 2001 From: Kosteev Eugene Date: Wed, 7 Dec 2022 07:48:00 +0200 Subject: [PATCH 028/158] Make BaseJob.most_recent_job favor "running" jobs (#28119) Co-authored-by: Tzu-ping Chung (cherry picked from commit 56c0871dce2fb2b7ed2252e4b2d1d8d5d0c07c58) --- airflow/jobs/base_job.py | 15 +++++++++++++-- tests/jobs/test_base_job.py | 20 ++++++++++++++++++++ 2 files changed, 33 insertions(+), 2 deletions(-) diff --git a/airflow/jobs/base_job.py b/airflow/jobs/base_job.py index e3d79d67b914d..d695615a5d8ba 100644 --- a/airflow/jobs/base_job.py +++ b/airflow/jobs/base_job.py @@ -19,7 +19,7 @@ from time import sleep -from sqlalchemy import Column, Index, Integer, String +from sqlalchemy import Column, Index, Integer, String, case from sqlalchemy.exc import OperationalError from sqlalchemy.orm import backref, foreign, relationship from sqlalchemy.orm.session import make_transient @@ -125,12 +125,23 @@ def most_recent_job(cls, session=None) -> BaseJob | None: Return the most recent job of this type, if any, based on last heartbeat received. + Jobs in "running" state take precedence over others to make sure alive + job is returned if it is available. This method should be called on a subclass (i.e. on SchedulerJob) to return jobs of that type. :param session: Database session """ - return session.query(cls).order_by(cls.latest_heartbeat.desc()).limit(1).first() + return ( + session.query(cls) + .order_by( + # Put "running" jobs at the front. + case({State.RUNNING: 0}, value=cls.state, else_=1), + cls.latest_heartbeat.desc(), + ) + .limit(1) + .first() + ) def is_alive(self, grace_multiplier=2.1): """ diff --git a/tests/jobs/test_base_job.py b/tests/jobs/test_base_job.py index 66715a9d635e3..6b0fbf7489f76 100644 --- a/tests/jobs/test_base_job.py +++ b/tests/jobs/test_base_job.py @@ -108,6 +108,26 @@ def test_most_recent_job(self): session.rollback() + def test_most_recent_job_running_precedence(self): + with create_session() as session: + old_running_state_job = MockJob(None, heartrate=10) + old_running_state_job.latest_heartbeat = timezone.utcnow() + old_running_state_job.state = State.RUNNING + new_failed_state_job = MockJob(None, heartrate=10) + new_failed_state_job.latest_heartbeat = timezone.utcnow() + new_failed_state_job.state = State.FAILED + new_null_state_job = MockJob(None, heartrate=10) + new_null_state_job.latest_heartbeat = timezone.utcnow() + new_null_state_job.state = None + session.add(old_running_state_job) + session.add(new_failed_state_job) + session.add(new_null_state_job) + session.flush() + + assert MockJob.most_recent_job(session=session) == old_running_state_job + + session.rollback() + def test_is_alive(self): job = MockJob(None, heartrate=10, state=State.RUNNING) assert job.is_alive() is True From a7bcb4ba7e127da172ed5d1edaf9e28f1c704a95 Mon Sep 17 00:00:00 2001 From: Brent Bovenzi Date: Tue, 6 Dec 2022 10:28:18 -0600 Subject: [PATCH 029/158] Improve run/task grid view actions (#28130) * improve run/task changes from grid * fix confirm return type (cherry picked from commit a0c85022b45c355f50ee2f820b7a32fd97c275e7) --- airflow/www/static/js/api/useClearRun.ts | 11 ++++++----- airflow/www/static/js/api/useClearTask.ts | 12 +++++++----- airflow/www/static/js/api/useMarkFailedRun.ts | 10 ++++++---- airflow/www/static/js/api/useMarkSuccessRun.ts | 10 ++++++---- airflow/www/static/js/api/useQueueRun.ts | 10 ++++++---- airflow/www/static/js/components/ConfirmDialog.tsx | 11 ++++++++--- .../www/static/js/dag/details/dagRun/ClearRun.tsx | 6 +++--- .../static/js/dag/details/dagRun/MarkFailedRun.tsx | 6 +++--- .../static/js/dag/details/dagRun/MarkSuccessRun.tsx | 6 +++--- .../www/static/js/dag/details/dagRun/QueueRun.tsx | 6 +++--- .../dag/details/taskInstance/taskActions/Clear.tsx | 6 +++--- .../details/taskInstance/taskActions/MarkFailed.tsx | 2 +- .../details/taskInstance/taskActions/MarkSuccess.tsx | 2 +- 13 files changed, 56 insertions(+), 42 deletions(-) diff --git a/airflow/www/static/js/api/useClearRun.ts b/airflow/www/static/js/api/useClearRun.ts index b2fa85eea2303..c7b32b79fb2a0 100644 --- a/airflow/www/static/js/api/useClearRun.ts +++ b/airflow/www/static/js/api/useClearRun.ts @@ -42,17 +42,18 @@ export default function useClearRun(dagId: string, runId: string) { dag_run_id: runId, }).toString(); - return axios.post(clearRunUrl, params, { + return axios.post(clearRunUrl, params, { headers: { 'Content-Type': 'application/x-www-form-urlencoded', }, }); }, { - onSuccess: () => { - // Invalidating the query will force a new API request - queryClient.invalidateQueries('gridData'); - startRefresh(); + onSuccess: (_, { confirmed }) => { + if (confirmed) { + queryClient.invalidateQueries('gridData'); + startRefresh(); + } }, onError: (error: Error) => errorToast({ error }), }, diff --git a/airflow/www/static/js/api/useClearTask.ts b/airflow/www/static/js/api/useClearTask.ts index 279bc7458868d..b6c10f1637e83 100644 --- a/airflow/www/static/js/api/useClearTask.ts +++ b/airflow/www/static/js/api/useClearTask.ts @@ -74,17 +74,19 @@ export default function useClearTask({ params.append('map_index', mi.toString()); }); - return axios.post(clearUrl, params.toString(), { + return axios.post(clearUrl, params.toString(), { headers: { 'Content-Type': 'application/x-www-form-urlencoded', }, }); }, { - onSuccess: () => { - queryClient.invalidateQueries('gridData'); - queryClient.invalidateQueries(['mappedInstances', dagId, runId, taskId]); - startRefresh(); + onSuccess: (_, { confirmed }) => { + if (confirmed) { + queryClient.invalidateQueries('gridData'); + queryClient.invalidateQueries(['mappedInstances', dagId, runId, taskId]); + startRefresh(); + } }, onError: (error: Error) => errorToast({ error }), }, diff --git a/airflow/www/static/js/api/useMarkFailedRun.ts b/airflow/www/static/js/api/useMarkFailedRun.ts index 9e58381da2896..e1b7f2a21efaa 100644 --- a/airflow/www/static/js/api/useMarkFailedRun.ts +++ b/airflow/www/static/js/api/useMarkFailedRun.ts @@ -42,16 +42,18 @@ export default function useMarkFailedRun(dagId: string, runId: string) { dag_run_id: runId, }).toString(); - return axios.post(markFailedUrl, params, { + return axios.post(markFailedUrl, params, { headers: { 'Content-Type': 'application/x-www-form-urlencoded', }, }); }, { - onSuccess: () => { - queryClient.invalidateQueries('gridData'); - startRefresh(); + onSuccess: (_, { confirmed }) => { + if (confirmed) { + queryClient.invalidateQueries('gridData'); + startRefresh(); + } }, onError: (error: Error) => errorToast({ error }), }, diff --git a/airflow/www/static/js/api/useMarkSuccessRun.ts b/airflow/www/static/js/api/useMarkSuccessRun.ts index 062326cca4ed9..545e42e9173bd 100644 --- a/airflow/www/static/js/api/useMarkSuccessRun.ts +++ b/airflow/www/static/js/api/useMarkSuccessRun.ts @@ -41,16 +41,18 @@ export default function useMarkSuccessRun(dagId: string, runId: string) { dag_run_id: runId, }).toString(); - return axios.post(markSuccessUrl, params, { + return axios.post(markSuccessUrl, params, { headers: { 'Content-Type': 'application/x-www-form-urlencoded', }, }); }, { - onSuccess: () => { - queryClient.invalidateQueries('gridData'); - startRefresh(); + onSuccess: (_, { confirmed }) => { + if (confirmed) { + queryClient.invalidateQueries('gridData'); + startRefresh(); + } }, onError: (error: Error) => errorToast({ error }), }, diff --git a/airflow/www/static/js/api/useQueueRun.ts b/airflow/www/static/js/api/useQueueRun.ts index 71d4027b5f09b..6438d2856874e 100644 --- a/airflow/www/static/js/api/useQueueRun.ts +++ b/airflow/www/static/js/api/useQueueRun.ts @@ -40,16 +40,18 @@ export default function useQueueRun(dagId: string, runId: string) { dag_id: dagId, dag_run_id: runId, }).toString(); - return axios.post(queuedUrl, params, { + return axios.post(queuedUrl, params, { headers: { 'Content-Type': 'application/x-www-form-urlencoded', }, }); }, { - onSuccess: () => { - queryClient.invalidateQueries('gridData'); - startRefresh(); + onSuccess: (_, { confirmed }) => { + if (confirmed) { + queryClient.invalidateQueries('gridData'); + startRefresh(); + } }, onError: (error: Error) => errorToast({ error }), }, diff --git a/airflow/www/static/js/components/ConfirmDialog.tsx b/airflow/www/static/js/components/ConfirmDialog.tsx index 5501627de3e8c..055186e6a2e68 100644 --- a/airflow/www/static/js/components/ConfirmDialog.tsx +++ b/airflow/www/static/js/components/ConfirmDialog.tsx @@ -37,13 +37,13 @@ interface Props extends PropsWithChildren { onClose: () => void; title?: string; description: string; - body?: string[] | string; + affectedTasks: string[]; onConfirm: () => void; isLoading?: boolean; } const ConfirmDialog = ({ - isOpen, onClose, title = 'Wait a minute', description, body = [], onConfirm, isLoading = false, children, + isOpen, onClose, title = 'Wait a minute', description, affectedTasks, onConfirm, isLoading = false, children, }: Props) => { const initialFocusRef = useRef(null); const containerRef = useContainerRef(); @@ -67,7 +67,12 @@ const ConfirmDialog = ({ {children} {description} - {Array.isArray(body) && body.map((ti) => ({ti}))} + {affectedTasks.map((ti) => ( + {ti} + ))} + {!affectedTasks.length && ( + No task instances to change. + )} diff --git a/airflow/www/static/js/dag/details/dagRun/ClearRun.tsx b/airflow/www/static/js/dag/details/dagRun/ClearRun.tsx index 5666dbc84a863..bd0a7d92346b1 100644 --- a/airflow/www/static/js/dag/details/dagRun/ClearRun.tsx +++ b/airflow/www/static/js/dag/details/dagRun/ClearRun.tsx @@ -32,7 +32,7 @@ interface Props { } const ClearRun = ({ dagId, runId }: Props) => { - const [affectedTasks, setAffectedTasks] = useState(''); + const [affectedTasks, setAffectedTasks] = useState([]); const { isOpen, onOpen, onClose } = useDisclosure(); const { mutateAsync: onClear, isLoading } = useClearRun(dagId, runId); @@ -44,7 +44,7 @@ const ClearRun = ({ dagId, runId }: Props) => { const onConfirm = async () => { await onClear({ confirmed: true }); - setAffectedTasks(''); + setAffectedTasks([]); onClose(); }; @@ -63,7 +63,7 @@ const ClearRun = ({ dagId, runId }: Props) => { onConfirm={onConfirm} isLoading={isLoading} description="Task instances you are about to clear:" - body={affectedTasks} + affectedTasks={affectedTasks} /> ); diff --git a/airflow/www/static/js/dag/details/dagRun/MarkFailedRun.tsx b/airflow/www/static/js/dag/details/dagRun/MarkFailedRun.tsx index 26bdce8fb815f..7096c10790a5c 100644 --- a/airflow/www/static/js/dag/details/dagRun/MarkFailedRun.tsx +++ b/airflow/www/static/js/dag/details/dagRun/MarkFailedRun.tsx @@ -32,7 +32,7 @@ interface Props { } const MarkFailedRun = ({ dagId, runId }: Props) => { - const [affectedTasks, setAffectedTasks] = useState(''); + const [affectedTasks, setAffectedTasks] = useState([]); const { isOpen, onOpen, onClose } = useDisclosure(); const { mutateAsync: markFailed, isLoading } = useMarkFailedRun(dagId, runId); @@ -44,7 +44,7 @@ const MarkFailedRun = ({ dagId, runId }: Props) => { const onConfirm = () => { markFailed({ confirmed: true }); - setAffectedTasks(''); + setAffectedTasks([]); onClose(); }; @@ -57,7 +57,7 @@ const MarkFailedRun = ({ dagId, runId }: Props) => { onConfirm={onConfirm} isLoading={isLoading} description="Task instances you are about to mark as failed or skipped:" - body={affectedTasks} + affectedTasks={affectedTasks} /> ); diff --git a/airflow/www/static/js/dag/details/dagRun/MarkSuccessRun.tsx b/airflow/www/static/js/dag/details/dagRun/MarkSuccessRun.tsx index 79a894b7a9ee0..38392e76fd0b1 100644 --- a/airflow/www/static/js/dag/details/dagRun/MarkSuccessRun.tsx +++ b/airflow/www/static/js/dag/details/dagRun/MarkSuccessRun.tsx @@ -32,7 +32,7 @@ interface Props { } const MarkSuccessRun = ({ dagId, runId }: Props) => { - const [affectedTasks, setAffectedTasks] = useState(''); + const [affectedTasks, setAffectedTasks] = useState([]); const { isOpen, onOpen, onClose } = useDisclosure(); const { mutateAsync: markSuccess, isLoading } = useMarkSuccessRun(dagId, runId); @@ -44,7 +44,7 @@ const MarkSuccessRun = ({ dagId, runId }: Props) => { const onConfirm = async () => { await markSuccess({ confirmed: true }); - setAffectedTasks(''); + setAffectedTasks([]); onClose(); }; @@ -57,7 +57,7 @@ const MarkSuccessRun = ({ dagId, runId }: Props) => { onConfirm={onConfirm} isLoading={isLoading} description="Task instances you are about to mark as success:" - body={affectedTasks} + affectedTasks={affectedTasks} /> ); diff --git a/airflow/www/static/js/dag/details/dagRun/QueueRun.tsx b/airflow/www/static/js/dag/details/dagRun/QueueRun.tsx index a973e0ddac0dc..e48187761a861 100644 --- a/airflow/www/static/js/dag/details/dagRun/QueueRun.tsx +++ b/airflow/www/static/js/dag/details/dagRun/QueueRun.tsx @@ -32,7 +32,7 @@ interface Props { } const QueueRun = ({ dagId, runId }: Props) => { - const [affectedTasks, setAffectedTasks] = useState(''); + const [affectedTasks, setAffectedTasks] = useState([]); const { isOpen, onOpen, onClose } = useDisclosure(); const { mutateAsync: onQueue, isLoading } = useQueueRun(dagId, runId); @@ -46,7 +46,7 @@ const QueueRun = ({ dagId, runId }: Props) => { // Confirm changes const onConfirm = async () => { await onQueue({ confirmed: true }); - setAffectedTasks(''); + setAffectedTasks([]); onClose(); }; @@ -67,7 +67,7 @@ const QueueRun = ({ dagId, runId }: Props) => { onConfirm={onConfirm} isLoading={isLoading} description="Task instances you are about to queue:" - body={affectedTasks} + affectedTasks={affectedTasks} /> ); diff --git a/airflow/www/static/js/dag/details/taskInstance/taskActions/Clear.tsx b/airflow/www/static/js/dag/details/taskInstance/taskActions/Clear.tsx index d6066c89bfe39..4d7a50f2f0bb0 100644 --- a/airflow/www/static/js/dag/details/taskInstance/taskActions/Clear.tsx +++ b/airflow/www/static/js/dag/details/taskInstance/taskActions/Clear.tsx @@ -44,7 +44,7 @@ const Run = ({ mapIndexes, isGroup, }: CommonActionProps) => { - const [affectedTasks, setAffectedTasks] = useState(''); + const [affectedTasks, setAffectedTasks] = useState([]); // Options check/unchecked const [past, setPast] = useState(false); @@ -98,7 +98,7 @@ const Run = ({ confirmed: true, mapIndexes, }); - setAffectedTasks(''); + setAffectedTasks([]); onClose(); }; @@ -127,7 +127,7 @@ const Run = ({ onConfirm={onConfirm} isLoading={isLoading} description={`Task instances you are about to clear (${affectedTasks.length}):`} - body={affectedTasks} + affectedTasks={affectedTasks} > { isGroup && (past || future) && ( diff --git a/airflow/www/static/js/dag/details/taskInstance/taskActions/MarkFailed.tsx b/airflow/www/static/js/dag/details/taskInstance/taskActions/MarkFailed.tsx index f3537369e89ab..bcada11089d97 100644 --- a/airflow/www/static/js/dag/details/taskInstance/taskActions/MarkFailed.tsx +++ b/airflow/www/static/js/dag/details/taskInstance/taskActions/MarkFailed.tsx @@ -116,7 +116,7 @@ const MarkFailed = ({ onConfirm={onConfirm} isLoading={isLoading} description="Task instances you are about to mark as failed:" - body={affectedTasks} + affectedTasks={affectedTasks} /> ); diff --git a/airflow/www/static/js/dag/details/taskInstance/taskActions/MarkSuccess.tsx b/airflow/www/static/js/dag/details/taskInstance/taskActions/MarkSuccess.tsx index 767ccbaa160be..79e6a07726100 100644 --- a/airflow/www/static/js/dag/details/taskInstance/taskActions/MarkSuccess.tsx +++ b/airflow/www/static/js/dag/details/taskInstance/taskActions/MarkSuccess.tsx @@ -111,7 +111,7 @@ const MarkSuccess = ({ onConfirm={onConfirm} isLoading={isLoading} description="Task instances you are about to mark as success:" - body={affectedTasks} + affectedTasks={affectedTasks} /> ); From b7f6d0c02ec7737e003ace801cfbfdfb03ac263e Mon Sep 17 00:00:00 2001 From: Ash Berlin-Taylor Date: Tue, 6 Dec 2022 07:52:47 +0000 Subject: [PATCH 030/158] Ensure the dagbag_size metric decreases when files are deleted (#28135) (cherry picked from commit 2c7bd921635fc2e6fdc5315d90769e355a67d7e6) --- airflow/dag_processing/manager.py | 35 +++++++++++++++++++++++----- tests/dag_processing/test_manager.py | 1 + 2 files changed, 30 insertions(+), 6 deletions(-) diff --git a/airflow/dag_processing/manager.py b/airflow/dag_processing/manager.py index a41b737837226..1d8a082869efc 100644 --- a/airflow/dag_processing/manager.py +++ b/airflow/dag_processing/manager.py @@ -951,6 +951,12 @@ def set_file_paths(self, new_file_paths): Stats.decr("dag_processing.processes") processor.terminate() self._file_stats.pop(file_path) + + to_remove = set(self._file_stats.keys()) - set(self._file_paths) + for key in to_remove: + # Remove the stats for any dag files that don't exist anymore + del self._file_stats[key] + self._processors = filtered_processors def wait_until_finished(self): @@ -1066,6 +1072,7 @@ def prepare_file_path_queue(self): is_mtime_mode = list_mode == "modified_time" file_paths_recently_processed = [] + file_paths_to_stop_watching = set() for file_path in self._file_paths: if is_mtime_mode: @@ -1073,6 +1080,8 @@ def prepare_file_path_queue(self): files_with_mtime[file_path] = os.path.getmtime(file_path) except FileNotFoundError: self.log.warning("Skipping processing of missing file: %s", file_path) + self._file_stats.pop(file_path, None) + file_paths_to_stop_watching.add(file_path) continue file_modified_time = timezone.make_aware(datetime.fromtimestamp(files_with_mtime[file_path])) else: @@ -1101,12 +1110,18 @@ def prepare_file_path_queue(self): # set of files. Since we set the seed, the sort order will remain same per host random.Random(get_hostname()).shuffle(file_paths) + if file_paths_to_stop_watching: + self.set_file_paths( + [path for path in self._file_paths if path not in file_paths_to_stop_watching] + ) + files_paths_at_run_limit = [ file_path for file_path, stat in self._file_stats.items() if stat.run_count == self._max_runs ] file_paths_to_exclude = set(file_paths_in_progress).union( - file_paths_recently_processed, files_paths_at_run_limit + file_paths_recently_processed, + files_paths_at_run_limit, ) # Do not convert the following list to set as set does not preserve the order @@ -1124,12 +1139,11 @@ def prepare_file_path_queue(self): self.log.debug("Queuing the following files for processing:\n\t%s", "\n\t".join(files_paths_to_queue)) + default = DagFileStat( + num_dags=0, import_errors=0, last_finish_time=None, last_duration=None, run_count=0 + ) for file_path in files_paths_to_queue: - if file_path not in self._file_stats: - self._file_stats[file_path] = DagFileStat( - num_dags=0, import_errors=0, last_finish_time=None, last_duration=None, run_count=0 - ) - + self._file_stats.setdefault(file_path, default) self._file_path_queue.extend(files_paths_to_queue) def _kill_timed_out_processors(self): @@ -1155,6 +1169,15 @@ def _kill_timed_out_processors(self): self.waitables.pop(processor.waitable_handle) processors_to_remove.append(file_path) + stat = DagFileStat( + num_dags=0, + import_errors=1, + last_finish_time=now, + last_duration=duration, + run_count=self.get_run_count(file_path) + 1, + ) + self._file_stats[processor.file_path] = stat + # Clean up `self._processors` after iterating over it for proc in processors_to_remove: self._processors.pop(proc) diff --git a/tests/dag_processing/test_manager.py b/tests/dag_processing/test_manager.py index 7df0e78840cd0..2e5c0c2345c5b 100644 --- a/tests/dag_processing/test_manager.py +++ b/tests/dag_processing/test_manager.py @@ -257,6 +257,7 @@ def test_set_file_paths_when_processor_file_path_not_in_new_file_paths(self): manager.set_file_paths(["abc.txt"]) assert manager._processors == {} + assert "missing_file.txt" not in manager._file_stats def test_set_file_paths_when_processor_file_path_is_in_new_file_paths(self): manager = DagFileProcessorManager( From 10112966e242a4771cca8560c470f174611ad46f Mon Sep 17 00:00:00 2001 From: Jarek Potiuk Date: Tue, 6 Dec 2022 12:27:11 +0100 Subject: [PATCH 031/158] Do not warn when airflow dags tests command is used (#28138) When you run airflow dags test command on a fresh, never run dag, the command tries to emit true scheduling delay but it fails as there are no task instances created in the DB. This change skips generation of the metrics if task instance cannot be found. Fixes: #28106 (cherry picked from commit b3d7e17e72c05fd149a5514e3796d46a241ac4f7) --- airflow/models/dagrun.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airflow/models/dagrun.py b/airflow/models/dagrun.py index c601193b272d9..35d4e1b61e88d 100644 --- a/airflow/models/dagrun.py +++ b/airflow/models/dagrun.py @@ -866,7 +866,7 @@ def _emit_true_scheduling_delay_stats_for_finished_state(self, finished_tis: lis ordered_tis_by_start_date = [ti for ti in finished_tis if ti.start_date] ordered_tis_by_start_date.sort(key=lambda ti: ti.start_date, reverse=False) - first_start_date = ordered_tis_by_start_date[0].start_date + first_start_date = ordered_tis_by_start_date[0].start_date if ordered_tis_by_start_date else None if first_start_date: # TODO: Logically, this should be DagRunInfo.run_after, but the # information is not stored on a DagRun, only before the actual From 524004e30a1d92a20ee6121ab91e81905b3e6b0e Mon Sep 17 00:00:00 2001 From: Ash Berlin-Taylor Date: Tue, 6 Dec 2022 17:11:45 +0000 Subject: [PATCH 032/158] Ensure that warnings from `@dag` decorator are reported in dag file (#28153) It's no use to a user if the warnings appear from `airflow/models/dag.py`! This hacky magic has been in place for the apply_defaults metaclass for a couple of releases, and for all it's evil hackiness, it works, I've just extracted it out to be reusable. Also I wish Python had something like this built in, as it's _really_ hard to get this right otherwise (cherry picked from commit 5fec7873a6b4d86fb2a1e66fa32f12cbd42431e0) --- airflow/models/baseoperator.py | 21 ++------------------- airflow/models/dag.py | 3 +++ airflow/utils/decorators.py | 22 ++++++++++++++++++++++ tests/models/test_dag.py | 22 +++++++++++++++++++++- 4 files changed, 48 insertions(+), 20 deletions(-) diff --git a/airflow/models/baseoperator.py b/airflow/models/baseoperator.py index a1b49abf66314..bb9b94b52f2aa 100644 --- a/airflow/models/baseoperator.py +++ b/airflow/models/baseoperator.py @@ -84,6 +84,7 @@ from airflow.triggers.base import BaseTrigger from airflow.utils import timezone from airflow.utils.context import Context +from airflow.utils.decorators import fixup_decorator_warning_stack from airflow.utils.helpers import validate_key from airflow.utils.operator_resources import Resources from airflow.utils.session import NEW_SESSION, provide_session @@ -340,25 +341,7 @@ def _apply_defaults(cls, func: T) -> T: if param.default == param.empty and name != "task_id" } - class autostacklevel_warn: - def __init__(self): - self.warnings = __import__("warnings") - - def __getattr__(self, name): - return getattr(self.warnings, name) - - def __dir__(self): - return dir(self.warnings) - - def warn(self, message, category=None, stacklevel=1, source=None): - self.warnings.warn(message, category, stacklevel + 2, source) - - if func.__globals__.get("warnings") is sys.modules["warnings"]: - # Yes, this is slightly hacky, but it _automatically_ sets the right - # stacklevel parameter to `warnings.warn` to ignore the decorator. Now - # that the decorator is applied automatically, this makes the needed - # stacklevel parameter less confusing. - func.__globals__["warnings"] = autostacklevel_warn() + fixup_decorator_warning_stack(func) @functools.wraps(func) def apply_defaults(self: BaseOperator, *args: Any, **kwargs: Any) -> Any: diff --git a/airflow/models/dag.py b/airflow/models/dag.py index 210f455f063ef..c08a7f236f5d1 100644 --- a/airflow/models/dag.py +++ b/airflow/models/dag.py @@ -90,6 +90,7 @@ from airflow.utils import timezone from airflow.utils.dag_cycle_tester import check_cycle from airflow.utils.dates import cron_presets, date_range as utils_date_range +from airflow.utils.decorators import fixup_decorator_warning_stack from airflow.utils.file import correct_maybe_zipped from airflow.utils.helpers import at_most_one, exactly_one, validate_key from airflow.utils.log.logging_mixin import LoggingMixin @@ -3543,6 +3544,8 @@ def factory(*args, **kwargs): # Return dag object such that it's accessible in Globals. return dag_obj + # Ensure that warnings from inside DAG() are emitted from the caller, not here + fixup_decorator_warning_stack(factory) return factory return wrapper diff --git a/airflow/utils/decorators.py b/airflow/utils/decorators.py index 1e576bb3fddd5..35dc23e85a0fa 100644 --- a/airflow/utils/decorators.py +++ b/airflow/utils/decorators.py @@ -17,6 +17,7 @@ # under the License. from __future__ import annotations +import sys import warnings from collections import deque from functools import wraps @@ -83,3 +84,24 @@ def _balance_parens(after_decorator): elif current == ")": num_paren = num_paren - 1 return "".join(after_decorator) + + +class _autostacklevel_warn: + def __init__(self): + self.warnings = __import__("warnings") + + def __getattr__(self, name): + return getattr(self.warnings, name) + + def __dir__(self): + return dir(self.warnings) + + def warn(self, message, category=None, stacklevel=1, source=None): + self.warnings.warn(message, category, stacklevel + 2, source) + + +def fixup_decorator_warning_stack(func): + if func.__globals__.get("warnings") is sys.modules["warnings"]: + # Yes, this is more than slightly hacky, but it _automatically_ sets the right stacklevel parameter to + # `warnings.warn` to ignore the decorator. + func.__globals__["warnings"] = _autostacklevel_warn() diff --git a/tests/models/test_dag.py b/tests/models/test_dag.py index 2bc8af8302b32..1c741be2d580e 100644 --- a/tests/models/test_dag.py +++ b/tests/models/test_dag.py @@ -23,6 +23,7 @@ import os import pickle import re +import sys from contextlib import redirect_stdout from datetime import timedelta from pathlib import Path @@ -42,7 +43,12 @@ from airflow.configuration import conf from airflow.datasets import Dataset from airflow.decorators import task as task_decorator -from airflow.exceptions import AirflowException, DuplicateTaskIdFound, ParamValidationError +from airflow.exceptions import ( + AirflowException, + DuplicateTaskIdFound, + ParamValidationError, + RemovedInAirflow3Warning, +) from airflow.models import DAG, DagModel, DagRun, DagTag, TaskFail, TaskInstance as TI from airflow.models.baseoperator import BaseOperator from airflow.models.dag import DagOwnerAttributes, dag as dag_decorator, get_dataset_triggered_next_run_info @@ -2740,6 +2746,20 @@ def return_num(num): dag = xcom_pass_to_op() assert dag.params["value"] == value + def test_warning_location(self): + # NOTE: This only works as long as there is some warning we can emit from `DAG()` + @dag_decorator(schedule_interval=None) + def mydag(): + ... + + with pytest.warns(RemovedInAirflow3Warning) as warnings: + line = sys._getframe().f_lineno + 1 + mydag() + + w = warnings.pop(RemovedInAirflow3Warning) + assert w.filename == __file__ + assert w.lineno == line + @pytest.mark.parametrize("timetable", [NullTimetable(), OnceTimetable()]) def test_dag_timetable_match_schedule_interval(timetable): From 824963a06a5612cb76018e6b1f696b44adeb3abb Mon Sep 17 00:00:00 2001 From: Brent Bovenzi Date: Tue, 6 Dec 2022 11:11:10 -0600 Subject: [PATCH 033/158] fix next run datasets error (#28165) (cherry picked from commit 196bab483655a24f0ef12a1049d21b8a9dead691) --- airflow/www/static/js/dags.js | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/airflow/www/static/js/dags.js b/airflow/www/static/js/dags.js index 72ae834bc41f0..97c3b2f12cf0d 100644 --- a/airflow/www/static/js/dags.js +++ b/airflow/www/static/js/dags.js @@ -273,14 +273,14 @@ function dagStatsHandler(selector, json) { }); } -function nextRunDatasetsSummaryHandler(json) { +function nextRunDatasetsSummaryHandler(_, json) { [...document.getElementsByClassName('next-dataset-triggered')].forEach((el) => { const dagId = $(el).attr('data-dag-id'); const previousSummary = $(el).attr('data-summary'); const nextDatasetsInfo = json[dagId]; // Only update dags that depend on multiple datasets - if (!nextDatasetsInfo.uri) { + if (nextDatasetsInfo && !nextDatasetsInfo.uri) { const newSummary = `${nextDatasetsInfo.ready} of ${nextDatasetsInfo.total} datasets updated`; // Only update the element if the summary has changed @@ -410,7 +410,7 @@ function handleRefresh({ activeDagsOnly = false } = {}) { .post(params, (error, json) => refreshDagStatsHandler(TASK_INSTANCE, json)); d3.json(nextRunDatasetsSummaryUrl) .header('X-CSRFToken', csrfToken) - .post(params, (error, json) => nextRunDatasetsSummaryHandler(json)); + .post(params, nextRunDatasetsSummaryHandler); } setTimeout(() => { $('#loading-dots').css('display', 'none'); From 2bfc0207edc97c357b9f0fc69692d994710c11d7 Mon Sep 17 00:00:00 2001 From: Tzu-ping Chung Date: Wed, 7 Dec 2022 15:37:39 +0800 Subject: [PATCH 034/158] Make invalid characters exception more readable (#28181) The InvalidStatsNameException message when a stats name contains invalid characters currently includes the set of allow characters, which is quite long and of underterministic order (bein formatted from a set), making the message impossible to decipher for a human. This commit changes the message to describe what characters are allowed in a human-readable fashion instead. (cherry picked from commit e3ae69e9a779f14ee70c5646d6e072bb2365e54f) --- airflow/stats.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/airflow/stats.py b/airflow/stats.py index d231964fd6e28..92ff3809f695b 100644 --- a/airflow/stats.py +++ b/airflow/stats.py @@ -194,7 +194,8 @@ def stat_name_default_handler(stat_name, max_length=250) -> str: ) if not all((c in ALLOWED_CHARACTERS) for c in stat_name): raise InvalidStatsNameException( - f"The stat name ({stat_name}) has to be composed with characters in {ALLOWED_CHARACTERS}." + f"The stat name ({stat_name}) has to be composed of ASCII " + f"alphabets, numbers, or the underscore, dot, or dash characters." ) return stat_name From d69af8e7d97f2d46886069c99ac8e870bd113bdc Mon Sep 17 00:00:00 2001 From: Patrick Ruoff <61174725+pruoff@users.noreply.github.com> Date: Wed, 7 Dec 2022 17:10:21 +0100 Subject: [PATCH 035/158] Fix typo in Best Practice documentation (#28190) (cherry picked from commit fd4ab1dc35fc0b4bb53b63080e58fe7829274293) --- docs/apache-airflow/best-practices.rst | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/apache-airflow/best-practices.rst b/docs/apache-airflow/best-practices.rst index 337815a63b14e..6ac50611e0d3e 100644 --- a/docs/apache-airflow/best-practices.rst +++ b/docs/apache-airflow/best-practices.rst @@ -846,7 +846,7 @@ The drawbacks: same worker might be affected by previous tasks creating/modifying files et.c You can think about the ``PythonVirtualenvOperator`` and ``ExternalPythonOperator`` as counterparts - -that make it smother to move from development phase to production phase. As a DAG author you'd normally +that make it smoother to move from development phase to production phase. As a DAG author you'd normally iterate with dependencies and develop your DAG using ``PythonVirtualenvOperator`` (thus decorating your tasks with ``@task.virtualenv`` decorators) while after the iteration and changes you would likely want to change it for production to switch to the ``ExternalPythonOperator`` (and ``@task.external_python``) From fa37bf7b8217eb85638247c31630d2e5fb902389 Mon Sep 17 00:00:00 2001 From: Tzu-ping Chung Date: Thu, 15 Dec 2022 17:20:25 +0800 Subject: [PATCH 036/158] Add custom pickling hooks to LazyXComAccess (#28191) fixes https://github.com/apache/airflow/issues/28146 (cherry picked from commit e981dfab4e0f4faf1fb932ac6993c3ecbd5318b2) --- airflow/models/xcom.py | 15 +++++++++++++++ tests/models/test_taskinstance.py | 15 +++++++++++++++ 2 files changed, 30 insertions(+) diff --git a/airflow/models/xcom.py b/airflow/models/xcom.py index 9d78057febf2b..3b4361842409f 100644 --- a/airflow/models/xcom.py +++ b/airflow/models/xcom.py @@ -723,6 +723,21 @@ def __eq__(self, other: Any) -> bool: return all(x == y for x, y in z) return NotImplemented + def __getstate__(self) -> Any: + # We don't want to go to the trouble of serializing the entire Query + # object, including its filters, hints, etc. (plus SQLAlchemy does not + # provide a public API to inspect a query's contents). Converting the + # query into a SQL string is the best we can get. Theoratically we can + # do the same for count(), but I think it should be performant enough to + # calculate only that eagerly. + with self._get_bound_query() as query: + statement = query.statement.compile(query.session.get_bind()) + return (str(statement), query.count()) + + def __setstate__(self, state: Any) -> None: + statement, self._len = state + self._query = Query(XCom.value).from_statement(text(statement)) + def __len__(self): if self._len is None: with self._get_bound_query() as query: diff --git a/tests/models/test_taskinstance.py b/tests/models/test_taskinstance.py index a7ce20c87c12c..17ce74178d1fb 100644 --- a/tests/models/test_taskinstance.py +++ b/tests/models/test_taskinstance.py @@ -21,6 +21,7 @@ import operator import os import pathlib +import pickle import signal import sys import urllib @@ -3591,6 +3592,20 @@ def cmds(): assert out_lines == ["hello FOO", "goodbye FOO", "hello BAR", "goodbye BAR"] +def test_lazy_xcom_access_does_not_pickle_session(dag_maker, session): + with dag_maker(session=session): + EmptyOperator(task_id="t") + + run: DagRun = dag_maker.create_dagrun() + run.get_task_instance("t", session=session).xcom_push("xxx", 123, session=session) + + original = LazyXComAccess.build_from_xcom_query(session.query(XCom)) + processed = pickle.loads(pickle.dumps(original)) + + assert len(processed) == 1 + assert list(processed) == [123] + + @mock.patch("airflow.models.taskinstance.XCom.deserialize_value", side_effect=XCom.deserialize_value) def test_ti_xcom_pull_on_mapped_operator_return_lazy_iterable(mock_deserialize_value, dag_maker, session): """Ensure we access XCom lazily when pulling from a mapped operator.""" From e21a31576b9e94139dc06fb4dcf8f39a9f8862fd Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Thu, 8 Dec 2022 12:23:21 -0800 Subject: [PATCH 037/158] Make live logs reading work for "other" k8s executors (#28213) (cherry picked from commit cbfbf8b843f178de1e1aa1066e5ea3377a8de774) --- airflow/utils/log/file_task_handler.py | 21 +++++++++++++-- tests/utils/test_log_handlers.py | 36 ++++++++++++++++++++++++++ 2 files changed, 55 insertions(+), 2 deletions(-) diff --git a/airflow/utils/log/file_task_handler.py b/airflow/utils/log/file_task_handler.py index e20da8cebc7dc..5a5b55d8a41a3 100644 --- a/airflow/utils/log/file_task_handler.py +++ b/airflow/utils/log/file_task_handler.py @@ -132,6 +132,24 @@ def _render_filename(self, ti: TaskInstance, try_number: int) -> str: def _read_grouped_logs(self): return False + @staticmethod + def _should_check_k8s(queue): + """ + If the task is running through kubernetes executor, return True. + + When logs aren't available locally, in this case we read from k8s pod logs. + """ + executor = conf.get("core", "executor") + if executor == "KubernetesExecutor": + return True + elif executor == "LocalKubernetesExecutor": + if queue == conf.get("local_kubernetes_executor", "kubernetes_queue"): + return True + elif executor == "CeleryKubernetesExecutor": + if queue == conf.get("celery_kubernetes_executor", "kubernetes_queue"): + return True + return False + def _read(self, ti: TaskInstance, try_number: int, metadata: dict[str, Any] | None = None): """ Template method that contains custom logic of reading @@ -163,7 +181,6 @@ def _read(self, ti: TaskInstance, try_number: int, metadata: dict[str, Any] | No location = os.path.join(self.local_base, log_relative_path) log = "" - if os.path.exists(location): try: with open(location, encoding="utf-8", errors="surrogateescape") as file: @@ -173,7 +190,7 @@ def _read(self, ti: TaskInstance, try_number: int, metadata: dict[str, Any] | No log = f"*** Failed to load local log file: {location}\n" log += f"*** {str(e)}\n" return log, {"end_of_log": True} - elif conf.get("core", "executor") == "KubernetesExecutor": + elif self._should_check_k8s(ti.queue): try: from airflow.kubernetes.kube_client import get_kube_client diff --git a/tests/utils/test_log_handlers.py b/tests/utils/test_log_handlers.py index 765b5d921aede..ee2ff2d9cea36 100644 --- a/tests/utils/test_log_handlers.py +++ b/tests/utils/test_log_handlers.py @@ -21,6 +21,9 @@ import logging.config import os import re +from unittest.mock import patch + +import pytest from airflow.config_templates.airflow_local_settings import DEFAULT_LOGGING_CONFIG from airflow.models import DAG, DagRun, TaskInstance @@ -264,3 +267,36 @@ def test_log_retrieval_valid(self, create_task_instance): log_url_ti.hostname = "hostname" url = FileTaskHandler._get_log_retrieval_url(log_url_ti, "DYNAMIC_PATH") assert url == "http://hostname:8793/log/DYNAMIC_PATH" + + +@pytest.mark.parametrize( + "config, queue, expected", + [ + (dict(AIRFLOW__CORE__EXECUTOR="LocalExecutor"), None, False), + (dict(AIRFLOW__CORE__EXECUTOR="LocalExecutor"), "kubernetes", False), + (dict(AIRFLOW__CORE__EXECUTOR="KubernetesExecutor"), None, True), + (dict(AIRFLOW__CORE__EXECUTOR="CeleryKubernetesExecutor"), "any", False), + (dict(AIRFLOW__CORE__EXECUTOR="CeleryKubernetesExecutor"), "kubernetes", True), + ( + dict( + AIRFLOW__CORE__EXECUTOR="CeleryKubernetesExecutor", + AIRFLOW__CELERY_KUBERNETES_EXECUTOR__KUBERNETES_QUEUE="hithere", + ), + "hithere", + True, + ), + (dict(AIRFLOW__CORE__EXECUTOR="LocalKubernetesExecutor"), "any", False), + (dict(AIRFLOW__CORE__EXECUTOR="LocalKubernetesExecutor"), "kubernetes", True), + ( + dict( + AIRFLOW__CORE__EXECUTOR="LocalKubernetesExecutor", + AIRFLOW__LOCAL_KUBERNETES_EXECUTOR__KUBERNETES_QUEUE="hithere", + ), + "hithere", + True, + ), + ], +) +def test__should_check_k8s(config, queue, expected): + with patch.dict("os.environ", **config): + assert FileTaskHandler._should_check_k8s(queue) == expected From 68a0fb7b4c7b5aaea404f19cf7731661843d3a99 Mon Sep 17 00:00:00 2001 From: Malthe Borch Date: Fri, 9 Dec 2022 13:29:39 +0000 Subject: [PATCH 038/158] Make arguments 'offset' and 'length' not required (#28234) * Make arguments 'offset' and 'length' not required * Disable implicit optional for azure-storage (mypy) (cherry picked from commit 7e776db254953076e932ed2183e1ed49a5ca20a6) --- airflow/providers/microsoft/azure/hooks/wasb.py | 2 +- setup.cfg | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/airflow/providers/microsoft/azure/hooks/wasb.py b/airflow/providers/microsoft/azure/hooks/wasb.py index 3a7f481a90dcb..27680a5b69408 100644 --- a/airflow/providers/microsoft/azure/hooks/wasb.py +++ b/airflow/providers/microsoft/azure/hooks/wasb.py @@ -385,7 +385,7 @@ def upload( return blob_client.upload_blob(data, blob_type, length=length, **kwargs) def download( - self, container_name, blob_name, offset: int, length: int, **kwargs + self, container_name, blob_name, offset: int | None = None, length: int | None = None, **kwargs ) -> StorageStreamDownloader: """ Downloads a blob to the StorageStreamDownloader diff --git a/setup.cfg b/setup.cfg index 59e6686ab55a8..2274d98e3a90a 100644 --- a/setup.cfg +++ b/setup.cfg @@ -191,5 +191,5 @@ no_implicit_optional = False #Let's assume all azure packages have no implicit optional [mypy-azure.batch.*] no_implicit_optional = False -[mypy-azure.batch.models.*] +[mypy-azure.storage.*] no_implicit_optional = False From 9220218114e715195a58ba1460d53b60accd0e13 Mon Sep 17 00:00:00 2001 From: Jed Cunningham <66968678+jedcunningham@users.noreply.github.com> Date: Fri, 9 Dec 2022 13:11:28 -0600 Subject: [PATCH 039/158] Fix db clean warnings (#28243) RTIF is already cleaned by TI's FK, and doesn't have a column to use as a recency column on it's own. db clean was added for 2.3, but cleaning RTIF was broken before 2.3 was released, meaning this never actually worked anyways. In addition, if users explicitly try and clear this table with `--tables`, it gets ignored, making this a safe change to make. Dataset event's created_at column was renamed to timestamp before 2.4 was released, so this one also never worked. Closes #26581 (cherry picked from commit 171ca66142887f59b1808fcdd6b19e7141a08d17) --- airflow/utils/db_cleanup.py | 3 +-- tests/utils/test_db_cleanup.py | 17 +++++++++++++++++ 2 files changed, 18 insertions(+), 2 deletions(-) diff --git a/airflow/utils/db_cleanup.py b/airflow/utils/db_cleanup.py index 26c6aea8f1c31..e03d1a28b0c06 100644 --- a/airflow/utils/db_cleanup.py +++ b/airflow/utils/db_cleanup.py @@ -94,10 +94,9 @@ def readable_config(self): keep_last_filters=[column("external_trigger") == false()], keep_last_group_by=["dag_id"], ), - _TableConfig(table_name="dataset_event", recency_column_name="created_at"), + _TableConfig(table_name="dataset_event", recency_column_name="timestamp"), _TableConfig(table_name="import_error", recency_column_name="timestamp"), _TableConfig(table_name="log", recency_column_name="dttm"), - _TableConfig(table_name="rendered_task_instance_fields", recency_column_name="execution_date"), _TableConfig(table_name="sla_miss", recency_column_name="timestamp"), _TableConfig(table_name="task_fail", recency_column_name="start_date"), _TableConfig(table_name="task_instance", recency_column_name="start_date"), diff --git a/tests/utils/test_db_cleanup.py b/tests/utils/test_db_cleanup.py index c505408a3bf13..b00fde82a8da3 100644 --- a/tests/utils/test_db_cleanup.py +++ b/tests/utils/test_db_cleanup.py @@ -18,6 +18,7 @@ from __future__ import annotations from contextlib import suppress +from datetime import datetime from importlib import import_module from pathlib import Path from unittest.mock import MagicMock, patch @@ -26,6 +27,7 @@ import pendulum import pytest from pytest import param +from sqlalchemy.exc import OperationalError from sqlalchemy.ext.declarative import DeclarativeMeta from airflow.models import DagModel, DagRun, TaskInstance @@ -280,6 +282,7 @@ def test_no_models_missing(self): "dataset_event_dag_run", # foreign keys "task_instance_note", # foreign keys "dag_run_note", # foreign keys + "rendered_task_instance_fields", # foreign key with TI } from airflow.utils.db_cleanup import config_dict @@ -289,6 +292,20 @@ def test_no_models_missing(self): assert set(all_models) - exclusion_list.union(config_dict) == set() assert exclusion_list.isdisjoint(config_dict) + def test_no_failure_warnings(self, caplog): + """ + Ensure every table we have configured (and that is present in the db) can be cleaned successfully. + For example, this checks that the recency column is actually a column. + """ + run_cleanup(clean_before_timestamp=datetime.utcnow(), dry_run=True) + assert "Encountered error when attempting to clean table" not in caplog.text + + # Lets check we have the right error message just in case + caplog.clear() + with patch("airflow.utils.db_cleanup._cleanup_table", side_effect=OperationalError("oops", {}, None)): + run_cleanup(clean_before_timestamp=datetime.utcnow(), table_names=["task_instance"], dry_run=True) + assert "Encountered error when attempting to clean table" in caplog.text + def create_tis(base_date, num_tis, external_trigger=False): with create_session() as session: From 805dbc2304b5d6d93ab242efa7eca669e7967d12 Mon Sep 17 00:00:00 2001 From: Jarek Potiuk Date: Fri, 9 Dec 2022 13:02:40 +0100 Subject: [PATCH 040/158] Convert test_task_command to Pytest and unquarantine tests in it (#28247) (cherry picked from commit 2f5c77b0baa0ab26d2c51fa010850653ded80a46) --- airflow/cli/commands/task_command.py | 1 - tests/cli/commands/test_task_command.py | 54 +++++++++++-------------- 2 files changed, 23 insertions(+), 32 deletions(-) diff --git a/airflow/cli/commands/task_command.py b/airflow/cli/commands/task_command.py index 078565dc38d1d..93e7c81146bb2 100644 --- a/airflow/cli/commands/task_command.py +++ b/airflow/cli/commands/task_command.py @@ -292,7 +292,6 @@ def _capture_task_logs(ti: TaskInstance) -> Generator[None, None, None]: """ modify = not settings.DONOT_MODIFY_HANDLERS - if modify: root_logger, task_logger = logging.getLogger(), logging.getLogger("airflow.task") diff --git a/tests/cli/commands/test_task_command.py b/tests/cli/commands/test_task_command.py index c565f601d720a..a062f31d4f65e 100644 --- a/tests/cli/commands/test_task_command.py +++ b/tests/cli/commands/test_task_command.py @@ -94,6 +94,7 @@ def setup_class(cls): def teardown_class(cls) -> None: clear_db_runs() + @pytest.mark.execution_timeout(120) def test_cli_list_tasks(self): for dag_id in self.dagbag.dags: args = self.parser.parse_args(["tasks", "list", dag_id]) @@ -498,8 +499,8 @@ def test_parentdag_downstream_clear(self): task_command.task_clear(args) -class TestLogsfromTaskRunCommand(unittest.TestCase): - def setUp(self) -> None: +class TestLogsfromTaskRunCommand: + def setup_method(self) -> None: self.dag_id = "test_logging_dag" self.task_id = "test_task" self.run_id = "test_run" @@ -531,7 +532,7 @@ def setUp(self) -> None: except OSError: pass - def tearDown(self) -> None: + def teardown_method(self) -> None: root = self.root_logger root.setLevel(self.root_level) root.handlers[:] = self.root_handlers @@ -598,9 +599,6 @@ def test_external_executor_id_present_for_process_run_task(self, mock_local_job) @unittest.skipIf(not hasattr(os, "fork"), "Forking not available") def test_logging_with_run_task(self): - # We are not using self.assertLogs as we want to verify what actually is stored in the Log file - # as that is what gets displayed - with conf_vars({("core", "dags_folder"): self.dag_path}): task_command.task_run(self.parser.parse_args(self.task_args)) @@ -644,12 +642,8 @@ def test_run_task_with_pool(self): session.delete(pool) session.commit() - # For this test memory spins out of control on Python 3.6. TODO(potiuk): FIXME") - @pytest.mark.quarantined @mock.patch("airflow.task.task_runner.standard_task_runner.CAN_FORK", False) def test_logging_with_run_task_subprocess(self): - # We are not using self.assertLogs as we want to verify what actually is stored in the Log file - # as that is what gets displayed with conf_vars({("core", "dags_folder"): self.dag_path}): task_command.task_run(self.parser.parse_args(self.task_args)) @@ -665,10 +659,7 @@ def test_logging_with_run_task_subprocess(self): self.assert_log_line("Log from TI Logger", logs_list) self.assert_log_line("Log from Print statement", logs_list, expect_from_logging_mixin=True) - assert ( - f"INFO - Running: ['airflow', 'tasks', 'run', '{self.dag_id}', " - f"'{self.task_id}', '{self.execution_date_str}'," in logs - ) + assert f"INFO - Running: ['airflow', 'tasks', 'run', '{self.dag_id}', '{self.task_id}'," in logs assert ( f"INFO - Marking task as SUCCESS. dag_id={self.dag_id}, " f"task_id={self.task_id}, execution_date=20170101T000000" in logs @@ -696,7 +687,7 @@ def test_log_file_template_with_run_task(self): pass @mock.patch.object(task_command, "_run_task_by_selected_method") - def test_root_logger_restored(self, run_task_mock): + def test_root_logger_restored(self, run_task_mock, caplog): """Verify that the root logging context is restored""" logger = logging.getLogger("foo.bar") @@ -711,19 +702,18 @@ def task_inner(*args, **kwargs): ("logging", "logging_level"): "INFO", } - with conf_vars(config): - with self.assertLogs(level=logging.WARNING) as captured: + with caplog.at_level(level=logging.WARNING): + with conf_vars(config): logger.warning("not redirected") task_command.task_run(self.parser.parse_args(self.task_args)) - - assert captured.output == ["WARNING:foo.bar:not redirected"] + assert "not redirected" in caplog.text assert self.root_logger.level == logging.WARNING assert self.root_logger.handlers == self.root_handlers - @pytest.mark.quarantined @mock.patch.object(task_command, "_run_task_by_selected_method") - def test_disable_handler_modifying(self, run_task_mock): + @pytest.mark.parametrize("do_not_modify_handler", [True, False]) + def test_disable_handler_modifying(self, run_task_mock, caplog, do_not_modify_handler): """If [core] donot_modify_handlers is set to True, the root logger is untouched""" from airflow import settings @@ -738,16 +728,18 @@ def task_inner(*args, **kwargs): ("core", "dags_folder"): self.dag_path, ("logging", "logging_level"): "INFO", } - old_value = settings.DONOT_MODIFY_HANDLERS - settings.DONOT_MODIFY_HANDLERS = True - - with conf_vars(config): - with self.assertLogs(level=logging.WARNING) as captured: - task_command.task_run(self.parser.parse_args(self.task_args)) - - assert captured.output == ["WARNING:foo.bar:not redirected"] - - settings.DONOT_MODIFY_HANDLERS = old_value + with caplog.at_level(logging.WARNING, logger="foo.bar"): + with conf_vars(config): + old_value = settings.DONOT_MODIFY_HANDLERS + settings.DONOT_MODIFY_HANDLERS = do_not_modify_handler + try: + task_command.task_run(self.parser.parse_args(self.task_args)) + if do_not_modify_handler: + assert "not redirected" in caplog.text + else: + assert "not redirected" not in caplog.text + finally: + settings.DONOT_MODIFY_HANDLERS = old_value def test_context_with_run(): From 215011493b679b5a85bb08b217c84c3451048b8a Mon Sep 17 00:00:00 2001 From: Jarek Potiuk Date: Wed, 21 Dec 2022 20:13:58 +0100 Subject: [PATCH 041/158] Trigger gevent monkeypatching via environment variable (#28283) Gevent needs to monkeypatch a number of system libraries as soon as possible when Python interpreter starts, in order to avoid other libraries monkey-patching them before. We should do it before any other initialization and it needs to be only run on webserver. So far it was done by local_settings monkeypatching but that has been rather brittle and some changes in Airflow made previous attempts to stop working because the "other" packages could be loaded by Airflow before - depending on installed providers and configuration (for example when you had AWS configured as logger, boto could have been loaded before and it could have monkey patch networking before gevent had a chance to do so. This change introduces different mechanism of triggering the patching - it could be triggered by setting an environment variable. This has the benefit that we do not need to initialize anything (including reading settings or setting up logging) before we determine if gevent patching should be performed. It has also the drawback that the user will have to set the environment variable in their deployment manually. However this is a small price to pay if they will get a stable and future-proof gevent monkeypatching built-in in Airflow. Fixes: #8212 (cherry picked from commit 2429d077d8c59299487562c8867cfc63cd969b9d) --- airflow/__init__.py | 9 +++++++++ airflow/config_templates/config.yml | 4 +++- airflow/config_templates/default_airflow.cfg | 4 +++- newsfragments/08212.misc.rst | 1 + 4 files changed, 16 insertions(+), 2 deletions(-) create mode 100644 newsfragments/08212.misc.rst diff --git a/airflow/__init__.py b/airflow/__init__.py index 20cc8550b84d3..19624252a3875 100644 --- a/airflow/__init__.py +++ b/airflow/__init__.py @@ -31,6 +31,14 @@ import sys from typing import Callable +if os.environ.get("_AIRFLOW_PATCH_GEVENT"): + # If you are using gevents and start airflow webserver, you might want to run gevent monkeypatching + # as one of the first thing when Airflow is started. This allows gevent to patch networking and other + # system libraries to make them gevent-compatible before anything else patches them (for example boto) + from gevent.monkey import patch_all + + patch_all() + from airflow import settings __all__ = ["__version__", "login", "DAG", "PY36", "PY37", "PY38", "PY39", "PY310", "XComArg"] @@ -40,6 +48,7 @@ # lib.) __path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore + # Perform side-effects unless someone has explicitly opted out before import # WARNING: DO NOT USE THIS UNLESS YOU REALLY KNOW WHAT YOU'RE DOING. if not os.environ.get("_AIRFLOW__AS_LIBRARY", None): diff --git a/airflow/config_templates/config.yml b/airflow/config_templates/config.yml index 009f6a48466c2..2c0721aa89fcd 100644 --- a/airflow/config_templates/config.yml +++ b/airflow/config_templates/config.yml @@ -1202,7 +1202,9 @@ - name: worker_class description: | The worker class gunicorn should use. Choices include - sync (default), eventlet, gevent + sync (default), eventlet, gevent. Note when using gevent you might also want to set the + "_AIRFLOW_PATCH_GEVENT" environment variable to "1" to make sure gevent patching is done as + early as possible. version_added: ~ type: string example: ~ diff --git a/airflow/config_templates/default_airflow.cfg b/airflow/config_templates/default_airflow.cfg index 1cff5f8dbe75f..80915d6643a56 100644 --- a/airflow/config_templates/default_airflow.cfg +++ b/airflow/config_templates/default_airflow.cfg @@ -621,7 +621,9 @@ secret_key = {SECRET_KEY} workers = 4 # The worker class gunicorn should use. Choices include -# sync (default), eventlet, gevent +# sync (default), eventlet, gevent. Note when using gevent you might also want to set the +# "_AIRFLOW_PATCH_GEVENT" environment variable to "1" to make sure gevent patching is done as +# early as possible. worker_class = sync # Log files for the gunicorn webserver. '-' means log to stderr. diff --git a/newsfragments/08212.misc.rst b/newsfragments/08212.misc.rst new file mode 100644 index 0000000000000..acce074f103ec --- /dev/null +++ b/newsfragments/08212.misc.rst @@ -0,0 +1 @@ +If you are using gevent for your webserver deployment and used local settings to monkeypatch gevent, you might want to replace local settings patching with an ``_AIRFLOW_PATCH_GEVENT`` environment variable set to 1 in your webserver. This ensures gevent patching is done as early as possible. From 7f065092b244a121212c5976bf45feaa2ce19e22 Mon Sep 17 00:00:00 2001 From: Ash Berlin-Taylor Date: Mon, 12 Dec 2022 22:34:03 +0000 Subject: [PATCH 042/158] Add SIGUSR2 handler for LocalTaskJob and workers to aid debugging (#28309) There have been multiple reports of people with tasks stuck in the running state, and no obvious activity from the running task, but the supervisor is still actively heart beating. In order to make it easier/possibly to tell _where_ the process is stuck we add a SIGUSR2 handler to the Task supervisor (that is purposefully inherited to the actual task process itself) that will print the current stack trace on receiving USR2 - is the same signal we use for causing a debug dump in the Scheduler. (cherry picked from commit f9e8969ff77d9a6498a779a912f78ca309c95aaa) --- airflow/jobs/local_task_job.py | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/airflow/jobs/local_task_job.py b/airflow/jobs/local_task_job.py index 07541ff3ca8f3..cc449e1fc56fb 100644 --- a/airflow/jobs/local_task_job.py +++ b/airflow/jobs/local_task_job.py @@ -120,8 +120,19 @@ def segfault_signal_handler(signum, frame): self.handle_task_exit(128 + signum) raise AirflowException("Segmentation Fault detected.") + def sigusr2_debug_handler(signum, frame): + import sys + import threading + import traceback + + id2name = {th.ident: th.name for th in threading.enumerate()} + for threadId, stack in sys._current_frames().items(): + print(id2name[threadId]) + traceback.print_stack(f=stack) + signal.signal(signal.SIGSEGV, segfault_signal_handler) signal.signal(signal.SIGTERM, signal_handler) + signal.signal(signal.SIGUSR2, sigusr2_debug_handler) if not self.task_instance.check_and_change_state_before_execution( mark_success=self.mark_success, From 4edd58845a15f99f56cccbeb7ba68f7201c8b490 Mon Sep 17 00:00:00 2001 From: Jarek Potiuk Date: Wed, 14 Dec 2022 00:06:51 +0100 Subject: [PATCH 043/158] Separate ER diagram dependencies to doc_gen extra (#28346) The eralchemy dependency requires pygraphviz which does not play well with MacOS/M1. For those who want to easily install `doc` extra, it is a blocker. Now the extra is separated. (cherry picked from commit b6bc318f6fb55fbaf61928ba67343d33e6fed73b) --- CONTRIBUTING.rst | 14 +++++++------- INSTALL | 14 +++++++------- docs/apache-airflow/extra-packages-ref.rst | 4 +++- setup.py | 6 +++++- 4 files changed, 22 insertions(+), 16 deletions(-) diff --git a/CONTRIBUTING.rst b/CONTRIBUTING.rst index f15127ce6c1ee..a5e00305c1297 100644 --- a/CONTRIBUTING.rst +++ b/CONTRIBUTING.rst @@ -615,13 +615,13 @@ apache.druid, apache.hdfs, apache.hive, apache.kylin, apache.livy, apache.pig, a apache.spark, apache.sqoop, apache.webhdfs, arangodb, asana, async, atlas, atlassian.jira, aws, azure, cassandra, celery, cgroups, cloudant, cncf.kubernetes, common.sql, crypto, dask, databricks, datadog, dbt.cloud, deprecated_api, devel, devel_all, devel_ci, devel_hadoop, dingding, discord, -doc, docker, druid, elasticsearch, exasol, facebook, ftp, gcp, gcp_api, github, github_enterprise, -google, google_auth, grpc, hashicorp, hdfs, hive, http, imap, influxdb, jdbc, jenkins, kerberos, -kubernetes, ldap, leveldb, microsoft.azure, microsoft.mssql, microsoft.psrp, microsoft.winrm, mongo, -mssql, mysql, neo4j, odbc, openfaas, opsgenie, oracle, pagerduty, pandas, papermill, password, -pinot, plexus, postgres, presto, qds, qubole, rabbitmq, redis, s3, salesforce, samba, segment, -sendgrid, sentry, sftp, singularity, slack, snowflake, spark, sqlite, ssh, statsd, tableau, tabular, -telegram, trino, vertica, virtualenv, webhdfs, winrm, yandex, zendesk +doc, doc_gen, docker, druid, elasticsearch, exasol, facebook, ftp, gcp, gcp_api, github, +github_enterprise, google, google_auth, grpc, hashicorp, hdfs, hive, http, imap, influxdb, jdbc, +jenkins, kerberos, kubernetes, ldap, leveldb, microsoft.azure, microsoft.mssql, microsoft.psrp, +microsoft.winrm, mongo, mssql, mysql, neo4j, odbc, openfaas, opsgenie, oracle, pagerduty, pandas, +papermill, password, pinot, plexus, postgres, presto, qds, qubole, rabbitmq, redis, s3, salesforce, +samba, segment, sendgrid, sentry, sftp, singularity, slack, snowflake, spark, sqlite, ssh, statsd, +tableau, tabular, telegram, trino, vertica, virtualenv, webhdfs, winrm, yandex, zendesk .. END EXTRAS HERE Provider packages diff --git a/INSTALL b/INSTALL index 618ab7cc9ea00..d1721a3a7152f 100644 --- a/INSTALL +++ b/INSTALL @@ -99,13 +99,13 @@ apache.druid, apache.hdfs, apache.hive, apache.kylin, apache.livy, apache.pig, a apache.spark, apache.sqoop, apache.webhdfs, arangodb, asana, async, atlas, atlassian.jira, aws, azure, cassandra, celery, cgroups, cloudant, cncf.kubernetes, common.sql, crypto, dask, databricks, datadog, dbt.cloud, deprecated_api, devel, devel_all, devel_ci, devel_hadoop, dingding, discord, -doc, docker, druid, elasticsearch, exasol, facebook, ftp, gcp, gcp_api, github, github_enterprise, -google, google_auth, grpc, hashicorp, hdfs, hive, http, imap, influxdb, jdbc, jenkins, kerberos, -kubernetes, ldap, leveldb, microsoft.azure, microsoft.mssql, microsoft.psrp, microsoft.winrm, mongo, -mssql, mysql, neo4j, odbc, openfaas, opsgenie, oracle, pagerduty, pandas, papermill, password, -pinot, plexus, postgres, presto, qds, qubole, rabbitmq, redis, s3, salesforce, samba, segment, -sendgrid, sentry, sftp, singularity, slack, snowflake, spark, sqlite, ssh, statsd, tableau, tabular, -telegram, trino, vertica, virtualenv, webhdfs, winrm, yandex, zendesk +doc, doc_gen, docker, druid, elasticsearch, exasol, facebook, ftp, gcp, gcp_api, github, +github_enterprise, google, google_auth, grpc, hashicorp, hdfs, hive, http, imap, influxdb, jdbc, +jenkins, kerberos, kubernetes, ldap, leveldb, microsoft.azure, microsoft.mssql, microsoft.psrp, +microsoft.winrm, mongo, mssql, mysql, neo4j, odbc, openfaas, opsgenie, oracle, pagerduty, pandas, +papermill, password, pinot, plexus, postgres, presto, qds, qubole, rabbitmq, redis, s3, salesforce, +samba, segment, sendgrid, sentry, sftp, singularity, slack, snowflake, spark, sqlite, ssh, statsd, +tableau, tabular, telegram, trino, vertica, virtualenv, webhdfs, winrm, yandex, zendesk # END EXTRAS HERE # For installing Airflow in development environments - see CONTRIBUTING.rst diff --git a/docs/apache-airflow/extra-packages-ref.rst b/docs/apache-airflow/extra-packages-ref.rst index d896321792bcb..e9f4bfb3996c8 100644 --- a/docs/apache-airflow/extra-packages-ref.rst +++ b/docs/apache-airflow/extra-packages-ref.rst @@ -324,13 +324,15 @@ you need to list explicitly all the non-bundle extras that you want to install. Doc extras ---------- -This is the extra that is needed to generated documentation for Airflow. This is used for development time only +Those are the extras that are needed to generated documentation for Airflow. This is used for development time only +---------------------+-----------------------------------------------------+----------------------------------------------------------------------+ | extra | install command | enables | +---------------------+-----------------------------------------------------+----------------------------------------------------------------------+ | doc | ``pip install 'apache-airflow[doc]'`` | Packages needed to build docs (included in ``devel``) | +---------------------+-----------------------------------------------------+----------------------------------------------------------------------+ +| doc_gen | ``pip install 'apache-airflow[doc_gen]'`` | Packages needed to generate er diagrams (included in ``devel_all``) | ++---------------------+-----------------------------------------------------+----------------------------------------------------------------------+ Deprecated 1.10 extras diff --git a/setup.py b/setup.py index 1f0e8e39feaa3..b3e7768225ff8 100644 --- a/setup.py +++ b/setup.py @@ -275,6 +275,9 @@ def write_version(filename: str = str(AIRFLOW_SOURCES_ROOT / "airflow" / "git_ve "sphinxcontrib-redoc>=1.6.0", "sphinxcontrib-spelling>=7.3", ] +doc_gen = [ + "eralchemy2", +] flask_appbuilder_oauth = [ "authlib>=1.0.0", # The version here should be upgraded at the same time as flask-appbuilder in setup.cfg @@ -624,7 +627,7 @@ def get_all_db_dependencies() -> list[str]: # This can be simplified to devel_hadoop + _all_dependencies due to inclusions # but we keep it for explicit sake. We are de-duplicating it anyway. devel_all = get_unique_dependency_list( - [_all_dependencies_without_airflow_providers, doc, devel, devel_hadoop] + [_all_dependencies_without_airflow_providers, doc, doc_gen, devel, devel_hadoop] ) # Those are packages excluded for "all" dependencies @@ -670,6 +673,7 @@ def remove_provider_limits(package: str) -> str: # Those are extras that we have to add for development purposes # They can be use to install some predefined set of dependencies. EXTRAS_DEPENDENCIES["doc"] = doc +EXTRAS_DEPENDENCIES["doc_gen"] = doc_gen EXTRAS_DEPENDENCIES["devel"] = devel # devel already includes doc EXTRAS_DEPENDENCIES["devel_hadoop"] = devel_hadoop # devel_hadoop already includes devel EXTRAS_DEPENDENCIES["devel_all"] = devel_all From 3a8768eb45497018add32e0f7a959b810317375b Mon Sep 17 00:00:00 2001 From: Jarek Potiuk Date: Wed, 14 Dec 2022 00:21:46 +0100 Subject: [PATCH 044/158] Remove docs package from Publish_docs.py and switch to python in shebang (#28347) (cherry picked from commit 81271424e364822ff094ff9a75d213b223ccd0be) --- docs/publish_docs.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/publish_docs.py b/docs/publish_docs.py index 6666b596af69b..bf337995043a9 100755 --- a/docs/publish_docs.py +++ b/docs/publish_docs.py @@ -1,4 +1,4 @@ -#!/usr/bin/env python3 +#!/usr/bin/env python # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file @@ -21,9 +21,9 @@ import argparse import os -from docs.exts.docs_build.docs_builder import AirflowDocsBuilder -from docs.exts.docs_build.package_filter import process_package_filters -from docs.exts.provider_yaml_utils import load_package_data +from exts.docs_build.docs_builder import AirflowDocsBuilder +from exts.docs_build.package_filter import process_package_filters +from exts.provider_yaml_utils import load_package_data AIRFLOW_SITE_DIR = os.environ.get("AIRFLOW_SITE_DIRECTORY") From 26533494072fcd2014963de2f87c9c8440fb19db Mon Sep 17 00:00:00 2001 From: Tzu-ping Chung Date: Fri, 16 Dec 2022 00:43:52 +0800 Subject: [PATCH 045/158] Correctly select a mapped task's "previous" task (#28379) (cherry picked from commit 8aac56656d29009dbca24a5948c2a2097043f4f3) --- airflow/ti_deps/deps/prev_dagrun_dep.py | 2 +- tests/models/test_dagrun.py | 35 +++++++++++++++++++++++++ 2 files changed, 36 insertions(+), 1 deletion(-) diff --git a/airflow/ti_deps/deps/prev_dagrun_dep.py b/airflow/ti_deps/deps/prev_dagrun_dep.py index 66dcdcc16b72c..dce4b8527f887 100644 --- a/airflow/ti_deps/deps/prev_dagrun_dep.py +++ b/airflow/ti_deps/deps/prev_dagrun_dep.py @@ -68,7 +68,7 @@ def _get_dep_statuses(self, ti: TI, session, dep_context): yield self._passing_status(reason="This task instance was the first task instance for its task.") return - previous_ti = last_dagrun.get_task_instance(ti.task_id, session=session) + previous_ti = last_dagrun.get_task_instance(ti.task_id, map_index=ti.map_index, session=session) if not previous_ti: if ti.task.ignore_first_depends_on_past: has_historical_ti = ( diff --git a/tests/models/test_dagrun.py b/tests/models/test_dagrun.py index c08b548870add..3d368ae962a64 100644 --- a/tests/models/test_dagrun.py +++ b/tests/models/test_dagrun.py @@ -2187,3 +2187,38 @@ def say_bye(): assert tis["add_one__1"] == TaskInstanceState.SKIPPED assert tis["add_one__2"] == TaskInstanceState.SKIPPED assert dr.state == State.SUCCESS + + +def test_mapped_task_depends_on_past(dag_maker, session): + with dag_maker(session=session): + + @task(depends_on_past=True) + def print_value(value): + print(value) + + print_value.expand_kwargs([{"value": i} for i in range(2)]) + + dr1: DagRun = dag_maker.create_dagrun(run_type=DagRunType.SCHEDULED) + dr2: DagRun = dag_maker.create_dagrun_after(dr1, run_type=DagRunType.SCHEDULED) + + # print_value in dr2 is not ready yet since the task depends on past. + decision = dr2.task_instance_scheduling_decisions(session=session) + assert len(decision.schedulable_tis) == 0 + + # Run print_value in dr1. + decision = dr1.task_instance_scheduling_decisions(session=session) + assert len(decision.schedulable_tis) == 2 + for ti in decision.schedulable_tis: + ti.run(session=session) + + # Now print_value in dr2 can run + decision = dr2.task_instance_scheduling_decisions(session=session) + assert len(decision.schedulable_tis) == 2 + for ti in decision.schedulable_tis: + ti.run(session=session) + + # Both runs are finished now. + decision = dr1.task_instance_scheduling_decisions(session=session) + assert len(decision.unfinished_tis) == 0 + decision = dr2.task_instance_scheduling_decisions(session=session) + assert len(decision.unfinished_tis) == 0 From 97f7de20412182e111cec596f56d95d40afb0752 Mon Sep 17 00:00:00 2001 From: Brent Bovenzi Date: Sun, 18 Dec 2022 19:00:34 -0600 Subject: [PATCH 046/158] Maintain manual scroll position in task logs (#28386) (cherry picked from commit 5c80d985a3102a46f198aec1c57a255e00784c51) --- airflow/www/static/js/api/useTaskLog.ts | 20 ++++- .../details/taskInstance/Logs/LogBlock.tsx | 90 +++++++++++++++++++ .../dag/details/taskInstance/Logs/index.tsx | 45 +++------- 3 files changed, 117 insertions(+), 38 deletions(-) create mode 100644 airflow/www/static/js/dag/details/taskInstance/Logs/LogBlock.tsx diff --git a/airflow/www/static/js/api/useTaskLog.ts b/airflow/www/static/js/api/useTaskLog.ts index 580c5e0ab4995..bbb6395878b4f 100644 --- a/airflow/www/static/js/api/useTaskLog.ts +++ b/airflow/www/static/js/api/useTaskLog.ts @@ -17,6 +17,7 @@ * under the License. */ +import { useState } from 'react'; import axios, { AxiosResponse } from 'axios'; import { useQuery } from 'react-query'; import { useAutoRefresh } from 'src/context/autorefresh'; @@ -34,6 +35,7 @@ const useTaskLog = ({ dagId, dagRunId, taskId, taskTryNumber, mapIndex, fullContent, state, }: Props) => { let url: string = ''; + const [isPreviousStatePending, setPrevState] = useState(true); if (taskLogApi) { url = taskLogApi.replace('_DAG_RUN_ID_', dagRunId).replace('_TASK_ID_', taskId).replace(/-1$/, taskTryNumber.toString()); } @@ -49,12 +51,24 @@ const useTaskLog = ({ || state === 'queued' || state === 'restarting'; + // We also want to get the last log when the task was finished + const expectingLogs = isStatePending || isPreviousStatePending; + return useQuery( - ['taskLogs', dagId, dagRunId, taskId, mapIndex, taskTryNumber, fullContent, state], - () => axios.get(url, { headers: { Accept: 'text/plain' }, params: { map_index: mapIndex, full_content: fullContent } }), + ['taskLogs', dagId, dagRunId, taskId, mapIndex, taskTryNumber, fullContent], + () => { + setPrevState(isStatePending); + return axios.get( + url, + { + headers: { Accept: 'text/plain' }, + params: { map_index: mapIndex, full_content: fullContent }, + }, + ); + }, { placeholderData: '', - refetchInterval: isStatePending && isRefreshOn && (autoRefreshInterval || 1) * 1000, + refetchInterval: expectingLogs && isRefreshOn && (autoRefreshInterval || 1) * 1000, }, ); }; diff --git a/airflow/www/static/js/dag/details/taskInstance/Logs/LogBlock.tsx b/airflow/www/static/js/dag/details/taskInstance/Logs/LogBlock.tsx new file mode 100644 index 0000000000000..0ffa76e21f905 --- /dev/null +++ b/airflow/www/static/js/dag/details/taskInstance/Logs/LogBlock.tsx @@ -0,0 +1,90 @@ +/*! + * 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 React, { + useRef, useEffect, useState, +} from 'react'; +import { + Code, +} from '@chakra-ui/react'; + +import useOffsetHeight from 'src/utils/useOffsetHeight'; + +interface Props { + parsedLogs: string; + wrap: boolean; + tryNumber: number; +} + +const LogBlock = ({ + parsedLogs, + wrap, + tryNumber, +}: Props) => { + const [autoScroll, setAutoScroll] = useState(true); + const logBoxRef = useRef(null); + + const maxHeight = useOffsetHeight(logBoxRef, parsedLogs); + + const codeBlockBottomDiv = useRef(null); + + const scrollToBottom = () => { + codeBlockBottomDiv.current?.scrollIntoView({ block: 'nearest', inline: 'nearest' }); + }; + + useEffect(() => { + // Always scroll to bottom when wrap or tryNumber change + scrollToBottom(); + }, [wrap, tryNumber]); + + useEffect(() => { + // When logs change, only scroll if autoScroll is enabled + if (autoScroll) scrollToBottom(); + }, [parsedLogs, autoScroll]); + + const onScroll = (e: React.UIEvent) => { + if (e.currentTarget) { + const { scrollTop, offsetHeight, scrollHeight } = e.currentTarget; + // Enable autoscroll if we've scrolled to the bottom of the logs + setAutoScroll(scrollTop + offsetHeight >= scrollHeight); + } + }; + + return ( + + {parsedLogs} +
+ + ); +}; + +export default LogBlock; diff --git a/airflow/www/static/js/dag/details/taskInstance/Logs/index.tsx b/airflow/www/static/js/dag/details/taskInstance/Logs/index.tsx index 43976a7b52974..0139d8e22392a 100644 --- a/airflow/www/static/js/dag/details/taskInstance/Logs/index.tsx +++ b/airflow/www/static/js/dag/details/taskInstance/Logs/index.tsx @@ -18,14 +18,13 @@ */ import React, { - useRef, useState, useEffect, useMemo, + useState, useEffect, useMemo, } from 'react'; import { Text, Box, Flex, Divider, - Code, Button, Checkbox, } from '@chakra-ui/react'; @@ -36,12 +35,12 @@ import LinkButton from 'src/components/LinkButton'; import { useTimezone } from 'src/context/timezone'; import type { Dag, DagRun, TaskInstance } from 'src/types'; import MultiSelect from 'src/components/MultiSelect'; -import useOffsetHeight from 'src/utils/useOffsetHeight'; import URLSearchParamsWrapper from 'src/utils/URLSearchParamWrapper'; import LogLink from './LogLink'; import { LogLevel, logLevelColorMapping, parseLogs } from './utils'; +import LogBlock from './LogBlock'; interface LogLevelOption { label: LogLevel; @@ -108,10 +107,9 @@ const Logs = ({ const [logLevelFilters, setLogLevelFilters] = useState>([]); const [fileSourceFilters, setFileSourceFilters] = useState>([]); const { timezone } = useTimezone(); - const logBoxRef = useRef(null); const taskTryNumber = selectedTryNumber || tryNumber || 1; - const { data, isSuccess } = useTaskLog({ + const { data } = useTaskLog({ dagId, dagRunId, taskId, @@ -121,8 +119,6 @@ const Logs = ({ state, }); - const offsetHeight = useOffsetHeight(logBoxRef, data); - const params = new URLSearchParamsWrapper({ task_id: taskId, execution_date: executionDate, @@ -142,14 +138,6 @@ const Logs = ({ [data, fileSourceFilters, logLevelFilters, timezone], ); - const codeBlockBottomDiv = useRef(null); - - useEffect(() => { - if (codeBlockBottomDiv.current && parsedLogs) { - codeBlockBottomDiv.current.scrollIntoView({ block: 'nearest', inline: 'nearest' }); - } - }, [wrap, parsedLogs]); - useEffect(() => { // Reset fileSourceFilters and selected attempt when changing to // a task that do not have those filters anymore. @@ -257,26 +245,13 @@ const Logs = ({ - - {isSuccess && ( - <> - {parsedLogs} -
- - )} - + {!!parsedLogs && ( + + )} )} {externalLogName && externalIndexes.length > 0 && ( From ae49fcf5207aa40b68a01618daa897e30de48320 Mon Sep 17 00:00:00 2001 From: Brent Bovenzi Date: Fri, 16 Dec 2022 12:04:36 -0600 Subject: [PATCH 047/158] Dont show task/run durations when there is no start_date (#28395) * Dont show broken task/run duration * add test (cherry picked from commit 11f30a887c77f9636e88e31dffd969056132ae8c) --- .../static/js/dag/InstanceTooltip.test.tsx | 14 +++++++++++ airflow/www/static/js/dag/InstanceTooltip.tsx | 24 +++++++++++-------- .../static/js/dag/details/dagRun/index.tsx | 14 ++++++----- .../js/dag/details/taskInstance/Details.tsx | 16 +++++++------ airflow/www/yarn.lock | 18 ++++---------- 5 files changed, 49 insertions(+), 37 deletions(-) diff --git a/airflow/www/static/js/dag/InstanceTooltip.test.tsx b/airflow/www/static/js/dag/InstanceTooltip.test.tsx index 8543057e7b2d0..4a8f2b9192ea5 100644 --- a/airflow/www/static/js/dag/InstanceTooltip.test.tsx +++ b/airflow/www/static/js/dag/InstanceTooltip.test.tsx @@ -48,6 +48,7 @@ describe('Test Task InstanceTooltip', () => { expect(getByText('Status: success')).toBeDefined(); expect(queryByText('Contains a note')).toBeNull(); + expect(getByText('Duration: 00:00:00')).toBeDefined(); }); test('Displays a mapped task with overall status', () => { @@ -111,4 +112,17 @@ describe('Test Task InstanceTooltip', () => { expect(getByText('Contains a note')).toBeInTheDocument(); }); + + test('Hides duration if there is no start date', () => { + const { queryByText, getByText } = render( + , + { wrapper: Wrapper }, + ); + + expect(getByText('Status: success')).toBeDefined(); + expect(queryByText('Duration: 00:00:00')).toBeNull(); + }); }); diff --git a/airflow/www/static/js/dag/InstanceTooltip.tsx b/airflow/www/static/js/dag/InstanceTooltip.tsx index f5aad277bbcca..ca1045fda9ae8 100644 --- a/airflow/www/static/js/dag/InstanceTooltip.tsx +++ b/airflow/www/static/js/dag/InstanceTooltip.tsx @@ -93,16 +93,20 @@ const InstanceTooltip = ({ {state || 'no status'} {(isGroup || isMapped) && summary} - - Started: - {' '} - - - Duration: - {' '} - {formatDuration(getDuration(startDate, endDate))} - + {startDate && ( + <> + + Started: + {' '} + + + Duration: + {' '} + {formatDuration(getDuration(startDate, endDate))} + + + )} {note && ( Contains a note )} diff --git a/airflow/www/static/js/dag/details/dagRun/index.tsx b/airflow/www/static/js/dag/details/dagRun/index.tsx index ffbf613da50ae..3f69e693d5bb6 100644 --- a/airflow/www/static/js/dag/details/dagRun/index.tsx +++ b/airflow/www/static/js/dag/details/dagRun/index.tsx @@ -144,12 +144,14 @@ const DagRun = ({ runId }: Props) => { {runType} - - Run duration - - {formatDuration(getDuration(startDate, endDate))} - - + {startDate && ( + + Run duration + + {formatDuration(getDuration(startDate, endDate))} + + + )} {lastSchedulingDecision && ( Last scheduling decision diff --git a/airflow/www/static/js/dag/details/taskInstance/Details.tsx b/airflow/www/static/js/dag/details/taskInstance/Details.tsx index 7d9dfa1a1c20b..ea8a59f657bee 100644 --- a/airflow/www/static/js/dag/details/taskInstance/Details.tsx +++ b/airflow/www/static/js/dag/details/taskInstance/Details.tsx @@ -199,13 +199,15 @@ const Details = ({ instance, group, dagId }: Props) => { {operator} )} - - - {isOverall} - Duration - - {formatDuration(getDuration(startDate, endDate))} - + {startDate && ( + + + {isOverall} + Duration + + {formatDuration(getDuration(startDate, endDate))} + + )} {startDate && ( Started diff --git a/airflow/www/yarn.lock b/airflow/www/yarn.lock index a30b381fe4945..5431b00d520d5 100644 --- a/airflow/www/yarn.lock +++ b/airflow/www/yarn.lock @@ -3947,20 +3947,10 @@ caniuse-api@^3.0.0: lodash.memoize "^4.1.2" lodash.uniq "^4.5.0" -caniuse-lite@^1.0.0, caniuse-lite@^1.0.30001109: - version "1.0.30001355" - resolved "https://registry.yarnpkg.com/caniuse-lite/-/caniuse-lite-1.0.30001355.tgz#e240b7177443ed0198c737a7f609536976701c77" - integrity sha512-Sd6pjJHF27LzCB7pT7qs+kuX2ndurzCzkpJl6Qct7LPSZ9jn0bkOA8mdgMgmqnQAWLVOOGjLpc+66V57eLtb1g== - -caniuse-lite@^1.0.30001219: - version "1.0.30001312" - resolved "https://registry.npmjs.org/caniuse-lite/-/caniuse-lite-1.0.30001312.tgz" - integrity sha512-Wiz1Psk2MEK0pX3rUzWaunLTZzqS2JYZFzNKqAiJGiuxIjRPLgV6+VDPOg6lQOUxmDwhTlh198JsTTi8Hzw6aQ== - -caniuse-lite@^1.0.30001349: - version "1.0.30001354" - resolved "https://registry.yarnpkg.com/caniuse-lite/-/caniuse-lite-1.0.30001354.tgz#95c5efdb64148bb4870771749b9a619304755ce5" - integrity sha512-mImKeCkyGDAHNywYFA4bqnLAzTUvVkqPvhY4DV47X+Gl2c5Z8c3KNETnXp14GQt11LvxE8AwjzGxJ+rsikiOzg== +caniuse-lite@^1.0.0, caniuse-lite@^1.0.30001109, caniuse-lite@^1.0.30001219, caniuse-lite@^1.0.30001349: + version "1.0.30001439" + resolved "https://registry.npmjs.org/caniuse-lite/-/caniuse-lite-1.0.30001439.tgz" + integrity sha512-1MgUzEkoMO6gKfXflStpYgZDlFM7M/ck/bgfVCACO5vnAf0fXoNVHdWtqGU+MYca+4bL9Z5bpOVmR33cWW9G2A== chakra-react-select@^4.0.0: version "4.0.3" From cf966c86336690a3c09c3d8f43e30b15f2a66160 Mon Sep 17 00:00:00 2001 From: Tzu-ping Chung Date: Thu, 22 Dec 2022 09:54:26 +0800 Subject: [PATCH 048/158] A manual run can't look like a scheduled one (#28397) Fix https://github.com/apache/airflow/issues/27818 (cherry picked from commit 7ccbe4e7eaa529641052779a89e34d54c5a20f72) --- airflow/models/dag.py | 23 ++++++++++++++++++----- tests/models/test_dag.py | 20 ++++++++++++++++++++ 2 files changed, 38 insertions(+), 5 deletions(-) diff --git a/airflow/models/dag.py b/airflow/models/dag.py index c08a7f236f5d1..63856349630a2 100644 --- a/airflow/models/dag.py +++ b/airflow/models/dag.py @@ -2549,7 +2549,7 @@ def create_dagrun( external_trigger: bool | None = False, conf: dict | None = None, run_type: DagRunType | None = None, - session=NEW_SESSION, + session: Session = NEW_SESSION, dag_hash: str | None = None, creating_job_id: int | None = None, data_interval: tuple[datetime, datetime] | None = None, @@ -2586,14 +2586,27 @@ def create_dagrun( else: data_interval = self.infer_automated_data_interval(logical_date) + if run_type is None or isinstance(run_type, DagRunType): + pass + elif isinstance(run_type, str): # Compatibility: run_type used to be a str. + run_type = DagRunType(run_type) + else: + raise ValueError(f"`run_type` should be a DagRunType, not {type(run_type)}") + if run_id: # Infer run_type from run_id if needed. if not isinstance(run_id, str): raise ValueError(f"`run_id` should be a str, not {type(run_id)}") - if not run_type: - run_type = DagRunType.from_run_id(run_id) + inferred_run_type = DagRunType.from_run_id(run_id) + if run_type is None: + # No explicit type given, use the inferred type. + run_type = inferred_run_type + elif run_type == DagRunType.MANUAL and inferred_run_type != DagRunType.MANUAL: + # Prevent a manual run from using an ID that looks like a scheduled run. + raise ValueError( + f"A {run_type.value} DAG run cannot use ID {run_id!r} since it " + f"is reserved for {inferred_run_type.value} runs" + ) elif run_type and logical_date is not None: # Generate run_id from run_type and execution_date. - if not isinstance(run_type, DagRunType): - raise ValueError(f"`run_type` should be a DagRunType, not {type(run_type)}") run_id = self.timetable.generate_run_id( run_type=run_type, logical_date=logical_date, data_interval=data_interval ) diff --git a/tests/models/test_dag.py b/tests/models/test_dag.py index 1c741be2d580e..0141d71ecd6e6 100644 --- a/tests/models/test_dag.py +++ b/tests/models/test_dag.py @@ -3147,3 +3147,23 @@ def generate_run_id(self, *, run_type, logical_date, data_interval, **extra) -> ) assert dag_run.run_id == "abc" + + +@pytest.mark.parametrize( + "run_id_type", + [DagRunType.BACKFILL_JOB, DagRunType.SCHEDULED, DagRunType.DATASET_TRIGGERED], +) +def test_create_dagrun_disallow_manual_to_use_automated_run_id(run_id_type: DagRunType) -> None: + dag = DAG(dag_id="test", start_date=DEFAULT_DATE, schedule="@daily") + run_id = run_id_type.generate_run_id(DEFAULT_DATE) + with pytest.raises(ValueError) as ctx: + dag.create_dagrun( + run_type=DagRunType.MANUAL, + run_id=run_id, + execution_date=DEFAULT_DATE, + data_interval=(DEFAULT_DATE, DEFAULT_DATE), + state=DagRunState.QUEUED, + ) + assert str(ctx.value) == ( + f"A manual DAG run cannot use ID {run_id!r} since it is reserved for {run_id_type.value} runs" + ) From d3824d58fbbcff2dce9b2318607cc453d10cf412 Mon Sep 17 00:00:00 2001 From: Brent Bovenzi Date: Thu, 22 Dec 2022 08:19:40 -0500 Subject: [PATCH 049/158] separate callModal from dag.js (#28410) (cherry picked from commit 2f0f02536f7773dd782bd980ae932091b7badc61) --- airflow/www/static/js/callModal.js | 353 +++++++++++++++++++++++++++++ airflow/www/static/js/dag.js | 327 +------------------------- airflow/www/static/js/gantt.js | 2 +- airflow/www/static/js/graph.js | 2 +- 4 files changed, 357 insertions(+), 327 deletions(-) create mode 100644 airflow/www/static/js/callModal.js diff --git a/airflow/www/static/js/callModal.js b/airflow/www/static/js/callModal.js new file mode 100644 index 0000000000000..4d68d0ba1a987 --- /dev/null +++ b/airflow/www/static/js/callModal.js @@ -0,0 +1,353 @@ +/*! + * 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. + */ + +/* global document, window, $ */ + +import { getMetaValue } from './utils'; +import { formatDateTime } from './datetime_utils'; + +function updateQueryStringParameter(uri, key, value) { + const re = new RegExp(`([?&])${key}=.*?(&|$)`, 'i'); + const separator = uri.indexOf('?') !== -1 ? '&' : '?'; + if (uri.match(re)) { + return uri.replace(re, `$1${key}=${value}$2`); + } + + return `${uri}${separator}${key}=${value}`; +} + +const dagId = getMetaValue('dag_id'); +const logsWithMetadataUrl = getMetaValue('logs_with_metadata_url'); +const externalLogUrl = getMetaValue('external_log_url'); +const extraLinksUrl = getMetaValue('extra_links_url'); +const showExternalLogRedirect = getMetaValue('show_external_log_redirect') === 'True'; + +const buttons = Array.from(document.querySelectorAll('a[id^="btn_"][data-base-url]')).reduce((obj, elm) => { + obj[elm.id.replace('btn_', '')] = elm; + return obj; +}, {}); + +function updateButtonUrl(elm, params) { + let url = elm.dataset.baseUrl; + if (params.dag_id && elm.dataset.baseUrl.indexOf(dagId) !== -1) { + url = url.replace(dagId, params.dag_id); + delete params.dag_id; + } + if (Object.prototype.hasOwnProperty.call(params, 'map_index') && params.map_index === undefined) { + delete params.map_index; + } + elm.setAttribute('href', `${url}?${$.param(params)}`); +} + +function updateModalUrls({ + executionDate, + subDagId, + taskId, + mapIndex, + dagRunId, +}) { + updateButtonUrl(buttons.subdag, { + dag_id: subDagId, + execution_date: executionDate, + }); + + updateButtonUrl(buttons.task, { + dag_id: dagId, + task_id: taskId, + execution_date: executionDate, + map_index: mapIndex, + }); + + updateButtonUrl(buttons.rendered, { + dag_id: dagId, + task_id: taskId, + execution_date: executionDate, + map_index: mapIndex, + }); + + updateButtonUrl(buttons.mapped, { + _flt_3_dag_id: dagId, + _flt_3_task_id: taskId, + _flt_3_run_id: dagRunId, + _oc_TaskInstanceModelView: 'map_index', + }); + + if (buttons.rendered_k8s) { + updateButtonUrl(buttons.rendered_k8s, { + dag_id: dagId, + task_id: taskId, + execution_date: executionDate, + map_index: mapIndex, + }); + } + + const tiButtonParams = { + _flt_3_dag_id: dagId, + _flt_3_task_id: taskId, + _oc_TaskInstanceModelView: 'dag_run.execution_date', + }; + // eslint-disable-next-line no-underscore-dangle + if (mapIndex >= 0) tiButtonParams._flt_0_map_index = mapIndex; + updateButtonUrl(buttons.ti, tiButtonParams); + + updateButtonUrl(buttons.log, { + dag_id: dagId, + task_id: taskId, + execution_date: executionDate, + map_index: mapIndex, + }); + + updateButtonUrl(buttons.xcom, { + dag_id: dagId, + task_id: taskId, + execution_date: executionDate, + map_index: mapIndex, + }); +} + +function callModal({ + taskId, + executionDate, + extraLinks, + tryNumber, + isSubDag, + dagRunId, + mapIndex = -1, + isMapped = false, + mappedStates = [], +}) { + // Turn off previous event listeners + $('.map_index_item').off('click'); + $('form[data-action]').off('submit'); + + const location = String(window.location); + $('#btn_filter').on('click', () => { + window.location = updateQueryStringParameter(location, 'root', taskId); + }); + $('#dag_run_id').text(dagRunId); + $('#task_id').text(taskId); + $('#execution_date').text(formatDateTime(executionDate)); + $('#taskInstanceModal').modal({}); + $('#taskInstanceModal').css('margin-top', '0'); + $('#extra_links').prev('hr').hide(); + $('#extra_links').empty().hide(); + if (mapIndex >= 0) { + $('#modal_map_index').show(); + $('#modal_map_index .value').text(mapIndex); + } else { + $('#modal_map_index').hide(); + $('#modal_map_index .value').text(''); + } + + let subDagId; + if (isSubDag) { + $('#div_btn_subdag').show(); + subDagId = `${dagId}.${taskId}`; + } else { + $('#div_btn_subdag').hide(); + } + + // Show a span or dropdown for mapIndex + if (mapIndex >= 0 && !mappedStates.length) { + $('#modal_map_index').show(); + $('#modal_map_index .value').text(mapIndex); + $('#mapped_dropdown').hide(); + } else if (mapIndex >= 0 || isMapped) { + $('#modal_map_index').show(); + $('#modal_map_index .value').text(''); + $('#mapped_dropdown').show(); + + const dropdownText = mapIndex > -1 + ? mapIndex + : `All ${mappedStates.length} Mapped Instances`; + $('#mapped_dropdown #dropdown-label').text(dropdownText); + $('#mapped_dropdown .dropdown-menu').empty(); + $('#mapped_dropdown .dropdown-menu') + .append(`
  • All ${mappedStates.length} Mapped Instances
  • `); + mappedStates.forEach((state, i) => { + $('#mapped_dropdown .dropdown-menu') + .append(`
  • ${i} - ${state}
  • `); + }); + } else { + $('#modal_map_index').hide(); + $('#modal_map_index .value').text(''); + $('#mapped_dropdown').hide(); + } + + if (isMapped) { + $('#task_actions').text(`Task Actions for all ${mappedStates.length} instances`); + $('#btn_mapped').show(); + $('#mapped_dropdown').css('display', 'inline-block'); + $('#btn_rendered').hide(); + $('#btn_xcom').hide(); + $('#btn_log').hide(); + $('#btn_task').hide(); + } else { + $('#task_actions').text('Task Actions'); + $('#btn_rendered').show(); + $('#btn_xcom').show(); + $('#btn_log').show(); + $('#btn_mapped').hide(); + $('#btn_task').show(); + } + + $('#dag_dl_logs').hide(); + $('#dag_redir_logs').hide(); + if (tryNumber > 0 && !isMapped) { + $('#dag_dl_logs').show(); + if (showExternalLogRedirect) { + $('#dag_redir_logs').show(); + } + } + + updateModalUrls({ + executionDate, + subDagId, + taskId, + mapIndex, + dagRunId, + }); + + $('#try_index > li').remove(); + $('#redir_log_try_index > li').remove(); + const startIndex = (tryNumber > 2 ? 0 : 1); + + const query = new URLSearchParams({ + dag_id: dagId, + task_id: taskId, + execution_date: executionDate, + metadata: 'null', + }); + if (mapIndex !== undefined) { + query.set('map_index', mapIndex); + } + for (let index = startIndex; index < tryNumber; index += 1) { + let showLabel = index; + if (index !== 0) { + query.set('try_number', index); + } else { + showLabel = 'All'; + } + + $('#try_index').append(`
  • + ${showLabel} +
  • `); + + if (index !== 0 || showExternalLogRedirect) { + $('#redir_log_try_index').append(`
  • + ${showLabel} +
  • `); + } + } + query.delete('try_number'); + + if (!isMapped && extraLinks && extraLinks.length > 0) { + const markupArr = []; + extraLinks.sort(); + $.each(extraLinks, (i, link) => { + query.set('link_name', link); + const externalLink = $(''); + const linkTooltip = $(''); + linkTooltip.append(externalLink); + externalLink.text(link); + + $.ajax( + { + url: `${extraLinksUrl}?${query}`, + cache: false, + success(data) { + externalLink.attr('href', data.url); + // open absolute (external) links in a new tab/window and relative (local) links + // directly + if (/^(?:[a-z]+:)?\/\//.test(data.url)) { + externalLink.attr('target', '_blank'); + } + externalLink.removeClass('disabled'); + linkTooltip.tooltip('disable'); + }, + error(data) { + linkTooltip.tooltip('hide').attr('title', data.responseJSON.error).tooltip('fixTitle'); + }, + }, + ); + + markupArr.push(linkTooltip); + }); + + const extraLinksSpan = $('#extra_links'); + extraLinksSpan.prev('hr').show(); + extraLinksSpan.append(markupArr).show(); + extraLinksSpan.find('[data-toggle="tooltip"]').tooltip(); + } + + // Switch the modal from a mapped task summary to a specific mapped task instance + function switchMapItem() { + const mi = $(this).attr('data-mapIndex'); + if (mi === 'all') { + callModal({ + taskId, + executionDate, + dagRunId, + extraLinks, + mapIndex: -1, + isMapped: true, + mappedStates, + }); + } else { + callModal({ + taskId, + executionDate, + dagRunId, + extraLinks, + mapIndex: mi, + }); + } + } + + // Task Instance Modal actions + function submit(e) { + e.preventDefault(); + const form = $(this).get(0); + if (dagRunId || executionDate) { + if (form.dag_run_id) { + form.dag_run_id.value = dagRunId; + } + if (form.execution_date) { + form.execution_date.value = executionDate; + } + form.origin.value = window.location; + if (form.task_id) { + form.task_id.value = taskId; + } + if (form.map_index && mapIndex >= 0) { + form.map_index.value = mapIndex; + } else if (form.map_index) { + form.map_index.remove(); + } + form.action = $(this).data('action'); + form.submit(); + } + } + + $('form[data-action]').on('submit', submit); + $('.map_index_item').on('click', switchMapItem); +} + +export default callModal; diff --git a/airflow/www/static/js/dag.js b/airflow/www/static/js/dag.js index f2c3ce39ee5d4..e205c1d8c55cf 100644 --- a/airflow/www/static/js/dag.js +++ b/airflow/www/static/js/dag.js @@ -23,36 +23,16 @@ import { getMetaValue } from './utils'; import { approxTimeFromNow, formatDateTime } from './datetime_utils'; import { openDatasetModal, getDatasetTooltipInfo } from './datasetUtils'; -function updateQueryStringParameter(uri, key, value) { - const re = new RegExp(`([?&])${key}=.*?(&|$)`, 'i'); - const separator = uri.indexOf('?') !== -1 ? '&' : '?'; - if (uri.match(re)) { - return uri.replace(re, `$1${key}=${value}$2`); - } - - return `${uri}${separator}${key}=${value}`; -} - const dagId = getMetaValue('dag_id'); -export const dagTZ = getMetaValue('dag_timezone'); -const logsWithMetadataUrl = getMetaValue('logs_with_metadata_url'); -const externalLogUrl = getMetaValue('external_log_url'); -const extraLinksUrl = getMetaValue('extra_links_url'); const pausedUrl = getMetaValue('paused_url'); +// eslint-disable-next-line import/prefer-default-export +export const dagTZ = getMetaValue('dag_timezone'); const datasetsUrl = getMetaValue('datasets_url'); const nextRun = { createAfter: getMetaValue('next_dagrun_create_after'), intervalStart: getMetaValue('next_dagrun_data_interval_start'), intervalEnd: getMetaValue('next_dagrun_data_interval_end'), }; -let taskId = ''; -let executionDate = ''; -let subdagId = ''; -let dagRunId = ''; -let mapIndex; -let mapStates = []; -let extraLinks; -const showExternalLogRedirect = getMetaValue('show_external_log_redirect') === 'True'; let nextDatasets = []; let nextDatasetsError; @@ -72,309 +52,6 @@ $(window).on('load', function onLoad() { } }); -const buttons = Array.from(document.querySelectorAll('a[id^="btn_"][data-base-url]')).reduce((obj, elm) => { - obj[elm.id.replace('btn_', '')] = elm; - return obj; -}, {}); - -function updateButtonUrl(elm, params) { - let url = elm.dataset.baseUrl; - if (params.dag_id && elm.dataset.baseUrl.indexOf(dagId) !== -1) { - url = url.replace(dagId, params.dag_id); - delete params.dag_id; - } - if (Object.prototype.hasOwnProperty.call(params, 'map_index') && params.map_index === undefined) { - delete params.map_index; - } - elm.setAttribute('href', `${url}?${$.param(params)}`); -} - -function updateModalUrls() { - updateButtonUrl(buttons.subdag, { - dag_id: subdagId, - execution_date: executionDate, - }); - - updateButtonUrl(buttons.task, { - dag_id: dagId, - task_id: taskId, - execution_date: executionDate, - map_index: mapIndex, - }); - - updateButtonUrl(buttons.rendered, { - dag_id: dagId, - task_id: taskId, - execution_date: executionDate, - map_index: mapIndex, - }); - - updateButtonUrl(buttons.mapped, { - _flt_3_dag_id: dagId, - _flt_3_task_id: taskId, - _flt_3_run_id: dagRunId, - _oc_TaskInstanceModelView: 'map_index', - }); - - if (buttons.rendered_k8s) { - updateButtonUrl(buttons.rendered_k8s, { - dag_id: dagId, - task_id: taskId, - execution_date: executionDate, - map_index: mapIndex, - }); - } - - const tiButtonParams = { - _flt_3_dag_id: dagId, - _flt_3_task_id: taskId, - _oc_TaskInstanceModelView: 'dag_run.execution_date', - }; - // eslint-disable-next-line no-underscore-dangle - if (mapIndex >= 0) tiButtonParams._flt_0_map_index = mapIndex; - updateButtonUrl(buttons.ti, tiButtonParams); - - updateButtonUrl(buttons.log, { - dag_id: dagId, - task_id: taskId, - execution_date: executionDate, - map_index: mapIndex, - }); - - updateButtonUrl(buttons.xcom, { - dag_id: dagId, - task_id: taskId, - execution_date: executionDate, - map_index: mapIndex, - }); -} - -// Update modal urls on toggle -document.addEventListener('click', (event) => { - if (event.target.matches('button[data-toggle="button"]')) { - updateModalUrls(); - } -}); - -export function callModal({ - taskId: t, - executionDate: d, - extraLinks: e, - tryNumber, - isSubDag, - dagRunId: drID, - mapIndex: mi, - isMapped = false, - mappedStates = [], -}) { - taskId = t; - const location = String(window.location); - $('#btn_filter').on('click', () => { - window.location = updateQueryStringParameter(location, 'root', taskId); - }); - executionDate = d; - dagRunId = drID; - mapIndex = mi; - extraLinks = e; - if (isMapped) { - mapStates = mappedStates; - } - $('#dag_run_id').text(drID); - $('#task_id').text(t); - $('#execution_date').text(formatDateTime(d)); - $('#taskInstanceModal').modal({}); - $('#taskInstanceModal').css('margin-top', '0'); - $('#extra_links').prev('hr').hide(); - $('#extra_links').empty().hide(); - if (mi >= 0) { - $('#modal_map_index').show(); - $('#modal_map_index .value').text(mi); - } else { - $('#modal_map_index').hide(); - $('#modal_map_index .value').text(''); - } - if (isSubDag) { - $('#div_btn_subdag').show(); - subdagId = `${dagId}.${t}`; - } else { - $('#div_btn_subdag').hide(); - subdagId = undefined; - } - - // Show a span or dropdown for mapIndex - if (mi >= 0 && !mapStates.length) { - $('#modal_map_index').show(); - $('#modal_map_index .value').text(mi); - $('#mapped_dropdown').hide(); - } else if (mi >= 0 || isMapped) { - $('#modal_map_index').show(); - $('#modal_map_index .value').text(''); - $('#mapped_dropdown').show(); - - const dropdownText = mapIndex > -1 ? mapIndex : `All ${mapStates.length} Mapped Instances`; - $('#mapped_dropdown #dropdown-label').text(dropdownText); - $('#mapped_dropdown .dropdown-menu').empty(); - $('#mapped_dropdown .dropdown-menu').append(`
  • All ${mapStates.length} Mapped Instances
  • `); - mapStates.forEach((state, i) => { - $('#mapped_dropdown .dropdown-menu').append(`
  • ${i} - ${state}
  • `); - }); - } else { - $('#modal_map_index').hide(); - $('#modal_map_index .value').text(''); - $('#mapped_dropdown').hide(); - } - - if (isMapped) { - $('#task_actions').text(`Task Actions for all ${mappedStates.length} instances`); - $('#btn_mapped').show(); - $('#mapped_dropdown').css('display', 'inline-block'); - $('#btn_rendered').hide(); - $('#btn_xcom').hide(); - $('#btn_log').hide(); - $('#btn_task').hide(); - } else { - $('#task_actions').text('Task Actions'); - $('#btn_rendered').show(); - $('#btn_xcom').show(); - $('#btn_log').show(); - $('#btn_mapped').hide(); - $('#btn_task').show(); - } - - $('#dag_dl_logs').hide(); - $('#dag_redir_logs').hide(); - if (tryNumber > 0 && !isMapped) { - $('#dag_dl_logs').show(); - if (showExternalLogRedirect) { - $('#dag_redir_logs').show(); - } - } - - updateModalUrls(); - - $('#try_index > li').remove(); - $('#redir_log_try_index > li').remove(); - const startIndex = (tryNumber > 2 ? 0 : 1); - - const query = new URLSearchParams({ - dag_id: dagId, - task_id: taskId, - execution_date: executionDate, - metadata: 'null', - }); - if (mi !== undefined) { - query.set('map_index', mi); - } - for (let index = startIndex; index < tryNumber; index += 1) { - let showLabel = index; - if (index !== 0) { - query.set('try_number', index); - } else { - showLabel = 'All'; - } - - $('#try_index').append(`
  • - ${showLabel} -
  • `); - - if (index !== 0 || showExternalLogRedirect) { - $('#redir_log_try_index').append(`
  • - ${showLabel} -
  • `); - } - } - query.delete('try_number'); - - if (!isMapped && extraLinks && extraLinks.length > 0) { - const markupArr = []; - extraLinks.sort(); - $.each(extraLinks, (i, link) => { - query.set('link_name', link); - const externalLink = $(''); - const linkTooltip = $(''); - linkTooltip.append(externalLink); - externalLink.text(link); - - $.ajax( - { - url: `${extraLinksUrl}?${query}`, - cache: false, - success(data) { - externalLink.attr('href', data.url); - // open absolute (external) links in a new tab/window and relative (local) links - // directly - if (/^(?:[a-z]+:)?\/\//.test(data.url)) { - externalLink.attr('target', '_blank'); - } - externalLink.removeClass('disabled'); - linkTooltip.tooltip('disable'); - }, - error(data) { - linkTooltip.tooltip('hide').attr('title', data.responseJSON.error).tooltip('fixTitle'); - }, - }, - ); - - markupArr.push(linkTooltip); - }); - - const extraLinksSpan = $('#extra_links'); - extraLinksSpan.prev('hr').show(); - extraLinksSpan.append(markupArr).show(); - extraLinksSpan.find('[data-toggle="tooltip"]').tooltip(); - } -} - -// Switch the modal from a mapped task summary to a specific mapped task instance -$(document).on('click', '.map_index_item', function mapItem() { - const mi = $(this).attr('data-mapIndex'); - if (mi === 'all') { - callModal({ - taskId, - executionDate, - dagRunId, - extraLinks, - mapIndex: -1, - isMapped: true, - mappedStates: mapStates, - }); - } else { - callModal({ - taskId, - executionDate, - dagRunId, - extraLinks, - mapIndex: mi, - }); - } -}); - -// Task Instance Modal actions -$('form[data-action]').on('submit', function submit(e) { - e.preventDefault(); - const form = $(this).get(0); - if (dagRunId || executionDate) { - if (form.dag_run_id) { - form.dag_run_id.value = dagRunId; - } - if (form.execution_date) { - form.execution_date.value = executionDate; - } - form.origin.value = window.location; - if (form.task_id) { - form.task_id.value = taskId; - } - if (form.map_index && mapIndex >= 0) { - form.map_index.value = mapIndex; - } else if (form.map_index) { - form.map_index.remove(); - } - form.action = $(this).data('action'); - form.submit(); - } -}); - $('#pause_resume').on('change', function onChange() { const $input = $(this); const id = $input.data('dag-id'); diff --git a/airflow/www/static/js/gantt.js b/airflow/www/static/js/gantt.js index 43704ea08196f..be87fb81772b7 100644 --- a/airflow/www/static/js/gantt.js +++ b/airflow/www/static/js/gantt.js @@ -33,7 +33,7 @@ /* global d3, document, moment, data $ */ import tiTooltip from './task_instances'; -import { callModal } from './dag'; +import callModal from './callModal'; const replacements = { a: 'ddd', diff --git a/airflow/www/static/js/graph.js b/airflow/www/static/js/graph.js index 715c46b3e2a8e..67370bd9c4b0d 100644 --- a/airflow/www/static/js/graph.js +++ b/airflow/www/static/js/graph.js @@ -27,7 +27,7 @@ import { getMetaValue, finalStatesMap } from './utils'; import { escapeHtml } from './main'; import tiTooltip, { taskNoInstanceTooltip } from './task_instances'; -import { callModal } from './dag'; +import callModal from './callModal'; // dagId comes from dag.html const dagId = getMetaValue('dag_id'); From b82f5af1e9f0ede56cce7a88f320bb51889da249 Mon Sep 17 00:00:00 2001 From: Brent Bovenzi Date: Thu, 5 Jan 2023 11:42:23 -0500 Subject: [PATCH 050/158] Fix ti._try_number for deferred and up_for_reschedule tasks (#26993) * have a consistent try_count util * Update airflow/www/utils.py Co-authored-by: pierrejeambrun * use _try_number, default to 1 * properly handle up_for_reschedule task try number * fix tests * add State.waiting for deferred and up for reschedule tasks * add State.pending var * remove confusing state aliases Co-authored-by: pierrejeambrun (cherry picked from commit f110cb11bf6fdf6ca9d0deecef9bd51fe370660a) --- airflow/api/common/mark_tasks.py | 8 ++++++-- airflow/models/taskinstance.py | 2 +- airflow/utils/log/file_task_handler.py | 2 +- airflow/utils/log/log_reader.py | 2 +- airflow/utils/state.py | 5 ----- airflow/www/utils.py | 12 +++++------- airflow/www/views.py | 17 ++++------------- tests/www/views/test_views_grid.py | 4 ++-- 8 files changed, 20 insertions(+), 32 deletions(-) diff --git a/airflow/api/common/mark_tasks.py b/airflow/api/common/mark_tasks.py index 46b912790b4a5..8e25e0f4fd660 100644 --- a/airflow/api/common/mark_tasks.py +++ b/airflow/api/common/mark_tasks.py @@ -154,6 +154,10 @@ def set_state( qry_sub_dag = all_subdag_tasks_query(sub_dag_run_ids, session, state, confirmed_dates) tis_altered += qry_sub_dag.with_for_update().all() for task_instance in tis_altered: + # The try_number was decremented when setting to up_for_reschedule and deferred. + # Increment it back when changing the state again + if task_instance.state in [State.DEFERRED, State.UP_FOR_RESCHEDULE]: + task_instance._try_number += 1 task_instance.set_state(state, session=session) session.flush() else: @@ -466,7 +470,7 @@ def set_dag_run_state_to_failed( TaskInstance.dag_id == dag.dag_id, TaskInstance.run_id == run_id, TaskInstance.task_id.in_(task_ids), - TaskInstance.state.in_(State.running), + TaskInstance.state.in_([State.RUNNING, State.DEFERRED, State.UP_FOR_RESCHEDULE]), ) task_ids_of_running_tis = [task_instance.task_id for task_instance in tis] @@ -482,7 +486,7 @@ def set_dag_run_state_to_failed( TaskInstance.dag_id == dag.dag_id, TaskInstance.run_id == run_id, TaskInstance.state.not_in(State.finished), - TaskInstance.state.not_in(State.running), + TaskInstance.state.not_in([State.RUNNING, State.DEFERRED, State.UP_FOR_RESCHEDULE]), ) tis = [ti for ti in tis] diff --git a/airflow/models/taskinstance.py b/airflow/models/taskinstance.py index 1be51834120e8..9708d73124b47 100644 --- a/airflow/models/taskinstance.py +++ b/airflow/models/taskinstance.py @@ -546,7 +546,7 @@ def try_number(self): database, in all other cases this will be incremented. """ # This is designed so that task logs end up in the right file. - if self.state in State.running: + if self.state == State.RUNNING: return self._try_number return self._try_number + 1 diff --git a/airflow/utils/log/file_task_handler.py b/airflow/utils/log/file_task_handler.py index 5a5b55d8a41a3..1dcdd745acd72 100644 --- a/airflow/utils/log/file_task_handler.py +++ b/airflow/utils/log/file_task_handler.py @@ -274,7 +274,7 @@ def _read(self, ti: TaskInstance, try_number: int, metadata: dict[str, Any] | No return log, {"end_of_log": True} # Process tailing if log is not at it's end - end_of_log = ti.try_number != try_number or ti.state not in State.running + end_of_log = ti.try_number != try_number or ti.state not in [State.RUNNING, State.DEFERRED] log_pos = len(log) if metadata and "log_pos" in metadata: previous_chars = metadata["log_pos"] diff --git a/airflow/utils/log/log_reader.py b/airflow/utils/log/log_reader.py index 0dbdddb90e3c8..7ad470019577c 100644 --- a/airflow/utils/log/log_reader.py +++ b/airflow/utils/log/log_reader.py @@ -78,7 +78,7 @@ def read_log_stream(self, ti: TaskInstance, try_number: int | None, metadata: di metadata.pop("offset", None) metadata.pop("log_pos", None) while "end_of_log" not in metadata or ( - not metadata["end_of_log"] and ti.state not in State.running + not metadata["end_of_log"] and ti.state not in [State.RUNNING, State.DEFERRED] ): logs, metadata = self.read_log_chunks(ti, current_try_number, metadata) for host, log in logs[0]: diff --git a/airflow/utils/state.py b/airflow/utils/state.py index 036bb9ab0c043..6558af9f2a995 100644 --- a/airflow/utils/state.py +++ b/airflow/utils/state.py @@ -137,11 +137,6 @@ def color_fg(cls, state): return "white" return "black" - running: frozenset[TaskInstanceState] = frozenset([TaskInstanceState.RUNNING, TaskInstanceState.DEFERRED]) - """ - A list of states indicating that a task is being executed. - """ - finished: frozenset[TaskInstanceState] = frozenset( [ TaskInstanceState.SUCCESS, diff --git a/airflow/www/utils.py b/airflow/www/utils.py index 971a2c7a82261..b1985a3c8a081 100644 --- a/airflow/www/utils.py +++ b/airflow/www/utils.py @@ -82,6 +82,10 @@ def get_instance_with_map(task_instance, session): return get_mapped_summary(task_instance, mapped_instances) +def get_try_count(try_number: int, state: State): + return try_number + 1 if state in [State.DEFERRED, State.UP_FOR_RESCHEDULE] else try_number + + priority = [ TaskInstanceState.FAILED, TaskInstanceState.UPSTREAM_FAILED, @@ -116,12 +120,6 @@ def get_mapped_summary(parent_instance, task_instances): max((ti.end_date for ti in task_instances if ti.end_date), default=None) ) - try_count = ( - parent_instance._try_number - if parent_instance._try_number != 0 or parent_instance.state in State.running - else parent_instance._try_number + 1 - ) - return { "task_id": parent_instance.task_id, "run_id": parent_instance.run_id, @@ -129,7 +127,7 @@ def get_mapped_summary(parent_instance, task_instances): "start_date": group_start_date, "end_date": group_end_date, "mapped_states": mapped_states, - "try_number": try_count, + "try_number": get_try_count(parent_instance._try_number, parent_instance.state), } diff --git a/airflow/www/views.py b/airflow/www/views.py index a22ca795695fd..8c649c9a32f1d 100644 --- a/airflow/www/views.py +++ b/airflow/www/views.py @@ -264,18 +264,18 @@ def dag_to_grid(dag, dag_runs, session): TaskInstance.task_id, TaskInstance.run_id, TaskInstance.state, + TaskInstance._try_number, func.min(TaskInstanceNote.content).label("note"), func.count(func.coalesce(TaskInstance.state, sqla.literal("no_status"))).label("state_count"), func.min(TaskInstance.start_date).label("start_date"), func.max(TaskInstance.end_date).label("end_date"), - func.max(TaskInstance._try_number).label("_try_number"), ) .join(TaskInstance.task_instance_note, isouter=True) .filter( TaskInstance.dag_id == dag.dag_id, TaskInstance.run_id.in_([dag_run.run_id for dag_run in dag_runs]), ) - .group_by(TaskInstance.task_id, TaskInstance.run_id, TaskInstance.state) + .group_by(TaskInstance.task_id, TaskInstance.run_id, TaskInstance.state, TaskInstance._try_number) .order_by(TaskInstance.task_id, TaskInstance.run_id) ) @@ -285,19 +285,13 @@ def task_group_to_grid(item, dag_runs, grouped_tis): if isinstance(item, AbstractOperator): def _get_summary(task_instance): - try_count = ( - task_instance._try_number - if task_instance._try_number != 0 or task_instance.state in State.running - else task_instance._try_number + 1 - ) - return { "task_id": task_instance.task_id, "run_id": task_instance.run_id, "state": task_instance.state, "start_date": task_instance.start_date, "end_date": task_instance.end_date, - "try_number": try_count, + "try_number": wwwutils.get_try_count(task_instance._try_number, task_instance.state), "note": task_instance.note, } @@ -1527,10 +1521,7 @@ def log(self, session=None): num_logs = 0 if ti is not None: - num_logs = ti.next_try_number - 1 - if ti.state in (State.UP_FOR_RESCHEDULE, State.DEFERRED): - # Tasks in reschedule state decremented the try number - num_logs += 1 + num_logs = wwwutils.get_try_count(ti._try_number, ti.state) logs = [""] * num_logs root = request.args.get("root", "") return self.render_template( diff --git a/tests/www/views/test_views_grid.py b/tests/www/views/test_views_grid.py index 2ff6db4751a7c..40dd1ce917368 100644 --- a/tests/www/views/test_views_grid.py +++ b/tests/www/views/test_views_grid.py @@ -216,7 +216,7 @@ def test_one_run(admin_client, dag_with_runs: list[DagRun], session): "note": None, "state": "success", "task_id": "task1", - "try_number": 1, + "try_number": 0, }, { "run_id": "run_2", @@ -225,7 +225,7 @@ def test_one_run(admin_client, dag_with_runs: list[DagRun], session): "note": None, "state": "success", "task_id": "task1", - "try_number": 1, + "try_number": 0, }, ], "is_mapped": False, From 17fc92f575e1cd8882222140f56b3a775ec89b36 Mon Sep 17 00:00:00 2001 From: cccs-seb <64980897+cccs-seb@users.noreply.github.com> Date: Tue, 13 Dec 2022 11:32:13 -0500 Subject: [PATCH 051/158] Add setters to MappedOperator on_*_callbacks (#28313) (cherry picked from commit 105dbd2ed140380b9af9e0c2d3086c3f052b9951) --- airflow/models/mappedoperator.py | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/airflow/models/mappedoperator.py b/airflow/models/mappedoperator.py index 9bdfd932de727..99e2b67f50018 100644 --- a/airflow/models/mappedoperator.py +++ b/airflow/models/mappedoperator.py @@ -450,18 +450,34 @@ def resources(self) -> Resources | None: def on_execute_callback(self) -> TaskStateChangeCallback | None: return self.partial_kwargs.get("on_execute_callback") + @on_execute_callback.setter + def on_execute_callback(self, value: TaskStateChangeCallback | None) -> None: + self.partial_kwargs["on_execute_callback"] = value + @property def on_failure_callback(self) -> TaskStateChangeCallback | None: return self.partial_kwargs.get("on_failure_callback") + @on_failure_callback.setter + def on_failure_callback(self, value: TaskStateChangeCallback | None) -> None: + self.partial_kwargs["on_failure_callback"] = value + @property def on_retry_callback(self) -> TaskStateChangeCallback | None: return self.partial_kwargs.get("on_retry_callback") + @on_retry_callback.setter + def on_retry_callback(self, value: TaskStateChangeCallback | None) -> None: + self.partial_kwargs["on_retry_callback"] = value + @property def on_success_callback(self) -> TaskStateChangeCallback | None: return self.partial_kwargs.get("on_success_callback") + @on_success_callback.setter + def on_success_callback(self, value: TaskStateChangeCallback | None) -> None: + self.partial_kwargs["on_success_callback"] = value + @property def run_as_user(self) -> str | None: return self.partial_kwargs.get("run_as_user") From 3113f89a3d2d4d27c58e1eac282ea6ff0c31913b Mon Sep 17 00:00:00 2001 From: Jarek Potiuk Date: Sun, 18 Dec 2022 21:40:29 +0100 Subject: [PATCH 052/158] Re-enable Plyvel on ARM as it now builds cleanly (#28443) Previously, Plyvel had to be disabled in order to get ARM compatibility (it did not have binary wheels released and it failed to compile cleanly on debian). But the last problem is fixed now, so we can re-enable it for ARM. (cherry picked from commit bea91b90180f075c974d58be438b80e3da8607ca) --- Dockerfile | 2 +- Dockerfile.ci | 2 +- scripts/docker/install_os_dependencies.sh | 2 +- setup.py | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/Dockerfile b/Dockerfile index d497d07c6d0ec..bab2cda06a41d 100644 --- a/Dockerfile +++ b/Dockerfile @@ -95,7 +95,7 @@ function get_dev_apt_deps() { if [[ "${DEV_APT_DEPS=}" == "" ]]; then DEV_APT_DEPS="apt-transport-https apt-utils build-essential ca-certificates dirmngr \ freetds-bin freetds-dev git gosu graphviz graphviz-dev krb5-user ldap-utils libffi-dev \ -libkrb5-dev libldap2-dev libsasl2-2 libsasl2-dev libsasl2-modules \ +libkrb5-dev libldap2-dev libleveldb1d libleveldb-dev libsasl2-2 libsasl2-dev libsasl2-modules \ libssl-dev locales lsb-release openssh-client sasl2-bin \ software-properties-common sqlite3 sudo unixodbc unixodbc-dev" export DEV_APT_DEPS diff --git a/Dockerfile.ci b/Dockerfile.ci index f2e64830fbbd6..0713cc8b715d3 100644 --- a/Dockerfile.ci +++ b/Dockerfile.ci @@ -55,7 +55,7 @@ function get_dev_apt_deps() { if [[ "${DEV_APT_DEPS=}" == "" ]]; then DEV_APT_DEPS="apt-transport-https apt-utils build-essential ca-certificates dirmngr \ freetds-bin freetds-dev git gosu graphviz graphviz-dev krb5-user ldap-utils libffi-dev \ -libkrb5-dev libldap2-dev libsasl2-2 libsasl2-dev libsasl2-modules \ +libkrb5-dev libldap2-dev libleveldb1d libleveldb-dev libsasl2-2 libsasl2-dev libsasl2-modules \ libssl-dev locales lsb-release openssh-client sasl2-bin \ software-properties-common sqlite3 sudo unixodbc unixodbc-dev" export DEV_APT_DEPS diff --git a/scripts/docker/install_os_dependencies.sh b/scripts/docker/install_os_dependencies.sh index 4ddeca0cf5f98..50aec97fa5895 100644 --- a/scripts/docker/install_os_dependencies.sh +++ b/scripts/docker/install_os_dependencies.sh @@ -37,7 +37,7 @@ function get_dev_apt_deps() { if [[ "${DEV_APT_DEPS=}" == "" ]]; then DEV_APT_DEPS="apt-transport-https apt-utils build-essential ca-certificates dirmngr \ freetds-bin freetds-dev git gosu graphviz graphviz-dev krb5-user ldap-utils libffi-dev \ -libkrb5-dev libldap2-dev libsasl2-2 libsasl2-dev libsasl2-modules \ +libkrb5-dev libldap2-dev libleveldb1d libleveldb-dev libsasl2-2 libsasl2-dev libsasl2-modules \ libssl-dev locales lsb-release openssh-client sasl2-bin \ software-properties-common sqlite3 sudo unixodbc unixodbc-dev" export DEV_APT_DEPS diff --git a/setup.py b/setup.py index b3e7768225ff8..2484b2076d0ec 100644 --- a/setup.py +++ b/setup.py @@ -304,7 +304,7 @@ def write_version(filename: str = str(AIRFLOW_SOURCES_ROOT / "airflow" / "git_ve "ldap3>=2.5.1", "python-ldap", ] -leveldb = ['plyvel; platform_machine != "aarch64"'] +leveldb = ["plyvel"] pandas = [ "pandas>=0.17.1", ] From 4657a63b78bef7f8a3d9a413291178e61ebdd090 Mon Sep 17 00:00:00 2001 From: Mansu Kim Date: Mon, 19 Dec 2022 15:09:36 +0900 Subject: [PATCH 053/158] Add `ensure_ascii=False` in trigger dag run API (#28451) * Add ensure_ascii=False in trigger dag run api * Fix static checks (cherry picked from commit c3eee4372556f9b09d3395a3f251c9ee21278846) --- airflow/www/views.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/airflow/www/views.py b/airflow/www/views.py index 8c649c9a32f1d..c645a56b58fb4 100644 --- a/airflow/www/views.py +++ b/airflow/www/views.py @@ -1898,7 +1898,9 @@ def trigger(self, session=None): else: try: default_conf = json.dumps( - {str(k): v.resolve(suppress_exception=True) for k, v in dag.params.items()}, indent=4 + {str(k): v.resolve(suppress_exception=True) for k, v in dag.params.items()}, + indent=4, + ensure_ascii=False, ) except TypeError: flash("Could not pre-populate conf field due to non-JSON-serializable data-types") From c57c7fcf3150e2f8dcf336cca25d73b732d7151d Mon Sep 17 00:00:00 2001 From: XuYicheng <109734725+xyc-pz@users.noreply.github.com> Date: Mon, 19 Dec 2022 18:58:28 +0800 Subject: [PATCH 054/158] Fix example import sentence in dates.py (#28453) The example import sentence in line 51 is incorrect (cherry picked from commit 0d12062198b2d4b0642b51da3c8f71e6f934d20d) --- airflow/utils/dates.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airflow/utils/dates.py b/airflow/utils/dates.py index 3a5b43df1e9c2..da84d792c65d0 100644 --- a/airflow/utils/dates.py +++ b/airflow/utils/dates.py @@ -48,7 +48,7 @@ def date_range( or a cron expression as a `str` .. code-block:: pycon - >>> from airflow.utils.dates import datterange + >>> from airflow.utils.dates import date_range >>> from datetime import datetime, timedelta >>> date_range(datetime(2016, 1, 1), datetime(2016, 1, 3), delta=timedelta(1)) [datetime.datetime(2016, 1, 1, 0, 0, tzinfo=Timezone('UTC')), From cf38e27af5a0c9dfa7c84e45cd6536e2c1933449 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Sun, 25 Dec 2022 23:56:13 -0800 Subject: [PATCH 055/158] Fix bad pods pickled in executor_config (#28454) We used to pickle raw pods objects but found that when unpickling across k8s lib versions we would get missing attr errors. Now, we serialize to json. But we still get reports of issues when people upgrade because it only solves the issue on a go-forward basis. But we can fix these old bad executor configs that keep popping up by roundtripping the pod to json in a more tolerant fashion than is done by the openapi-generated code, i.e. by populating missing attrs with None. (cherry picked from commit 27f07b0bf5ed088c4186296668a36dc89da25617) --- airflow/utils/sqlalchemy.py | 101 ++++++++++++++++++++++++++++++++- tests/utils/test_sqlalchemy.py | 53 ++++++++++++++++- 2 files changed, 151 insertions(+), 3 deletions(-) diff --git a/airflow/utils/sqlalchemy.py b/airflow/utils/sqlalchemy.py index 08a4b890b0a16..1ee482cc515d4 100644 --- a/airflow/utils/sqlalchemy.py +++ b/airflow/utils/sqlalchemy.py @@ -21,7 +21,7 @@ import datetime import json import logging -from typing import Any, Iterable +from typing import TYPE_CHECKING, Any, Iterable import pendulum from dateutil import relativedelta @@ -37,6 +37,9 @@ from airflow.configuration import conf from airflow.serialization.enums import Encoding +if TYPE_CHECKING: + from kubernetes.client.models.v1_pod import V1Pod + log = logging.getLogger(__name__) utc = pendulum.tz.timezone("UTC") @@ -153,6 +156,93 @@ def process_result_value(self, value, dialect): return BaseSerialization.deserialize(value) +def sanitize_for_serialization(obj: V1Pod): + """ + Convert pod to dict.... but *safely*. + + When pod objects created with one k8s version are unpickled in a python + env with a more recent k8s version (in which the object attrs may have + changed) the unpickled obj may throw an error because the attr + expected on new obj may not be there on the unpickled obj. + + This function still converts the pod to a dict; the only difference is + it populates missing attrs with None. You may compare with + https://github.com/kubernetes-client/python/blob/5a96bbcbe21a552cc1f9cda13e0522fafb0dbac8/kubernetes/client/api_client.py#L202 + + If obj is None, return None. + If obj is str, int, long, float, bool, return directly. + If obj is datetime.datetime, datetime.date + convert to string in iso8601 format. + If obj is list, sanitize each element in the list. + If obj is dict, return the dict. + If obj is OpenAPI model, return the properties dict. + + :param obj: The data to serialize. + :return: The serialized form of data. + + :meta private: + """ + if obj is None: + return None + elif isinstance(obj, (float, bool, bytes, str, int)): + return obj + elif isinstance(obj, list): + return [sanitize_for_serialization(sub_obj) for sub_obj in obj] + elif isinstance(obj, tuple): + return tuple(sanitize_for_serialization(sub_obj) for sub_obj in obj) + elif isinstance(obj, (datetime.datetime, datetime.date)): + return obj.isoformat() + + if isinstance(obj, dict): + obj_dict = obj + else: + obj_dict = { + obj.attribute_map[attr]: getattr(obj, attr) + for attr, _ in obj.openapi_types.items() + # below is the only line we change, and we just add default=None for getattr + if getattr(obj, attr, None) is not None + } + + return {key: sanitize_for_serialization(val) for key, val in obj_dict.items()} + + +def ensure_pod_is_valid_after_unpickling(pod: V1Pod) -> V1Pod | None: + """ + Convert pod to json and back so that pod is safe. + + The pod_override in executor_config is a V1Pod object. + Such objects created with one k8s version, when unpickled in + an env with upgraded k8s version, may blow up when + `to_dict` is called, because openapi client code gen calls + getattr on all attrs in openapi_types for each object, and when + new attrs are added to that list, getattr will fail. + + Here we re-serialize it to ensure it is not going to blow up. + + :meta private: + """ + try: + # if to_dict works, the pod is fine + pod.to_dict() + return pod + except AttributeError: + pass + try: + from kubernetes.client.models.v1_pod import V1Pod + except ImportError: + return None + if not isinstance(pod, V1Pod): + return None + try: + from airflow.kubernetes.pod_generator import PodGenerator + + # now we actually reserialize / deserialize the pod + pod_dict = sanitize_for_serialization(pod) + return PodGenerator.deserialize_model_dict(pod_dict) + except Exception: + return None + + class ExecutorConfigType(PickleType): """ Adds special handling for K8s executor config. If we unpickle a k8s object that was @@ -188,9 +278,16 @@ def process(value): if isinstance(value, dict) and "pod_override" in value: pod_override = value["pod_override"] - # If pod_override was serialized with Airflow's BaseSerialization, deserialize it if isinstance(pod_override, dict) and pod_override.get(Encoding.TYPE): + # If pod_override was serialized with Airflow's BaseSerialization, deserialize it value["pod_override"] = BaseSerialization.deserialize(pod_override) + else: + # backcompat path + # we no longer pickle raw pods but this code may be reached + # when accessing executor configs created in a prior version + new_pod = ensure_pod_is_valid_after_unpickling(pod_override) + if new_pod: + value["pod_override"] = new_pod return value return process diff --git a/tests/utils/test_sqlalchemy.py b/tests/utils/test_sqlalchemy.py index 2650aea6bc502..2bf4ad1be5bbd 100644 --- a/tests/utils/test_sqlalchemy.py +++ b/tests/utils/test_sqlalchemy.py @@ -19,6 +19,7 @@ import datetime import pickle +from copy import deepcopy from unittest import mock from unittest.mock import MagicMock @@ -32,7 +33,14 @@ from airflow.serialization.enums import DagAttributeTypes, Encoding from airflow.serialization.serialized_objects import BaseSerialization from airflow.settings import Session -from airflow.utils.sqlalchemy import ExecutorConfigType, nowait, prohibit_commit, skip_locked, with_row_locks +from airflow.utils.sqlalchemy import ( + ExecutorConfigType, + ensure_pod_is_valid_after_unpickling, + nowait, + prohibit_commit, + skip_locked, + with_row_locks, +) from airflow.utils.state import State from airflow.utils.timezone import utcnow @@ -324,3 +332,46 @@ def __eq__(self, other): instance = ExecutorConfigType() assert instance.compare_values(a, a) is False assert instance.compare_values("a", "a") is True + + def test_result_processor_bad_pickled_obj(self): + """ + If unpickled obj is missing attrs that curr lib expects + """ + test_container = k8s.V1Container(name="base") + test_pod = k8s.V1Pod(spec=k8s.V1PodSpec(containers=[test_container])) + copy_of_test_pod = deepcopy(test_pod) + # curr api expects attr `tty` + assert "tty" in test_container.openapi_types + # it lives in protected attr _tty + assert hasattr(test_container, "_tty") + # so, let's remove it before pickling, to simulate what happens in real life + del test_container._tty + # now let's prove that this blows up when calling to_dict + with pytest.raises(AttributeError): + test_pod.to_dict() + # no such problem with the copy + assert copy_of_test_pod.to_dict() + # so we need to roundtrip it through json + fixed_pod = ensure_pod_is_valid_after_unpickling(test_pod) + # and, since the missing attr was None anyway, we actually have the same pod + assert fixed_pod.to_dict() == copy_of_test_pod.to_dict() + + # now, let's verify that result processor makes this all work + # first, check that bad pod is still bad + with pytest.raises(AttributeError): + test_pod.to_dict() + # define what will be retrieved from db + input = pickle.dumps({"pod_override": TEST_POD}) + + # get the result processor method + config_type = ExecutorConfigType() + mock_dialect = MagicMock() + mock_dialect.dbapi = None + process = config_type.result_processor(mock_dialect, None) + + # apply the result processor + result = process(input) + + # show that the pickled (bad) pod is now a good pod, and same as the copy made + # before making it bad + assert result["pod_override"].to_dict() == copy_of_test_pod.to_dict() From d18367bf207b9964bffe0abe56288ee8c04e1261 Mon Sep 17 00:00:00 2001 From: Ping Zhang Date: Fri, 6 Jan 2023 11:07:00 -0800 Subject: [PATCH 056/158] Ensure correct log dir in file task handler (#28477) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit since Path.mkdir combines with the process’ umask value to determine the file mode and access flags, thus the newly created folder isn't 0o777 (cherry picked from commit bda39188bd127d0dd933cdff6c7e8d11ec6bf41b) --- airflow/utils/log/file_task_handler.py | 37 +++++++++++++++++++++----- 1 file changed, 30 insertions(+), 7 deletions(-) diff --git a/airflow/utils/log/file_task_handler.py b/airflow/utils/log/file_task_handler.py index 1dcdd745acd72..f8bf2043abba5 100644 --- a/airflow/utils/log/file_task_handler.py +++ b/airflow/utils/log/file_task_handler.py @@ -328,6 +328,35 @@ def read(self, task_instance, try_number=None, metadata=None): return logs, metadata_array + def _prepare_log_folder(self, directory: Path): + """ + Prepare the log folder and ensure its mode is 777. + + To handle log writing when tasks are impersonated, the log files need to + be writable by the user that runs the Airflow command and the user + that is impersonated. This is mainly to handle corner cases with the + SubDagOperator. When the SubDagOperator is run, all of the operators + run under the impersonated user and create appropriate log files + as the impersonated user. However, if the user manually runs tasks + of the SubDagOperator through the UI, then the log files are created + by the user that runs the Airflow command. For example, the Airflow + run command may be run by the `airflow_sudoable` user, but the Airflow + tasks may be run by the `airflow` user. If the log files are not + writable by both users, then it's possible that re-running a task + via the UI (or vice versa) results in a permission error as the task + tries to write to a log file created by the other user. + + Create the log file and give it group writable permissions + TODO(aoen): Make log dirs and logs globally readable for now since the SubDag + operator is not compatible with impersonation (e.g. if a Celery executor is used + for a SubDag operator and the SubDag operator has a different owner than the + parent DAG) + """ + mode = 0o777 + directory.mkdir(mode=mode, parents=True, exist_ok=True) + if directory.stat().st_mode != mode: + directory.chmod(mode) + def _init_file(self, ti): """ Create log directory and give it correct permissions. @@ -350,13 +379,7 @@ def _init_file(self, ti): # tries to write to a log file created by the other user. relative_path = self._render_filename(ti, ti.try_number) full_path = os.path.join(self.local_base, relative_path) - directory = os.path.dirname(full_path) - # Create the log file and give it group writable permissions - # TODO(aoen): Make log dirs and logs globally readable for now since the SubDag - # operator is not compatible with impersonation (e.g. if a Celery executor is used - # for a SubDag operator and the SubDag operator has a different owner than the - # parent DAG) - Path(directory).mkdir(mode=0o777, parents=True, exist_ok=True) + self._prepare_log_folder(Path(full_path).parent) if not os.path.exists(full_path): open(full_path, "a").close() From df1d322cf4fc4307a11c83f651f4e0783937a6a0 Mon Sep 17 00:00:00 2001 From: Michael Petro <40223998+michaelmicheal@users.noreply.github.com> Date: Thu, 5 Jan 2023 00:58:07 -0500 Subject: [PATCH 057/158] Add retry to purge_inactive_dag_warnings (#28481) Co-authored-by: Ephraim Anierobi (cherry picked from commit 5289938ec1d9011a9ff8625705cffd1708f9274d) --- airflow/models/dagwarning.py | 6 +++ tests/models/test_dagwarning.py | 69 +++++++++++++++++++++++++++++++++ 2 files changed, 75 insertions(+) create mode 100644 tests/models/test_dagwarning.py diff --git a/airflow/models/dagwarning.py b/airflow/models/dagwarning.py index fb77e42a9d0c7..db93aafb0fba5 100644 --- a/airflow/models/dagwarning.py +++ b/airflow/models/dagwarning.py @@ -24,6 +24,7 @@ from airflow.models.base import Base, StringID from airflow.utils import timezone +from airflow.utils.retries import retry_db_transaction from airflow.utils.session import NEW_SESSION, provide_session from airflow.utils.sqlalchemy import UtcDateTime @@ -72,6 +73,11 @@ def purge_inactive_dag_warnings(cls, session: Session = NEW_SESSION) -> None: :return: None """ + cls._purge_inactive_dag_warnings_with_retry(session) + + @classmethod + @retry_db_transaction + def _purge_inactive_dag_warnings_with_retry(cls, session: Session) -> None: from airflow.models.dag import DagModel if session.get_bind().dialect.name == "sqlite": diff --git a/tests/models/test_dagwarning.py b/tests/models/test_dagwarning.py new file mode 100644 index 0000000000000..7ae2962b1f758 --- /dev/null +++ b/tests/models/test_dagwarning.py @@ -0,0 +1,69 @@ +# 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 __future__ import annotations + +from unittest.mock import MagicMock + +from sqlalchemy.exc import OperationalError + +from airflow.models import DagModel +from airflow.models.dagwarning import DagWarning +from tests.test_utils.db import clear_db_dags + + +class TestDagWarning: + def setup_method(self): + clear_db_dags() + + def test_purge_inactive_dag_warnings(self, session): + """ + Test that the purge_inactive_dag_warnings method deletes inactive dag warnings + """ + + dags = [DagModel(dag_id="dag_1", is_active=False), DagModel(dag_id="dag_2", is_active=True)] + session.add_all(dags) + session.commit() + + dag_warnings = [ + DagWarning("dag_1", "non-existent pool", "non-existent pool"), + DagWarning("dag_2", "non-existent pool", "non-existent pool"), + ] + session.add_all(dag_warnings) + session.commit() + + DagWarning.purge_inactive_dag_warnings(session) + + remaining_dag_warnings = session.query(DagWarning).all() + assert len(remaining_dag_warnings) == 1 + assert remaining_dag_warnings[0].dag_id == "dag_2" + + def test_retry_purge_inactive_dag_warnings(self): + """ + Test that the purge_inactive_dag_warnings method calls the delete method twice + if the query throws an operationalError on the first call and works on the second attempt + """ + self.session_mock = MagicMock() + self.delete_mock = MagicMock() + self.session_mock.query.return_value.filter.return_value.delete = self.delete_mock + + self.delete_mock.side_effect = [OperationalError(None, None, "database timeout"), None] + + DagWarning.purge_inactive_dag_warnings(self.session_mock) + + # Assert that the delete method was called twice + assert self.delete_mock.call_count == 2 From 8157c0b868cb229eeccee547627a3ec3672fbfa7 Mon Sep 17 00:00:00 2001 From: Jarek Potiuk Date: Tue, 20 Dec 2022 19:33:46 +0100 Subject: [PATCH 058/158] Move MyPY plugins of ours to dev folder (#28498) The Plugins are only used in the static check phase. The problem with having them in "airflow" package is that mypy imports "airlfow" during loading of the plugins and it means that it has to have fully working Airflow configuration to work - otherwise this import fails while reading the configuration values. Moving the whole mypy plugins to dev solves the problem entirely. (cherry picked from commit 1f75e9ffcf0e61115ea141bc1c5de5002ef8f2c0) --- .github/boring-cyborg.yml | 1 - {airflow => dev}/mypy/__init__.py | 0 {airflow => dev}/mypy/plugin/__init__.py | 0 {airflow => dev}/mypy/plugin/decorators.py | 0 {airflow => dev}/mypy/plugin/outputs.py | 0 setup.cfg | 4 ++-- 6 files changed, 2 insertions(+), 3 deletions(-) rename {airflow => dev}/mypy/__init__.py (100%) rename {airflow => dev}/mypy/plugin/__init__.py (100%) rename {airflow => dev}/mypy/plugin/decorators.py (100%) rename {airflow => dev}/mypy/plugin/outputs.py (100%) diff --git a/.github/boring-cyborg.yml b/.github/boring-cyborg.yml index 0776ca1d0f5c8..90f80674ec538 100644 --- a/.github/boring-cyborg.yml +++ b/.github/boring-cyborg.yml @@ -83,7 +83,6 @@ labelPRBasedOnFilePath: - tests/www/api/**/* area:dev-tools: - - airflow/mypy/**/* - scripts/**/* - dev/**/* - .github/**/* diff --git a/airflow/mypy/__init__.py b/dev/mypy/__init__.py similarity index 100% rename from airflow/mypy/__init__.py rename to dev/mypy/__init__.py diff --git a/airflow/mypy/plugin/__init__.py b/dev/mypy/plugin/__init__.py similarity index 100% rename from airflow/mypy/plugin/__init__.py rename to dev/mypy/plugin/__init__.py diff --git a/airflow/mypy/plugin/decorators.py b/dev/mypy/plugin/decorators.py similarity index 100% rename from airflow/mypy/plugin/decorators.py rename to dev/mypy/plugin/decorators.py diff --git a/airflow/mypy/plugin/outputs.py b/dev/mypy/plugin/outputs.py similarity index 100% rename from airflow/mypy/plugin/outputs.py rename to dev/mypy/plugin/outputs.py diff --git a/setup.cfg b/setup.cfg index 2274d98e3a90a..5cc2a1342bf63 100644 --- a/setup.cfg +++ b/setup.cfg @@ -174,8 +174,8 @@ no_implicit_optional = True warn_redundant_casts = True warn_unused_ignores = False plugins = - airflow.mypy.plugin.decorators, - airflow.mypy.plugin.outputs + dev.mypy.plugin.decorators, + dev.mypy.plugin.outputs pretty = True show_error_codes = True From 505cfc459ef79ffc96b2e98fa9d9e0b8180ffb68 Mon Sep 17 00:00:00 2001 From: Jarek Potiuk Date: Tue, 20 Dec 2022 21:44:36 +0100 Subject: [PATCH 059/158] Remove outdated Optional Provider Feature outdated documentation (#28506) After bumping min_airflow_version to 2.3 the section about optional provider feature and the way to add it for pre-2.3 compatible providers is outdated and should be removed. (cherry picked from commit 9ac76ec52604486d41d0c70984fea51ab2764525) --- .../howto/create-update-providers.rst | 33 +------------------ 1 file changed, 1 insertion(+), 32 deletions(-) diff --git a/docs/apache-airflow-providers/howto/create-update-providers.rst b/docs/apache-airflow-providers/howto/create-update-providers.rst index 173f95c9b5073..5231b69a68f23 100644 --- a/docs/apache-airflow-providers/howto/create-update-providers.rst +++ b/docs/apache-airflow-providers/howto/create-update-providers.rst @@ -267,7 +267,7 @@ main Airflow documentation that involves some steps with the providers is also w Optional provider features -------------------------- - .. note:: + .. versionadded:: 2.3.0 This feature is available in Airflow 2.3+. @@ -280,37 +280,6 @@ Airflow 2.1 and 2.2 silently ignored all ImportErrors coming from providers with ignoring even important import errors - without giving the clue to Airflow users that there is something missing in provider dependencies. -In Airflow 2.3, new exception :class:`~airflow.exceptions.OptionalProviderFeatureException` has been -introduced and Providers can use the exception to signal that the ImportError (or any other error) should -be ignored by Airflow ProvidersManager. However this Exception is only available in Airflow 2.3 so if -providers would like to remain compatible with 2.2, they should continue throwing -the ImportError exception. - -Example code (from Plyvel Hook, part of the Google Provider) explains how such conditional error handling -should be implemented to keep compatibility with 2.2 - - .. code-block:: python - - try: - import plyvel - from plyvel import DB - - from airflow.exceptions import AirflowException - from airflow.hooks.base import BaseHook - - except ImportError as e: - # Plyvel is an optional feature and if imports are missing, it should be silently ignored - # As of Airflow 2.3 and above the operator can throw OptionalProviderFeatureException - try: - from airflow.exceptions import AirflowOptionalProviderFeatureException - except ImportError: - # However, in order to keep backwards-compatibility with Airflow 2.1 and 2.2, if the - # 2.3 exception cannot be imported, the original ImportError should be raised. - # This try/except can be removed when the provider depends on Airflow >= 2.3.0 - raise e - raise AirflowOptionalProviderFeatureException(e) - - Using Providers with dynamic task mapping ----------------------------------------- From f0625d4516684067944a4be18246f16b3c2c3f8f Mon Sep 17 00:00:00 2001 From: Kfir Gollan Date: Thu, 22 Dec 2022 09:05:55 +0200 Subject: [PATCH 060/158] Add AIRFLOW_PROJ_DIR to docker-compose example (#28517) Add an environment variable called AIRFLOW_PROJ_DIR that allows controlling the base directory for volumes. This allows custom folder structure when working with the example docker-compose. (cherry picked from commit 3f88148080157d1d8f9e0495e7c79ad81d475fca) --- .../howto/docker-compose/docker-compose.yaml | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/docs/apache-airflow/howto/docker-compose/docker-compose.yaml b/docs/apache-airflow/howto/docker-compose/docker-compose.yaml index 99e675a42ca73..38e8e8b471c9f 100644 --- a/docs/apache-airflow/howto/docker-compose/docker-compose.yaml +++ b/docs/apache-airflow/howto/docker-compose/docker-compose.yaml @@ -27,6 +27,8 @@ # Default: apache/airflow:|version| # AIRFLOW_UID - User ID in Airflow containers # Default: 50000 +# AIRFLOW_PROJ_DIR - Base path to which all the files will be volumed. +# Default: . # Those configurations are useful mostly in case of standalone testing/running Airflow in test/try-out mode # # _AIRFLOW_WWW_USER_USERNAME - Username for the administrator account (if requested). @@ -60,9 +62,9 @@ x-airflow-common: AIRFLOW__API__AUTH_BACKENDS: 'airflow.api.auth.backend.basic_auth,airflow.api.auth.backend.session' _PIP_ADDITIONAL_REQUIREMENTS: ${_PIP_ADDITIONAL_REQUIREMENTS:-} volumes: - - ./dags:/opt/airflow/dags - - ./logs:/opt/airflow/logs - - ./plugins:/opt/airflow/plugins + - ${AIRFLOW_PROJ_DIR:-.}/dags:/opt/airflow/dags + - ${AIRFLOW_PROJ_DIR:-.}/logs:/opt/airflow/logs + - ${AIRFLOW_PROJ_DIR:-.}/plugins:/opt/airflow/plugins user: "${AIRFLOW_UID:-50000}:0" depends_on: &airflow-common-depends-on @@ -238,7 +240,7 @@ services: _PIP_ADDITIONAL_REQUIREMENTS: '' user: "0:0" volumes: - - .:/sources + - ${AIRFLOW_PROJ_DIR:-.}:/sources airflow-cli: <<: *airflow-common From 70f8ee37823b9295f69a3f7933930643480a6ea0 Mon Sep 17 00:00:00 2001 From: Jarek Potiuk Date: Thu, 22 Dec 2022 14:02:24 +0100 Subject: [PATCH 061/158] Add MSSQL support for ARM Docker images (#28533) The #28530 re-enabled support for pymssql installation on ARM. Even if we have no binary wheels, pymssql seems to build cleanly. This is a follow up, to enable mssql client installation for ARM for our images. It also adds docuemntation in Docker image docs to explain it. (cherry picked from commit 9c3734bb127ff0d71a0321d0578e556552cfc934) --- Dockerfile | 5 ----- Dockerfile.ci | 5 ----- docs/docker-stack/changelog.rst | 8 ++++++++ docs/docker-stack/index.rst | 14 ++++++++++++++ docs/spelling_wordlist.txt | 1 + scripts/docker/install_mssql.sh | 5 ----- 6 files changed, 23 insertions(+), 15 deletions(-) diff --git a/Dockerfile b/Dockerfile index bab2cda06a41d..bc8502982c29a 100644 --- a/Dockerfile +++ b/Dockerfile @@ -275,11 +275,6 @@ function install_mssql_client() { apt-get clean && rm -rf /var/lib/apt/lists/* } -if [[ $(uname -m) == "arm64" || $(uname -m) == "aarch64" ]]; then - # disable MSSQL for ARM64 - INSTALL_MSSQL_CLIENT="false" -fi - install_mssql_client "${@}" EOF diff --git a/Dockerfile.ci b/Dockerfile.ci index 0713cc8b715d3..95ab395053969 100644 --- a/Dockerfile.ci +++ b/Dockerfile.ci @@ -235,11 +235,6 @@ function install_mssql_client() { apt-get clean && rm -rf /var/lib/apt/lists/* } -if [[ $(uname -m) == "arm64" || $(uname -m) == "aarch64" ]]; then - # disable MSSQL for ARM64 - INSTALL_MSSQL_CLIENT="false" -fi - install_mssql_client "${@}" EOF diff --git a/docs/docker-stack/changelog.rst b/docs/docker-stack/changelog.rst index 37a44202660a7..412d5b61370c2 100644 --- a/docs/docker-stack/changelog.rst +++ b/docs/docker-stack/changelog.rst @@ -55,6 +55,14 @@ here so that users affected can find the reason for the changes. | | | | on 17 Jan 2022 | | +--------------+---------------------+-----------------------------------------+------------------------+----------------------------------------------+ +Airflow 2.5.1 +~~~~~~~~~~~~~ + +* 2.5.1 + + * The ARM experimental image adds support for MSSQL + + Airflow 2.5 ~~~~~~~~~~~ diff --git a/docs/docker-stack/index.rst b/docs/docker-stack/index.rst index 2ce60a37eac79..be8877be06f88 100644 --- a/docs/docker-stack/index.rst +++ b/docs/docker-stack/index.rst @@ -83,6 +83,20 @@ are also images published from branches but they are used mainly for development See `Airflow Git Branching `_ for details. +Support +======= + +The reference Docker Image supports the following platforms and database: + + +* Intel platform (x86_64) + * Postgres Client + * MySQL Client + * MSSQL Client + +* ARM platform (aarch64) - experimental support, might change any time + * Postgres Client + * MSSQL Client Usage ===== diff --git a/docs/spelling_wordlist.txt b/docs/spelling_wordlist.txt index 5b186ae821fb5..f019d2a8c0120 100644 --- a/docs/spelling_wordlist.txt +++ b/docs/spelling_wordlist.txt @@ -1,3 +1,4 @@ +aarch abc accessor accountmaking diff --git a/scripts/docker/install_mssql.sh b/scripts/docker/install_mssql.sh index 6765b060d463e..fa0cfc0033cb6 100644 --- a/scripts/docker/install_mssql.sh +++ b/scripts/docker/install_mssql.sh @@ -51,9 +51,4 @@ function install_mssql_client() { apt-get clean && rm -rf /var/lib/apt/lists/* } -if [[ $(uname -m) == "arm64" || $(uname -m) == "aarch64" ]]; then - # disable MSSQL for ARM64 - INSTALL_MSSQL_CLIENT="false" -fi - install_mssql_client "${@}" From 287dd0d39a0f2daaa2d9ee63be43a6f76d405f0e Mon Sep 17 00:00:00 2001 From: Tzu-ping Chung Date: Mon, 26 Dec 2022 17:30:02 +0800 Subject: [PATCH 062/158] Consider previous run in CronTriggerTimetable (#28532) (cherry picked from commit 6dc28fb0278c1bdb096b75b6e19acbcb1019db02) --- airflow/timetables/trigger.py | 21 +++++++------ tests/timetables/test_trigger_timetable.py | 36 +++++++++++++++++----- 2 files changed, 40 insertions(+), 17 deletions(-) diff --git a/airflow/timetables/trigger.py b/airflow/timetables/trigger.py index 86ab58cc4df25..7807542da5a63 100644 --- a/airflow/timetables/trigger.py +++ b/airflow/timetables/trigger.py @@ -82,18 +82,19 @@ def next_dagrun_info( restriction: TimeRestriction, ) -> DagRunInfo | None: if restriction.catchup: - if last_automated_data_interval is None: - if restriction.earliest is None: - return None - next_start_time = self._align_to_next(restriction.earliest) - else: + if last_automated_data_interval is not None: next_start_time = self._get_next(last_automated_data_interval.end) - else: - current_time = DateTime.utcnow() - if restriction.earliest is not None and current_time < restriction.earliest: - next_start_time = self._align_to_next(restriction.earliest) + elif restriction.earliest is None: + return None # Don't know where to catch up from, give up. else: - next_start_time = self._align_to_next(current_time) + next_start_time = self._align_to_next(restriction.earliest) + else: + start_time_candidates = [self._align_to_next(DateTime.utcnow())] + if last_automated_data_interval is not None: + start_time_candidates.append(self._get_next(last_automated_data_interval.end)) + if restriction.earliest is not None: + start_time_candidates.append(self._align_to_next(restriction.earliest)) + next_start_time = max(start_time_candidates) if restriction.latest is not None and restriction.latest < next_start_time: return None return DagRunInfo.interval(next_start_time - self._interval, next_start_time) diff --git a/tests/timetables/test_trigger_timetable.py b/tests/timetables/test_trigger_timetable.py index 3e3c97a5e0c08..cabb1198efd79 100644 --- a/tests/timetables/test_trigger_timetable.py +++ b/tests/timetables/test_trigger_timetable.py @@ -44,12 +44,29 @@ @pytest.mark.parametrize( - "last_automated_data_interval", - [pytest.param(None, id="first-run"), pytest.param(PREV_DATA_INTERVAL_EXACT, id="subsequent")], + "last_automated_data_interval, next_start_time", + [ + pytest.param( + None, + CURRENT_TIME + DELTA_FROM_MIDNIGHT, + id="first-run", + ), + pytest.param( + PREV_DATA_INTERVAL_EXACT, + CURRENT_TIME + DELTA_FROM_MIDNIGHT, + id="before-now", + ), + pytest.param( + DataInterval.exact(CURRENT_TIME + DELTA_FROM_MIDNIGHT), + CURRENT_TIME + datetime.timedelta(days=1) + DELTA_FROM_MIDNIGHT, + id="after-now", + ), + ], ) @freezegun.freeze_time(CURRENT_TIME) def test_daily_cron_trigger_no_catchup_first_starts_at_next_schedule( last_automated_data_interval: DataInterval | None, + next_start_time: pendulum.DateTime, ) -> None: """If ``catchup=False`` and start_date is a day before""" timetable = CronTriggerTimetable("30 16 * * *", timezone=TIMEZONE) @@ -57,8 +74,7 @@ def test_daily_cron_trigger_no_catchup_first_starts_at_next_schedule( last_automated_data_interval=last_automated_data_interval, restriction=TimeRestriction(earliest=YESTERDAY, latest=None, catchup=False), ) - expected = CURRENT_TIME + DELTA_FROM_MIDNIGHT - assert next_info == DagRunInfo.exact(expected) + assert next_info == DagRunInfo.exact(next_start_time) @pytest.mark.parametrize( @@ -124,12 +140,18 @@ def test_hourly_cron_trigger_no_catchup_next_info( DagRunInfo.exact(pendulum.DateTime(2022, 7, 27, 1, 0, 0, tzinfo=TIMEZONE)), id="no_last_automated_with_earliest_not_on_boundary", ), + pytest.param( + None, + None, + None, + id="no_last_automated_no_earliest", + ), ], ) def test_hourly_cron_trigger_catchup_next_info( - last_automated_data_interval: DataInterval, - earliest: pendulum.DateTime, - expected: DagRunInfo, + last_automated_data_interval: DataInterval | None, + earliest: pendulum.DateTime | None, + expected: DagRunInfo | None, ) -> None: next_info = HOURLY_CRON_TRIGGER_TIMETABLE.next_dagrun_info( last_automated_data_interval=last_automated_data_interval, From 2a34ec41b584f99a7cb111bb1b8ab2103354a07b Mon Sep 17 00:00:00 2001 From: Ash Berlin-Taylor Date: Fri, 30 Dec 2022 17:53:46 +0000 Subject: [PATCH 063/158] Emit warnings for `conf.get*` from the right source location (#28543) `getboolean` and other typed get functions were issuing warnings from "inside" themselves. Before: ``` $ python ./airflow/airflow/kubernetes/kube_client.py /home/ash/code/airflow/airflow/airflow/configuration.py:722 DeprecationWarning: The in_cluster option in [kubernetes] has been moved to the in_cluster option in [kubernetes_executor] - the old setting has been used, but please update your config. ``` After: ``` $ python ./airflow/airflow/kubernetes/kube_client.py /home/ash/code/airflow/airflow/airflow/kubernetes/kube_client.py:89 DeprecationWarning: The in_cluster option in [kubernetes] has been moved to the in_cluster option in [kubernetes_executor] - the old setting has been used, but please update your config. ``` (cherry picked from commit f0ae250527c0494d32227ad8433c15e691f004d1) --- airflow/configuration.py | 77 ++++++++++++++++++++++++++++------------ 1 file changed, 55 insertions(+), 22 deletions(-) diff --git a/airflow/configuration.py b/airflow/configuration.py index 2fd96d1710738..4cea1a53f6646 100644 --- a/airflow/configuration.py +++ b/airflow/configuration.py @@ -395,7 +395,7 @@ def _upgrade_postgres_metastore_conn(self): must be replaced with `postgresql`. """ section, key = "database", "sql_alchemy_conn" - old_value = self.get(section, key) + old_value = self.get(section, key, _extra_stacklevel=1) bad_schemes = ["postgres+psycopg2", "postgres"] good_scheme = "postgresql" parsed = urlsplit(old_value) @@ -543,7 +543,7 @@ def _get_secret_option_from_config_sources( return None def get_mandatory_value(self, section: str, key: str, **kwargs) -> str: - value = self.get(section, key, **kwargs) + value = self.get(section, key, _extra_stacklevel=1, **kwargs) if value is None: raise ValueError(f"The value {section}/{key} should be set!") return value @@ -558,7 +558,13 @@ def get(self, section: str, key: str, **kwargs) -> str | None: # type: ignore[o ... - def get(self, section: str, key: str, **kwargs) -> str | None: # type: ignore[override, misc] + def get( # type: ignore[override, misc] + self, + section: str, + key: str, + _extra_stacklevel: int = 0, + **kwargs, + ) -> str | None: section = str(section).lower() key = str(key).lower() warning_emitted = False @@ -574,7 +580,7 @@ def get(self, section: str, key: str, **kwargs) -> str | None: # type: ignore[o f"The config section [{deprecated_section}] has been renamed to " f"[{section}]. Please update your `conf.get*` call to use the new name", FutureWarning, - stacklevel=2, + stacklevel=2 + _extra_stacklevel, ) # Don't warn about individual rename if the whole section is renamed warning_emitted = True @@ -587,7 +593,7 @@ def get(self, section: str, key: str, **kwargs) -> str | None: # type: ignore[o f"[{new_section}/{new_key}] instead. Please update your `conf.get*` call to use the " "new name", FutureWarning, - stacklevel=2, + stacklevel=2 + _extra_stacklevel, ) warning_emitted = True deprecated_section, deprecated_key = section, key @@ -603,28 +609,49 @@ def get(self, section: str, key: str, **kwargs) -> str | None: # type: ignore[o # first check environment variables option = self._get_environment_variables( - deprecated_key, deprecated_section, key, section, issue_warning=not warning_emitted + deprecated_key, + deprecated_section, + key, + section, + issue_warning=not warning_emitted, + extra_stacklevel=_extra_stacklevel, ) if option is not None: return option # ...then the config file option = self._get_option_from_config_file( - deprecated_key, deprecated_section, key, kwargs, section, issue_warning=not warning_emitted + deprecated_key, + deprecated_section, + key, + kwargs, + section, + issue_warning=not warning_emitted, + extra_stacklevel=_extra_stacklevel, ) if option is not None: return option # ...then commands option = self._get_option_from_commands( - deprecated_key, deprecated_section, key, section, issue_warning=not warning_emitted + deprecated_key, + deprecated_section, + key, + section, + issue_warning=not warning_emitted, + extra_stacklevel=_extra_stacklevel, ) if option is not None: return option # ...then from secret backends option = self._get_option_from_secrets( - deprecated_key, deprecated_section, key, section, issue_warning=not warning_emitted + deprecated_key, + deprecated_section, + key, + section, + issue_warning=not warning_emitted, + extra_stacklevel=_extra_stacklevel, ) if option is not None: return option @@ -644,6 +671,7 @@ def _get_option_from_secrets( key: str, section: str, issue_warning: bool = True, + extra_stacklevel: int = 0, ) -> str | None: option = self._get_secret_option(section, key) if option: @@ -653,7 +681,7 @@ def _get_option_from_secrets( option = self._get_secret_option(deprecated_section, deprecated_key) if option: if issue_warning: - self._warn_deprecate(section, key, deprecated_section, deprecated_key) + self._warn_deprecate(section, key, deprecated_section, deprecated_key, extra_stacklevel) return option return None @@ -664,6 +692,7 @@ def _get_option_from_commands( key: str, section: str, issue_warning: bool = True, + extra_stacklevel: int = 0, ) -> str | None: option = self._get_cmd_option(section, key) if option: @@ -673,7 +702,7 @@ def _get_option_from_commands( option = self._get_cmd_option(deprecated_section, deprecated_key) if option: if issue_warning: - self._warn_deprecate(section, key, deprecated_section, deprecated_key) + self._warn_deprecate(section, key, deprecated_section, deprecated_key, extra_stacklevel) return option return None @@ -685,6 +714,7 @@ def _get_option_from_config_file( kwargs: dict[str, Any], section: str, issue_warning: bool = True, + extra_stacklevel: int = 0, ) -> str | None: if super().has_option(section, key): # Use the parent's methods to get the actual config here to be able to @@ -693,7 +723,7 @@ def _get_option_from_config_file( if deprecated_section and deprecated_key: if super().has_option(deprecated_section, deprecated_key): if issue_warning: - self._warn_deprecate(section, key, deprecated_section, deprecated_key) + self._warn_deprecate(section, key, deprecated_section, deprecated_key, extra_stacklevel) with self.suppress_future_warnings(): return expand_env_var(super().get(deprecated_section, deprecated_key, **kwargs)) return None @@ -705,6 +735,7 @@ def _get_environment_variables( key: str, section: str, issue_warning: bool = True, + extra_stacklevel: int = 0, ) -> str | None: option = self._get_env_var_option(section, key) if option is not None: @@ -714,12 +745,12 @@ def _get_environment_variables( option = self._get_env_var_option(deprecated_section, deprecated_key) if option is not None: if issue_warning: - self._warn_deprecate(section, key, deprecated_section, deprecated_key) + self._warn_deprecate(section, key, deprecated_section, deprecated_key, extra_stacklevel) return option return None def getboolean(self, section: str, key: str, **kwargs) -> bool: # type: ignore[override] - val = str(self.get(section, key, **kwargs)).lower().strip() + val = str(self.get(section, key, _extra_stacklevel=1, **kwargs)).lower().strip() if "#" in val: val = val.split("#")[0].strip() if val in ("t", "true", "1"): @@ -733,7 +764,7 @@ def getboolean(self, section: str, key: str, **kwargs) -> bool: # type: ignore[ ) def getint(self, section: str, key: str, **kwargs) -> int: # type: ignore[override] - val = self.get(section, key, **kwargs) + val = self.get(section, key, _extra_stacklevel=1, **kwargs) if val is None: raise AirflowConfigException( f"Failed to convert value None to int. " @@ -748,7 +779,7 @@ def getint(self, section: str, key: str, **kwargs) -> int: # type: ignore[overr ) def getfloat(self, section: str, key: str, **kwargs) -> float: # type: ignore[override] - val = self.get(section, key, **kwargs) + val = self.get(section, key, _extra_stacklevel=1, **kwargs) if val is None: raise AirflowConfigException( f"Failed to convert value None to float. " @@ -799,7 +830,7 @@ def getjson( fallback = _UNSET try: - data = self.get(section=section, key=key, fallback=fallback, **kwargs) + data = self.get(section=section, key=key, fallback=fallback, _extra_stacklevel=1, **kwargs) except (NoSectionError, NoOptionError): return default @@ -825,7 +856,7 @@ def gettimedelta( :raises AirflowConfigException: raised because ValueError or OverflowError :return: datetime.timedelta(seconds=) or None """ - val = self.get(section, key, fallback=fallback, **kwargs) + val = self.get(section, key, fallback=fallback, _extra_stacklevel=1, **kwargs) if val: # the given value must be convertible to integer @@ -867,7 +898,7 @@ def has_option(self, section: str, option: str) -> bool: # Using self.get() to avoid reimplementing the priority order # of config variables (env, config, cmd, defaults) # UNSET to avoid logging a warning about missing values - self.get(section, option, fallback=_UNSET) + self.get(section, option, fallback=_UNSET, _extra_stacklevel=1) return True except (NoOptionError, NoSectionError): return False @@ -1342,20 +1373,22 @@ def load_test_config(self): self.read(TEST_CONFIG_FILE) @staticmethod - def _warn_deprecate(section: str, key: str, deprecated_section: str, deprecated_name: str): + def _warn_deprecate( + section: str, key: str, deprecated_section: str, deprecated_name: str, extra_stacklevel: int + ): if section == deprecated_section: warnings.warn( f"The {deprecated_name} option in [{section}] has been renamed to {key} - " f"the old setting has been used, but please update your config.", DeprecationWarning, - stacklevel=4, + stacklevel=4 + extra_stacklevel, ) else: warnings.warn( f"The {deprecated_name} option in [{deprecated_section}] has been moved to the {key} option " f"in [{section}] - the old setting has been used, but please update your config.", DeprecationWarning, - stacklevel=4, + stacklevel=4 + extra_stacklevel, ) def __getstate__(self): From ccbdab8b6edb29b05b1771eebfb19c085e16d4cb Mon Sep 17 00:00:00 2001 From: Michael Petro <40223998+michaelmicheal@users.noreply.github.com> Date: Thu, 29 Dec 2022 20:50:45 -0500 Subject: [PATCH 064/158] navbar, cap dropdown size, and add scroll bar (#28561) * navbar, cap dropdown size, and add scroll bar * Formatting css * main.css, remove extra newline (cherry picked from commit 2aa52f4ce78e1be7f34b0995d40be996b4826f26) --- airflow/www/static/css/main.css | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/airflow/www/static/css/main.css b/airflow/www/static/css/main.css index 2557f521e6d26..9e63eb5ad06ed 100644 --- a/airflow/www/static/css/main.css +++ b/airflow/www/static/css/main.css @@ -61,6 +61,13 @@ div.container { color: #e2d2e2; } +/* If a category has a lot of menu items, we cap it, and + add a scroll bar */ +.navbar li.dropdown .dropdown-menu { + max-height: 300px; + overflow-y: auto; +} + .navbar-nav li.dropdown:hover > .dropdown-menu, .navbar-nav li.dropdown:focus-within > .dropdown-menu { display: block; From 21e6df87f3393316eb6f5bf1cd160c1e5d63dc73 Mon Sep 17 00:00:00 2001 From: Jed Cunningham <66968678+jedcunningham@users.noreply.github.com> Date: Fri, 23 Dec 2022 17:35:20 -0600 Subject: [PATCH 065/158] Move TI setNote endpoints under TaskInstance in OpenAPI (#28566) These endpoints were accidentally under DAG instead of TaskInstance where they belong. (cherry picked from commit b3e26560c7fd835570a0b3a9d65670c87c8cfe0a) --- airflow/api_connexion/openapi/v1.yaml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/airflow/api_connexion/openapi/v1.yaml b/airflow/api_connexion/openapi/v1.yaml index 0094e0d7f1157..9db4e7647dce0 100644 --- a/airflow/api_connexion/openapi/v1.yaml +++ b/airflow/api_connexion/openapi/v1.yaml @@ -599,7 +599,7 @@ paths: *New in version 2.5.0* x-openapi-router-controller: airflow.api_connexion.endpoints.task_instance_endpoint operationId: set_task_instance_note - tags: [DAG] + tags: [TaskInstance] requestBody: description: Parameters of set Task Instance note. required: true @@ -639,7 +639,7 @@ paths: *New in version 2.5.0* x-openapi-router-controller: airflow.api_connexion.endpoints.task_instance_endpoint operationId: set_mapped_task_instance_note - tags: [DAG] + tags: [TaskInstance] requestBody: description: Parameters of set Task Instance note. required: true From 9a4645f17454b18cf34c773e12d08b45c124595e Mon Sep 17 00:00:00 2001 From: abouklila Date: Thu, 29 Dec 2022 20:30:43 +0100 Subject: [PATCH 066/158] Update baseoperator.py (#28574) Fix baseoperator chain docstring (cherry picked from commit 9fefbad54fe50b4ddb9d4eaba46ed051ef7960b3) --- airflow/models/baseoperator.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/airflow/models/baseoperator.py b/airflow/models/baseoperator.py index bb9b94b52f2aa..ff400804a4840 100644 --- a/airflow/models/baseoperator.py +++ b/airflow/models/baseoperator.py @@ -1563,12 +1563,12 @@ def chain(*tasks: DependencyMixin | Sequence[DependencyMixin]) -> None: .. code-block:: python - chain(t1, [Label("branch one"), Label("branch two")], [x1(), x2()], task_group1, t2()) + chain(t1, [Label("branch one"), Label("branch two")], [x1(), x2()], task_group1, x3()) is equivalent to:: / "branch one" -> x1 \ - t1 -> t2 -> x3 + t1 -> task_group1 -> x3 \ "branch two" -> x2 / .. code-block:: python From 80e7dece744d7a0c2809e5e24e91f2f36f933f34 Mon Sep 17 00:00:00 2001 From: Tzu-ping Chung Date: Wed, 28 Dec 2022 23:47:11 +0800 Subject: [PATCH 067/158] Guard not-yet-expanded ti in trigger rule dep (#28592) Previously, if a mapped task is not yet expanded when the trigger rule dep is evaluated, it would raise an exception and fail the scheduler. This adds an additional try-except to guard against this. The problematic scenario is when a mapped task depends on another mapped task, and its trigger rule is evaluated before that other mapped task is expanded (e.g. the other task also has a task-mapping dependency that is not yet finished). Since we can be certain the upstream task has not yet satisfy the expansion dep, we can simply declare the task we're checking as unsatisfied. (cherry picked from commit d4dbb0077aec33e5b3b4793bf9e2902e6cbdaa7f) --- airflow/ti_deps/deps/trigger_rule_dep.py | 8 +++++- tests/ti_deps/deps/test_trigger_rule_dep.py | 30 +++++++++++++++++++++ 2 files changed, 37 insertions(+), 1 deletion(-) diff --git a/airflow/ti_deps/deps/trigger_rule_dep.py b/airflow/ti_deps/deps/trigger_rule_dep.py index d932a6dd211d8..7d78b591af323 100644 --- a/airflow/ti_deps/deps/trigger_rule_dep.py +++ b/airflow/ti_deps/deps/trigger_rule_dep.py @@ -105,6 +105,8 @@ def _evaluate_trigger_rule( :param dep_context: The current dependency context. :param session: Database session. """ + from airflow.models.abstractoperator import NotMapped + from airflow.models.expandinput import NotFullyPopulated from airflow.models.operator import needs_expansion from airflow.models.taskinstance import TaskInstance @@ -129,9 +131,13 @@ def _get_relevant_upstream_map_indexes(upstream_id: str) -> int | range | None: and at most once for each task (instead of once for each expanded task instance of the same task). """ + try: + expanded_ti_count = _get_expanded_ti_count() + except (NotFullyPopulated, NotMapped): + return None return ti.get_relevant_upstream_map_indexes( upstream_tasks[upstream_id], - _get_expanded_ti_count(), + expanded_ti_count, session=session, ) diff --git a/tests/ti_deps/deps/test_trigger_rule_dep.py b/tests/ti_deps/deps/test_trigger_rule_dep.py index 509909d97434a..42c979c93ae3b 100644 --- a/tests/ti_deps/deps/test_trigger_rule_dep.py +++ b/tests/ti_deps/deps/test_trigger_rule_dep.py @@ -22,6 +22,7 @@ import pytest +from airflow.decorators import task, task_group from airflow.models.baseoperator import BaseOperator from airflow.models.dagrun import DagRun from airflow.models.taskinstance import TaskInstance @@ -947,3 +948,32 @@ def _one_scheduling_decision_iteration() -> dict[tuple[str, int], TaskInstance]: tis["tg.t2", 1].run() tis = _one_scheduling_decision_iteration() assert sorted(tis) == [("t3", -1)] + + +def test_mapped_task_check_before_expand(dag_maker, session): + with dag_maker(session=session): + + @task + def t(x): + return x + + @task_group + def tg(a): + b = t.override(task_id="t2")(a) + c = t.override(task_id="t3")(b) + return c + + tg.expand(a=t([1, 2, 3])) + + dr: DagRun = dag_maker.create_dagrun() + result_iterator = TriggerRuleDep()._evaluate_trigger_rule( + # t3 depends on t2, which depends on t1 for expansion. Since t1 has not + # yet run, t2 has not expanded yet, and we need to guarantee this lack + # of expansion does not fail the dependency-checking logic. + ti=next(ti for ti in dr.task_instances if ti.task_id == "tg.t3" and ti.map_index == -1), + dep_context=DepContext(), + session=session, + ) + results = list(result_iterator) + assert len(results) == 1 + assert results[0].passed is False From cb2435e07d3e2b945c47cb0a22c363f646abc8ff Mon Sep 17 00:00:00 2001 From: Michael T Date: Wed, 28 Dec 2022 19:42:38 +0400 Subject: [PATCH 068/158] Fix ``Connection.get_extra`` type (#28594) (cherry picked from commit 5dcbbd6f17c6b287a512b461d16a8a473b194ed9) --- airflow/models/connection.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airflow/models/connection.py b/airflow/models/connection.py index 8ed46e325d150..5f7406d9d4835 100644 --- a/airflow/models/connection.py +++ b/airflow/models/connection.py @@ -281,7 +281,7 @@ def password(cls): """Password. The value is decrypted/encrypted when reading/setting the value.""" return synonym("_password", descriptor=property(cls.get_password, cls.set_password)) - def get_extra(self) -> dict: + def get_extra(self) -> str: """Return encrypted extra-data.""" if self._extra and self.is_extra_encrypted: fernet = get_fernet() From 2edf3badef9c9b0e232c02f79ace18aed1e8bb4f Mon Sep 17 00:00:00 2001 From: Jihoon Park <62637758+aaronsatae@users.noreply.github.com> Date: Wed, 28 Dec 2022 04:00:39 +0900 Subject: [PATCH 069/158] Use docker compose v2 command (#28605) (cherry picked from commit cb1d798fb80f1e7e38e4300eb7fe9b1e1a5bcee8) --- .../howto/docker-compose/index.rst | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/docs/apache-airflow/howto/docker-compose/index.rst b/docs/apache-airflow/howto/docker-compose/index.rst index f9ec22d74fdc3..33e06e4b9629b 100644 --- a/docs/apache-airflow/howto/docker-compose/index.rst +++ b/docs/apache-airflow/howto/docker-compose/index.rst @@ -72,7 +72,7 @@ This file contains several service definitions: - ``postgres`` - The database. - ``redis`` - `The redis `__ - broker that forwards messages from scheduler to worker. -Optionally, you can enable flower by adding ``--profile flower`` option, e.g. ``docker-compose --profile flower up``, or by explicitly specifying it on the command line e.g. ``docker-compose up flower``. +Optionally, you can enable flower by adding ``--profile flower`` option, e.g. ``docker compose --profile flower up``, or by explicitly specifying it on the command line e.g. ``docker compose up flower``. - ``flower`` - `The flower app `__ for monitoring the environment. It is available at ``http://localhost:5555``. @@ -147,7 +147,7 @@ up and restart from scratch. The best way to do this is to: -* Run ``docker-compose down --volumes --remove-orphans`` command in the directory you downloaded the +* Run ``docker compose down --volumes --remove-orphans`` command in the directory you downloaded the ``docker-compose.yaml`` file * Remove the entire directory where you downloaded the ``docker-compose.yaml`` file ``rm -rf ''`` @@ -160,7 +160,7 @@ Now you can start all services: .. code-block:: bash - docker-compose up + docker compose up In a second terminal you can check the condition of the containers and make sure that no containers are in an unhealthy condition: @@ -191,7 +191,7 @@ You can also run :doc:`CLI commands `, but you have to do it in one .. code-block:: bash - docker-compose run airflow-worker airflow info + docker compose run airflow-worker airflow info If you have Linux or Mac OS, you can make your work easier and download a optional wrapper scripts that will allow you to run commands with a simpler command. @@ -252,7 +252,7 @@ To stop and delete containers, delete volumes with database data and download im .. code-block:: bash - docker-compose down --volumes --rmi all + docker compose down --volumes --rmi all Using custom images =================== @@ -260,9 +260,9 @@ Using custom images When you want to run Airflow locally, you might want to use an extended image, containing some additional dependencies - for example you might add new python packages, or upgrade airflow providers to a later version. This can be done very easily by specifying ``build: .`` in your ``docker-compose.yaml`` and placing a custom Dockerfile alongside your -``docker-compose.yaml``. Then you can use ``docker-compose build`` command -to build your image (you need to do it only once). You can also add the ``--build`` flag to your ``docker-compose`` commands -to rebuild the images on-the-fly when you run other ``docker-compose`` commands. +``docker-compose.yaml``. Then you can use ``docker compose build`` command +to build your image (you need to do it only once). You can also add the ``--build`` flag to your ``docker compose`` commands +to rebuild the images on-the-fly when you run other ``docker compose`` commands. Examples of how you can extend the image with custom providers, python packages, apt packages and more can be found in :doc:`Building the image `. From 652c6acd528f6391655e4a13f53e6944514c1561 Mon Sep 17 00:00:00 2001 From: Jared Fallt Date: Wed, 28 Dec 2022 11:00:26 -0500 Subject: [PATCH 070/158] Fix Incorrect Example (#28609) Following the existing example to create a decorator results in this error: AttributeError: 'list' object has no attribute 'rsplit' Changing it to a list fixes this (cherry picked from commit 761aa59f9256fb2901039ac8ef7fd6e69af8528f) --- docs/apache-airflow/howto/create-custom-decorator.rst | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/apache-airflow/howto/create-custom-decorator.rst b/docs/apache-airflow/howto/create-custom-decorator.rst index 182e87bb3f7bc..dc7efd21c44cc 100644 --- a/docs/apache-airflow/howto/create-custom-decorator.rst +++ b/docs/apache-airflow/howto/create-custom-decorator.rst @@ -77,7 +77,7 @@ tasks. The steps to create and register ``@task.foo`` are: { "name": "foo", # "Import path" and function name of the `foo_task` - "class-name": ["name.of.python.package.foo_task"], + "class-name": "name.of.python.package.foo_task", } ], # ... From 274304fb30e26634e7b55992ba930c09ecd9569b Mon Sep 17 00:00:00 2001 From: Max Ho Date: Wed, 28 Dec 2022 23:45:22 +0800 Subject: [PATCH 071/158] Minor update to Scheduler documentation (#28620) (cherry picked from commit 48a051acc40469ce43e28353fffd265e5326926a) --- docs/apache-airflow/concepts/scheduler.rst | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/apache-airflow/concepts/scheduler.rst b/docs/apache-airflow/concepts/scheduler.rst index e6b4f14575027..d9ba22be825c8 100644 --- a/docs/apache-airflow/concepts/scheduler.rst +++ b/docs/apache-airflow/concepts/scheduler.rst @@ -239,7 +239,7 @@ There are several areas of resource usage that you should pay attention to: but if your problems with performance come from distributed filesystem performance, they might be the best approach to follow. * Database connections and Database usage might become a problem as you want to increase performance and - process more things in parallel. Airflow is known from being "database-connection hungry" - the more DAGs + process more things in parallel. Airflow is known for being "database-connection hungry" - the more DAGs you have and the more you want to process in parallel, the more database connections will be opened. This is generally not a problem for MySQL as its model of handling connections is thread-based, but this might be a problem for Postgres, where connection handling is process-based. It is a general consensus @@ -257,8 +257,8 @@ There are several areas of resource usage that you should pay attention to: usage. If you have more CPUs available, you can increase number of processing threads :ref:`config:scheduler__parsing_processes`, Also Airflow Scheduler scales almost linearly with several instances, so you can also add more Schedulers if your Scheduler's performance is CPU-bound. -* Airflow might use quite significant amount of memory when you try to get more performance out of it. - Often more performance is achieved in Airflow by increasing number of processes handling the load, +* Airflow might use quite a significant amount of memory when you try to get more performance out of it. + Often more performance is achieved in Airflow by increasing the number of processes handling the load, and each process requires whole interpreter of Python loaded, a lot of classes imported, temporary in-memory storage. A lot of it is optimized by Airflow by using forking and copy-on-write memory used but in case new classes are imported after forking this can lead to extra memory pressure. From a4fef1fbb90758745e950d4ae7279ae7b4da47ae Mon Sep 17 00:00:00 2001 From: Stanislav Kazanov <94890848+stamixthereal@users.noreply.github.com> Date: Wed, 28 Dec 2022 15:01:23 +0300 Subject: [PATCH 072/158] Fix code docstrings (#28622) Fix docstrings grammar Co-authored-by: kazanau (cherry picked from commit 76186bb58854d851e3599fac3ff9f20feff43bfd) --- airflow/models/baseoperator.py | 16 ++++++++-------- airflow/models/dag.py | 14 +++++++------- airflow/models/dagbag.py | 4 ++-- airflow/models/dagrun.py | 12 ++++++------ 4 files changed, 23 insertions(+), 23 deletions(-) diff --git a/airflow/models/baseoperator.py b/airflow/models/baseoperator.py index ff400804a4840..9448c4a415db9 100644 --- a/airflow/models/baseoperator.py +++ b/airflow/models/baseoperator.py @@ -413,8 +413,8 @@ def apply_defaults(self: BaseOperator, *args: Any, **kwargs: Any) -> Any: def __new__(cls, name, bases, namespace, **kwargs): new_cls = super().__new__(cls, name, bases, namespace, **kwargs) with contextlib.suppress(KeyError): - # Update the partial descriptor with the class method so it call call the actual function (but let - # subclasses override it if they need to) + # Update the partial descriptor with the class method, so it calls the actual function + # (but let subclasses override it if they need to) partial_desc = vars(new_cls)["partial"] if isinstance(partial_desc, _PartialDescriptor): partial_desc.class_method = classmethod(partial) @@ -448,7 +448,7 @@ class derived from this one results in the creation of a task object, (e.g. user/person/team/role name) to clarify ownership is recommended. :param email: the 'to' email address(es) used in email alerts. This can be a single email or multiple ones. Multiple addresses can be specified as a - comma or semi-colon separated string or by passing a list of strings. + comma or semicolon separated string or by passing a list of strings. :param email_on_retry: Indicates whether email alerts should be sent when a task is retried :param email_on_failure: Indicates whether email alerts should be sent when @@ -932,7 +932,7 @@ def __hash__(self): def __or__(self, other): """ Called for [This Operator] | [Operator], The inlets of other - will be set to pickup the outlets from this operator. Other will + will be set to pick up the outlets from this operator. Other will be set as a downstream task of this operator. """ if isinstance(other, BaseOperator): @@ -1124,9 +1124,9 @@ def post_execute(self, context: Any, result: Any = None): def on_kill(self) -> None: """ - Override this method to cleanup subprocesses when a task instance + Override this method to clean up subprocesses when a task instance gets killed. Any use of the threading, subprocess or multiprocessing - module within an operator needs to be cleaned up or it will leave + module within an operator needs to be cleaned up, or it will leave ghost processes behind. """ @@ -1172,7 +1172,7 @@ def render_template_fields( This mutates the attributes in-place and is irreversible. :param context: Context dict with values to apply on content. - :param jinja_env: Jinja environment to use for rendering. + :param jinja_env: Jinja's environment to use for rendering. """ if not jinja_env: jinja_env = self.get_template_env() @@ -1463,7 +1463,7 @@ def inherits_from_empty_operator(self): """Used to determine if an Operator is inherited from EmptyOperator""" # This looks like `isinstance(self, EmptyOperator) would work, but this also # needs to cope when `self` is a Serialized instance of a EmptyOperator or one - # of its sub-classes (which don't inherit from anything but BaseOperator). + # of its subclasses (which don't inherit from anything but BaseOperator). return getattr(self, "_is_empty", False) def defer( diff --git a/airflow/models/dag.py b/airflow/models/dag.py index 63856349630a2..4cb59754468d2 100644 --- a/airflow/models/dag.py +++ b/airflow/models/dag.py @@ -189,7 +189,7 @@ def create_timetable(interval: ScheduleIntervalArg, timezone: Timezone) -> Timet def get_last_dagrun(dag_id, session, include_externally_triggered=False): """ Returns the last dag run for a dag, None if there was none. - Last dag run can be any type of run eg. scheduled or backfilled. + Last dag run can be any type of run e.g. scheduled or backfilled. Overridden DagRuns are ignored. """ DR = DagRun @@ -276,8 +276,8 @@ class DAG(LoggingMixin): :param start_date: The timestamp from which the scheduler will attempt to backfill :param end_date: A date beyond which your DAG won't run, leave to None - for open ended scheduling - :param template_searchpath: This list of folders (non relative) + for open-ended scheduling + :param template_searchpath: This list of folders (non-relative) defines where jinja will look for your templates. Order matters. Note that jinja/airflow includes the path of your DAG file by default @@ -807,7 +807,7 @@ def get_next_data_interval(self, dag_model: DagModel) -> DataInterval | None: schedule if the run does not have an explicit one set, which is possible for runs created prior to AIP-39. - This function is private to Airflow core and should not be depended as a + This function is private to Airflow core and should not be depended on as a part of the Python API. :meta private: @@ -832,7 +832,7 @@ def get_run_data_interval(self, run: DagRun) -> DataInterval: schedule if the run does not have an explicit one set, which is possible for runs created prior to AIP-39. - This function is private to Airflow core and should not be depended as a + This function is private to Airflow core and should not be depended on as a part of the Python API. :meta private: @@ -2806,7 +2806,7 @@ def bulk_write_to_db( # here we go through dags and tasks to check for dataset references # if there are now None and previously there were some, we delete them - # if there are now *any*, we add them to the above data structures and + # if there are now *any*, we add them to the above data structures, and # later we'll persist them to the database. for dag in dags: curr_orm_dag = existing_dags.get(dag.dag_id) @@ -3496,7 +3496,7 @@ def wrapper(f: Callable) -> Callable[..., DAG]: @functools.wraps(f) def factory(*args, **kwargs): # Generate signature for decorated function and bind the arguments when called - # we do this to extract parameters so we can annotate them on the DAG object. + # we do this to extract parameters, so we can annotate them on the DAG object. # In addition, this fails if we are missing any args/kwargs with TypeError as expected. f_sig = signature(f).bind(*args, **kwargs) # Apply defaults to capture default values if set. diff --git a/airflow/models/dagbag.py b/airflow/models/dagbag.py index 8f11d7996f6d5..f78125ebd9d46 100644 --- a/airflow/models/dagbag.py +++ b/airflow/models/dagbag.py @@ -154,7 +154,7 @@ def size(self) -> int: @property def store_serialized_dags(self) -> bool: - """Whether or not to read dags from DB""" + """Whether to read dags from DB""" warnings.warn( "The store_serialized_dags property has been deprecated. Use read_dags_from_db instead.", RemovedInAirflow3Warning, @@ -176,7 +176,7 @@ def get_dag(self, dag_id, session: Session = None): """ Gets the DAG out of the dictionary, and refreshes it if expired - :param dag_id: DAG Id + :param dag_id: DAG ID """ # Avoid circular import from airflow.models.dag import DagModel diff --git a/airflow/models/dagrun.py b/airflow/models/dagrun.py index 35d4e1b61e88d..2c736c4c2efe5 100644 --- a/airflow/models/dagrun.py +++ b/airflow/models/dagrun.py @@ -529,7 +529,7 @@ def update_state( of its TaskInstances. :param session: Sqlalchemy ORM Session - :param execute_callbacks: Should dag callbacks (success/failure, SLA etc) be invoked + :param execute_callbacks: Should dag callbacks (success/failure, SLA etc.) be invoked directly (default: true) or recorded as a pending request in the ``returned_callback`` property :return: Tuple containing tis that can be scheduled in the current loop & `returned_callback` that needs to be executed @@ -713,7 +713,7 @@ def _filter_tis_and_exclude_removed(dag: DAG, tis: list[TI]) -> Iterable[TI]: session=session, ) - # During expansion we may change some tis into non-schedulable + # During expansion, we may change some tis into non-schedulable # states, so we need to re-compute. if expansion_happened: changed_tis = True @@ -829,8 +829,8 @@ def _are_premature_tis( ignore_in_reschedule_period=True, finished_tis=finished_tis, ) - # there might be runnable tasks that are up for retry and for some reason(retry delay, etc) are - # not ready yet so we set the flags to count them in + # there might be runnable tasks that are up for retry and for some reason(retry delay, etc.) are + # not ready yet, so we set the flags to count them in return ( any(ut.are_dependencies_met(dep_context=dep_context, session=session) for ut in unfinished_tis), dep_context.have_changed_ti_states, @@ -844,7 +844,7 @@ def _emit_true_scheduling_delay_stats_for_finished_state(self, finished_tis: lis is updated to a completed status (either success or failure). The method will find the first started task within the DAG and calculate the expected DagRun start time (based on dag.execution_date & dag.timetable), and minus these two values to get the delay. - The emitted data may contains outlier (e.g. when the first task was cleared, so + The emitted data may contain outlier (e.g. when the first task was cleared, so the second task's start_date will be used), but we can get rid of the outliers on the stats side through the dashboards tooling built. Note, the stat will only be emitted if the DagRun is a scheduler triggered one @@ -993,7 +993,7 @@ def _check_for_removed_or_restored_tasks( ) ti.state = State.REMOVED else: - # Check if the number of mapped literals has changed and we need to mark this TI as removed. + # Check if the number of mapped literals has changed, and we need to mark this TI as removed. if ti.map_index >= num_mapped_tis: self.log.debug( "Removing task '%s' as the map_index is longer than the literal mapping list (%s)", From 958169118b9af01b07db92cb09096b7887f1f411 Mon Sep 17 00:00:00 2001 From: John Bampton Date: Thu, 29 Dec 2022 01:44:52 +1000 Subject: [PATCH 073/158] Fix typo (#28623) (cherry picked from commit 4f7ac623c881ae0d5304d6a9d57e3e0f2aa65865) --- airflow/utils/db.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airflow/utils/db.py b/airflow/utils/db.py index 8f8e0437baae7..8f22bc1dfbb55 100644 --- a/airflow/utils/db.py +++ b/airflow/utils/db.py @@ -1570,7 +1570,7 @@ def upgradedb( log.info("Creating tables") val = os.environ.get("AIRFLOW__DATABASE__SQL_ALCHEMY_MAX_SIZE") try: - # Reconfigure the ORM ot use _EXACTLY_ one connection, otherwise some db engines hang forever + # Reconfigure the ORM to use _EXACTLY_ one connection, otherwise some db engines hang forever # trying to ALTER TABLEs os.environ["AIRFLOW__DATABASE__SQL_ALCHEMY_MAX_SIZE"] = "1" settings.reconfigure_orm(pool_class=sqlalchemy.pool.SingletonThreadPool) From c442215bc2640ce3f22ba099540c87632779bb61 Mon Sep 17 00:00:00 2001 From: Victor Chiapaikeo Date: Mon, 2 Jan 2023 16:01:28 -0500 Subject: [PATCH 074/158] Fix UI caret direction (#28624) * Fix UI caret direction * Rename ids and fix tests (cherry picked from commit 0ab881a4ab78ca7d30712c893a6f01b83eb60e9e) --- airflow/www/static/css/flash.css | 8 ++++---- airflow/www/static/js/dag/grid/TaskName.test.tsx | 2 +- airflow/www/static/js/dag/grid/TaskName.tsx | 2 +- airflow/www/static/js/dag/grid/index.test.tsx | 14 +++++++------- 4 files changed, 13 insertions(+), 13 deletions(-) diff --git a/airflow/www/static/css/flash.css b/airflow/www/static/css/flash.css index c121bc0e628dd..272d46c79f913 100644 --- a/airflow/www/static/css/flash.css +++ b/airflow/www/static/css/flash.css @@ -20,7 +20,7 @@ .panel-heading #alerts-accordion-toggle::after { /* symbol for "opening" panels */ font-family: FontAwesome;/* stylelint-disable-line font-family-no-missing-generic-family-keyword */ - content: "\f078"; + content: "\f077"; float: right; color: grey; } @@ -28,7 +28,7 @@ .panel-heading #alerts-accordion-toggle.collapsed::after { /* symbol for "closing" panels */ font-family: FontAwesome;/* stylelint-disable-line font-family-no-missing-generic-family-keyword */ - content: "\f054"; + content: "\f078"; float: right; color: grey; } @@ -53,7 +53,7 @@ .dag-import-error::after { /* symbol for "opening" panels */ font-family: FontAwesome;/* stylelint-disable-line font-family-no-missing-generic-family-keyword */ - content: "\f054"; + content: "\f078"; float: right; color: #e43921; position: absolute; @@ -63,5 +63,5 @@ .dag-import-error.expanded-error::after { /* symbol for "closing" panels */ - content: "\f078"; + content: "\f077"; } diff --git a/airflow/www/static/js/dag/grid/TaskName.test.tsx b/airflow/www/static/js/dag/grid/TaskName.test.tsx index b5e869add593f..d2d8f4a7758d9 100644 --- a/airflow/www/static/js/dag/grid/TaskName.test.tsx +++ b/airflow/www/static/js/dag/grid/TaskName.test.tsx @@ -43,6 +43,6 @@ describe('Test TaskName', () => { const { getByText, getByTestId } = render( {}} />, { wrapper: ChakraWrapper }); expect(getByText('test')).toBeDefined(); - expect(getByTestId('closed-group')).toBeDefined(); + expect(getByTestId('open-group')).toBeDefined(); }); }); diff --git a/airflow/www/static/js/dag/grid/TaskName.tsx b/airflow/www/static/js/dag/grid/TaskName.tsx index d7c074bfa1625..ae0c0acb6a56f 100644 --- a/airflow/www/static/js/dag/grid/TaskName.tsx +++ b/airflow/www/static/js/dag/grid/TaskName.tsx @@ -57,7 +57,7 @@ const TaskName = ({ )} {isGroup && ( - isOpen ? : + isOpen ? : )} ); diff --git a/airflow/www/static/js/dag/grid/index.test.tsx b/airflow/www/static/js/dag/grid/index.test.tsx index 3e6f4536bb6a4..4475a75a345a7 100644 --- a/airflow/www/static/js/dag/grid/index.test.tsx +++ b/airflow/www/static/js/dag/grid/index.test.tsx @@ -144,7 +144,7 @@ describe('Test ToggleGroups', () => { expect(getAllByTestId('task-instance')).toHaveLength(1); expect(groupName).toBeInTheDocument(); - expect(getByTestId('closed-group')).toBeInTheDocument(); + expect(getByTestId('open-group')).toBeInTheDocument(); }); test('Buttons are disabled if all groups are expanded or collapsed', () => { @@ -179,21 +179,21 @@ describe('Test ToggleGroups', () => { expect(queryAllByTestId('task-instance')).toHaveLength(3); expect(groupName).toBeInTheDocument(); - expect(queryAllByTestId('open-group')).toHaveLength(2); - expect(queryAllByTestId('closed-group')).toHaveLength(0); + expect(queryAllByTestId('close-group')).toHaveLength(2); + expect(queryAllByTestId('open-group')).toHaveLength(0); fireEvent.click(collapseButton); await waitFor(() => expect(queryAllByTestId('task-instance')).toHaveLength(1)); - expect(queryAllByTestId('open-group')).toHaveLength(0); + expect(queryAllByTestId('close-group')).toHaveLength(0); // Since the groups are nested, only the parent row is rendered - expect(queryAllByTestId('closed-group')).toHaveLength(1); + expect(queryAllByTestId('open-group')).toHaveLength(1); fireEvent.click(expandButton); await waitFor(() => expect(queryAllByTestId('task-instance')).toHaveLength(3)); - expect(queryAllByTestId('open-group')).toHaveLength(2); - expect(queryAllByTestId('closed-group')).toHaveLength(0); + expect(queryAllByTestId('close-group')).toHaveLength(2); + expect(queryAllByTestId('open-group')).toHaveLength(0); }); test('Hovered effect on task state', async () => { From 73169bf025434889c431a879a4d99eb9c616dcae Mon Sep 17 00:00:00 2001 From: Stanislav Kazanov <94890848+stamixthereal@users.noreply.github.com> Date: Thu, 29 Dec 2022 21:37:48 +0300 Subject: [PATCH 075/158] Add doc-strings and small improvement to email util (#28634) (cherry picked from commit 906264dd904a8f44a1533ffb70ec0cd6b9f92a4b) --- airflow/utils/email.py | 123 ++++++++++++++++++++++++++++++----------- 1 file changed, 90 insertions(+), 33 deletions(-) diff --git a/airflow/utils/email.py b/airflow/utils/email.py index 67c6a90b2610d..ee570399e7f21 100644 --- a/airflow/utils/email.py +++ b/airflow/utils/email.py @@ -20,6 +20,7 @@ import collections.abc import logging import os +import re import smtplib import warnings from email.mime.application import MIMEApplication @@ -47,8 +48,26 @@ def send_email( conn_id: str | None = None, custom_headers: dict[str, Any] | None = None, **kwargs, -): - """Send email using backend specified in EMAIL_BACKEND.""" +) -> None: + """ + Send an email using the backend specified in the *EMAIL_BACKEND* configuration option. + + :param to: A list or iterable of email addresses to send the email to. + :param subject: The subject of the email. + :param html_content: The content of the email in HTML format. + :param files: A list of paths to files to attach to the email. + :param dryrun: If *True*, the email will not actually be sent. Default: *False*. + :param cc: A string or iterable of strings containing email addresses to send a copy of the email to. + :param bcc: A string or iterable of strings containing email addresses to send a + blind carbon copy of the email to. + :param mime_subtype: The subtype of the MIME message. Default: "mixed". + :param mime_charset: The charset of the email. Default: "utf-8". + :param conn_id: The connection ID to use for the backend. If not provided, the default connection + specified in the *EMAIL_CONN_ID* configuration option will be used. + :param custom_headers: A dictionary of additional headers to add to the MIME message. + No validations are run on these values, and they should be able to be encoded. + :param kwargs: Additional keyword arguments to pass to the backend. + """ backend = conf.getimport("email", "EMAIL_BACKEND") backend_conn_id = conn_id or conf.get("email", "EMAIL_CONN_ID") from_email = conf.get("email", "from_email", fallback=None) @@ -87,9 +106,22 @@ def send_email_smtp( from_email: str | None = None, custom_headers: dict[str, Any] | None = None, **kwargs, -): - """ - Send an email with html content +) -> None: + """Send an email with html content. + + :param to: Recipient email address or list of addresses. + :param subject: Email subject. + :param html_content: Email body in HTML format. + :param files: List of file paths to attach to the email. + :param dryrun: If True, the email will not be sent, but all other actions will be performed. + :param cc: Carbon copy recipient email address or list of addresses. + :param bcc: Blind carbon copy recipient email address or list of addresses. + :param mime_subtype: MIME subtype of the email. + :param mime_charset: MIME charset of the email. + :param conn_id: Connection ID of the SMTP server. + :param from_email: Sender email address. + :param custom_headers: Dictionary of custom headers to include in the email. + :param kwargs: Additional keyword arguments. >>> send_email('test@example.com', 'foo', 'Foo bar', ['/dev/null'], dryrun=True) """ @@ -133,21 +165,20 @@ def build_mime_message( custom_headers: dict[str, Any] | None = None, ) -> tuple[MIMEMultipart, list[str]]: """ - Build a MIME message that can be used to send an email and - returns full list of recipients. - - :param mail_from: Email address to set as email's from - :param to: List of email addresses to set as email's to - :param subject: Email's subject - :param html_content: Content of email in HTML format - :param files: List of paths of files to be attached - :param cc: List of email addresses to set as email's CC - :param bcc: List of email addresses to set as email's BCC - :param mime_subtype: Can be used to specify the subtype of the message. Default = mixed - :param mime_charset: Email's charset. Default = UTF-8. - :param custom_headers: Additional headers to add to the MIME message. - No validations are run on these values and they should be able to be encoded. - :return: Email as MIMEMultipart and list of recipients' addresses. + Build a MIME message that can be used to send an email and returns a full list of recipients. + + :param mail_from: Email address to set as the email's "From" field. + :param to: A string or iterable of strings containing email addresses to set as the email's "To" field. + :param subject: The subject of the email. + :param html_content: The content of the email in HTML format. + :param files: A list of paths to files to be attached to the email. + :param cc: A string or iterable of strings containing email addresses to set as the email's "CC" field. + :param bcc: A string or iterable of strings containing email addresses to set as the email's "BCC" field. + :param mime_subtype: The subtype of the MIME message. Default: "mixed". + :param mime_charset: The charset of the email. Default: "utf-8". + :param custom_headers: Additional headers to add to the MIME message. No validations are run on these + values, and they should be able to be encoded. + :return: A tuple containing the email as a MIMEMultipart object and a list of recipient email addresses. """ to = get_email_address_list(to) @@ -159,12 +190,12 @@ def build_mime_message( if cc: cc = get_email_address_list(cc) msg["CC"] = ", ".join(cc) - recipients = recipients + cc + recipients += cc if bcc: # don't add bcc in header bcc = get_email_address_list(bcc) - recipients = recipients + bcc + recipients += bcc msg["Date"] = formatdate(localtime=True) mime_text = MIMEText(html_content, "html", mime_charset) @@ -192,7 +223,15 @@ def send_mime_email( conn_id: str = "smtp_default", dryrun: bool = False, ) -> None: - """Send MIME email.""" + """ + Send a MIME email. + + :param e_from: The email address of the sender. + :param e_to: The email address or a list of email addresses of the recipient(s). + :param mime_msg: The MIME message to send. + :param conn_id: The ID of the SMTP connection to use. + :param dryrun: If True, the email will not be sent, but a log message will be generated. + """ smtp_host = conf.get_mandatory_value("smtp", "SMTP_HOST") smtp_port = conf.getint("smtp", "SMTP_PORT") smtp_starttls = conf.getboolean("smtp", "SMTP_STARTTLS") @@ -245,20 +284,33 @@ def send_mime_email( def get_email_address_list(addresses: str | Iterable[str]) -> list[str]: - """Get list of email addresses.""" + """ + Returns a list of email addresses from the provided input. + + :param addresses: A string or iterable of strings containing email addresses. + :return: A list of email addresses. + :raises TypeError: If the input is not a string or iterable of strings. + """ if isinstance(addresses, str): return _get_email_list_from_str(addresses) - elif isinstance(addresses, collections.abc.Iterable): if not all(isinstance(item, str) for item in addresses): raise TypeError("The items in your iterable must be strings.") return list(addresses) - - received_type = type(addresses).__name__ - raise TypeError(f"Unexpected argument type: Received '{received_type}'.") + else: + raise TypeError(f"Unexpected argument type: Received '{type(addresses).__name__}'.") def _get_smtp_connection(host: str, port: int, timeout: int, with_ssl: bool) -> smtplib.SMTP: + """ + Returns an SMTP connection to the specified host and port, with optional SSL encryption. + + :param host: The hostname or IP address of the SMTP server. + :param port: The port number to connect to on the SMTP server. + :param timeout: The timeout in seconds for the connection. + :param with_ssl: Whether to use SSL encryption for the connection. + :return: An SMTP connection to the specified host and port. + """ return ( smtplib.SMTP_SSL(host=host, port=port, timeout=timeout) if with_ssl @@ -267,8 +319,13 @@ def _get_smtp_connection(host: str, port: int, timeout: int, with_ssl: bool) -> def _get_email_list_from_str(addresses: str) -> list[str]: - delimiters = [",", ";"] - for delimiter in delimiters: - if delimiter in addresses: - return [address.strip() for address in addresses.split(delimiter)] - return [addresses] + """ + Extract a list of email addresses from a string. The string + can contain multiple email addresses separated by + any of the following delimiters: ',' or ';'. + + :param addresses: A string containing one or more email addresses. + :return: A list of email addresses. + """ + pattern = r"\s*[,;]\s*" + return [address for address in re.split(pattern, addresses)] From e3bce861af76eaec0888471eda62163b31e38a51 Mon Sep 17 00:00:00 2001 From: Victor Chiapaikeo Date: Wed, 4 Jan 2023 16:19:28 -0500 Subject: [PATCH 076/158] Nest header blocks in divs to fix dagid copy nit on dag.html (#28643) (cherry picked from commit 9aea857343c231319df4c5f47e8b4d9c8c3975e6) --- airflow/www/templates/airflow/dag.html | 132 +++++++++++++------------ 1 file changed, 68 insertions(+), 64 deletions(-) diff --git a/airflow/www/templates/airflow/dag.html b/airflow/www/templates/airflow/dag.html index e6da97e68d5db..9fc7d516c0ae1 100644 --- a/airflow/www/templates/airflow/dag.html +++ b/airflow/www/templates/airflow/dag.html @@ -102,74 +102,78 @@ {% endif %}
    -

    - {% if dag.parent_dag is defined and dag.parent_dag %} - SUBDAG: {{ dag.dag_id }} - {% else %} - {% set can_edit = appbuilder.sm.can_edit_dag(dag.dag_id) %} - {% if appbuilder.sm.can_edit_dag(dag.dag_id) %} - {% set switch_tooltip = 'Pause/Unpause DAG' %} +
    +

    + {% if dag.parent_dag is defined and dag.parent_dag %} + SUBDAG: {{ dag.dag_id }} {% else %} - {% set switch_tooltip = 'DAG is Paused' if dag_is_paused else 'DAG is Active' %} + {% set can_edit = appbuilder.sm.can_edit_dag(dag.dag_id) %} + {% if appbuilder.sm.can_edit_dag(dag.dag_id) %} + {% set switch_tooltip = 'Pause/Unpause DAG' %} + {% else %} + {% set switch_tooltip = 'DAG is Paused' if dag_is_paused else 'DAG is Active' %} + {% endif %} + + DAG: {{ dag.dag_id }} + {{ dag.description[0:150] + '…' if dag.description and dag.description|length > 150 else dag.description|default('', true) }} + {% endif %} + {% if root %} + ROOT: {{ root }} + {% endif %} +

    +
    +
    +

    + {% if state_token is defined and state_token %} + {{ state_token }} + {% endif %} + + Schedule: {{ dag_model is defined and dag_model and dag_model.schedule_interval }} + + {% if dag_model is defined and dag_model and dag_model.timetable_description %} + {% endif %} - - DAG: {{ dag.dag_id }} - {{ dag.description[0:150] + '…' if dag.description and dag.description|length > 150 else dag.description|default('', true) }} - {% endif %} - {% if root %} - ROOT: {{ root }} - {% endif %} -

    -

    - {% if state_token is defined and state_token %} - {{ state_token }} - {% endif %} - - Schedule: {{ dag_model is defined and dag_model and dag_model.schedule_interval }} - - {% if dag_model is defined and dag_model and dag_model.timetable_description %} - - {% endif %} - {% if dag_model is defined and dag_model.next_dagrun is defined and dag_model.schedule_interval != 'Dataset' %} -

    - Next Run: -

    - {% endif %} - {% if dag_model is defined and dag_model.schedule_interval is defined and dag_model.schedule_interval == 'Dataset' %} - {%- with ds_info = dag_model.get_dataset_triggered_next_run_info() -%} - -

    + Next Run: +

    + {% endif %} + {% if dag_model is defined and dag_model.schedule_interval is defined and dag_model.schedule_interval == 'Dataset' %} + {%- with ds_info = dag_model.get_dataset_triggered_next_run_info() -%} + +

    + {% if ds_info.total == 1 -%} + On {{ ds_info.uri }} {%- else -%} {{ ds_info.ready }} of {{ ds_info.total }} datasets updated - {%- endif -%}" - > - {% if ds_info.total == 1 -%} - On {{ ds_info.uri }} - {%- else -%} - {{ ds_info.ready }} of {{ ds_info.total }} datasets updated - {%- endif %} -

    -
    - {%- endwith -%} - {% endif %} -

    + {%- endif %} +

    + + {%- endwith -%} + {% endif %} +

    +

    From 922963f19b66b7285e8c536bc25166a76c29988f Mon Sep 17 00:00:00 2001 From: Pankaj Singh <98807258+pankajastro@users.noreply.github.com> Date: Sat, 31 Dec 2022 01:01:26 +0530 Subject: [PATCH 077/158] Remove outdated comments from base hook (#28649) (cherry picked from commit 29a74699eb740f2df3bc6160fe3d747b175c76cd) --- airflow/hooks/base.py | 2 -- 1 file changed, 2 deletions(-) diff --git a/airflow/hooks/base.py b/airflow/hooks/base.py index 85467e7312774..9298a686889f7 100644 --- a/airflow/hooks/base.py +++ b/airflow/hooks/base.py @@ -81,8 +81,6 @@ def get_hook(cls, conn_id: str) -> BaseHook: :param conn_id: connection id :return: default hook for this connection """ - # TODO: set method return type to BaseHook class when on 3.7+. - # See https://stackoverflow.com/a/33533514/3066428 connection = cls.get_connection(conn_id) return connection.get_hook() From 7e34004b8cd6a053bc0b7f3089475cb0f16e2fff Mon Sep 17 00:00:00 2001 From: Bob Du Date: Thu, 22 Dec 2022 20:36:39 +0800 Subject: [PATCH 078/158] Fix `DetachedInstanceError` when finding zombies in Dag Parsing process (#28198) ``` [2022-12-06T14:20:21.622+0000] {base_job.py:229} DEBUG - [heartbeat] [2022-12-06T14:20:21.623+0000] {scheduler_job.py:1495} DEBUG - Finding 'running' jobs without a recent heartbeat [2022-12-06T14:20:21.637+0000] {scheduler_job.py:1515} WARNING - Failing (2) jobs without heartbeat after 2022-12-06 14:15:21.623199+00:00 [2022-12-06T14:20:21.641+0000] {scheduler_job.py:1526} ERROR - Detected zombie job: {'full_filepath': '/opt/airflow/dags/xxx_dag.py', 'processor_subdir': '/opt/airflow/dags', 'msg': "{'DAG Id': 'xxx', 'Task Id': 'xxx', 'Run Id': 'scheduled__2022-12-05T00:15:00+00:00', 'Hostname': 'airflow-worker-0.airflow-worker.airflow2.svc.cluster.local', 'External Executor Id': '9520cb9f-3245-497a-8e17-e9dec29d4549'}", 'simple_task_instance': , 'is_failure_callback': True} [2022-12-06T14:20:21.645+0000] {scheduler_job.py:763} ERROR - Exception when executing SchedulerJob._run_scheduler_loop Traceback (most recent call last): File "/home/airflow/.local/lib/python3.10/site-packages/airflow/jobs/scheduler_job.py", line 746, in _execute self._run_scheduler_loop() File "/home/airflow/.local/lib/python3.10/site-packages/airflow/jobs/scheduler_job.py", line 878, in _run_scheduler_loop next_event = timers.run(blocking=False) File "/usr/local/lib/python3.10/sched.py", line 151, in run action(*argument, **kwargs) File "/home/airflow/.local/lib/python3.10/site-packages/airflow/utils/event_scheduler.py", line 37, in repeat action(*args, **kwargs) File "/home/airflow/.local/lib/python3.10/site-packages/airflow/utils/session.py", line 75, in wrapper return func(*args, session=session, **kwargs) File "/home/airflow/.local/lib/python3.10/site-packages/airflow/jobs/scheduler_job.py", line 1522, in _find_zombies processor_subdir=ti.dag_model.processor_subdir, File "/home/airflow/.local/lib/python3.10/site-packages/sqlalchemy/orm/attributes.py", line 481, in __get__ return self.impl.get(state, dict_) File "/home/airflow/.local/lib/python3.10/site-packages/sqlalchemy/orm/attributes.py", line 926, in get value = self._fire_loader_callables(state, key, passive) File "/home/airflow/.local/lib/python3.10/site-packages/sqlalchemy/orm/attributes.py", line 962, in _fire_loader_callables return self.callable_(state, passive) File "/home/airflow/.local/lib/python3.10/site-packages/sqlalchemy/orm/strategies.py", line 861, in _load_for_state raise orm_exc.DetachedInstanceError( sqlalchemy.orm.exc.DetachedInstanceError: Parent instance is not bound to a Session; lazy load operation of attribute 'dag_model' cannot proceed (Background on this error at: https://sqlalche.me/e/14/bhk3) [2022-12-06T14:20:21.647+0000] {celery_executor.py:443} DEBUG - Inquiring about 5 celery task(s) [2022-12-06T14:20:21.669+0000] {celery_executor.py:602} DEBUG - Fetched 5 state(s) for 5 task(s) [2022-12-06T14:20:21.669+0000] {celery_executor.py:446} DEBUG - Inquiries completed. [2022-12-06T14:20:21.669+0000] {scheduler_job.py:775} INFO - Exited execute loop [2022-12-06T14:20:21.674+0000] {cli_action_loggers.py:83} DEBUG - Calling callbacks: [] Traceback (most recent call last): File "/home/airflow/.local/bin/airflow", line 8, in sys.exit(main()) File "/home/airflow/.local/lib/python3.10/site-packages/airflow/__main__.py", line 39, in main args.func(args) File "/home/airflow/.local/lib/python3.10/site-packages/airflow/cli/cli_parser.py", line 52, in command return func(*args, **kwargs) File "/home/airflow/.local/lib/python3.10/site-packages/airflow/utils/cli.py", line 103, in wrapper return f(*args, **kwargs) File "/home/airflow/.local/lib/python3.10/site-packages/airflow/cli/commands/scheduler_command.py", line 85, in scheduler _run_scheduler_job(args=args) File "/home/airflow/.local/lib/python3.10/site-packages/airflow/cli/commands/scheduler_command.py", line 50, in _run_scheduler_job job.run() File "/home/airflow/.local/lib/python3.10/site-packages/airflow/jobs/base_job.py", line 247, in run self._execute() File "/home/airflow/.local/lib/python3.10/site-packages/airflow/jobs/scheduler_job.py", line 746, in _execute self._run_scheduler_loop() File "/home/airflow/.local/lib/python3.10/site-packages/airflow/jobs/scheduler_job.py", line 878, in _run_scheduler_loop next_event = timers.run(blocking=False) File "/usr/local/lib/python3.10/sched.py", line 151, in run action(*argument, **kwargs) File "/home/airflow/.local/lib/python3.10/site-packages/airflow/utils/event_scheduler.py", line 37, in repeat action(*args, **kwargs) File "/home/airflow/.local/lib/python3.10/site-packages/airflow/utils/session.py", line 75, in wrapper return func(*args, session=session, **kwargs) File "/home/airflow/.local/lib/python3.10/site-packages/airflow/jobs/scheduler_job.py", line 1522, in _find_zombies processor_subdir=ti.dag_model.processor_subdir, File "/home/airflow/.local/lib/python3.10/site-packages/sqlalchemy/orm/attributes.py", line 481, in __get__ return self.impl.get(state, dict_) File "/home/airflow/.local/lib/python3.10/site-packages/sqlalchemy/orm/attributes.py", line 926, in get value = self._fire_loader_callables(state, key, passive) File "/home/airflow/.local/lib/python3.10/site-packages/sqlalchemy/orm/attributes.py", line 962, in _fire_loader_callables return self.callable_(state, passive) File "/home/airflow/.local/lib/python3.10/site-packages/sqlalchemy/orm/strategies.py", line 861, in _load_for_state raise orm_exc.DetachedInstanceError( sqlalchemy.orm.exc.DetachedInstanceError: Parent instance is not bound to a Session; lazy load operation of attribute 'dag_model' cannot proceed (Background on this error at: https://sqlalche.me/e/14/bhk3) ``` When in standalone dag processor mode, will use `DatabaseCallbackSink` `_find_zombies` func call `self.executor.send_callback(request)` func. But not propagation orm `session` , provide_session in `send` func again. ``` class DatabaseCallbackSink(BaseCallbackSink): """Sends callbacks to database.""" @provide_session def send(self, callback: CallbackRequest, session: Session = NEW_SESSION) -> None: """Sends callback for execution.""" db_callback = DbCallbackRequest(callback=callback, priority_weight=10) session.add(db_callback) ``` Signed-off-by: BobDu (cherry picked from commit 4b340b7561e9db0055bf69ad0fc8b3a508ea7667) --- airflow/jobs/scheduler_job.py | 33 +++++++++++++------------ tests/jobs/test_scheduler_job.py | 41 ++++++++++++++++---------------- 2 files changed, 36 insertions(+), 38 deletions(-) diff --git a/airflow/jobs/scheduler_job.py b/airflow/jobs/scheduler_job.py index 6fe8d3d710cf9..baeabdc2ecbd0 100644 --- a/airflow/jobs/scheduler_job.py +++ b/airflow/jobs/scheduler_job.py @@ -1510,8 +1510,7 @@ def check_trigger_timeouts(self, session: Session = NEW_SESSION) -> None: if num_timed_out_tasks: self.log.info("Timed out %i deferred tasks without fired triggers", num_timed_out_tasks) - @provide_session - def _find_zombies(self, session: Session) -> None: + def _find_zombies(self) -> None: """ Find zombie task instances, which are tasks haven't heartbeated for too long or have a no-longer-running LocalTaskJob, and create a TaskCallbackRequest @@ -1522,30 +1521,30 @@ def _find_zombies(self, session: Session) -> None: self.log.debug("Finding 'running' jobs without a recent heartbeat") limit_dttm = timezone.utcnow() - timedelta(seconds=self._zombie_threshold_secs) - zombies = ( - session.query(TaskInstance, DagModel.fileloc) - .with_hint(TI, "USE INDEX (ti_state)", dialect_name="mysql") - .join(LocalTaskJob, TaskInstance.job_id == LocalTaskJob.id) - .join(DagModel, TaskInstance.dag_id == DagModel.dag_id) - .filter(TaskInstance.state == TaskInstanceState.RUNNING) - .filter( - or_( - LocalTaskJob.state != State.RUNNING, - LocalTaskJob.latest_heartbeat < limit_dttm, + with create_session() as session: + zombies: list[tuple[TI, str, str]] = ( + session.query(TI, DM.fileloc, DM.processor_subdir) + .with_hint(TI, "USE INDEX (ti_state)", dialect_name="mysql") + .join(LocalTaskJob, TaskInstance.job_id == LocalTaskJob.id) + .filter(TI.state == TaskInstanceState.RUNNING) + .filter( + or_( + LocalTaskJob.state != State.RUNNING, + LocalTaskJob.latest_heartbeat < limit_dttm, + ) ) + .filter(TI.queued_by_job_id == self.id) + .all() ) - .filter(TaskInstance.queued_by_job_id == self.id) - .all() - ) if zombies: self.log.warning("Failing (%s) jobs without heartbeat after %s", len(zombies), limit_dttm) - for ti, file_loc in zombies: + for ti, file_loc, processor_subdir in zombies: zombie_message_details = self._generate_zombie_message_details(ti) request = TaskCallbackRequest( full_filepath=file_loc, - processor_subdir=ti.dag_model.processor_subdir, + processor_subdir=processor_subdir, simple_task_instance=SimpleTaskInstance.from_ti(ti), msg=str(zombie_message_details), ) diff --git a/tests/jobs/test_scheduler_job.py b/tests/jobs/test_scheduler_job.py index b0c8e016c8321..f04a76f57e235 100644 --- a/tests/jobs/test_scheduler_job.py +++ b/tests/jobs/test_scheduler_job.py @@ -4129,14 +4129,13 @@ def test_timeout_triggers(self, dag_maker): assert ti2.state == State.DEFERRED def test_find_zombies_nothing(self): - with create_session() as session: - executor = MockExecutor(do_update=False) - self.scheduler_job = SchedulerJob(executor=executor) - self.scheduler_job.processor_agent = mock.MagicMock() + executor = MockExecutor(do_update=False) + self.scheduler_job = SchedulerJob(executor=executor) + self.scheduler_job.processor_agent = mock.MagicMock() - self.scheduler_job._find_zombies(session=session) + self.scheduler_job._find_zombies() - self.scheduler_job.executor.callback_sink.send.assert_not_called() + self.scheduler_job.executor.callback_sink.send.assert_not_called() def test_find_zombies(self, load_examples): dagbag = DagBag(TEST_DAG_FOLDER, read_dags_from_db=False) @@ -4179,20 +4178,21 @@ def test_find_zombies(self, load_examples): ti.queued_by_job_id = self.scheduler_job.id session.flush() - self.scheduler_job._find_zombies(session=session) + self.scheduler_job._find_zombies() - self.scheduler_job.executor.callback_sink.send.assert_called_once() - requests = self.scheduler_job.executor.callback_sink.send.call_args[0] - assert 1 == len(requests) - assert requests[0].full_filepath == dag.fileloc - assert requests[0].msg == str(self.scheduler_job._generate_zombie_message_details(ti)) - assert requests[0].is_failure_callback is True - assert isinstance(requests[0].simple_task_instance, SimpleTaskInstance) - assert ti.dag_id == requests[0].simple_task_instance.dag_id - assert ti.task_id == requests[0].simple_task_instance.task_id - assert ti.run_id == requests[0].simple_task_instance.run_id - assert ti.map_index == requests[0].simple_task_instance.map_index + self.scheduler_job.executor.callback_sink.send.assert_called_once() + requests = self.scheduler_job.executor.callback_sink.send.call_args[0] + assert 1 == len(requests) + assert requests[0].full_filepath == dag.fileloc + assert requests[0].msg == str(self.scheduler_job._generate_zombie_message_details(ti)) + assert requests[0].is_failure_callback is True + assert isinstance(requests[0].simple_task_instance, SimpleTaskInstance) + assert ti.dag_id == requests[0].simple_task_instance.dag_id + assert ti.task_id == requests[0].simple_task_instance.task_id + assert ti.run_id == requests[0].simple_task_instance.run_id + assert ti.map_index == requests[0].simple_task_instance.map_index + with create_session() as session: session.query(TaskInstance).delete() session.query(LocalTaskJob).delete() @@ -4267,12 +4267,11 @@ def test_find_zombies_handle_failure_callbacks_are_correctly_passed_to_dag_proce Check that the same set of failure callback with zombies are passed to the dag file processors until the next zombie detection logic is invoked. """ - with conf_vars({("core", "load_examples"): "False"}): + with conf_vars({("core", "load_examples"): "False"}), create_session() as session: dagbag = DagBag( dag_folder=os.path.join(settings.DAGS_FOLDER, "test_example_bash_operator.py"), read_dags_from_db=False, ) - session = settings.Session() session.query(LocalTaskJob).delete() dag = dagbag.get_dag("test_example_bash_operator") dag.sync_to_db(processor_subdir=TEST_DAG_FOLDER) @@ -4301,7 +4300,7 @@ def test_find_zombies_handle_failure_callbacks_are_correctly_passed_to_dag_proce self.scheduler_job.executor = MockExecutor() self.scheduler_job.processor_agent = mock.MagicMock() - self.scheduler_job._find_zombies(session=session) + self.scheduler_job._find_zombies() self.scheduler_job.executor.callback_sink.send.assert_called_once() From 8b5ce052f7bf92b2b0493b3304b1ea2ce163049a Mon Sep 17 00:00:00 2001 From: Amogh Desai Date: Fri, 16 Dec 2022 01:59:15 +0530 Subject: [PATCH 079/158] Adding an example dag for dynamic task mapping (#28325) (cherry picked from commit b263dbcb0f84fd9029591d1447a7c843cb970f15) --- .../example_dynamic_task_mapping.py | 38 +++++++++ .../test_docker_compose_quick_start.py | 1 - .../concepts/dynamic-task-mapping.rst | 23 +----- docs/build_docs.py | 5 +- docs/exts/docs_build/spelling_checks.py | 2 +- tests/serialization/test_dag_serialization.py | 77 +++++++++++++------ 6 files changed, 99 insertions(+), 47 deletions(-) create mode 100644 airflow/example_dags/example_dynamic_task_mapping.py diff --git a/airflow/example_dags/example_dynamic_task_mapping.py b/airflow/example_dags/example_dynamic_task_mapping.py new file mode 100644 index 0000000000000..dce6cda20972c --- /dev/null +++ b/airflow/example_dags/example_dynamic_task_mapping.py @@ -0,0 +1,38 @@ +# +# 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 DAG demonstrating the usage of dynamic task mapping.""" +from __future__ import annotations + +from datetime import datetime + +from airflow import DAG +from airflow.decorators import task + +with DAG(dag_id="example_dynamic_task_mapping", start_date=datetime(2022, 3, 4)) as dag: + + @task + def add_one(x: int): + return x + 1 + + @task + def sum_it(values): + total = sum(values) + print(f"Total was {total}") + + added_values = add_one.expand(x=[1, 2, 3]) + sum_it(added_values) diff --git a/docker_tests/test_docker_compose_quick_start.py b/docker_tests/test_docker_compose_quick_start.py index 6f25f625788d4..fd553ed1756eb 100644 --- a/docker_tests/test_docker_compose_quick_start.py +++ b/docker_tests/test_docker_compose_quick_start.py @@ -27,7 +27,6 @@ from unittest import mock import requests - from docker_tests.command_utils import run_command from docker_tests.constants import SOURCE_ROOT from docker_tests.docker_tests_utils import docker_image diff --git a/docs/apache-airflow/concepts/dynamic-task-mapping.rst b/docs/apache-airflow/concepts/dynamic-task-mapping.rst index 5ae0e9fb82031..d15c0ada77155 100644 --- a/docs/apache-airflow/concepts/dynamic-task-mapping.rst +++ b/docs/apache-airflow/concepts/dynamic-task-mapping.rst @@ -30,27 +30,10 @@ Simple mapping In its simplest form you can map over a list defined directly in your DAG file using the ``expand()`` function instead of calling your task directly. -.. code-block:: python - - from datetime import datetime - - from airflow import DAG - from airflow.decorators import task - - - with DAG(dag_id="simple_mapping", start_date=datetime(2022, 3, 4)) as dag: - - @task - def add_one(x: int): - return x + 1 - - @task - def sum_it(values): - total = sum(values) - print(f"Total was {total}") +If you want to see a simple usage of Dynamic Task Mapping, you can look below: - added_values = add_one.expand(x=[1, 2, 3]) - sum_it(added_values) +.. exampleinclude:: /../../airflow/example_dags/example_dynamic_task_mapping.py + :language: python This will show ``Total was 9`` in the task logs when executed. diff --git a/docs/build_docs.py b/docs/build_docs.py index d1fb06ccacad1..cd6c83249d4f3 100755 --- a/docs/build_docs.py +++ b/docs/build_docs.py @@ -25,9 +25,6 @@ from itertools import filterfalse, tee from typing import Callable, Iterable, NamedTuple, TypeVar -from rich.console import Console -from tabulate import tabulate - from docs.exts.docs_build import dev_index_generator, lint_checks from docs.exts.docs_build.code_utils import CONSOLE_WIDTH, PROVIDER_INIT_FILE from docs.exts.docs_build.docs_builder import DOCS_DIR, AirflowDocsBuilder, get_available_packages @@ -36,6 +33,8 @@ from docs.exts.docs_build.github_action_utils import with_group from docs.exts.docs_build.package_filter import process_package_filters from docs.exts.docs_build.spelling_checks import SpellingError, display_spelling_error_summary +from rich.console import Console +from tabulate import tabulate TEXT_RED = "\033[31m" TEXT_RESET = "\033[0m" diff --git a/docs/exts/docs_build/spelling_checks.py b/docs/exts/docs_build/spelling_checks.py index bbaa9fa5dde79..f89bfa50dc587 100644 --- a/docs/exts/docs_build/spelling_checks.py +++ b/docs/exts/docs_build/spelling_checks.py @@ -21,10 +21,10 @@ from functools import total_ordering from typing import NamedTuple +from docs.exts.docs_build.code_utils import CONSOLE_WIDTH from rich.console import Console from airflow.utils.code_utils import prepare_code_snippet -from docs.exts.docs_build.code_utils import CONSOLE_WIDTH CURRENT_DIR = os.path.abspath(os.path.join(os.path.dirname(__file__))) DOCS_DIR = os.path.abspath(os.path.join(CURRENT_DIR, os.pardir, os.pardir)) diff --git a/tests/serialization/test_dag_serialization.py b/tests/serialization/test_dag_serialization.py index 44411f5c075e8..ec07d609541e1 100644 --- a/tests/serialization/test_dag_serialization.py +++ b/tests/serialization/test_dag_serialization.py @@ -30,6 +30,7 @@ from pathlib import Path from unittest import mock +import attr import pendulum import pytest from dateutil.relativedelta import FR, relativedelta @@ -42,6 +43,7 @@ from airflow.kubernetes.pod_generator import PodGenerator from airflow.models import DAG, Connection, DagBag, Operator from airflow.models.baseoperator import BaseOperator, BaseOperatorLink +from airflow.models.expandinput import EXPAND_INPUT_EMPTY from airflow.models.mappedoperator import MappedOperator from airflow.models.param import Param, ParamsDict from airflow.models.xcom import XCOM_RETURN_KEY, XCom @@ -534,32 +536,47 @@ def validate_deserialized_task( serialized_task, task, ): - """Verify non-airflow operators are casted to BaseOperator.""" - assert isinstance(serialized_task, SerializedBaseOperator) + """Verify non-Airflow operators are casted to BaseOperator or MappedOperator.""" assert not isinstance(task, SerializedBaseOperator) - assert isinstance(task, BaseOperator) + assert isinstance(task, (BaseOperator, MappedOperator)) # Every task should have a task_group property -- even if it's the DAG's root task group assert serialized_task.task_group - fields_to_check = task.get_serialized_fields() - { - # Checked separately - "_task_type", - "_operator_name", - "subdag", - # Type is excluded, so don't check it - "_log", - # List vs tuple. Check separately - "template_ext", - "template_fields", - # We store the string, real dag has the actual code - "on_failure_callback", - "on_success_callback", - "on_retry_callback", - # Checked separately - "resources", - "params", - } + if isinstance(task, BaseOperator): + assert isinstance(serialized_task, SerializedBaseOperator) + fields_to_check = task.get_serialized_fields() - { + # Checked separately + "_task_type", + "_operator_name", + "subdag", + # Type is excluded, so don't check it + "_log", + # List vs tuple. Check separately + "template_ext", + "template_fields", + # We store the string, real dag has the actual code + "on_failure_callback", + "on_success_callback", + "on_retry_callback", + # Checked separately + "resources", + } + else: # Promised to be mapped by the assert above. + assert isinstance(serialized_task, MappedOperator) + fields_to_check = {f.name for f in attr.fields(MappedOperator)} + fields_to_check -= { + # Matching logic in BaseOperator.get_serialized_fields(). + "dag", + "task_group", + # List vs tuple. Check separately. + "operator_extra_links", + "template_ext", + "template_fields", + # Checked separately. + "operator_class", + "partial_kwargs", + } assert serialized_task.task_type == task.task_type @@ -580,9 +597,25 @@ def validate_deserialized_task( assert serialized_task.resources == task.resources # Ugly hack as some operators override params var in their init - if isinstance(task.params, ParamsDict): + if isinstance(task.params, ParamsDict) and isinstance(serialized_task.params, ParamsDict): assert serialized_task.params.dump() == task.params.dump() + if isinstance(task, MappedOperator): + # MappedOperator.operator_class holds a backup of the serialized + # data; checking its entirety basically duplicates this validation + # function, so we just do some satiny checks. + serialized_task.operator_class["_task_type"] == type(task).__name__ + serialized_task.operator_class["_operator_name"] == task._operator_name + + # Serialization cleans up default values in partial_kwargs, this + # adds them back to both sides. + default_partial_kwargs = ( + BaseOperator.partial(task_id="_")._expand(EXPAND_INPUT_EMPTY, strict=False).partial_kwargs + ) + serialized_partial_kwargs = {**default_partial_kwargs, **serialized_task.partial_kwargs} + original_partial_kwargs = {**default_partial_kwargs, **task.partial_kwargs} + assert serialized_partial_kwargs == original_partial_kwargs + # Check that for Deserialized task, task.subdag is None for all other Operators # except for the SubDagOperator where task.subdag is an instance of DAG object if task.task_type == "SubDagOperator": From c18fe56d22f07744b7d616a66eec909ee9168e66 Mon Sep 17 00:00:00 2001 From: Alex Vicol Date: Wed, 11 Jan 2023 13:33:12 -0800 Subject: [PATCH 080/158] After running the DAG the employees table is empty. (#28353) The reason is the sql syntax error: ERROR: subquery in FROM must have an alias LINE 3: FROM ( ^ HINT: For example, FROM (SELECT ...) [AS] foo. SQL state: 42601 Character: 37 The change fixes the sql syntax in merge_data. (cherry picked from commit 53893f13b2391c005eacf33115e5e50fd558a396) --- docs/apache-airflow/tutorial/pipeline.rst | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/apache-airflow/tutorial/pipeline.rst b/docs/apache-airflow/tutorial/pipeline.rst index 26bbfb83e3783..5fa854d0485ad 100644 --- a/docs/apache-airflow/tutorial/pipeline.rst +++ b/docs/apache-airflow/tutorial/pipeline.rst @@ -168,7 +168,7 @@ Here we select completely unique records from the retrieved data, then we check FROM ( SELECT DISTINCT * FROM employees_temp - ) + ) t ON CONFLICT ("Serial Number") DO UPDATE SET "Serial Number" = excluded."Serial Number"; """ @@ -283,7 +283,7 @@ Putting all of the pieces together, we have our completed DAG. FROM ( SELECT DISTINCT * FROM employees_temp - ) + ) t ON CONFLICT ("Serial Number") DO UPDATE SET "Serial Number" = excluded."Serial Number"; """ From 214a2ff8080ade14160c38adb65b83537fd45db0 Mon Sep 17 00:00:00 2001 From: Brent Bovenzi Date: Wed, 28 Dec 2022 00:52:54 -0500 Subject: [PATCH 081/158] Fix calendar view for CronTriggerTimeTable dags (#28411) Co-authored-by: Tzu-ping Chung (cherry picked from commit 467a5e3ab287013db2a5381ef4a642e912f8b45b) --- airflow/www/views.py | 49 ++++++++++++++++++++++++-------------------- 1 file changed, 27 insertions(+), 22 deletions(-) diff --git a/airflow/www/views.py b/airflow/www/views.py index c645a56b58fb4..14a57b5b4f460 100644 --- a/airflow/www/views.py +++ b/airflow/www/views.py @@ -33,7 +33,7 @@ from functools import wraps from json import JSONDecodeError from operator import itemgetter -from typing import Any, Callable, Collection +from typing import Any, Callable, Collection, Iterator from urllib.parse import unquote, urljoin, urlsplit import configupdater @@ -107,8 +107,8 @@ from airflow.security import permissions from airflow.ti_deps.dep_context import DepContext from airflow.ti_deps.dependencies_deps import RUNNING_DEPS, SCHEDULER_QUEUED_DEPS +from airflow.timetables._cron import CronMixin from airflow.timetables.base import DataInterval, TimeRestriction -from airflow.timetables.interval import CronDataIntervalTimetable from airflow.utils import json as utils_json, timezone, yaml from airflow.utils.airflow_flask_app import get_airflow_app from airflow.utils.dag_edges import dag_edges @@ -979,7 +979,6 @@ def task_stats(self, session=None): ) if conf.getboolean("webserver", "SHOW_RECENT_STATS_FOR_COMPLETED_RUNS", fallback=True): - last_dag_run = ( session.query(DagRun.dag_id, sqla.func.max(DagRun.execution_date).label("execution_date")) .join(DagModel, DagModel.dag_id == DagRun.dag_id) @@ -2730,28 +2729,37 @@ def _convert_to_date(session, column): restriction = TimeRestriction(dag.start_date, dag.end_date, False) dates = collections.Counter() - if isinstance(dag.timetable, CronDataIntervalTimetable): - for next in croniter( - dag.timetable.summary, start_time=last_automated_data_interval.end, ret_type=datetime - ): - if next is None: + if isinstance(dag.timetable, CronMixin): + # Optimized calendar generation for timetables based on a cron expression. + dates_iter: Iterator[datetime | None] = croniter( + dag.timetable._expression, + start_time=last_automated_data_interval.end, + ret_type=datetime, + ) + for dt in dates_iter: + if dt is None: break - if next.year != year: + if dt.year != year: break - if dag.end_date and next > dag.end_date: + if dag.end_date and dt > dag.end_date: break - dates[next.date()] += 1 + dates[dt.date()] += 1 else: + prev_logical_date = datetime.min while True: - info = dag.timetable.next_dagrun_info( - last_automated_data_interval=last_automated_data_interval, restriction=restriction + curr_info = dag.timetable.next_dagrun_info( + last_automated_data_interval=last_automated_data_interval, + restriction=restriction, ) - if info is None: - break - if info.logical_date.year != year: - break - last_automated_data_interval = info.data_interval - dates[info.logical_date] += 1 + if curr_info is None: + break # Reached the end. + if curr_info.logical_date <= prev_logical_date: + break # We're not progressing. Maybe a malformed timetable? Give up. + if curr_info.logical_date.year != year: + break # Crossed the year boundary. + last_automated_data_interval = curr_info.data_interval + dates[curr_info.logical_date] += 1 + prev_logical_date = curr_info.logical_date data_dag_states.extend( {"date": date.isoformat(), "state": "planned", "count": count} @@ -3845,7 +3853,6 @@ def conf(self): for key, (value, source) in parameters.items() ] elif expose_config.lower() in ["true", "t", "1"]: - with open(AIRFLOW_CONFIG) as file: config = file.read() table = [ @@ -4345,7 +4352,6 @@ def action_mulduplicate(self, connections, session=None): "warning", ) else: - dup_conn = Connection( new_conn_id, selected_conn.conn_type, @@ -5557,7 +5563,6 @@ def list(self): ) def _calculate_graph(self): - nodes_dict: dict[str, Any] = {} edge_tuples: set[dict[str, str]] = set() From 319234bf32785a2974f82c2aeb561602aa284639 Mon Sep 17 00:00:00 2001 From: Jarek Potiuk Date: Thu, 22 Dec 2022 03:25:08 +0100 Subject: [PATCH 082/158] Improve provider validation pre-commit (#28516) (cherry picked from commit e47c472e632effbfe3ddc784788a956c4ca44122) --- .pre-commit-config.yaml | 21 +- STATIC_CODE_CHECKS.rst | 2 +- airflow/cli/commands/info_command.py | 1 + .../pre_commit_check_provider_yaml_files.py | 417 ++------------- .../run_provider_yaml_files_check.py | 482 ++++++++++++++++++ 5 files changed, 528 insertions(+), 395 deletions(-) create mode 100755 scripts/in_container/run_provider_yaml_files_check.py diff --git a/.pre-commit-config.yaml b/.pre-commit-config.yaml index 5df89e4fc7b4d..a6ed9b1f4dd25 100644 --- a/.pre-commit-config.yaml +++ b/.pre-commit-config.yaml @@ -627,19 +627,6 @@ repos: entry: ./scripts/ci/pre_commit/pre_commit_check_providers_subpackages_all_have_init.py language: python require_serial: true - - id: check-provider-yaml-valid - name: Validate providers.yaml files - pass_filenames: false - entry: ./scripts/ci/pre_commit/pre_commit_check_provider_yaml_files.py - language: python - require_serial: true - files: ^docs/|provider\.yaml$|^scripts/ci/pre_commit/pre_commit_check_provider_yaml_files\.py$ - additional_dependencies: - - 'PyYAML==5.3.1' - - 'jsonschema>=3.2.0,<5.0.0' - - 'tabulate==0.8.8' - - 'jsonpath-ng==1.5.3' - - 'rich>=12.4.4' - id: check-pre-commit-information-consistent name: Update information re pre-commit hooks and verify ids and names entry: ./scripts/ci/pre_commit/pre_commit_check_pre_commit_hooks.py @@ -888,6 +875,14 @@ repos: pass_filenames: true exclude: ^airflow/_vendor/ additional_dependencies: ['rich>=12.4.4', 'inputimeout'] + - id: check-provider-yaml-valid + name: Validate provider.yaml files + pass_filenames: false + entry: ./scripts/ci/pre_commit/pre_commit_check_provider_yaml_files.py + language: python + require_serial: true + files: ^docs/|provider\.yaml$|^scripts/ci/pre_commit/pre_commit_check_provider_yaml_files\.py$ + additional_dependencies: ['rich>=12.4.4', 'inputimeout', 'markdown-it-py'] - id: update-migration-references name: Update migration ref doc language: python diff --git a/STATIC_CODE_CHECKS.rst b/STATIC_CODE_CHECKS.rst index 7495044f3dcd8..b2b6081b5f976 100644 --- a/STATIC_CODE_CHECKS.rst +++ b/STATIC_CODE_CHECKS.rst @@ -195,7 +195,7 @@ require Breeze Docker image to be build locally. +--------------------------------------------------------+------------------------------------------------------------------+---------+ | check-provide-create-sessions-imports | Check provide_session and create_session imports | | +--------------------------------------------------------+------------------------------------------------------------------+---------+ -| check-provider-yaml-valid | Validate providers.yaml files | | +| check-provider-yaml-valid | Validate provider.yaml files | * | +--------------------------------------------------------+------------------------------------------------------------------+---------+ | check-providers-init-file-missing | Provider init file is missing | | +--------------------------------------------------------+------------------------------------------------------------------+---------+ diff --git a/airflow/cli/commands/info_command.py b/airflow/cli/commands/info_command.py index 124271c8c9e61..a8a7c760abad6 100644 --- a/airflow/cli/commands/info_command.py +++ b/airflow/cli/commands/info_command.py @@ -176,6 +176,7 @@ def get_current(): "arm64": Architecture.ARM, "armv7": Architecture.ARM, "armv7l": Architecture.ARM, + "aarch64": Architecture.ARM, } diff --git a/scripts/ci/pre_commit/pre_commit_check_provider_yaml_files.py b/scripts/ci/pre_commit/pre_commit_check_provider_yaml_files.py index 5622212f46965..f1888792003f3 100755 --- a/scripts/ci/pre_commit/pre_commit_check_provider_yaml_files.py +++ b/scripts/ci/pre_commit/pre_commit_check_provider_yaml_files.py @@ -17,392 +17,47 @@ # under the License. from __future__ import annotations -import json -import pathlib +import os import sys -import textwrap -from collections import Counter -from itertools import chain, product -from typing import Any, Iterable +from pathlib import Path -import jsonschema -import yaml -from jsonpath_ng.ext import parse -from rich.console import Console -from tabulate import tabulate - -try: - from yaml import CSafeLoader as SafeLoader -except ImportError: - from yaml import SafeLoader # type: ignore - -if __name__ != "__main__": - raise Exception( +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" ) -ROOT_DIR = pathlib.Path(__file__).resolve().parents[3] -DOCS_DIR = ROOT_DIR.joinpath("docs") -PROVIDER_DATA_SCHEMA_PATH = ROOT_DIR.joinpath("airflow", "provider.yaml.schema.json") -PROVIDER_ISSUE_TEMPLATE_PATH = ROOT_DIR.joinpath( - ".github", "ISSUE_TEMPLATE", "airflow_providers_bug_report.yml" -) -CORE_INTEGRATIONS = ["SQL", "Local"] - -errors = [] - - -def _filepath_to_module(filepath: pathlib.Path) -> str: - p = filepath.resolve().relative_to(ROOT_DIR).as_posix() - if p.endswith(".py"): - p = p[:-3] - return p.replace("/", ".") - - -def _load_schema() -> dict[str, Any]: - with PROVIDER_DATA_SCHEMA_PATH.open() as schema_file: - content = json.load(schema_file) - return content - - -def _load_package_data(package_paths: Iterable[str]): - schema = _load_schema() - result = {} - for provider_yaml_path in package_paths: - with open(provider_yaml_path) as yaml_file: - provider = yaml.load(yaml_file, SafeLoader) - rel_path = pathlib.Path(provider_yaml_path).relative_to(ROOT_DIR).as_posix() - try: - jsonschema.validate(provider, schema=schema) - except jsonschema.ValidationError: - raise Exception(f"Unable to parse: {rel_path}.") - result[rel_path] = provider - return result - - -def get_all_integration_names(yaml_files) -> list[str]: - all_integrations = [ - i["integration-name"] for f in yaml_files.values() if "integrations" in f for i in f["integrations"] - ] - all_integrations += ["Local"] - return all_integrations - - -def check_integration_duplicates(yaml_files: dict[str, dict]): - """Integration names must be globally unique.""" - print("Checking integration duplicates") - all_integrations = get_all_integration_names(yaml_files) - - duplicates = [(k, v) for (k, v) in Counter(all_integrations).items() if v > 1] - - if duplicates: - print( - "Duplicate integration names found. Integration names must be globally unique. " - "Please delete duplicates." - ) - print(tabulate(duplicates, headers=["Integration name", "Number of occurrences"])) - sys.exit(3) - +AIRFLOW_SOURCES = Path(__file__).parents[3].resolve() +GITHUB_REPOSITORY = os.environ.get("GITHUB_REPOSITORY", "apache/airflow") +os.environ["SKIP_GROUP_OUTPUT"] = "true" -def assert_sets_equal(set1, set2): - try: - difference1 = set1.difference(set2) - except TypeError as e: - raise AssertionError(f"invalid type when attempting set difference: {e}") - except AttributeError as e: - raise AssertionError(f"first argument does not support set difference: {e}") - - try: - difference2 = set2.difference(set1) - except TypeError as e: - raise AssertionError(f"invalid type when attempting set difference: {e}") - except AttributeError as e: - raise AssertionError(f"second argument does not support set difference: {e}") - - if not (difference1 or difference2): - return - - lines = [] - if difference1: - lines.append(" -- Items in the left set but not the right:") - for item in sorted(difference1): - lines.append(f" {item!r}") - if difference2: - lines.append(" -- Items in the right set but not the left:") - for item in sorted(difference2): - lines.append(f" {item!r}") - - standard_msg = "\n".join(lines) - raise AssertionError(standard_msg) - - -def check_if_objects_belongs_to_package( - object_names: set[str], provider_package: str, yaml_file_path: str, resource_type: str -): - for object_name in object_names: - if not object_name.startswith(provider_package): - errors.append( - f"The `{object_name}` object in {resource_type} list in {yaml_file_path} does not start" - f" with the expected {provider_package}." - ) - - -def parse_module_data(provider_data, resource_type, yaml_file_path): - package_dir = ROOT_DIR.joinpath(yaml_file_path).parent - provider_package = pathlib.Path(yaml_file_path).parent.as_posix().replace("/", ".") - py_files = chain( - package_dir.glob(f"**/{resource_type}/*.py"), - package_dir.glob(f"{resource_type}/*.py"), - package_dir.glob(f"**/{resource_type}/**/*.py"), - package_dir.glob(f"{resource_type}/**/*.py"), +if __name__ == "__main__": + sys.path.insert(0, str(AIRFLOW_SOURCES / "dev" / "breeze" / "src")) + from airflow_breeze.global_constants import MOUNT_SELECTED + from airflow_breeze.utils.console import get_console + from airflow_breeze.utils.docker_command_utils import get_extra_docker_flags + from airflow_breeze.utils.run_utils import get_ci_image_for_pre_commits, run_command + + airflow_image = get_ci_image_for_pre_commits() + cmd_result = run_command( + [ + "docker", + "run", + "-t", + *get_extra_docker_flags(MOUNT_SELECTED), + "-e", + "SKIP_ENVIRONMENT_INITIALIZATION=true", + "--pull", + "never", + airflow_image, + "-c", + "python3 /opt/airflow/scripts/in_container/run_provider_yaml_files_check.py", + ], + check=False, ) - expected_modules = {_filepath_to_module(f) for f in py_files if f.name != "__init__.py"} - resource_data = provider_data.get(resource_type, []) - return expected_modules, provider_package, resource_data - - -def check_completeness_of_list_of_hooks_sensors_hooks(yaml_files: dict[str, dict]): - print("Checking completeness of list of {sensors, hooks, operators}") - print(" -- {sensors, hooks, operators} - Expected modules (left) : Current modules (right)") - for (yaml_file_path, provider_data), resource_type in product( - yaml_files.items(), ["sensors", "operators", "hooks"] - ): - expected_modules, provider_package, resource_data = parse_module_data( - provider_data, resource_type, yaml_file_path + if cmd_result.returncode != 0: + get_console().print( + "[warning]If you see strange stacktraces above, " + "run `breeze ci-image build --python 3.7` and try again." ) - - current_modules = {str(i) for r in resource_data for i in r.get("python-modules", [])} - check_if_objects_belongs_to_package(current_modules, provider_package, yaml_file_path, resource_type) - try: - assert_sets_equal(set(expected_modules), set(current_modules)) - except AssertionError as ex: - nested_error = textwrap.indent(str(ex), " ") - errors.append( - f"Incorrect content of key '{resource_type}/python-modules' " - f"in file: {yaml_file_path}\n{nested_error}" - ) - - -def check_duplicates_in_integrations_names_of_hooks_sensors_operators(yaml_files: dict[str, dict]): - print("Checking for duplicates in list of {sensors, hooks, operators}") - for (yaml_file_path, provider_data), resource_type in product( - yaml_files.items(), ["sensors", "operators", "hooks"] - ): - resource_data = provider_data.get(resource_type, []) - current_integrations = [r.get("integration-name", "") for r in resource_data] - if len(current_integrations) != len(set(current_integrations)): - for integration in current_integrations: - if current_integrations.count(integration) > 1: - errors.append( - f"Duplicated content of '{resource_type}/integration-name/{integration}' " - f"in file: {yaml_file_path}" - ) - - -def check_completeness_of_list_of_transfers(yaml_files: dict[str, dict]): - print("Checking completeness of list of transfers") - resource_type = "transfers" - - print(" -- Expected transfers modules(Left): Current transfers Modules(Right)") - for yaml_file_path, provider_data in yaml_files.items(): - expected_modules, provider_package, resource_data = parse_module_data( - provider_data, resource_type, yaml_file_path - ) - - current_modules = {r.get("python-module") for r in resource_data} - check_if_objects_belongs_to_package(current_modules, provider_package, yaml_file_path, resource_type) - try: - assert_sets_equal(set(expected_modules), set(current_modules)) - except AssertionError as ex: - nested_error = textwrap.indent(str(ex), " ") - errors.append( - f"Incorrect content of key '{resource_type}/python-module' " - f"in file: {yaml_file_path}\n{nested_error}" - ) - - -def check_hook_classes(yaml_files: dict[str, dict]): - print("Checking connection classes belong to package") - resource_type = "hook-class-names" - for yaml_file_path, provider_data in yaml_files.items(): - provider_package = pathlib.Path(yaml_file_path).parent.as_posix().replace("/", ".") - hook_class_names = provider_data.get(resource_type) - if hook_class_names: - check_if_objects_belongs_to_package( - hook_class_names, provider_package, yaml_file_path, resource_type - ) - - -def check_duplicates_in_list_of_transfers(yaml_files: dict[str, dict]): - print("Checking for duplicates in list of transfers") - errors = [] - resource_type = "transfers" - for yaml_file_path, provider_data in yaml_files.items(): - resource_data = provider_data.get(resource_type, []) - - source_target_integrations = [ - (r.get("source-integration-name", ""), r.get("target-integration-name", "")) - for r in resource_data - ] - if len(source_target_integrations) != len(set(source_target_integrations)): - for integration_couple in source_target_integrations: - if source_target_integrations.count(integration_couple) > 1: - errors.append( - f"Duplicated content of \n" - f" '{resource_type}/source-integration-name/{integration_couple[0]}' " - f" '{resource_type}/target-integration-name/{integration_couple[1]}' " - f"in file: {yaml_file_path}" - ) - - -def check_invalid_integration(yaml_files: dict[str, dict]): - print("Detect unregistered integrations") - all_integration_names = set(get_all_integration_names(yaml_files)) - - for (yaml_file_path, provider_data), resource_type in product( - yaml_files.items(), ["sensors", "operators", "hooks"] - ): - resource_data = provider_data.get(resource_type, []) - current_names = {r["integration-name"] for r in resource_data} - invalid_names = current_names - all_integration_names - if invalid_names: - errors.append( - f"Incorrect content of key '{resource_type}/integration-name' in file: {yaml_file_path}. " - f"Invalid values: {invalid_names}" - ) - - for (yaml_file_path, provider_data), key in product( - yaml_files.items(), ["source-integration-name", "target-integration-name"] - ): - resource_data = provider_data.get("transfers", []) - current_names = {r[key] for r in resource_data} - invalid_names = current_names - all_integration_names - if invalid_names: - errors.append( - f"Incorrect content of key 'transfers/{key}' in file: {yaml_file_path}. " - f"Invalid values: {invalid_names}" - ) - - -def check_doc_files(yaml_files: dict[str, dict]): - print("Checking doc files") - current_doc_urls: list[str] = [] - current_logo_urls: list[str] = [] - for provider in yaml_files.values(): - if "integrations" in provider: - current_doc_urls.extend( - guide - for guides in provider["integrations"] - if "how-to-guide" in guides - for guide in guides["how-to-guide"] - ) - current_logo_urls.extend( - integration["logo"] for integration in provider["integrations"] if "logo" in integration - ) - if "transfers" in provider: - current_doc_urls.extend( - op["how-to-guide"] for op in provider["transfers"] if "how-to-guide" in op - ) - - expected_doc_urls = { - f"/docs/{f.relative_to(DOCS_DIR).as_posix()}" - for f in DOCS_DIR.glob("apache-airflow-providers-*/operators/**/*.rst") - if f.name != "index.rst" and "_partials" not in f.parts - } | { - f"/docs/{f.relative_to(DOCS_DIR).as_posix()}" - for f in DOCS_DIR.glob("apache-airflow-providers-*/operators.rst") - } - expected_logo_urls = { - f"/{f.relative_to(DOCS_DIR).as_posix()}" - for f in DOCS_DIR.glob("integration-logos/**/*") - if f.is_file() - } - - try: - print(" -- Checking document urls: expected (left), current (right)") - assert_sets_equal(set(expected_doc_urls), set(current_doc_urls)) - - print(" -- Checking logo urls: expected (left), current (right)") - assert_sets_equal(set(expected_logo_urls), set(current_logo_urls)) - except AssertionError as ex: - print(ex) - sys.exit(1) - - -def check_unique_provider_name(yaml_files: dict[str, dict]): - provider_names = [d["name"] for d in yaml_files.values()] - duplicates = {x for x in provider_names if provider_names.count(x) > 1} - if duplicates: - errors.append(f"Provider name must be unique. Duplicates: {duplicates}") - - -def check_providers_are_mentioned_in_issue_template(yaml_files: dict[str, dict]): - prefix_len = len("apache-airflow-providers-") - short_provider_names = [d["package-name"][prefix_len:] for d in yaml_files.values()] - # exclude deprecated provider that shouldn't be in issue template - deprecated_providers: list[str] = [] - for item in deprecated_providers: - short_provider_names.remove(item) - jsonpath_expr = parse('$.body[?(@.attributes.label == "Apache Airflow Provider(s)")]..options[*]') - with PROVIDER_ISSUE_TEMPLATE_PATH.open() as issue_file: - issue_template = yaml.safe_load(issue_file) - all_mentioned_providers = [match.value for match in jsonpath_expr.find(issue_template)] - try: - print( - f" -- Checking providers: present in code (left), " - f"mentioned in {PROVIDER_ISSUE_TEMPLATE_PATH} (right)" - ) - assert_sets_equal(set(short_provider_names), set(all_mentioned_providers)) - except AssertionError as ex: - print(ex) - sys.exit(1) - - -def check_providers_have_all_documentation_files(yaml_files: dict[str, dict]): - expected_files = ["commits.rst", "index.rst", "installing-providers-from-sources.rst"] - for package_info in yaml_files.values(): - package_name = package_info["package-name"] - provider_dir = DOCS_DIR.joinpath(package_name) - for file in expected_files: - if not provider_dir.joinpath(file).is_file(): - errors.append( - f"The provider {package_name} misses `{file}` in documentation. " - f"Please add the file to {provider_dir}" - ) - - -if __name__ == "__main__": - provider_files_pattern = pathlib.Path(ROOT_DIR).glob("airflow/providers/**/provider.yaml") - all_provider_files = sorted(str(path) for path in provider_files_pattern) - - if len(sys.argv) > 1: - paths = sorted(sys.argv[1:]) - else: - paths = all_provider_files - - all_parsed_yaml_files: dict[str, dict] = _load_package_data(paths) - - all_files_loaded = len(all_provider_files) == len(paths) - check_integration_duplicates(all_parsed_yaml_files) - - check_completeness_of_list_of_hooks_sensors_hooks(all_parsed_yaml_files) - check_duplicates_in_integrations_names_of_hooks_sensors_operators(all_parsed_yaml_files) - - check_completeness_of_list_of_transfers(all_parsed_yaml_files) - check_duplicates_in_list_of_transfers(all_parsed_yaml_files) - check_hook_classes(all_parsed_yaml_files) - check_unique_provider_name(all_parsed_yaml_files) - check_providers_are_mentioned_in_issue_template(all_parsed_yaml_files) - check_providers_have_all_documentation_files(all_parsed_yaml_files) - - if all_files_loaded: - # Only check those if all provider files are loaded - check_doc_files(all_parsed_yaml_files) - check_invalid_integration(all_parsed_yaml_files) - - if errors: - console = Console(width=400, color_system="standard") - console.print(f"[red]Found {len(errors)} errors in providers[/]") - for error in errors: - console.print(f"[red]Error:[/] {error}") - sys.exit(1) + sys.exit(cmd_result.returncode) diff --git a/scripts/in_container/run_provider_yaml_files_check.py b/scripts/in_container/run_provider_yaml_files_check.py new file mode 100755 index 0000000000000..cab365eb21744 --- /dev/null +++ b/scripts/in_container/run_provider_yaml_files_check.py @@ -0,0 +1,482 @@ +#!/usr/bin/env python +# 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 __future__ import annotations + +import importlib +import inspect +import json +import os +import pathlib +import platform +import sys +import textwrap +from collections import Counter +from enum import Enum +from itertools import chain, product +from typing import Any, Iterable + +import jsonschema +import yaml +from jsonpath_ng.ext import parse +from rich.console import Console +from tabulate import tabulate + +from airflow.cli.commands.info_command import Architecture + +try: + from yaml import CSafeLoader as SafeLoader +except ImportError: + from yaml import SafeLoader # type: ignore + +if __name__ != "__main__": + raise Exception( + "This file is intended to be executed as an executable program. You cannot use it as a module." + ) + +ROOT_DIR = pathlib.Path(__file__).resolve().parents[2] +DOCS_DIR = ROOT_DIR.joinpath("docs") +PROVIDER_DATA_SCHEMA_PATH = ROOT_DIR.joinpath("airflow", "provider.yaml.schema.json") +PROVIDER_ISSUE_TEMPLATE_PATH = ROOT_DIR.joinpath( + ".github", "ISSUE_TEMPLATE", "airflow_providers_bug_report.yml" +) +CORE_INTEGRATIONS = ["SQL", "Local"] + +errors = [] + +console = Console(width=400, color_system="standard") + + +def _filepath_to_module(filepath: pathlib.Path) -> str: + p = filepath.resolve().relative_to(ROOT_DIR).as_posix() + if p.endswith(".py"): + p = p[:-3] + return p.replace("/", ".") + + +def _load_schema() -> dict[str, Any]: + with PROVIDER_DATA_SCHEMA_PATH.open() as schema_file: + content = json.load(schema_file) + return content + + +def _load_package_data(package_paths: Iterable[str]): + schema = _load_schema() + result = {} + for provider_yaml_path in package_paths: + with open(provider_yaml_path) as yaml_file: + provider = yaml.load(yaml_file, SafeLoader) + rel_path = pathlib.Path(provider_yaml_path).relative_to(ROOT_DIR).as_posix() + try: + jsonschema.validate(provider, schema=schema) + except jsonschema.ValidationError: + raise Exception(f"Unable to parse: {rel_path}.") + result[rel_path] = provider + return result + + +def get_all_integration_names(yaml_files) -> list[str]: + all_integrations = [ + i["integration-name"] for f in yaml_files.values() if "integrations" in f for i in f["integrations"] + ] + all_integrations += ["Local"] + return all_integrations + + +def check_integration_duplicates(yaml_files: dict[str, dict]): + """Integration names must be globally unique.""" + print("Checking integration duplicates") + all_integrations = get_all_integration_names(yaml_files) + + duplicates = [(k, v) for (k, v) in Counter(all_integrations).items() if v > 1] + + if duplicates: + print( + "Duplicate integration names found. Integration names must be globally unique. " + "Please delete duplicates." + ) + print(tabulate(duplicates, headers=["Integration name", "Number of occurrences"])) + sys.exit(3) + + +def assert_sets_equal(set1, set2): + try: + difference1 = set1.difference(set2) + except TypeError as e: + raise AssertionError(f"invalid type when attempting set difference: {e}") + except AttributeError as e: + raise AssertionError(f"first argument does not support set difference: {e}") + + try: + difference2 = set2.difference(set1) + except TypeError as e: + raise AssertionError(f"invalid type when attempting set difference: {e}") + except AttributeError as e: + raise AssertionError(f"second argument does not support set difference: {e}") + + if not (difference1 or difference2): + return + + lines = [] + if difference1: + lines.append(" -- Items in the left set but not the right:") + for item in sorted(difference1): + lines.append(f" {item!r}") + if difference2: + lines.append(" -- Items in the right set but not the left:") + for item in sorted(difference2): + lines.append(f" {item!r}") + + standard_msg = "\n".join(lines) + raise AssertionError(standard_msg) + + +class ObjectType(Enum): + MODULE = "module" + CLASS = "class" + + +def check_if_object_exist(object_name: str, resource_type: str, yaml_file_path: str, object_type: ObjectType): + try: + if object_type == ObjectType.CLASS: + module_name, object_name = object_name.rsplit(".", maxsplit=1) + the_class = getattr(importlib.import_module(module_name), object_name) + if the_class and inspect.isclass(the_class): + return + elif object_type == ObjectType.MODULE: + module = importlib.import_module(object_name) + if inspect.ismodule(module): + return + else: + raise RuntimeError(f"Wrong enum {object_type}???") + except Exception as e: + if architecture == Architecture.ARM: + if "pymssql" in str(e) or "MySQLdb" in str(e): + console.print( + f"[yellow]The imports fail on ARM: {object_name} in {resource_type} {e}, " + f"but it is expected.[/]" + ) + return + errors.append( + f"The `{object_name}` object in {resource_type} list in {yaml_file_path} does not exist " + f"or is not a class: {e}" + ) + else: + errors.append( + f"The `{object_name}` object in {resource_type} list in {yaml_file_path} does not exist " + f"or is not a {object_type.value}." + ) + + +def check_if_objects_exist_and_belong_to_package( + object_names: set[str], + provider_package: str, + yaml_file_path: str, + resource_type: str, + object_type: ObjectType, +): + for object_name in object_names: + if os.environ.get("VERBOSE"): + console.print( + f"[bright_blue]Checking if {object_name} of {resource_type} " + f"in {yaml_file_path} is {object_type.value} and belongs to {provider_package} package" + ) + if not object_name.startswith(provider_package): + errors.append( + f"The `{object_name}` object in {resource_type} list in {yaml_file_path} does not start" + f" with the expected {provider_package}." + ) + check_if_object_exist(object_name, resource_type, yaml_file_path, object_type) + + +def parse_module_data(provider_data, resource_type, yaml_file_path): + package_dir = ROOT_DIR.joinpath(yaml_file_path).parent + provider_package = pathlib.Path(yaml_file_path).parent.as_posix().replace("/", ".") + py_files = chain( + package_dir.glob(f"**/{resource_type}/*.py"), + package_dir.glob(f"{resource_type}/*.py"), + package_dir.glob(f"**/{resource_type}/**/*.py"), + package_dir.glob(f"{resource_type}/**/*.py"), + ) + expected_modules = {_filepath_to_module(f) for f in py_files if f.name != "__init__.py"} + resource_data = provider_data.get(resource_type, []) + return expected_modules, provider_package, resource_data + + +def check_correctness_of_list_of_sensors_operators_hook_modules(yaml_files: dict[str, dict]): + print("Checking completeness of list of {sensors, hooks, operators}") + print(" -- {sensors, hooks, operators} - Expected modules (left) : Current modules (right)") + for (yaml_file_path, provider_data), resource_type in product( + yaml_files.items(), ["sensors", "operators", "hooks"] + ): + expected_modules, provider_package, resource_data = parse_module_data( + provider_data, resource_type, yaml_file_path + ) + + current_modules = {str(i) for r in resource_data for i in r.get("python-modules", [])} + check_if_objects_exist_and_belong_to_package( + current_modules, provider_package, yaml_file_path, resource_type, ObjectType.MODULE + ) + try: + assert_sets_equal(set(expected_modules), set(current_modules)) + except AssertionError as ex: + nested_error = textwrap.indent(str(ex), " ") + errors.append( + f"Incorrect content of key '{resource_type}/python-modules' " + f"in file: {yaml_file_path}\n{nested_error}" + ) + + +def check_duplicates_in_integrations_names_of_hooks_sensors_operators(yaml_files: dict[str, dict]): + print("Checking for duplicates in list of {sensors, hooks, operators}") + for (yaml_file_path, provider_data), resource_type in product( + yaml_files.items(), ["sensors", "operators", "hooks"] + ): + resource_data = provider_data.get(resource_type, []) + current_integrations = [r.get("integration-name", "") for r in resource_data] + if len(current_integrations) != len(set(current_integrations)): + for integration in current_integrations: + if current_integrations.count(integration) > 1: + errors.append( + f"Duplicated content of '{resource_type}/integration-name/{integration}' " + f"in file: {yaml_file_path}" + ) + + +def check_completeness_of_list_of_transfers(yaml_files: dict[str, dict]): + print("Checking completeness of list of transfers") + resource_type = "transfers" + + print(" -- Expected transfers modules(Left): Current transfers Modules(Right)") + for yaml_file_path, provider_data in yaml_files.items(): + expected_modules, provider_package, resource_data = parse_module_data( + provider_data, resource_type, yaml_file_path + ) + + current_modules = {r.get("python-module") for r in resource_data} + check_if_objects_exist_and_belong_to_package( + current_modules, provider_package, yaml_file_path, resource_type, ObjectType.MODULE + ) + try: + assert_sets_equal(set(expected_modules), set(current_modules)) + except AssertionError as ex: + nested_error = textwrap.indent(str(ex), " ") + errors.append( + f"Incorrect content of key '{resource_type}/python-module' " + f"in file: {yaml_file_path}\n{nested_error}" + ) + + +def check_hook_classes(yaml_files: dict[str, dict]): + print("Checking connection classes belong to package, exist and are classes") + resource_type = "hook-class-names" + for yaml_file_path, provider_data in yaml_files.items(): + provider_package = pathlib.Path(yaml_file_path).parent.as_posix().replace("/", ".") + hook_class_names = provider_data.get(resource_type) + if hook_class_names: + check_if_objects_exist_and_belong_to_package( + hook_class_names, provider_package, yaml_file_path, resource_type, ObjectType.CLASS + ) + + +def check_extra_link_classes(yaml_files: dict[str, dict]): + print("Checking extra-links belong to package, exist and are classes") + resource_type = "extra-links" + for yaml_file_path, provider_data in yaml_files.items(): + provider_package = pathlib.Path(yaml_file_path).parent.as_posix().replace("/", ".") + extra_links = provider_data.get(resource_type) + if extra_links: + check_if_objects_exist_and_belong_to_package( + extra_links, provider_package, yaml_file_path, resource_type, ObjectType.CLASS + ) + + +def check_duplicates_in_list_of_transfers(yaml_files: dict[str, dict]): + print("Checking for duplicates in list of transfers") + errors = [] + resource_type = "transfers" + for yaml_file_path, provider_data in yaml_files.items(): + resource_data = provider_data.get(resource_type, []) + + source_target_integrations = [ + (r.get("source-integration-name", ""), r.get("target-integration-name", "")) + for r in resource_data + ] + if len(source_target_integrations) != len(set(source_target_integrations)): + for integration_couple in source_target_integrations: + if source_target_integrations.count(integration_couple) > 1: + errors.append( + f"Duplicated content of \n" + f" '{resource_type}/source-integration-name/{integration_couple[0]}' " + f" '{resource_type}/target-integration-name/{integration_couple[1]}' " + f"in file: {yaml_file_path}" + ) + + +def check_invalid_integration(yaml_files: dict[str, dict]): + print("Detect unregistered integrations") + all_integration_names = set(get_all_integration_names(yaml_files)) + + for (yaml_file_path, provider_data), resource_type in product( + yaml_files.items(), ["sensors", "operators", "hooks"] + ): + resource_data = provider_data.get(resource_type, []) + current_names = {r["integration-name"] for r in resource_data} + invalid_names = current_names - all_integration_names + if invalid_names: + errors.append( + f"Incorrect content of key '{resource_type}/integration-name' in file: {yaml_file_path}. " + f"Invalid values: {invalid_names}" + ) + + for (yaml_file_path, provider_data), key in product( + yaml_files.items(), ["source-integration-name", "target-integration-name"] + ): + resource_data = provider_data.get("transfers", []) + current_names = {r[key] for r in resource_data} + invalid_names = current_names - all_integration_names + if invalid_names: + errors.append( + f"Incorrect content of key 'transfers/{key}' in file: {yaml_file_path}. " + f"Invalid values: {invalid_names}" + ) + + +def check_doc_files(yaml_files: dict[str, dict]): + print("Checking doc files") + current_doc_urls: list[str] = [] + current_logo_urls: list[str] = [] + for provider in yaml_files.values(): + if "integrations" in provider: + current_doc_urls.extend( + guide + for guides in provider["integrations"] + if "how-to-guide" in guides + for guide in guides["how-to-guide"] + ) + current_logo_urls.extend( + integration["logo"] for integration in provider["integrations"] if "logo" in integration + ) + if "transfers" in provider: + current_doc_urls.extend( + op["how-to-guide"] for op in provider["transfers"] if "how-to-guide" in op + ) + + expected_doc_urls = { + f"/docs/{f.relative_to(DOCS_DIR).as_posix()}" + for f in DOCS_DIR.glob("apache-airflow-providers-*/operators/**/*.rst") + if f.name != "index.rst" and "_partials" not in f.parts + } | { + f"/docs/{f.relative_to(DOCS_DIR).as_posix()}" + for f in DOCS_DIR.glob("apache-airflow-providers-*/operators.rst") + } + expected_logo_urls = { + f"/{f.relative_to(DOCS_DIR).as_posix()}" + for f in DOCS_DIR.glob("integration-logos/**/*") + if f.is_file() + } + + try: + print(" -- Checking document urls: expected (left), current (right)") + assert_sets_equal(set(expected_doc_urls), set(current_doc_urls)) + + print(" -- Checking logo urls: expected (left), current (right)") + assert_sets_equal(set(expected_logo_urls), set(current_logo_urls)) + except AssertionError as ex: + print(ex) + sys.exit(1) + + +def check_unique_provider_name(yaml_files: dict[str, dict]): + provider_names = [d["name"] for d in yaml_files.values()] + duplicates = {x for x in provider_names if provider_names.count(x) > 1} + if duplicates: + errors.append(f"Provider name must be unique. Duplicates: {duplicates}") + + +def check_providers_are_mentioned_in_issue_template(yaml_files: dict[str, dict]): + prefix_len = len("apache-airflow-providers-") + short_provider_names = [d["package-name"][prefix_len:] for d in yaml_files.values()] + # exclude deprecated provider that shouldn't be in issue template + deprecated_providers: list[str] = [] + for item in deprecated_providers: + short_provider_names.remove(item) + jsonpath_expr = parse('$.body[?(@.attributes.label == "Apache Airflow Provider(s)")]..options[*]') + with PROVIDER_ISSUE_TEMPLATE_PATH.open() as issue_file: + issue_template = yaml.safe_load(issue_file) + all_mentioned_providers = [match.value for match in jsonpath_expr.find(issue_template)] + try: + print( + f" -- Checking providers: present in code (left), " + f"mentioned in {PROVIDER_ISSUE_TEMPLATE_PATH} (right)" + ) + assert_sets_equal(set(short_provider_names), set(all_mentioned_providers)) + except AssertionError as ex: + print(ex) + sys.exit(1) + + +def check_providers_have_all_documentation_files(yaml_files: dict[str, dict]): + expected_files = ["commits.rst", "index.rst", "installing-providers-from-sources.rst"] + for package_info in yaml_files.values(): + package_name = package_info["package-name"] + provider_dir = DOCS_DIR.joinpath(package_name) + for file in expected_files: + if not provider_dir.joinpath(file).is_file(): + errors.append( + f"The provider {package_name} misses `{file}` in documentation. " + f"Please add the file to {provider_dir}" + ) + + +if __name__ == "__main__": + architecture = Architecture().get_current() + console.print(f"Verifying packages on {architecture} architecture. Platform: {platform.machine()}.") + provider_files_pattern = pathlib.Path(ROOT_DIR).glob("airflow/providers/**/provider.yaml") + all_provider_files = sorted(str(path) for path in provider_files_pattern) + + if len(sys.argv) > 1: + paths = sorted(sys.argv[1:]) + else: + paths = all_provider_files + + all_parsed_yaml_files: dict[str, dict] = _load_package_data(paths) + + all_files_loaded = len(all_provider_files) == len(paths) + check_integration_duplicates(all_parsed_yaml_files) + + check_duplicates_in_integrations_names_of_hooks_sensors_operators(all_parsed_yaml_files) + + check_completeness_of_list_of_transfers(all_parsed_yaml_files) + check_duplicates_in_list_of_transfers(all_parsed_yaml_files) + check_hook_classes(all_parsed_yaml_files) + check_extra_link_classes(all_parsed_yaml_files) + check_correctness_of_list_of_sensors_operators_hook_modules(all_parsed_yaml_files) + check_unique_provider_name(all_parsed_yaml_files) + check_providers_are_mentioned_in_issue_template(all_parsed_yaml_files) + check_providers_have_all_documentation_files(all_parsed_yaml_files) + + if all_files_loaded: + # Only check those if all provider files are loaded + check_doc_files(all_parsed_yaml_files) + check_invalid_integration(all_parsed_yaml_files) + + if errors: + console.print(f"[red]Found {len(errors)} errors in providers[/]") + for error in errors: + console.print(f"[red]Error:[/] {error}") + sys.exit(1) From 0d42c51172be7cbffc86319b09b49fe9fa63b2f6 Mon Sep 17 00:00:00 2001 From: Jed Cunningham <66968678+jedcunningham@users.noreply.github.com> Date: Fri, 23 Dec 2022 09:24:30 -0600 Subject: [PATCH 083/158] Add back join to zombie query that was dropped in #28198 (#28544) #28198 accidentally dropped a join in a query, leading to this: airflow/jobs/scheduler_job.py:1547 SAWarning: SELECT statement has a cartesian product between FROM element(s) "dag_run_1", "task_instance", "job" and FROM element "dag". Apply join condition(s) between each element to resolve. (cherry picked from commit a24d18a534ddbcefbcf0d8790d140ff496781f8b) --- airflow/jobs/scheduler_job.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/airflow/jobs/scheduler_job.py b/airflow/jobs/scheduler_job.py index baeabdc2ecbd0..b8b608efcd847 100644 --- a/airflow/jobs/scheduler_job.py +++ b/airflow/jobs/scheduler_job.py @@ -1525,7 +1525,8 @@ def _find_zombies(self) -> None: zombies: list[tuple[TI, str, str]] = ( session.query(TI, DM.fileloc, DM.processor_subdir) .with_hint(TI, "USE INDEX (ti_state)", dialect_name="mysql") - .join(LocalTaskJob, TaskInstance.job_id == LocalTaskJob.id) + .join(LocalTaskJob, TI.job_id == LocalTaskJob.id) + .join(DM, TI.dag_id == DM.dag_id) .filter(TI.state == TaskInstanceState.RUNNING) .filter( or_( From 5d6b44598a405a181e1a006eefa1baf337fec62e Mon Sep 17 00:00:00 2001 From: KarshVashi <41749592+KarshVashi@users.noreply.github.com> Date: Sat, 24 Dec 2022 01:03:59 +0000 Subject: [PATCH 084/158] Update pre-commit hooks (#28567) (cherry picked from commit 837e0fe2ea8859ae879d8382142c29a6416f02b9) --- .pre-commit-config.yaml | 8 ++++---- airflow/www/fab_security/manager.py | 2 +- .../airflow_breeze/commands/testing_commands.py | 2 +- .../prepare_provider_packages.py | 16 ++++++++-------- docs/exts/docs_build/docs_builder.py | 4 ++-- docs/exts/extra_files_with_substitutions.py | 2 +- docs/exts/provider_init_hack.py | 2 +- kubernetes_tests/test_base.py | 2 +- tests/jobs/test_triggerer_job.py | 2 +- 9 files changed, 20 insertions(+), 20 deletions(-) diff --git a/.pre-commit-config.yaml b/.pre-commit-config.yaml index a6ed9b1f4dd25..577f0a1ddab76 100644 --- a/.pre-commit-config.yaml +++ b/.pre-commit-config.yaml @@ -148,7 +148,7 @@ repos: \.cfg$|\.conf$|\.ini$|\.ldif$|\.properties$|\.readthedocs$|\.service$|\.tf$|Dockerfile.*$ # Keep version of black in sync wit blacken-docs and pre-commit-hook-names - repo: https://github.com/psf/black - rev: 22.3.0 + rev: 22.12.0 hooks: - id: black name: Run black (python formatter) @@ -210,7 +210,7 @@ repos: pass_filenames: true # TODO: Bump to Python 3.8 when support for Python 3.7 is dropped in Airflow. - repo: https://github.com/asottile/pyupgrade - rev: v2.32.1 + rev: v3.3.1 hooks: - id: pyupgrade name: Upgrade Python code automatically @@ -259,7 +259,7 @@ repos: ^airflow/_vendor/ additional_dependencies: ['toml'] - repo: https://github.com/asottile/yesqa - rev: v1.3.0 + rev: v1.4.0 hooks: - id: yesqa name: Remove unnecessary noqa statements @@ -268,7 +268,7 @@ repos: ^airflow/_vendor/ additional_dependencies: ['flake8>=4.0.1'] - repo: https://github.com/ikamensh/flynt - rev: '0.76' + rev: '0.77' hooks: - id: flynt name: Run flynt string format converter for Python diff --git a/airflow/www/fab_security/manager.py b/airflow/www/fab_security/manager.py index 96649b046b20a..ea8918053c00f 100644 --- a/airflow/www/fab_security/manager.py +++ b/airflow/www/fab_security/manager.py @@ -1013,7 +1013,7 @@ def _ldap_bind(ldap, con, dn: str, password: str) -> bool: @staticmethod def ldap_extract(ldap_dict: dict[str, list[bytes]], field_name: str, fallback: str) -> str: - raw_value = ldap_dict.get(field_name, [bytes()]) + raw_value = ldap_dict.get(field_name, [b""]) # decode - if empty string, default to fallback, otherwise take first element return raw_value[0].decode("utf-8") or fallback diff --git a/dev/breeze/src/airflow_breeze/commands/testing_commands.py b/dev/breeze/src/airflow_breeze/commands/testing_commands.py index 33781e33735bf..58d0b509a917f 100644 --- a/dev/breeze/src/airflow_breeze/commands/testing_commands.py +++ b/dev/breeze/src/airflow_breeze/commands/testing_commands.py @@ -181,7 +181,7 @@ def _run_test( for container_id in container_ids: dump_path = FILES_DIR / f"container_logs_{container_id}_{date_str}.log" get_console(output=output).print(f"[info]Dumping container {container_id} to {dump_path}") - with open(dump_path, "wt") as outfile: + with open(dump_path, "w") as outfile: run_command(["docker", "logs", container_id], check=False, stdout=outfile) finally: run_command( diff --git a/dev/provider_packages/prepare_provider_packages.py b/dev/provider_packages/prepare_provider_packages.py index 2ef0859c8980e..ed1afb1e8f141 100755 --- a/dev/provider_packages/prepare_provider_packages.py +++ b/dev/provider_packages/prepare_provider_packages.py @@ -1110,7 +1110,7 @@ def prepare_readme_file(context): template_name="PROVIDER_README", context=context, extension=".rst" ) readme_file_path = os.path.join(TARGET_PROVIDER_PACKAGES_PATH, "README.rst") - with open(readme_file_path, "wt") as readme_file: + with open(readme_file_path, "w") as readme_file: readme_file.write(readme_content) @@ -1182,7 +1182,7 @@ def mark_latest_changes_as_documentation_only(provider_package_id: str, latest_c "as doc-only changes!" ) with open( - os.path.join(provider_details.source_provider_package_path, ".latest-doc-only-change.txt"), "tw" + os.path.join(provider_details.source_provider_package_path, ".latest-doc-only-change.txt"), "w" ) as f: f.write(latest_change.full_hash + "\n") # exit code 66 marks doc-only change marked @@ -1311,7 +1311,7 @@ def replace_content(file_path, old_text, new_text, provider_package_id): try: if os.path.isfile(file_path): copyfile(file_path, temp_file_path) - with open(file_path, "wt") as readme_file: + with open(file_path, "w") as readme_file: readme_file.write(new_text) console.print() console.print(f"Generated {file_path} file for the {provider_package_id} provider") @@ -1401,7 +1401,7 @@ def prepare_setup_py_file(context): setup_py_content = render_template( template_name=setup_py_template_name, context=context, extension=".py", autoescape=False ) - with open(setup_py_file_path, "wt") as setup_py_file: + with open(setup_py_file_path, "w") as setup_py_file: setup_py_file.write(black_format(setup_py_content)) @@ -1415,7 +1415,7 @@ def prepare_setup_cfg_file(context): autoescape=False, keep_trailing_newline=True, ) - with open(setup_cfg_file_path, "wt") as setup_cfg_file: + with open(setup_cfg_file_path, "w") as setup_cfg_file: setup_cfg_file.write(setup_cfg_content) @@ -1434,7 +1434,7 @@ def prepare_get_provider_info_py_file(context, provider_package_id: str): autoescape=False, keep_trailing_newline=True, ) - with open(get_provider_file_path, "wt") as get_provider_file: + with open(get_provider_file_path, "w") as get_provider_file: get_provider_file.write(black_format(get_provider_content)) @@ -1447,7 +1447,7 @@ def prepare_manifest_in_file(context): autoescape=False, keep_trailing_newline=True, ) - with open(target, "wt") as fh: + with open(target, "w") as fh: fh.write(content) @@ -1840,7 +1840,7 @@ def generate_new_changelog(package_id, provider_details, changelog_path, changes console.print( f"[green]Appending the provider {package_id} changelog for `{latest_version}` version.[/]" ) - with open(changelog_path, "wt") as changelog: + with open(changelog_path, "w") as changelog: changelog.write("\n".join(new_changelog_lines)) changelog.write("\n") diff --git a/docs/exts/docs_build/docs_builder.py b/docs/exts/docs_build/docs_builder.py index d6b01d7239d0e..90baffe2ba26f 100644 --- a/docs/exts/docs_build/docs_builder.py +++ b/docs/exts/docs_build/docs_builder.py @@ -162,7 +162,7 @@ def check_spelling(self, verbose: bool) -> list[SpellingError]: " ".join(shlex.quote(c) for c in build_cmd), ) console.print(f"[info]{self.package_name:60}:[/] The output is hidden until an error occurs.") - with open(self.log_spelling_filename, "wt") as output: + with open(self.log_spelling_filename, "w") as output: completed_proc = run( build_cmd, cwd=self._src_dir, @@ -241,7 +241,7 @@ def build_sphinx_docs(self, verbose: bool) -> list[DocBuildError]: f"[info]{self.package_name:60}:[/] Running sphinx. " f"The output is hidden until an error occurs." ) - with open(self.log_build_filename, "wt") as output: + with open(self.log_build_filename, "w") as output: completed_proc = run( build_cmd, cwd=self._src_dir, diff --git a/docs/exts/extra_files_with_substitutions.py b/docs/exts/extra_files_with_substitutions.py index 5cdaadd610860..a2f0d8f9ce70c 100644 --- a/docs/exts/extra_files_with_substitutions.py +++ b/docs/exts/extra_files_with_substitutions.py @@ -38,7 +38,7 @@ def copy_docker_compose(app, exception): with open(os.path.join(app.outdir, os.path.dirname(path), os.path.basename(path))) as input_file: content = input_file.readlines() with open( - os.path.join(app.outdir, os.path.dirname(path), os.path.basename(path)), "wt" + os.path.join(app.outdir, os.path.dirname(path), os.path.basename(path)), "w" ) as output_file: for line in content: output_file.write(line.replace("|version|", app.config.version)) diff --git a/docs/exts/provider_init_hack.py b/docs/exts/provider_init_hack.py index e9ff142e827ea..be34d13b3abcb 100644 --- a/docs/exts/provider_init_hack.py +++ b/docs/exts/provider_init_hack.py @@ -37,7 +37,7 @@ def _create_init_py(app, config): del config # This file is deleted by /docs/build_docs.py. If you are not using the script, the file will be # deleted by pre-commit. - with open(PROVIDER_INIT_FILE, "wt"): + with open(PROVIDER_INIT_FILE, "w"): pass diff --git a/kubernetes_tests/test_base.py b/kubernetes_tests/test_base.py index a5a690881d5d7..0601b2ff55e3e 100644 --- a/kubernetes_tests/test_base.py +++ b/kubernetes_tests/test_base.py @@ -52,7 +52,7 @@ def _describe_resources(self, namespace: str): ci = os.environ.get("CI") if ci and ci.lower() == "true": print("The resource dump will be uploaded as artifact of the CI job") - with open(output_file_path, "wt") as output_file: + with open(output_file_path, "w") as output_file: print("=" * 80, file=output_file) print(f"Describe resources for namespace {namespace}", file=output_file) print(f"Datetime: {datetime.utcnow()}", file=output_file) diff --git a/tests/jobs/test_triggerer_job.py b/tests/jobs/test_triggerer_job.py index b84392366aa80..5fa64c9c47993 100644 --- a/tests/jobs/test_triggerer_job.py +++ b/tests/jobs/test_triggerer_job.py @@ -44,7 +44,7 @@ def __init__(self, delta, filename): self.delta = delta async def run(self): - with open(self.filename, "at") as f: + with open(self.filename, "a") as f: f.write("hi\n") async for event in super().run(): yield event From a910d39ee17a431b781539df7a2e63a5b26ea1fa Mon Sep 17 00:00:00 2001 From: Jarek Potiuk Date: Mon, 2 Jan 2023 05:58:54 +0100 Subject: [PATCH 085/158] Change Architecture and OperatingSystem classies into Enums (#28627) Since they are objects already, there is a very little overhead into making them Enums and it has the nice property of being able to add type hinting for the returned values. (cherry picked from commit 8a15557f6fe73feab0e49f97b295160820ad7cfd) --- airflow/cli/commands/info_command.py | 22 +++++++++++-------- .../run_provider_yaml_files_check.py | 2 +- 2 files changed, 14 insertions(+), 10 deletions(-) diff --git a/airflow/cli/commands/info_command.py b/airflow/cli/commands/info_command.py index a8a7c760abad6..7261dfc484156 100644 --- a/airflow/cli/commands/info_command.py +++ b/airflow/cli/commands/info_command.py @@ -23,6 +23,7 @@ import platform import subprocess import sys +from enum import Enum from urllib.parse import urlsplit, urlunsplit import httpx @@ -124,16 +125,17 @@ def process_url(self, value) -> str: return urlunsplit((url_parts.scheme, netloc, url_parts.path, url_parts.query, url_parts.fragment)) -class OperatingSystem: +class OperatingSystem(Enum): """Operating system.""" WINDOWS = "Windows" LINUX = "Linux" MACOSX = "Mac OS" CYGWIN = "Cygwin" + UNKNOWN = "Unknown" @staticmethod - def get_current() -> str | None: + def get_current() -> OperatingSystem: """Get current operating system.""" if os.name == "nt": return OperatingSystem.WINDOWS @@ -143,24 +145,26 @@ def get_current() -> str | None: return OperatingSystem.MACOSX elif "cygwin" in sys.platform: return OperatingSystem.CYGWIN - return None + return OperatingSystem.UNKNOWN -class Architecture: +class Architecture(Enum): """Compute architecture.""" X86_64 = "x86_64" X86 = "x86" PPC = "ppc" ARM = "arm" + UNKNOWN = "unknown" @staticmethod - def get_current(): + def get_current() -> Architecture: """Get architecture.""" - return _MACHINE_TO_ARCHITECTURE.get(platform.machine().lower()) + current_architecture = _MACHINE_TO_ARCHITECTURE.get(platform.machine().lower()) + return current_architecture if current_architecture else Architecture.UNKNOWN -_MACHINE_TO_ARCHITECTURE = { +_MACHINE_TO_ARCHITECTURE: dict[str, Architecture] = { "amd64": Architecture.X86_64, "x86_64": Architecture.X86_64, "i686-64": Architecture.X86_64, @@ -259,8 +263,8 @@ def _system_info(self): python_version = sys.version.replace("\n", " ") return [ - ("OS", operating_system or "NOT AVAILABLE"), - ("architecture", arch or "NOT AVAILABLE"), + ("OS", operating_system.value), + ("architecture", arch.value), ("uname", str(uname)), ("locale", str(_locale)), ("python_version", python_version), diff --git a/scripts/in_container/run_provider_yaml_files_check.py b/scripts/in_container/run_provider_yaml_files_check.py index cab365eb21744..c2cfe565aefd9 100755 --- a/scripts/in_container/run_provider_yaml_files_check.py +++ b/scripts/in_container/run_provider_yaml_files_check.py @@ -444,7 +444,7 @@ def check_providers_have_all_documentation_files(yaml_files: dict[str, dict]): if __name__ == "__main__": - architecture = Architecture().get_current() + architecture = Architecture.get_current() console.print(f"Verifying packages on {architecture} architecture. Platform: {platform.machine()}.") provider_files_pattern = pathlib.Path(ROOT_DIR).glob("airflow/providers/**/provider.yaml") all_provider_files = sorted(str(path) for path in provider_files_pattern) From c0145d08cc8f3842dbd091f687109adc5229e802 Mon Sep 17 00:00:00 2001 From: Mark H <06.swivel-robots@icloud.com> Date: Fri, 30 Dec 2022 12:05:25 -1000 Subject: [PATCH 086/158] Fix minor typo in taskflow.rst (#28656) Case change to match logging API. getlogger -> getLogger (cherry picked from commit 068886231ac0759d3ae9dd13fc2b2727d87b2f60) --- docs/apache-airflow/concepts/taskflow.rst | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/apache-airflow/concepts/taskflow.rst b/docs/apache-airflow/concepts/taskflow.rst index 7efa38b8430c7..11554dabf4bec 100644 --- a/docs/apache-airflow/concepts/taskflow.rst +++ b/docs/apache-airflow/concepts/taskflow.rst @@ -77,7 +77,7 @@ To use logging from your task functions, simply import and use Python's logging .. code-block:: python - logger = logging.getlogger("airflow.task") + logger = logging.getLogger("airflow.task") Every logging line created this way will be recorded in the task log. From 0468ee7f92e633e2e159a2cc27e97ff0e22fa77c Mon Sep 17 00:00:00 2001 From: Jarek Potiuk Date: Tue, 3 Jan 2023 10:05:09 +0100 Subject: [PATCH 087/158] Fix description of output redirection for access_log for gunicorn (#28672) As of gunicorn 19.7.0, default for access_log is stdout not stderr and our documentation has not been updated to reflect that. We are already past that (min version of gunicorn is 20.1.0, so the documentation of access-log flag of ours was wrong. Having the access_log in stdout rather than stderr also allows to redirect the access log to a separate log sink in deployments like K8S. (cherry picked from commit 675af73ceb5bc8b03d46a7cd903a73f9b8faba6f) --- airflow/cli/cli_parser.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airflow/cli/cli_parser.py b/airflow/cli/cli_parser.py index a6ec776bd2c23..33e513b586484 100644 --- a/airflow/cli/cli_parser.py +++ b/airflow/cli/cli_parser.py @@ -647,7 +647,7 @@ def string_lower_type(val): ARG_ACCESS_LOGFILE = Arg( ("-A", "--access-logfile"), default=conf.get("webserver", "ACCESS_LOGFILE"), - help="The logfile to store the webserver access log. Use '-' to print to stderr", + help="The logfile to store the webserver access log. Use '-' to print to stdout", ) ARG_ERROR_LOGFILE = Arg( ("-E", "--error-logfile"), From 6a6a258e99c432a0c2f5c941338ce060afc6e44c Mon Sep 17 00:00:00 2001 From: Max Ho Date: Tue, 3 Jan 2023 19:53:52 +0800 Subject: [PATCH 088/158] Handle ConnectionReset exception in Executor cleanup (#28685) (cherry picked from commit a3de721e2f084913e853aff39d04adc00f0b82ea) --- airflow/executors/kubernetes_executor.py | 17 ++++++++++------- 1 file changed, 10 insertions(+), 7 deletions(-) diff --git a/airflow/executors/kubernetes_executor.py b/airflow/executors/kubernetes_executor.py index 16cf1b282f19a..65e463a948350 100644 --- a/airflow/executors/kubernetes_executor.py +++ b/airflow/executors/kubernetes_executor.py @@ -843,13 +843,16 @@ def end(self) -> None: assert self.kube_scheduler self.log.info("Shutting down Kubernetes executor") - self.log.debug("Flushing task_queue...") - self._flush_task_queue() - self.log.debug("Flushing result_queue...") - self._flush_result_queue() - # Both queues should be empty... - self.task_queue.join() - self.result_queue.join() + try: + self.log.debug("Flushing task_queue...") + self._flush_task_queue() + self.log.debug("Flushing result_queue...") + self._flush_result_queue() + # Both queues should be empty... + self.task_queue.join() + self.result_queue.join() + except ConnectionResetError: + self.log.exception("Connection Reset error while flushing task_queue and result_queue.") if self.kube_scheduler: self.kube_scheduler.terminate() self._manager.shutdown() From 064c5c8f3792694cee541060d7471003d10c382c Mon Sep 17 00:00:00 2001 From: Adylzhan Khashtamov Date: Tue, 3 Jan 2023 12:11:34 +0300 Subject: [PATCH 089/158] Fixed typo (#28687) (cherry picked from commit e598a1b294956448928c82a444e081ff67c6aa47) --- docs/apache-airflow/concepts/scheduler.rst | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/apache-airflow/concepts/scheduler.rst b/docs/apache-airflow/concepts/scheduler.rst index d9ba22be825c8..10057c386de55 100644 --- a/docs/apache-airflow/concepts/scheduler.rst +++ b/docs/apache-airflow/concepts/scheduler.rst @@ -280,7 +280,7 @@ When you know what your resource usage is, the improvements that you can conside parsed continuously so optimizing that code might bring tremendous improvements, especially if you try to reach out to some external databases etc. while parsing DAGs (this should be avoided at all cost). The :ref:`best_practices/top_level_code` explains what are the best practices for writing your top-level - Python code. The :ref:`best_practices/reducing_dag_complexity` document provides some ares that you might + Python code. The :ref:`best_practices/reducing_dag_complexity` document provides some areas that you might look at when you want to reduce complexity of your code. * improve utilization of your resources. This is when you have a free capacity in your system that seems underutilized (again CPU, memory I/O, networking are the prime candidates) - you can take From d213afbd36f4ad25fd2692e0844f675127e3c385 Mon Sep 17 00:00:00 2001 From: Ephraim Anierobi Date: Wed, 4 Jan 2023 07:21:44 +0100 Subject: [PATCH 090/158] Row-lock TIs to be removed during mapped task expansion (#28689) Instead of query-update, we row lock the TI to apply the update. This protects against updating a row that has been updated by another process. (cherry picked from commit a055d8fd9b42ae662e0c696e29066926b5346f6a) --- airflow/models/abstractoperator.py | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/airflow/models/abstractoperator.py b/airflow/models/abstractoperator.py index ba0a8954ae183..d693f8bfc95cb 100644 --- a/airflow/models/abstractoperator.py +++ b/airflow/models/abstractoperator.py @@ -31,6 +31,7 @@ from airflow.utils.log.logging_mixin import LoggingMixin from airflow.utils.mixins import ResolveMixin from airflow.utils.session import NEW_SESSION, provide_session +from airflow.utils.sqlalchemy import skip_locked, with_row_locks from airflow.utils.state import State, TaskInstanceState from airflow.utils.task_group import MappedTaskGroup from airflow.utils.trigger_rule import TriggerRule @@ -548,13 +549,15 @@ def expand_mapped_task(self, run_id: str, *, session: Session) -> tuple[Sequence # Any (old) task instances with inapplicable indexes (>= the total # number we need) are set to "REMOVED". - session.query(TaskInstance).filter( + query = session.query(TaskInstance).filter( TaskInstance.dag_id == self.dag_id, TaskInstance.task_id == self.task_id, TaskInstance.run_id == run_id, TaskInstance.map_index >= total_expanded_ti_count, - ).update({TaskInstance.state: TaskInstanceState.REMOVED}) - + ) + to_update = with_row_locks(query, of=TaskInstance, session=session, **skip_locked(session=session)) + for ti in to_update: + ti.state = TaskInstanceState.REMOVED session.flush() return all_expanded_tis, total_expanded_ti_count - 1 From 2528a5bce968fc47c1b0768846068bbe4541601b Mon Sep 17 00:00:00 2001 From: Victor Chiapaikeo Date: Wed, 4 Jan 2023 06:39:53 -0500 Subject: [PATCH 091/158] Allow XComArgs for external_task_ids of ExternalTaskSensor (#28692) (cherry picked from commit 7f18fa96e434c64288d801904caf1fcde18e2cbf) --- airflow/sensors/external_task.py | 6 +- tests/sensors/test_external_task_sensor.py | 72 +++++++++++++++++++--- 2 files changed, 68 insertions(+), 10 deletions(-) diff --git a/airflow/sensors/external_task.py b/airflow/sensors/external_task.py index e9573a0671237..967bb5a276ce1 100644 --- a/airflow/sensors/external_task.py +++ b/airflow/sensors/external_task.py @@ -162,8 +162,7 @@ def __init__( f"when `external_task_id` or `external_task_ids` or `external_task_group_id` " f"is not `None`: {State.task_states}" ) - if external_task_ids and len(external_task_ids) > len(set(external_task_ids)): - raise ValueError("Duplicate task_ids passed in external_task_ids parameter") + elif not total_states <= set(State.dag_states): raise ValueError( f"Valid values for `allowed_states` and `failed_states` " @@ -196,6 +195,9 @@ def _get_dttm_filter(self, context): @provide_session def poke(self, context, session=None): + if self.external_task_ids and len(self.external_task_ids) > len(set(self.external_task_ids)): + raise ValueError("Duplicate task_ids passed in external_task_ids parameter") + dttm_filter = self._get_dttm_filter(context) serialized_dttm_filter = ",".join(dt.isoformat() for dt in dttm_filter) diff --git a/tests/sensors/test_external_task_sensor.py b/tests/sensors/test_external_task_sensor.py index 80f538e8680ec..b594210b13817 100644 --- a/tests/sensors/test_external_task_sensor.py +++ b/tests/sensors/test_external_task_sensor.py @@ -33,8 +33,10 @@ from airflow.models import DagBag, DagRun, TaskInstance from airflow.models.dag import DAG from airflow.models.serialized_dag import SerializedDagModel +from airflow.models.xcom_arg import XComArg from airflow.operators.bash import BashOperator from airflow.operators.empty import EmptyOperator +from airflow.operators.python import PythonOperator from airflow.sensors.external_task import ExternalTaskMarker, ExternalTaskSensor, ExternalTaskSensorLink from airflow.sensors.time_sensor import TimeSensor from airflow.serialization.serialized_objects import SerializedBaseOperator @@ -45,6 +47,7 @@ from airflow.utils.types import DagRunType from tests.models import TEST_DAGS_FOLDER from tests.test_utils.db import clear_db_runs +from tests.test_utils.mock_operators import MockOperator DEFAULT_DATE = datetime(2015, 1, 1) TEST_DAG_ID = "unit_test_dag" @@ -579,17 +582,70 @@ def test_external_task_sensor_error_task_id_and_task_ids(self): dag=self.dag, ) + def test_external_task_sensor_with_xcom_arg_does_not_fail_on_init(self): + self.add_time_sensor() + op1 = MockOperator(task_id="op1", dag=self.dag) + op2 = ExternalTaskSensor( + task_id="test_external_task_sensor_with_xcom_arg_does_not_fail_on_init", + external_dag_id=TEST_DAG_ID, + external_task_ids=XComArg(op1), + allowed_states=["success"], + dag=self.dag, + ) + assert isinstance(op2.external_task_ids, XComArg) + def test_catch_duplicate_task_ids(self): self.add_time_sensor() - # Test By passing same task_id multiple times + op1 = ExternalTaskSensor( + task_id="test_external_task_duplicate_task_ids", + external_dag_id=TEST_DAG_ID, + external_task_ids=[TEST_TASK_ID, TEST_TASK_ID], + allowed_states=["success"], + dag=self.dag, + ) with pytest.raises(ValueError): - ExternalTaskSensor( - task_id="test_external_task_duplicate_task_ids", - external_dag_id=TEST_DAG_ID, - external_task_ids=[TEST_TASK_ID, TEST_TASK_ID], - allowed_states=["success"], - dag=self.dag, - ) + op1.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE) + + def test_catch_duplicate_task_ids_with_xcom_arg(self): + self.add_time_sensor() + op1 = PythonOperator( + python_callable=lambda: ["dupe_value", "dupe_value"], + task_id="op1", + do_xcom_push=True, + dag=self.dag, + ) + + op2 = ExternalTaskSensor( + task_id="test_external_task_duplicate_task_ids_with_xcom_arg", + external_dag_id=TEST_DAG_ID, + external_task_ids=XComArg(op1), + allowed_states=["success"], + dag=self.dag, + ) + with pytest.raises(ValueError): + op1.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE) + op2.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE) + + def test_catch_duplicate_task_ids_with_multiple_xcom_args(self): + self.add_time_sensor() + + op1 = PythonOperator( + python_callable=lambda: "value", + task_id="op1", + do_xcom_push=True, + dag=self.dag, + ) + + op2 = ExternalTaskSensor( + task_id="test_external_task_duplicate_task_ids_with_xcom_arg", + external_dag_id=TEST_DAG_ID, + external_task_ids=[XComArg(op1), XComArg(op1)], + allowed_states=["success"], + dag=self.dag, + ) + with pytest.raises(ValueError): + op1.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE) + op2.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE) def test_catch_invalid_allowed_states(self): with pytest.raises(ValueError): From b4d65eb0ed854f808e07e6bf5a118003fc6b662c Mon Sep 17 00:00:00 2001 From: Ephraim Anierobi Date: Wed, 4 Jan 2023 21:43:20 +0100 Subject: [PATCH 092/158] Fix "airflow tasks render" cli command for mapped task instances (#28698) The fix was to use the 'template_fields' attr directly since both mapped and unmapped tasks now have that attribute. I also had to use ti.task instead of the task from dag.get_task due to this error: `AttributeError: 'DecoratedMappedOperator' object has no attribute 'templates_dict'` and I wonder if this is a bug (cherry picked from commit 1da17be37627385fed7fc06584d72e0abda6a1b5) --- airflow/cli/commands/task_command.py | 9 ++-- tests/cli/commands/test_task_command.py | 62 +++++++++++++++++++++++++ 2 files changed, 67 insertions(+), 4 deletions(-) diff --git a/airflow/cli/commands/task_command.py b/airflow/cli/commands/task_command.py index 93e7c81146bb2..2f37579c351d3 100644 --- a/airflow/cli/commands/task_command.py +++ b/airflow/cli/commands/task_command.py @@ -591,21 +591,22 @@ def task_test(args, dag=None): @cli_utils.action_cli(check_db=False) @suppress_logs_and_warning -def task_render(args): +def task_render(args, dag=None): """Renders and displays templated fields for a given task.""" - dag = get_dag(args.subdir, args.dag_id) + if not dag: + dag = get_dag(args.subdir, args.dag_id) task = dag.get_task(task_id=args.task_id) ti, _ = _get_ti( task, args.map_index, exec_date_or_run_id=args.execution_date_or_run_id, create_if_necessary="memory" ) ti.render_templates() - for attr in task.__class__.template_fields: + for attr in task.template_fields: print( textwrap.dedent( f""" # ---------------------------------------------------------- # property: {attr} # ---------------------------------------------------------- - {getattr(task, attr)} + {getattr(ti.task, attr)} """ ) ) diff --git a/tests/cli/commands/test_task_command.py b/tests/cli/commands/test_task_command.py index a062f31d4f65e..864ea10408f3c 100644 --- a/tests/cli/commands/test_task_command.py +++ b/tests/cli/commands/test_task_command.py @@ -41,6 +41,7 @@ from airflow.exceptions import AirflowException, DagRunNotFound from airflow.models import DagBag, DagRun, Pool, TaskInstance from airflow.models.serialized_dag import SerializedDagModel +from airflow.operators.bash import BashOperator from airflow.utils import timezone from airflow.utils.session import create_session from airflow.utils.state import State @@ -389,6 +390,67 @@ def test_task_render(self): assert 'echo "2016-01-01"' in output assert 'echo "2016-01-08"' in output + def test_mapped_task_render(self): + """ + tasks render should render and displays templated fields for a given mapping task + """ + with redirect_stdout(io.StringIO()) as stdout: + task_command.task_render( + self.parser.parse_args( + [ + "tasks", + "render", + "test_mapped_classic", + "consumer_literal", + "2022-01-01", + "--map-index", + "0", + ] + ) + ) + # the dag test_mapped_classic has op_args=[[1], [2], [3]], so the first mapping task should have + # op_args=[1] + output = stdout.getvalue() + assert "[1]" in output + assert "[2]" not in output + assert "[3]" not in output + assert "property: op_args" in output + + def test_mapped_task_render_with_template(self, dag_maker): + """ + tasks render should render and displays templated fields for a given mapping task + """ + with dag_maker() as dag: + templated_command = """ + {% for i in range(5) %} + echo "{{ ds }}" + echo "{{ macros.ds_add(ds, 7)}}" + {% endfor %} + """ + commands = [templated_command, "echo 1"] + + BashOperator.partial(task_id="some_command").expand(bash_command=commands) + + with redirect_stdout(io.StringIO()) as stdout: + task_command.task_render( + self.parser.parse_args( + [ + "tasks", + "render", + "test_dag", + "some_command", + "2022-01-01", + "--map-index", + "0", + ] + ), + dag=dag, + ) + + output = stdout.getvalue() + assert 'echo "2022-01-01"' in output + assert 'echo "2022-01-08"' in output + def test_cli_run_when_pickle_and_dag_cli_method_selected(self): """ tasks run should return an AirflowException when invalid pickle_id is passed From b33b87edd29f86934f335994b921d0d6ccf4df01 Mon Sep 17 00:00:00 2001 From: Charles Machalow Date: Wed, 4 Jan 2023 03:43:15 -0800 Subject: [PATCH 093/158] Fix some docs on using sensors with taskflow (#28708) Also add in testing to ensure that returning bool from taskflow sensors works as expected (cherry picked from commit 12a065a38d19f4b5698962db67f5fe9ab50d420a) --- airflow/decorators/sensor.py | 2 +- airflow/sensors/python.py | 2 +- docs/apache-airflow/concepts/taskflow.rst | 9 +++ docs/apache-airflow/tutorial/taskflow.rst | 6 +- tests/decorators/test_sensor.py | 74 +++++++++++++++++++++++ 5 files changed, 90 insertions(+), 3 deletions(-) diff --git a/airflow/decorators/sensor.py b/airflow/decorators/sensor.py index 291c41298838d..20339686201a9 100644 --- a/airflow/decorators/sensor.py +++ b/airflow/decorators/sensor.py @@ -56,7 +56,7 @@ def __init__( kwargs["task_id"] = get_unique_task_id(task_id, kwargs.get("dag"), kwargs.get("task_group")) super().__init__(**kwargs) - def poke(self, context: Context) -> PokeReturnValue: + def poke(self, context: Context) -> PokeReturnValue | bool: return self.python_callable(*self.op_args, **self.op_kwargs) diff --git a/airflow/sensors/python.py b/airflow/sensors/python.py index 374df243d7335..615e4e20eea18 100644 --- a/airflow/sensors/python.py +++ b/airflow/sensors/python.py @@ -65,7 +65,7 @@ def __init__( self.op_kwargs = op_kwargs or {} self.templates_dict = templates_dict - def poke(self, context: Context) -> PokeReturnValue: + def poke(self, context: Context) -> PokeReturnValue | bool: context_merge(context, self.op_kwargs, templates_dict=self.templates_dict) self.op_kwargs = determine_kwargs(self.python_callable, self.op_args, context) diff --git a/docs/apache-airflow/concepts/taskflow.rst b/docs/apache-airflow/concepts/taskflow.rst index 11554dabf4bec..97847455ab93a 100644 --- a/docs/apache-airflow/concepts/taskflow.rst +++ b/docs/apache-airflow/concepts/taskflow.rst @@ -182,6 +182,15 @@ for deserialization ensure that ``deserialize(data: dict, version: int)`` is spe Note: Typing of ``version`` is required and needs to be ``ClassVar[int]`` + +Sensors and the TaskFlow API +-------------------------------------- + +.. versionadded:: 2.5.0 + +For an example of writing a Sensor using the TaskFlow API, see +:ref:`Using the TaskFlow API with Sensor operators `. + History ------- diff --git a/docs/apache-airflow/tutorial/taskflow.rst b/docs/apache-airflow/tutorial/taskflow.rst index 9db581200effe..66255936f5f85 100644 --- a/docs/apache-airflow/tutorial/taskflow.rst +++ b/docs/apache-airflow/tutorial/taskflow.rst @@ -365,7 +365,11 @@ You can apply the ``@task.sensor`` decorator to convert a regular Python functio BaseSensorOperator class. The Python function implements the poke logic and returns an instance of the ``PokeReturnValue`` class as the ``poke()`` method in the BaseSensorOperator does. The ``PokeReturnValue`` is a new feature in Airflow 2.3 that allows a sensor operator to push an XCom value as described in -section "Having sensors return XOM values" of :doc:`apache-airflow-providers:howto/create-update-providers`. +section "Having sensors return XCOM values" of :doc:`apache-airflow-providers:howto/create-update-providers`. + +Alternatively in cases where the sensor doesn't need to push XCOM values: both ``poke()`` and the wrapped +function can return a boolean-like value where ``True`` designates the sensor's operation as complete and +``False`` designates the sensor's operation as incomplete. .. _taskflow/task_sensor_example: diff --git a/tests/decorators/test_sensor.py b/tests/decorators/test_sensor.py index d58fb486aad35..a6dd9106cf38d 100644 --- a/tests/decorators/test_sensor.py +++ b/tests/decorators/test_sensor.py @@ -63,6 +63,30 @@ def dummy_f(): ) assert actual_xcom_value == sensor_xcom_value + def test_basic_sensor_success_returns_bool(self, dag_maker): + @task.sensor + def sensor_f(): + return True + + @task + def dummy_f(): + pass + + with dag_maker(): + sf = sensor_f() + df = dummy_f() + sf >> df + + dr = dag_maker.create_dagrun() + sf.operator.run(start_date=dr.execution_date, end_date=dr.execution_date, ignore_ti_state=True) + tis = dr.get_task_instances() + assert len(tis) == 2 + for ti in tis: + if ti.task_id == "sensor_f": + assert ti.state == State.SUCCESS + if ti.task_id == "dummy_f": + assert ti.state == State.NONE + def test_basic_sensor_failure(self, dag_maker): @task.sensor(timeout=0) def sensor_f(): @@ -89,6 +113,32 @@ def dummy_f(): if ti.task_id == "dummy_f": assert ti.state == State.NONE + def test_basic_sensor_failure_returns_bool(self, dag_maker): + @task.sensor(timeout=0) + def sensor_f(): + return False + + @task + def dummy_f(): + pass + + with dag_maker(): + sf = sensor_f() + df = dummy_f() + sf >> df + + dr = dag_maker.create_dagrun() + with pytest.raises(AirflowSensorTimeout): + sf.operator.run(start_date=dr.execution_date, end_date=dr.execution_date, ignore_ti_state=True) + + tis = dr.get_task_instances() + assert len(tis) == 2 + for ti in tis: + if ti.task_id == "sensor_f": + assert ti.state == State.FAILED + if ti.task_id == "dummy_f": + assert ti.state == State.NONE + def test_basic_sensor_soft_fail(self, dag_maker): @task.sensor(timeout=0, soft_fail=True) def sensor_f(): @@ -113,6 +163,30 @@ def dummy_f(): if ti.task_id == "dummy_f": assert ti.state == State.NONE + def test_basic_sensor_soft_fail_returns_bool(self, dag_maker): + @task.sensor(timeout=0, soft_fail=True) + def sensor_f(): + return False + + @task + def dummy_f(): + pass + + with dag_maker(): + sf = sensor_f() + df = dummy_f() + sf >> df + + dr = dag_maker.create_dagrun() + sf.operator.run(start_date=dr.execution_date, end_date=dr.execution_date, ignore_ti_state=True) + tis = dr.get_task_instances() + assert len(tis) == 2 + for ti in tis: + if ti.task_id == "sensor_f": + assert ti.state == State.SKIPPED + if ti.task_id == "dummy_f": + assert ti.state == State.NONE + def test_basic_sensor_get_upstream_output(self, dag_maker): ret_val = 100 sensor_xcom_value = "xcom_value" From 329aad6e56335a14bc413c59eaa244f5f601dba8 Mon Sep 17 00:00:00 2001 From: Jed Cunningham <66968678+jedcunningham@users.noreply.github.com> Date: Tue, 3 Jan 2023 18:23:55 -0600 Subject: [PATCH 094/158] Add Niko to committers (#28712) (cherry picked from commit 56fb1f1b8cd73b4328df5b6fc6d232788b1f7d13) --- .github/workflows/ci.yml | 1 + docs/apache-airflow/project.rst | 1 + 2 files changed, 2 insertions(+) diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index 961fad71e5147..6814d69f2ce63 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -102,6 +102,7 @@ jobs: "milton0825", "mistercrunch", "msumit", + "o-nikolas", "pierrejeambrun", "pingzh", "potiuk", diff --git a/docs/apache-airflow/project.rst b/docs/apache-airflow/project.rst index 9c6fe09e250aa..a89779f2b5e5e 100644 --- a/docs/apache-airflow/project.rst +++ b/docs/apache-airflow/project.rst @@ -71,6 +71,7 @@ Committers - Leah Cole (@leahecole) - Malthe Borch (@malthe) - Maxime "Max" Beauchemin (@mistercrunch) +- Niko Oliveira (@o-nikolas) - Patrick Leo Tardif (@pltardif) - Pierre Jeambrun (@pierrejeambrun) - Ping Zhang (@pingzh) From 694c8c00578d3608d51a93ce429b646328c380a9 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 4 Jan 2023 09:32:52 -0500 Subject: [PATCH 095/158] Bump json5 from 1.0.1 to 1.0.2 in /airflow/www (#28715) Bumps [json5](https://github.com/json5/json5) from 1.0.1 to 1.0.2. - [Release notes](https://github.com/json5/json5/releases) - [Changelog](https://github.com/json5/json5/blob/main/CHANGELOG.md) - [Commits](https://github.com/json5/json5/compare/v1.0.1...v1.0.2) --- updated-dependencies: - dependency-name: json5 dependency-type: indirect ... Signed-off-by: dependabot[bot] Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> (cherry picked from commit e4bc5e54b1f41c991542850045bcfd060bac7395) --- airflow/www/yarn.lock | 13 ++++--------- 1 file changed, 4 insertions(+), 9 deletions(-) diff --git a/airflow/www/yarn.lock b/airflow/www/yarn.lock index 5431b00d520d5..17125472c3abb 100644 --- a/airflow/www/yarn.lock +++ b/airflow/www/yarn.lock @@ -7235,9 +7235,9 @@ json-stringify-safe@^5.0.1: integrity sha512-ZClg6AaYvamvYEE82d3Iyd3vSSIjQ+odgjaTzRuO3s7toCdFKczob2i0zCh7JE8kWn17yvAWhUVxvqGwUalsRA== json5@^1.0.1: - version "1.0.1" - resolved "https://registry.yarnpkg.com/json5/-/json5-1.0.1.tgz#779fb0018604fa854eacbf6252180d83543e3dbe" - integrity sha512-aKS4WQjPenRxiQsC93MNfjx+nbF4PAdYzmd/1JIj8HYzqfbu86beTuNgXDzPknWk0n0uARlyewZo4s++ES36Ow== + version "1.0.2" + resolved "https://registry.yarnpkg.com/json5/-/json5-1.0.2.tgz#63d98d60f21b313b77c4d6da18bfa69d80e1d593" + integrity sha512-g1MWMLBiz8FKi1e4w0UyVL3w+iJceWAFBAaBnnGKOpNa5f8TLktkbre1+s6oICydWAm+HRUGTmI+//xv2hvXYA== dependencies: minimist "^1.2.0" @@ -7684,16 +7684,11 @@ minimist-options@4.1.0: is-plain-obj "^1.1.0" kind-of "^6.0.3" -minimist@^1.2.0: +minimist@^1.2.0, minimist@^1.2.5, minimist@^1.2.6: version "1.2.7" resolved "https://registry.yarnpkg.com/minimist/-/minimist-1.2.7.tgz#daa1c4d91f507390437c6a8bc01078e7000c4d18" integrity sha512-bzfL1YUZsP41gmu/qjrEk0Q6i2ix/cVeAhbCbqH9u3zYutS1cLg00qhrD0M2MVdCcx4Sc0UpP2eBWo9rotpq6g== -minimist@^1.2.5, minimist@^1.2.6: - version "1.2.6" - resolved "https://registry.yarnpkg.com/minimist/-/minimist-1.2.6.tgz#8637a5b759ea0d6e98702cfb3a9283323c93af44" - integrity sha512-Jsjnk4bw3YJqYzbdyBiNsPWHPfO++UGG749Cxs6peCu5Xg4nrena6OVxOYxrQTqww0Jmwt+Ref8rggumkTLz9Q== - minipass-collect@^1.0.2: version "1.0.2" resolved "https://registry.yarnpkg.com/minipass-collect/-/minipass-collect-1.0.2.tgz#22b813bf745dc6edba2576b940022ad6edc8c617" From 65ea160f7026f244478457cc5bcbccf799fee885 Mon Sep 17 00:00:00 2001 From: Tzu-ping Chung Date: Wed, 4 Jan 2023 15:41:47 +0800 Subject: [PATCH 096/158] Ignore Blackification commit from Blame (#28719) (cherry picked from commit 8cb69bb05417075adebef19cd28b2409dbba3f33) --- .git-blame-ignore-revs | 1 + 1 file changed, 1 insertion(+) diff --git a/.git-blame-ignore-revs b/.git-blame-ignore-revs index 3cbfd2db16eb2..639153b3c0326 100644 --- a/.git-blame-ignore-revs +++ b/.git-blame-ignore-revs @@ -8,3 +8,4 @@ d67ac5932dabbf06ae733fc57b48491a8029b8c2 # Mass converting string literals to use double quotes. 2a34dc9e8470285b0ed2db71109ef4265e29688b bfcae349b88fd959e32bfacd027a5be976fe2132 +01a819a42daa7990c30ab9776208b3dcb9f3a28b From 42644011e23ab87c0217af172ad4212e3219776e Mon Sep 17 00:00:00 2001 From: Jarek Potiuk Date: Wed, 4 Jan 2023 13:39:50 +0100 Subject: [PATCH 097/158] Limit SQLAlchemy to below 2.0 (#28725) SQLAlchemy is about to release 2.0 version and in 1.46 version it started to warn about deprecated features that are used. This (nicely) started to fail our builds - so our canary tests caught it early and gave us a chance to prepare for the 2.0 release and limit Airflow's dependencies beforehand. This PR adds the deprecation as "known" and limits SQLAlchemy to be <2.0 (and links to appropriate issues and documentation) (cherry picked from commit 93fed0cf5eeed5dbea9f261370149206232fca98) --- scripts/in_container/verify_providers.py | 7 ++++--- setup.cfg | 6 +++++- 2 files changed, 9 insertions(+), 4 deletions(-) diff --git a/scripts/in_container/verify_providers.py b/scripts/in_container/verify_providers.py index 87d79ee6ca123..d46e32caa38ea 100755 --- a/scripts/in_container/verify_providers.py +++ b/scripts/in_container/verify_providers.py @@ -231,6 +231,7 @@ class ProviderPackageDetails(NamedTuple): "Please use `schedule` instead. ", "'urllib3.contrib.pyopenssl' module is deprecated and will be removed in a future " "release of urllib3 2.x. Read more in this issue: https://github.com/urllib3/urllib3/issues/2680", + "Deprecated API features detected! These feature(s) are not compatible with SQLAlchemy 2.0", } # The set of warning messages generated by direct importing of some deprecated modules. We should only @@ -260,7 +261,7 @@ class ProviderPackageDetails(NamedTuple): def filter_known_warnings(warn: warnings.WarningMessage) -> bool: msg_string = str(warn.message).replace("\n", " ") for message, origin in KNOWN_DEPRECATED_MESSAGES: - if msg_string == message and warn.filename.find(f"/{origin}/") != -1: + if message in msg_string and warn.filename.find(f"/{origin}/") != -1: return False return True @@ -268,7 +269,7 @@ def filter_known_warnings(warn: warnings.WarningMessage) -> bool: def filter_direct_importlib_warning(warn: warnings.WarningMessage) -> bool: msg_string = str(warn.message).replace("\n", " ") for m in KNOWN_DEPRECATED_DIRECT_IMPORTS: - if msg_string == m and warn.filename.find("/importlib/") != -1: + if m in msg_string and warn.filename.find("/importlib/") != -1: return False return True @@ -276,7 +277,7 @@ def filter_direct_importlib_warning(warn: warnings.WarningMessage) -> bool: def filter_known_common_deprecated_messages(warn: warnings.WarningMessage) -> bool: msg_string = str(warn.message).replace("\n", " ") for m in KNOWN_COMMON_DEPRECATED_MESSAGES: - if msg_string == m: + if m in msg_string: return False return True diff --git a/setup.cfg b/setup.cfg index 5cc2a1342bf63..868065c3984c4 100644 --- a/setup.cfg +++ b/setup.cfg @@ -125,7 +125,11 @@ install_requires = python-slugify>=5.0 rich>=12.4.4 setproctitle>=1.1.8 - sqlalchemy>=1.4 + # We use some deprecated features of sqlalchemy 2.0 and we should replace them before we can upgrade + # See https://sqlalche.me/e/b8d9 for details of deprecated features + # you can set environment variable SQLALCHEMY_WARN_20=1 to show all deprecation warnings. + # The issue tracking it is https://github.com/apache/airflow/issues/28723 + sqlalchemy>=1.4,<2.0 sqlalchemy_jsonfield>=1.0 tabulate>=0.7.5 tenacity>=6.2.0 From e70f640433387805ea52d32032f756d265127063 Mon Sep 17 00:00:00 2001 From: eladkal <45845474+eladkal@users.noreply.github.com> Date: Wed, 4 Jan 2023 18:05:58 +0200 Subject: [PATCH 098/158] Clarify about docker compose (#28729) We got several requests to update syntax https://github.com/apache/airflow/pull/28728 https://github.com/apache/airflow/pull/27792 https://github.com/apache/airflow/pull/28194 lets clarify that this is not a mistake (cherry picked from commit df0e4c9ad447377073af1ed60fb0dfad731be059) --- docs/apache-airflow/howto/docker-compose/index.rst | 3 +++ 1 file changed, 3 insertions(+) diff --git a/docs/apache-airflow/howto/docker-compose/index.rst b/docs/apache-airflow/howto/docker-compose/index.rst index 33e06e4b9629b..cc7a67e2973cd 100644 --- a/docs/apache-airflow/howto/docker-compose/index.rst +++ b/docs/apache-airflow/howto/docker-compose/index.rst @@ -162,6 +162,9 @@ Now you can start all services: docker compose up +.. note:: + docker-compose is old syntax. Please check `Stackoverflow `__. + In a second terminal you can check the condition of the containers and make sure that no containers are in an unhealthy condition: .. code-block:: text From 62d5e4788e1e1b0235defcb88aa689227659e37d Mon Sep 17 00:00:00 2001 From: Max Ho Date: Wed, 11 Jan 2023 07:25:29 +0800 Subject: [PATCH 099/158] Update CSRF token to expire with session (#28730) (cherry picked from commit 543e9a592e6b9dc81467c55169725e192fe95e89) --- airflow/config_templates/default_webserver_config.py | 1 + 1 file changed, 1 insertion(+) diff --git a/airflow/config_templates/default_webserver_config.py b/airflow/config_templates/default_webserver_config.py index ac999a0deafb6..aa22b125fa98c 100644 --- a/airflow/config_templates/default_webserver_config.py +++ b/airflow/config_templates/default_webserver_config.py @@ -32,6 +32,7 @@ # Flask-WTF flag for CSRF WTF_CSRF_ENABLED = True +WTF_CSRF_TIME_LIMIT = None # ---------------------------------------------------- # AUTHENTICATION CONFIG From 44dc2d815d660d984413cbcdf3a6d81ee02c8bc4 Mon Sep 17 00:00:00 2001 From: Jarek Potiuk Date: Fri, 6 Jan 2023 10:29:28 +0100 Subject: [PATCH 100/158] Clarify that versioned constraints are fixed at release time (#28762) We received a number of requests to upgrade individual dependencies in the constraint files (mostly due to those dependencies releasing version with vulnerabilities fixed). This is not how our constraint works, their main purpose is to provide "consistent installation" mechanism for anyone who installs airflow from the scratch, we are not going to keep such relased versions up-to-date with versions of dependencies released after the release. This PR provides additional explanation about that in both constraint files as well as in reference container images which follow similar patterns. (cherry picked from commit 8290ade26deba02ca6cf3d8254981b31cf89ee5b) --- .../installation/installing-from-pypi.rst | 34 +++++++++++++++++- docs/docker-stack/index.rst | 35 +++++++++++++++++++ 2 files changed, 68 insertions(+), 1 deletion(-) diff --git a/docs/apache-airflow/installation/installing-from-pypi.rst b/docs/apache-airflow/installation/installing-from-pypi.rst index 3ec492b0edb9a..e6cf162bc3017 100644 --- a/docs/apache-airflow/installation/installing-from-pypi.rst +++ b/docs/apache-airflow/installation/installing-from-pypi.rst @@ -53,7 +53,7 @@ and both at the same time. We decided to keep our dependencies as open as possib version of libraries if needed. This means that from time to time plain ``pip install apache-airflow`` will not work or will produce an unusable Airflow installation. -In order to have a repeatable installation, we also keep a set of "known-to-be-working" constraint files in the +In order to have a repeatable installation (and only for that reason), we also keep a set of "known-to-be-working" constraint files in the ``constraints-main``, ``constraints-2-0``, ``constraints-2-1`` etc. orphan branches and then we create a tag for each released version e.g. :subst-code:`constraints-|version|`. This way, we keep a tested and working set of dependencies. @@ -88,6 +88,38 @@ constraints always points to the "latest" released Airflow version constraints: https://raw.githubusercontent.com/apache/airflow/constraints-latest/constraints-3.7.txt + +Fixing Constraint files at release time +''''''''''''''''''''''''''''''''''''''' + +The released "versioned" constraints are mostly ``fixed`` when we release Airflow version and we only +update them in exceptional circumstances. For example when we find out that the released constraints might prevent +Airflow from being installed consistently from the scratch. In normal circumstances, the constraint files +are not going to change if new version of Airflow dependencies are released - not even when those +versions contain critical security fixes. The process of Airflow releases is designed around upgrading +dependencies automatically where applicable but only when we release a new version of Airflow, +not for already released versions. + +If you want to make sure that Airflow dependencies are upgraded to the latest released versions containing +latest security fixes, you should implement your own process to upgrade those yourself when +you detect the need for that. Airflow usually does not upper-bound versions of its dependencies via +requirements, so you should be able to upgrade them to the latest versions - usually without any problems. + +Obviously - since we have no control over what gets released in new versions of the dependencies, we +cannot give any guarantees that tests and functionality of those dependencies will be compatible with +Airflow after you upgrade them - testing if Airflow still works with those is in your hands, +and in case of any problems, you should raise issue with the authors of the dependencies that are problematic. +You can also - in such cases - look at the `Airflow issues `_ +`Airflow Pull Requests `_ and +`Airflow Discussions `_, searching for similar +problems to see if there are any fixes or workarounds found in the ``main`` version of Airflow and apply them +to your deployment. + +The easiest way to keep-up with the latest released dependencies is however, to upgrade to the latest released +Airflow version. Whenever we release a new version of Airflow, we upgrade all dependencies to the latest +applicable versions and test them together, so if you want to keep up with those tests - staying up-to-date +with latest version of Airflow is the easiest way to update those dependencies. + Installation and upgrade scenarios '''''''''''''''''''''''''''''''''' diff --git a/docs/docker-stack/index.rst b/docs/docker-stack/index.rst index be8877be06f88..d54c374b702a7 100644 --- a/docs/docker-stack/index.rst +++ b/docs/docker-stack/index.rst @@ -83,6 +83,41 @@ are also images published from branches but they are used mainly for development See `Airflow Git Branching `_ for details. +Fixing images at release time +============================= + +The released "versioned" reference images are mostly ``fixed`` when we release Airflow version and we only +update them in exceptional circumstances. For example when we find out that there are dependency errors +that might prevent important Airflow or embedded provider's functionalities working. In normal circumstances, +the images are not going to change after release, even if new version of Airflow dependencies are released - +not even when those versions contain critical security fixes. The process of Airflow releases is designed +around upgrading dependencies automatically where applicable but only when we release a new version of Airflow, +not for already released versions. + +If you want to make sure that Airflow dependencies are upgraded to the latest released versions containing +latest security fixes in the image you use, you should implement your own process to upgrade +those yourself when you build custom image based on the Airflow reference one. Airflow usually does not +upper-bound versions of its dependencies via requirements, so you should be able to upgrade them to the +latest versions - usually without any problems. And you can follow the process described in +:ref:`Building the image ` to do it (even in automated way). + +Obviously - since we have no control over what gets released in new versions of the dependencies, we +cannot give any guarantees that tests and functionality of those dependencies will be compatible with +Airflow after you upgrade them - testing if Airflow still works with those is in your hands, +and in case of any problems, you should raise issue with the authors of the dependencies that are problematic. +You can also - in such cases - look at the `Airflow issues `_ +`Airflow Pull Requests `_ and +`Airflow Discussions `_, searching for similar +problems to see if there are any fixes or workarounds found in the ``main`` version of Airflow and apply them +to your custom image. + +The easiest way to keep-up with the latest released dependencies is however, to upgrade to the latest released +Airflow version via switching to newly released images as base for your images, when a new version of +Airflow is released. Whenever we release a new version of Airflow, we upgrade all dependencies to the latest +applicable versions and test them together, so if you want to keep up with those tests - staying up-to-date +with latest version of Airflow is the easiest way to update those dependencies. + + Support ======= From 589c83b16c86e4346d3134994b0e1a97a814cf31 Mon Sep 17 00:00:00 2001 From: Jed Cunningham <66968678+jedcunningham@users.noreply.github.com> Date: Mon, 9 Jan 2023 18:07:56 -0600 Subject: [PATCH 101/158] Only patch single label when adopting pod (#28776) When KubernetesExecutor adopts pods, it was patching the pod with the pod it retrieved from the k8s api, while just updating a single label. Normally this works just fine, but there are cases where the pod you pull from the k8s api can't be used as-is when patching - it results in a 422 `Forbidden: pod updates may not change fields other than ...`. Instead we now just pass the single label we need to update to patch, allowing us to avoid accidentally "updating" other fields. Closes #24015 (cherry picked from commit 9922953bcd9e11a1412a3528aef938444d62f7fe) --- airflow/executors/kubernetes_executor.py | 17 +++-- tests/executors/test_kubernetes_executor.py | 80 ++++++++++++++++++--- 2 files changed, 77 insertions(+), 20 deletions(-) diff --git a/airflow/executors/kubernetes_executor.py b/airflow/executors/kubernetes_executor.py index 65e463a948350..28f720f35e1c2 100644 --- a/airflow/executors/kubernetes_executor.py +++ b/airflow/executors/kubernetes_executor.py @@ -636,7 +636,6 @@ def sync(self) -> None: ) self.fail(task[0], e) except ApiException as e: - # These codes indicate something is wrong with pod definition; otherwise we assume pod # definition is ok, and that retrying may work if e.status in (400, 422): @@ -748,27 +747,28 @@ def adopt_launched_task( assert self.scheduler_job_id self.log.info("attempting to adopt pod %s", pod.metadata.name) - pod.metadata.labels["airflow-worker"] = pod_generator.make_safe_label_value(self.scheduler_job_id) pod_id = annotations_to_key(pod.metadata.annotations) if pod_id not in pod_ids: self.log.error("attempting to adopt taskinstance which was not specified by database: %s", pod_id) return + new_worker_id_label = pod_generator.make_safe_label_value(self.scheduler_job_id) try: kube_client.patch_namespaced_pod( name=pod.metadata.name, namespace=pod.metadata.namespace, - body=PodGenerator.serialize_pod(pod), + body={"metadata": {"labels": {"airflow-worker": new_worker_id_label}}}, ) - pod_ids.pop(pod_id) - self.running.add(pod_id) except ApiException as e: self.log.info("Failed to adopt pod %s. Reason: %s", pod.metadata.name, e) + return + + del pod_ids[pod_id] + self.running.add(pod_id) def _adopt_completed_pods(self, kube_client: client.CoreV1Api) -> None: """ - - Patch completed pod so that the KubernetesJobWatcher can delete it. + Patch completed pods so that the KubernetesJobWatcher can delete them. :param kube_client: kubernetes client for speaking to kube API """ @@ -783,12 +783,11 @@ def _adopt_completed_pods(self, kube_client: client.CoreV1Api) -> None: pod_list = kube_client.list_namespaced_pod(namespace=self.kube_config.kube_namespace, **kwargs) for pod in pod_list.items: self.log.info("Attempting to adopt pod %s", pod.metadata.name) - pod.metadata.labels["airflow-worker"] = new_worker_id_label try: kube_client.patch_namespaced_pod( name=pod.metadata.name, namespace=pod.metadata.namespace, - body=PodGenerator.serialize_pod(pod), + body={"metadata": {"labels": {"airflow-worker": new_worker_id_label}}}, ) except ApiException as e: self.log.info("Failed to adopt pod %s. Reason: %s", pod.metadata.name, e) diff --git a/tests/executors/test_kubernetes_executor.py b/tests/executors/test_kubernetes_executor.py index 367f1cb2c4f4a..97619225e65ec 100644 --- a/tests/executors/test_kubernetes_executor.py +++ b/tests/executors/test_kubernetes_executor.py @@ -654,20 +654,78 @@ def test_adopt_launched_task(self, mock_kube_client): pod_ids = {ti_key: {}} executor.adopt_launched_task(mock_kube_client, pod=pod, pod_ids=pod_ids) - assert mock_kube_client.patch_namespaced_pod.call_args[1] == { - "body": { - "metadata": { - "labels": {"airflow-worker": "modified"}, - "annotations": annotations, - "name": "foo", - } - }, - "name": "foo", - "namespace": None, - } + mock_kube_client.patch_namespaced_pod.assert_called_once_with( + body={"metadata": {"labels": {"airflow-worker": "modified"}}}, + name="foo", + namespace=None, + ) assert pod_ids == {} assert executor.running == {ti_key} + @mock.patch("airflow.executors.kubernetes_executor.get_kube_client") + def test_adopt_launched_task_api_exception(self, mock_kube_client): + """We shouldn't think we are running the task if aren't able to patch the pod""" + executor = self.kubernetes_executor + executor.scheduler_job_id = "modified" + annotations = { + "dag_id": "dag", + "run_id": "run_id", + "task_id": "task", + "try_number": "1", + } + ti_key = annotations_to_key(annotations) + pod = k8s.V1Pod(metadata=k8s.V1ObjectMeta(name="foo", annotations=annotations)) + pod_ids = {ti_key: {}} + + mock_kube_client.patch_namespaced_pod.side_effect = ApiException(status=400) + executor.adopt_launched_task(mock_kube_client, pod=pod, pod_ids=pod_ids) + mock_kube_client.patch_namespaced_pod.assert_called_once_with( + body={"metadata": {"labels": {"airflow-worker": "modified"}}}, + name="foo", + namespace=None, + ) + assert pod_ids == {ti_key: {}} + assert executor.running == set() + + @mock.patch("airflow.executors.kubernetes_executor.get_kube_client") + def test_adopt_completed_pods(self, mock_kube_client): + """We should adopt all completed pods from other schedulers""" + executor = self.kubernetes_executor + executor.scheduler_job_id = "modified" + executor.kube_client = mock_kube_client + executor.kube_config.kube_namespace = "somens" + pod_names = ["one", "two"] + mock_kube_client.list_namespaced_pod.return_value.items = [ + k8s.V1Pod( + metadata=k8s.V1ObjectMeta( + name=pod_name, + labels={"airflow-worker": pod_name}, + annotations={"some_annotation": "hello"}, + namespace="somens", + ) + ) + for pod_name in pod_names + ] + + executor._adopt_completed_pods(mock_kube_client) + mock_kube_client.list_namespaced_pod.assert_called_once_with( + namespace="somens", + field_selector="status.phase=Succeeded", + label_selector="kubernetes_executor=True,airflow-worker!=modified", + ) + assert len(pod_names) == mock_kube_client.patch_namespaced_pod.call_count + mock_kube_client.patch_namespaced_pod.assert_has_calls( + [ + mock.call( + body={"metadata": {"labels": {"airflow-worker": "modified"}}}, + name=pod_name, + namespace="somens", + ) + for pod_name in pod_names + ], + any_order=True, + ) + @mock.patch("airflow.executors.kubernetes_executor.get_kube_client") def test_not_adopt_unassigned_task(self, mock_kube_client): """ From 2f0916e83e27dbe05277fd499cf84a040664af73 Mon Sep 17 00:00:00 2001 From: Victor Chiapaikeo Date: Tue, 10 Jan 2023 00:51:53 -0500 Subject: [PATCH 102/158] Fix UIAlert should_show when AUTH_ROLE_PUBLIC set (#28781) * Fix UIAlert should_show when AUTH_ROLE_PUBLIC set Co-authored-by: Tzu-ping Chung (cherry picked from commit f17e2ba48b59525655a92e04684db664a672918f) --- airflow/www/utils.py | 22 +++++++++++++++++++--- tests/test_utils/www.py | 7 +++++++ tests/www/views/conftest.py | 7 ++++++- tests/www/views/test_views_home.py | 5 +++++ 4 files changed, 37 insertions(+), 4 deletions(-) diff --git a/airflow/www/utils.py b/airflow/www/utils.py index b1985a3c8a081..5381709e35c83 100644 --- a/airflow/www/utils.py +++ b/airflow/www/utils.py @@ -55,6 +55,8 @@ from sqlalchemy.orm.query import Query from sqlalchemy.sql.operators import ColumnOperators + from airflow.www.fab_security.sqla.manager import SecurityManager + def datetime_to_string(value: DateTime | None) -> str | None: if value is None: @@ -812,10 +814,24 @@ def __init__( self.html = html self.message = Markup(message) if html else message - def should_show(self, securitymanager) -> bool: - """Determine if the user should see the message based on their role membership""" + def should_show(self, securitymanager: SecurityManager) -> bool: + """Determine if the user should see the message. + + The decision is based on the user's role. If ``AUTH_ROLE_PUBLIC`` is + set in ``webserver_config.py``, An anonymous user would have the + ``AUTH_ROLE_PUBLIC`` role. + """ if self.roles: - user_roles = {r.name for r in securitymanager.current_user.roles} + current_user = securitymanager.current_user + if current_user is not None: + user_roles = {r.name for r in securitymanager.current_user.roles} + elif "AUTH_ROLE_PUBLIC" in securitymanager.appbuilder.get_app.config: + # If the current_user is anonymous, assign AUTH_ROLE_PUBLIC role (if it exists) to them + user_roles = {securitymanager.appbuilder.get_app.config["AUTH_ROLE_PUBLIC"]} + else: + # Unable to obtain user role - default to not showing + return False + if not user_roles.intersection(set(self.roles)): return False return True diff --git a/tests/test_utils/www.py b/tests/test_utils/www.py index 8491d54094051..55699c70516ce 100644 --- a/tests/test_utils/www.py +++ b/tests/test_utils/www.py @@ -32,6 +32,13 @@ def client_with_login(app, **kwargs): return client +def client_without_login(app): + # Anonymous users can only view if AUTH_ROLE_PUBLIC is set to non-Public + app.config["AUTH_ROLE_PUBLIC"] = "Viewer" + client = app.test_client() + return client + + def check_content_in_response(text, resp, resp_code=200): resp_html = resp.data.decode("utf-8") assert resp_code == resp.status_code diff --git a/tests/www/views/conftest.py b/tests/www/views/conftest.py index 4166b1c961add..e04b5ec96ee7e 100644 --- a/tests/www/views/conftest.py +++ b/tests/www/views/conftest.py @@ -29,7 +29,7 @@ from airflow.www.app import create_app from tests.test_utils.api_connexion_utils import delete_user from tests.test_utils.decorators import dont_initialize_flask_app_submodules -from tests.test_utils.www import client_with_login +from tests.test_utils.www import client_with_login, client_without_login @pytest.fixture(autouse=True, scope="module") @@ -123,6 +123,11 @@ def user_client(app): return client_with_login(app, username="test_user", password="test_user") +@pytest.fixture() +def anonymous_client(app): + return client_without_login(app) + + class _TemplateWithContext(NamedTuple): template: jinja2.environment.Template context: dict[str, Any] diff --git a/tests/www/views/test_views_home.py b/tests/www/views/test_views_home.py index 1041a71117798..4f6e354a7bad4 100644 --- a/tests/www/views/test_views_home.py +++ b/tests/www/views/test_views_home.py @@ -203,6 +203,11 @@ def test_home_robots_header_in_response(user_client): @pytest.mark.parametrize( "client, flash_message, expected", [ + ("anonymous_client", UIAlert("hello world"), True), + ("anonymous_client", UIAlert("hello world", roles=["Viewer"]), True), + ("anonymous_client", UIAlert("hello world", roles=["User"]), False), + ("anonymous_client", UIAlert("hello world", roles=["Viewer", "User"]), True), + ("anonymous_client", UIAlert("hello world", roles=["Admin"]), False), ("user_client", UIAlert("hello world"), True), ("user_client", UIAlert("hello world", roles=["User"]), True), ("user_client", UIAlert("hello world", roles=["User", "Admin"]), True), From e659595b9f1346fd918defe7a18b5a34fe889cc8 Mon Sep 17 00:00:00 2001 From: Josh Goldman Date: Tue, 10 Jan 2023 05:24:17 -0500 Subject: [PATCH 103/158] Remove swagger-ui extra from connexion and install swagger-ui-dist via npm package (#28788) (cherry picked from commit 35ad16dc0f6b764322b1eb289709e493fbbb0ae0) --- Dockerfile.ci | 2 +- airflow/www/extensions/init_views.py | 7 +- airflow/www/package.json | 1 + airflow/www/templates/swagger-ui/index.j2 | 87 +++++++++++++++++++++++ airflow/www/webpack.config.js | 8 +++ airflow/www/yarn.lock | 5 ++ setup.cfg | 2 +- 7 files changed, 109 insertions(+), 3 deletions(-) create mode 100644 airflow/www/templates/swagger-ui/index.j2 diff --git a/Dockerfile.ci b/Dockerfile.ci index 95ab395053969..74123f9427a0a 100644 --- a/Dockerfile.ci +++ b/Dockerfile.ci @@ -1003,7 +1003,7 @@ ARG PYTHON_BASE_IMAGE ARG AIRFLOW_IMAGE_REPOSITORY="https://github.com/apache/airflow" # By increasing this number we can do force build of all dependencies -ARG DEPENDENCIES_EPOCH_NUMBER="6" +ARG DEPENDENCIES_EPOCH_NUMBER="7" # Make sure noninteractive debian install is used and language variables set ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE} \ diff --git a/airflow/www/extensions/init_views.py b/airflow/www/extensions/init_views.py index 25d5d5898c642..7db06a06c138d 100644 --- a/airflow/www/extensions/init_views.py +++ b/airflow/www/extensions/init_views.py @@ -208,7 +208,12 @@ def _handle_method_not_allowed(ex): return views.method_not_allowed(ex) spec_dir = path.join(ROOT_APP_DIR, "api_connexion", "openapi") - connexion_app = App(__name__, specification_dir=spec_dir, skip_error_handlers=True) + swagger_ui_dir = path.join(ROOT_APP_DIR, "www", "static", "dist", "swagger-ui") + options = { + "swagger_ui": conf.getboolean("webserver", "enable_swagger_ui", fallback=True), + "swagger_path": swagger_ui_dir, + } + connexion_app = App(__name__, specification_dir=spec_dir, skip_error_handlers=True, options=options) connexion_app.app = app api_bp = connexion_app.add_api( specification="v1.yaml", base_path=base_path, validate_responses=True, strict_validation=True diff --git a/airflow/www/package.json b/airflow/www/package.json index f694089ee6baa..37565483c2397 100644 --- a/airflow/www/package.json +++ b/airflow/www/package.json @@ -118,6 +118,7 @@ "react-table": "^7.8.0", "react-textarea-autosize": "^8.3.4", "redoc": "^2.0.0-rc.72", + "swagger-ui-dist": "3.52.0", "type-fest": "^2.17.0", "url-search-params-polyfill": "^8.1.0" }, diff --git a/airflow/www/templates/swagger-ui/index.j2 b/airflow/www/templates/swagger-ui/index.j2 new file mode 100644 index 0000000000000..62661a369aac6 --- /dev/null +++ b/airflow/www/templates/swagger-ui/index.j2 @@ -0,0 +1,87 @@ +{# + 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. +#} + + + + + + + {{ title | default('Swagger UI') }} + + + + + + + +
    + + + + + diff --git a/airflow/www/webpack.config.js b/airflow/www/webpack.config.js index 49b546ab40da9..9a889da6f8253 100644 --- a/airflow/www/webpack.config.js +++ b/airflow/www/webpack.config.js @@ -259,6 +259,14 @@ const config = { from: 'node_modules/jshint/dist/jshint.js', flatten: true, }, + { + from: 'templates/swagger-ui', + to: `${BUILD_DIR}/swagger-ui`, + }, + { + from: 'node_modules/swagger-ui-dist', + to: `${BUILD_DIR}/swagger-ui`, + }, ], }), new LicensePlugin({ diff --git a/airflow/www/yarn.lock b/airflow/www/yarn.lock index 17125472c3abb..f25e2647bd4c3 100644 --- a/airflow/www/yarn.lock +++ b/airflow/www/yarn.lock @@ -9880,6 +9880,11 @@ svgo@^2.7.0: picocolors "^1.0.0" stable "^0.1.8" +swagger-ui-dist@3.52.0: + version "3.52.0" + resolved "https://registry.yarnpkg.com/swagger-ui-dist/-/swagger-ui-dist-3.52.0.tgz#cb16ec6dcdf134ff47cbfe57cba7be7748429142" + integrity sha512-SGfhW8FCih00QG59PphdeAUtTNw7HS5k3iPqDZowerPw9mcbhKchUb12kbROk99c1X6RTWW1gB1kqgfnYGuCSg== + swagger2openapi@^7.0.6: version "7.0.6" resolved "https://registry.yarnpkg.com/swagger2openapi/-/swagger2openapi-7.0.6.tgz#20a2835b8edfc0f4c08036b20cb51e8f78a420bf" diff --git a/setup.cfg b/setup.cfg index 868065c3984c4..4a499dccac2c3 100644 --- a/setup.cfg +++ b/setup.cfg @@ -75,7 +75,7 @@ install_requires = # Update CustomTTYColoredFormatter to remove colorlog>=4.0.2, <5.0 configupdater>=3.1.1 - connexion[swagger-ui,flask]>=2.10.0 + connexion[flask]>=2.10.0 cron-descriptor>=1.2.24 croniter>=0.3.17 cryptography>=0.9.3 From 292592c4a94e190cf1fde2e7a72c663e8aab803c Mon Sep 17 00:00:00 2001 From: Elena Sadler <36947886+sadler-elena@users.noreply.github.com> Date: Mon, 9 Jan 2023 02:16:12 -0500 Subject: [PATCH 104/158] Update dynamic-task-mapping.rst (#28797) (cherry picked from commit 6ca67ba98ee74c1b42a93f9812ddb8a0e02c041d) --- docs/apache-airflow/concepts/dynamic-task-mapping.rst | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/apache-airflow/concepts/dynamic-task-mapping.rst b/docs/apache-airflow/concepts/dynamic-task-mapping.rst index d15c0ada77155..8f5f2ee6e2eaa 100644 --- a/docs/apache-airflow/concepts/dynamic-task-mapping.rst +++ b/docs/apache-airflow/concepts/dynamic-task-mapping.rst @@ -338,7 +338,7 @@ There are a couple of things to note: Combining upstream data (aka "zipping") ======================================= -It is also to want to combine multiple input sources into one task mapping iterable. This is generally known as "zipping" (like Python's built-in ``zip()`` function), and is also performed as pre-processing of the downstream task. +It is also common to want to combine multiple input sources into one task mapping iterable. This is generally known as "zipping" (like Python's built-in ``zip()`` function), and is also performed as pre-processing of the downstream task. This is especially useful for conditional logic in task mapping. For example, if you want to download files from S3, but rename those files, something like this would be possible: From 7b3d5917db4a482fa2928455d5f576e5ec960a8d Mon Sep 17 00:00:00 2001 From: Ephraim Anierobi Date: Mon, 9 Jan 2023 17:43:51 +0100 Subject: [PATCH 105/158] Fix masking of non-sensitive environment variables (#28802) Environment variables are hidden even when we set expose-config to non-sensitive-only. This PR changes it to work like every other source, the items are only hidden when they are sensitive (cherry picked from commit 0a8d0ab56689c341e65a36c0287c9d635bae1242) --- airflow/configuration.py | 4 ++-- tests/core/test_configuration.py | 8 +++++--- 2 files changed, 7 insertions(+), 5 deletions(-) diff --git a/airflow/configuration.py b/airflow/configuration.py index 4cea1a53f6646..ce55aa45c6075 100644 --- a/airflow/configuration.py +++ b/airflow/configuration.py @@ -1142,8 +1142,8 @@ def _include_envs( if not display_sensitive and env_var != self._env_var_name("core", "unit_test_mode"): # Don't hide cmd/secret values here if not env_var.lower().endswith("cmd") and not env_var.lower().endswith("secret"): - opt = "< hidden >" - + if (section, key) in self.sensitive_config_values: + opt = "< hidden >" elif raw: opt = opt.replace("%", "%%") if display_source: diff --git a/tests/core/test_configuration.py b/tests/core/test_configuration.py index 6cbe0e9076988..558634de1434c 100644 --- a/tests/core/test_configuration.py +++ b/tests/core/test_configuration.py @@ -65,6 +65,7 @@ def restore_env(): "os.environ", { "AIRFLOW__TESTSECTION__TESTKEY": "testvalue", + "AIRFLOW__CORE__FERNET_KEY": "testvalue", "AIRFLOW__TESTSECTION__TESTPERCENT": "with%percent", "AIRFLOW__TESTCMDENV__ITSACOMMAND_CMD": 'echo -n "OK"', "AIRFLOW__TESTCMDENV__NOTACOMMAND_CMD": 'echo -n "NOT OK"', @@ -136,15 +137,16 @@ def test_conf_as_dict(self): assert cfg_dict["core"]["percent"] == "with%inside" # test env vars - assert cfg_dict["testsection"]["testkey"] == "< hidden >" - assert cfg_dict["kubernetes_environment_variables"]["AIRFLOW__TESTSECTION__TESTKEY"] == "< hidden >" + assert cfg_dict["testsection"]["testkey"] == "testvalue" + assert cfg_dict["kubernetes_environment_variables"]["AIRFLOW__TESTSECTION__TESTKEY"] == "nested" def test_conf_as_dict_source(self): # test display_source cfg_dict = conf.as_dict(display_source=True) assert cfg_dict["core"]["load_examples"][1] == "airflow.cfg" assert cfg_dict["database"]["load_default_connections"][1] == "airflow.cfg" - assert cfg_dict["testsection"]["testkey"] == ("< hidden >", "env var") + assert cfg_dict["testsection"]["testkey"] == ("testvalue", "env var") + assert cfg_dict["core"]["fernet_key"] == ("< hidden >", "env var") def test_conf_as_dict_sensitive(self): # test display_sensitive From 116ca056c93876ebd814dd393833a668422ce350 Mon Sep 17 00:00:00 2001 From: eladkal <45845474+eladkal@users.noreply.github.com> Date: Wed, 11 Jan 2023 13:25:31 +0200 Subject: [PATCH 106/158] Update scheduler docs about low priority tasks (#28831) Gathered insights from discussion in https://github.com/apache/airflow/issues/26933 into a paragraph in scheduler docs to clarify why sometimes low priority tasks are scheduled before high priority tasks (cherry picked from commit 493b433ad57088a5f5cabc466c949445e500b4c1) --- docs/apache-airflow/concepts/scheduler.rst | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/docs/apache-airflow/concepts/scheduler.rst b/docs/apache-airflow/concepts/scheduler.rst index 10057c386de55..b07c5c16c9d22 100644 --- a/docs/apache-airflow/concepts/scheduler.rst +++ b/docs/apache-airflow/concepts/scheduler.rst @@ -59,6 +59,14 @@ In the UI, it appears as if Airflow is running your tasks a day **late** You should refer to :doc:`/dag-run` for details on scheduling a DAG. +.. note:: + The scheduler is designed for high throughput. This is an informed design decision to achieve scheduling + tasks as soon as possible. The scheduler checks how many free slots available in a pool and schedule at most that number of tasks instances in one iteration. + This means that task priority will only come in to effect when there are more scheduled tasks + waiting than the queue slots. Thus there can be cases where low priority tasks will be schedule before high priority tasks if they share the same batch. + For more read about that you can reference `this GitHub discussion `__. + + DAG File Processing ------------------- From 7e9467ff9f010277c4a0f6e0203bbdc7d0007cc1 Mon Sep 17 00:00:00 2001 From: itaymaslo <62256194+itaymaslo@users.noreply.github.com> Date: Wed, 11 Jan 2023 20:44:39 +0200 Subject: [PATCH 107/158] Fix taskflow.rst duplicated "or" (#28839) Co-authored-by: Jed Cunningham <66968678+jedcunningham@users.noreply.github.com> (cherry picked from commit 7d57f5696885eb2a4cd64d56bf79d6a8e5a5d638) --- docs/apache-airflow/tutorial/taskflow.rst | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/docs/apache-airflow/tutorial/taskflow.rst b/docs/apache-airflow/tutorial/taskflow.rst index 66255936f5f85..d859257d73e18 100644 --- a/docs/apache-airflow/tutorial/taskflow.rst +++ b/docs/apache-airflow/tutorial/taskflow.rst @@ -234,8 +234,7 @@ Using the TaskFlow API with complex/conflicting Python dependencies ------------------------------------------------------------------- If you have tasks that require complex or conflicting requirements then you will have the ability to use the -TaskFlow API with either Python virtual environment (since 2.0.2), Docker container (since version 2.2.0) or -or ExternalPythonOperator or KubernetesPodOperator (since 2.4.0). +TaskFlow API with either Python virtual environment (since 2.0.2), Docker container (since 2.2.0), ExternalPythonOperator (since 2.4.0) or KubernetesPodOperator (since 2.4.0). This functionality allows a much more comprehensive range of use-cases for the TaskFlow API, as you are not limited to the packages and system libraries of the Airflow worker. For all cases of From 92235e1727a9b7addd46b02c3f706c2888764959 Mon Sep 17 00:00:00 2001 From: Jarek Potiuk Date: Tue, 6 Dec 2022 22:14:10 +0100 Subject: [PATCH 108/158] Update config hash in Breeze's README.md during reinstalllation (#28148) Previously we updated Breeze's config hash using pre-commit whenever setup files changed. This has proven to be brittle. When you locally work and add new dependencies, breeze would keep reinstalling every time you run it locally - without the README being updated. You'd have to manually run pre-commit in order to get it regenerated. This PR adds a new flow. Whenever you automatically re-install breeze, the README.md file of the folder from which you reinstall breeze gets updated with the new hash **just** before reinstalling. This means that after installation the new hash is already present in the package, and next time you run breeze it will match the changed hash of your dependencies. The only thing left is to commit the changed README to the repo together with setup.py/cfg changes of yours. Pre-commit is still run on commit to verify that the hash of the config files is good. (cherry picked from commit 5bac5b39ffa415d535d629ddc4992337317a9c0e) --- .pre-commit-config.yaml | 2 +- .../src/airflow_breeze/utils/path_utils.py | 12 + images/breeze/output_static-checks.svg | 248 +++++++++--------- 3 files changed, 141 insertions(+), 121 deletions(-) diff --git a/.pre-commit-config.yaml b/.pre-commit-config.yaml index 577f0a1ddab76..bcef7889cf988 100644 --- a/.pre-commit-config.yaml +++ b/.pre-commit-config.yaml @@ -641,7 +641,7 @@ repos: name: Update Breeze README.md with config files hash language: python entry: ./scripts/ci/pre_commit/pre_commit_update_breeze_config_hash.py - files: ^dev/breeze/setup.*$|^dev/breeze/pyproject.toml$|^dev/breeze/README.md$ + files: dev/breeze/setup.py|dev/breeze/setup.cfg|dev/breeze/pyproject.toml|dev/breeze/README.md pass_filenames: false require_serial: true - id: check-breeze-top-dependencies-limited diff --git a/dev/breeze/src/airflow_breeze/utils/path_utils.py b/dev/breeze/src/airflow_breeze/utils/path_utils.py index 8ad789e1e2bf0..28c0f4579634e 100644 --- a/dev/breeze/src/airflow_breeze/utils/path_utils.py +++ b/dev/breeze/src/airflow_breeze/utils/path_utils.py @@ -134,6 +134,17 @@ def set_forced_answer_for_upgrade_check(): set_forced_answer("quit") +def process_breeze_readme(breeze_sources: Path, sources_hash: str): + breeze_readme = breeze_sources / "README.md" + lines = breeze_readme.read_text().splitlines(keepends=True) + result_lines = [] + for line in lines: + if line.startswith("Package config hash:"): + line = f"Package config hash: {sources_hash}\n" + result_lines.append(line) + breeze_readme.write_text("".join(result_lines)) + + def reinstall_if_setup_changed() -> bool: """ Prints warning if detected airflow sources are not the ones that Breeze was installed with. @@ -156,6 +167,7 @@ def reinstall_if_setup_changed() -> bool: if installation_sources is not None: breeze_sources = installation_sources / "dev" / "breeze" warn_dependencies_changed() + process_breeze_readme(breeze_sources, sources_hash) set_forced_answer_for_upgrade_check() reinstall_breeze(breeze_sources) set_forced_answer(None) diff --git a/images/breeze/output_static-checks.svg b/images/breeze/output_static-checks.svg index b9c725f1ad8ac..acb8c2e3195a7 100644 --- a/images/breeze/output_static-checks.svg +++ b/images/breeze/output_static-checks.svg @@ -1,4 +1,4 @@ - + - - + + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + + + + + + + - Command: static-checks + Command: static-checks - - - - - -Usage: breeze static-checks [OPTIONS] [PRECOMMIT_ARGS]... - -Run static checks. - -╭─ Pre-commit flags ───────────────────────────────────────────────────────────────────────────────────────────────────╮ ---type-tType(s) of the static checks to run (multiple can be added).                             -(all | black | blacken-docs | check-airflow-config-yaml-consistent |                     -check-airflow-provider-compatibility | check-apache-license-rat |                        -check-base-operator-partial-arguments | check-base-operator-usage |                      -check-boring-cyborg-configuration | check-breeze-top-dependencies-limited |              -check-builtin-literals | check-changelog-has-no-duplicates |                             -check-core-deprecation-classes | check-daysago-import-from-utils |                       -check-decorated-operator-implements-custom-name | check-docstring-param-types |          -check-example-dags-urls | check-executables-have-shebangs |                              -check-extra-packages-references | check-extras-order | check-for-inclusive-language |    -check-hooks-apply | check-incorrect-use-of-LoggingMixin | check-init-decorator-arguments -| check-lazy-logging | check-merge-conflict | check-newsfragments-are-valid |            -check-no-providers-in-core-examples | check-no-relative-imports |                        -check-persist-credentials-disabled-in-github-workflows |                                 -check-pre-commit-information-consistent | check-provide-create-sessions-imports |        -check-provider-yaml-valid | check-providers-init-file-missing |                          -check-providers-subpackages-init-file-exist | check-pydevd-left-in-code |                -check-revision-heads-map | check-safe-filter-usage-in-html | check-setup-order |         -check-start-date-not-used-in-defaults | check-system-tests-present |                     -check-system-tests-tocs | check-xml | codespell | compile-www-assets |                   -compile-www-assets-dev | create-missing-init-py-files-tests | debug-statements |         -detect-private-key | doctoc | end-of-file-fixer | fix-encoding-pragma | flynt | identity -| insert-license | isort | lint-chart-schema | lint-css | lint-dockerfile |              -lint-helm-chart | lint-json-schema | lint-markdown | lint-openapi | mixed-line-ending |  -pretty-format-json | pydocstyle | python-no-log-warn | pyupgrade |                       -replace-bad-characters | rst-backticks | run-flake8 | run-mypy | run-shellcheck |        -static-check-autoflake | trailing-whitespace | ts-compile-and-lint-javascript |          -update-breeze-cmd-output | update-breeze-readme-config-hash | update-er-diagram |        -update-extras | update-in-the-wild-to-be-sorted | update-inlined-dockerfile-scripts |    -update-local-yml-file | update-migration-references | update-providers-dependencies |    -update-spelling-wordlist-to-be-sorted | update-supported-versions |                      -update-vendored-in-k8s-json-schema | update-version | yamllint | yesqa)                  ---file-fList of files to run the checks on.(PATH) ---all-files-aRun checks on all files. ---show-diff-on-failure-sShow diff for files modified by the checks. ---last-commit-cRun checks for all files in last commit. Mutually exclusive with --commit-ref. ---commit-ref-rRun checks for this commit reference only (can be any git commit-ish reference).         -Mutually exclusive with --last-commit.                                                   -(TEXT)                                                                                   ---github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---verbose-vPrint verbose information about performed steps. ---dry-run-DIf dry-run is set, commands are only printed, not executed. ---help-hShow this message and exit. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + + + + +Usage: breeze static-checks [OPTIONS] [PRECOMMIT_ARGS]... + +Run static checks. + +╭─ Pre-commit flags ───────────────────────────────────────────────────────────────────────────────────────────────────╮ +--type-tType(s) of the static checks to run (multiple can be added).                             +(all | black | blacken-docs | check-airflow-config-yaml-consistent |                     +check-airflow-provider-compatibility | check-apache-license-rat |                        +check-base-operator-partial-arguments | check-base-operator-usage |                      +check-boring-cyborg-configuration | check-breeze-top-dependencies-limited |              +check-builtin-literals | check-changelog-has-no-duplicates |                             +check-core-deprecation-classes | check-daysago-import-from-utils |                       +check-decorated-operator-implements-custom-name | check-docstring-param-types |          +check-example-dags-urls | check-executables-have-shebangs |                              +check-extra-packages-references | check-extras-order | check-for-inclusive-language |    +check-hooks-apply | check-incorrect-use-of-LoggingMixin | check-init-decorator-arguments +| check-lazy-logging | check-links-to-example-dags-do-not-use-hardcoded-versions |       +check-merge-conflict | check-newsfragments-are-valid |                                   +check-no-providers-in-core-examples | check-no-relative-imports |                        +check-persist-credentials-disabled-in-github-workflows |                                 +check-pre-commit-information-consistent | check-provide-create-sessions-imports |        +check-provider-yaml-valid | check-providers-init-file-missing |                          +check-providers-subpackages-init-file-exist | check-pydevd-left-in-code |                +check-revision-heads-map | check-safe-filter-usage-in-html | check-setup-order |         +check-start-date-not-used-in-defaults | check-system-tests-present |                     +check-system-tests-tocs | check-xml | codespell | compile-www-assets |                   +compile-www-assets-dev | create-missing-init-py-files-tests | debug-statements |         +detect-private-key | doctoc | end-of-file-fixer | fix-encoding-pragma | flynt | identity +| insert-license | isort | lint-chart-schema | lint-css | lint-dockerfile |              +lint-helm-chart | lint-json-schema | lint-markdown | lint-openapi | mixed-line-ending |  +pretty-format-json | pydocstyle | python-no-log-warn | pyupgrade |                       +replace-bad-characters | rst-backticks | run-flake8 | run-mypy | run-shellcheck |        +static-check-autoflake | trailing-whitespace | ts-compile-and-lint-javascript |          +update-breeze-cmd-output | update-breeze-readme-config-hash |                            +update-common-sql-api-stubs | update-er-diagram | update-extras |                        +update-in-the-wild-to-be-sorted | update-inlined-dockerfile-scripts |                    +update-local-yml-file | update-migration-references | update-providers-dependencies |    +update-spelling-wordlist-to-be-sorted | update-supported-versions |                      +update-vendored-in-k8s-json-schema | update-version | yamllint | yesqa)                  +--file-fList of files to run the checks on.(PATH) +--all-files-aRun checks on all files. +--show-diff-on-failure-sShow diff for files modified by the checks. +--last-commit-cRun checks for all files in last commit. Mutually exclusive with --commit-ref. +--commit-ref-rRun checks for this commit reference only (can be any git commit-ish reference).         +Mutually exclusive with --last-commit.                                                   +(TEXT)                                                                                   +--github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ From 06807b27c96bd2550b4889d1649bbaa49e5d0b0e Mon Sep 17 00:00:00 2001 From: Jarek Potiuk Date: Mon, 12 Dec 2022 12:00:34 +0100 Subject: [PATCH 109/158] Add inputimeout as dependency to breeze-cmd-line pre-commit deps (#28299) (cherry picked from commit 504e2c29ef1ea070291f14d1284de403a433f157) --- .pre-commit-config.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.pre-commit-config.yaml b/.pre-commit-config.yaml index bcef7889cf988..810ac0fc1bb1e 100644 --- a/.pre-commit-config.yaml +++ b/.pre-commit-config.yaml @@ -796,7 +796,7 @@ repos: files: ^BREEZE\.rst$|^dev/breeze/.*$|^\.pre-commit-config\.yaml$ require_serial: true pass_filenames: false - additional_dependencies: ['rich>=12.4.4', 'rich-click>=1.5'] + additional_dependencies: ['rich>=12.4.4', 'rich-click>=1.5', 'inputimeout'] - id: check-example-dags-urls name: Check that example dags url include provider versions entry: ./scripts/ci/pre_commit/pre_commit_update_example_dags_paths.py From d3aeea633441081abda94720f39c08ed7eebb624 Mon Sep 17 00:00:00 2001 From: Brent Bovenzi Date: Mon, 19 Dec 2022 09:31:27 -0600 Subject: [PATCH 110/158] Show mapped task groups in graph (#28392) * Show mapped task groups in graph * generate correct group tooltip summary * fix tests (cherry picked from commit 78b72f4fa07cac009ddd6d43d54627381e3e9c21) --- airflow/utils/task_group.py | 2 ++ airflow/www/static/js/graph.js | 57 ++++++++++++++++++++++------------ tests/utils/test_task_group.py | 3 ++ 3 files changed, 42 insertions(+), 20 deletions(-) diff --git a/airflow/utils/task_group.py b/airflow/utils/task_group.py index a0cbea5176e07..c9635c40c8339 100644 --- a/airflow/utils/task_group.py +++ b/airflow/utils/task_group.py @@ -617,6 +617,7 @@ def task_group_to_dict(task_item_or_group): }, } task_group = task_item_or_group + is_mapped = isinstance(task_group, MappedTaskGroup) children = [ task_group_to_dict(child) for child in sorted(task_group.children.values(), key=lambda t: t.label) ] @@ -658,6 +659,7 @@ def task_group_to_dict(task_item_or_group): "ry": 5, "clusterLabelPos": "top", "tooltip": task_group.tooltip, + "isMapped": is_mapped, }, "children": children, } diff --git a/airflow/www/static/js/graph.js b/airflow/www/static/js/graph.js index 67370bd9c4b0d..19a7b4ec1f035 100644 --- a/airflow/www/static/js/graph.js +++ b/airflow/www/static/js/graph.js @@ -89,11 +89,19 @@ const updateNodeLabels = (node, instances) => { let haveLabelsChanged = false; let { label } = node.value; // Check if there is a count of mapped instances - if (tasks[node.id] && tasks[node.id].is_mapped) { - const count = instances[node.id] - && instances[node.id].mapped_states - ? instances[node.id].mapped_states.length - : ' '; + if ((tasks[node.id] && tasks[node.id].is_mapped) || node.value.isMapped) { + const firstChildId = node.children[0].id; + + let count = ' '; + + // get count from mapped_states or the first child's mapped_states + // TODO: update this count for when we can nest mapped tasks inside of mapped task groups + if (instances[node.id] && instances[node.id].mapped_states) { + count = instances[node.id].mapped_states.length; + } else if (firstChildId && instances[firstChildId]) { + count = instances[firstChildId].mapped_states.length; + } + if (!label.includes(`[${count}]`)) { label = `${label} [${count}]`; } @@ -184,8 +192,7 @@ function draw() { // A task node const task = tasks[nodeId]; const tryNumber = taskInstances[nodeId].try_number || 0; - let mappedStates = []; - if (task.is_mapped) mappedStates = taskInstances[nodeId].mapped_states; + const mappedStates = taskInstances[nodeId].mapped_states || []; callModal({ taskId: nodeId, @@ -195,7 +202,7 @@ function draw() { isSubDag: task.task_type === 'SubDagOperator', dagRunId, mapIndex: task.map_index, - isMapped: task.is_mapped, + isMapped: task.is_mapped || !!taskInstances[nodeId].mapped_states, mappedStates, }); } @@ -487,19 +494,29 @@ function groupTooltip(node, tis) { let minStart; let maxEnd; - getChildrenIds(node).forEach((child) => { - if (child in tis) { - const ti = tis[child]; - if (!minStart || moment(ti.start_date).isBefore(minStart)) { - minStart = moment(ti.start_date); - } - if (!maxEnd || moment(ti.end_date).isAfter(maxEnd)) { - maxEnd = moment(ti.end_date); + if (node.isMapped) { + const firstChildId = node.children[0].id; + const mappedLength = tis[firstChildId].mapped_states.length; + [...Array(mappedLength).keys()].forEach((mapIndex) => { + const groupStates = getChildrenIds(node).map((child) => tis[child].mapped_states[mapIndex]); + const overallState = priority.find((state) => groupStates.includes(state)) || 'no_status'; + if (numMap.has(overallState)) numMap.set(overallState, numMap.get(overallState) + 1); + }); + } else { + getChildrenIds(node).forEach((child) => { + if (child in tis) { + const ti = tis[child]; + if (!minStart || moment(ti.start_date).isBefore(minStart)) { + minStart = moment(ti.start_date); + } + if (!maxEnd || moment(ti.end_date).isAfter(maxEnd)) { + maxEnd = moment(ti.end_date); + } + const stateKey = ti.state == null ? 'no_status' : ti.state; + if (numMap.has(stateKey)) numMap.set(stateKey, numMap.get(stateKey) + 1); } - const stateKey = ti.state == null ? 'no_status' : ti.state; - if (numMap.has(stateKey)) numMap.set(stateKey, numMap.get(stateKey) + 1); - } - }); + }); + } const groupDuration = convertSecsToHumanReadable(moment(maxEnd).diff(minStart, 'second')); const tooltipText = node.tooltip ? `

    ${node.tooltip}

    ` : ''; diff --git a/tests/utils/test_task_group.py b/tests/utils/test_task_group.py index ea04ce97a3afe..ba6f174773f0a 100644 --- a/tests/utils/test_task_group.py +++ b/tests/utils/test_task_group.py @@ -40,6 +40,7 @@ "rx": 5, "ry": 5, "clusterLabelPos": "top", + "isMapped": False, "tooltip": "", }, "children": [ @@ -53,6 +54,7 @@ "ry": 5, "clusterLabelPos": "top", "tooltip": "", + "isMapped": False, }, "children": [ { @@ -65,6 +67,7 @@ "ry": 5, "clusterLabelPos": "top", "tooltip": "", + "isMapped": False, }, "children": [ { From 825fdf35fde28ba9f1ade249fd3ec1d48b2b8101 Mon Sep 17 00:00:00 2001 From: Ephraim Anierobi Date: Thu, 24 Nov 2022 23:17:51 +0100 Subject: [PATCH 111/158] Only get changelog for core commits (#27900) (cherry picked from commit bad875b58d7768581f97cd432295ed303d4e4c09) --- dev/airflow-github | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/dev/airflow-github b/dev/airflow-github index d0422524131e2..0e06a185734ba 100755 --- a/dev/airflow-github +++ b/dev/airflow-github @@ -300,7 +300,10 @@ def changelog(previous_version, target_version, github_token): if tickets: issue = gh_repo.get_issue(number=int(tickets[0][1:])) issue_type = get_issue_type(issue) - sections[issue_type].append(commit["subject"]) + files = files_touched(repo, commit["id"]) + if is_core_commit(files): + sections[issue_type].append(commit["subject"]) + else: sections[DEFAULT_SECTION_NAME].append(commit["subject"]) From ab37ae5686514f4b13b2d6a83d45e52d8fbbb674 Mon Sep 17 00:00:00 2001 From: Jarek Potiuk Date: Sat, 26 Nov 2022 13:46:30 +0100 Subject: [PATCH 112/158] Strip provider test details for mssql tests in CI (#27938) When we have a big number of providers changed in a PR and test are run in self-hosted environment, the file path generated for mssql data volume was too long - because it contained all providers that were changed. However this is not necessary because there is at most one Providers test type run during those tests and "Providers" is enough to guarantee uniqueness of the volume name. This PR strips provider details from the volume name. (cherry picked from commit 0d0a62de94668530ae64a2a183a8e88718d6069f) --- dev/breeze/src/airflow_breeze/params/shell_params.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/dev/breeze/src/airflow_breeze/params/shell_params.py b/dev/breeze/src/airflow_breeze/params/shell_params.py index 9e36e54345824..9a1813e780bfe 100644 --- a/dev/breeze/src/airflow_breeze/params/shell_params.py +++ b/dev/breeze/src/airflow_breeze/params/shell_params.py @@ -266,7 +266,9 @@ def command_passed(self): @property def mssql_data_volume(self) -> str: docker_filesystem = get_filesystem_type("/var/lib/docker") - volume_name = f"tmp-mssql-volume-{self.test_type}" if self.test_type else "tmp-mssql-volume" + # in case of Providers[....], only leave Providers + base_test_type = self.test_type.split("[")[0] if self.test_type else None + volume_name = f"tmp-mssql-volume-{base_test_type}" if base_test_type else "tmp-mssql-volume" if docker_filesystem == "tmpfs": return os.fspath(Path.home() / MSSQL_TMP_DIR_NAME / f"{volume_name}-{self.mssql_version}") else: From bf41ca437309ad2b8cdeea60a51b6c29ea6d3d4e Mon Sep 17 00:00:00 2001 From: Ash Berlin-Taylor Date: Mon, 28 Nov 2022 11:52:30 +0000 Subject: [PATCH 113/158] Make updating breeze deps via breeze easier. (#27958) In trying to update the deps of breeze, I made a change to `setup.cfg` and then rand `breeze static-checks update-breeze-readme-config-hash` and ran in to two problems that this fixes 1. It prevents a self-update loop, where the change to `setup.cfg` was detected, but the hash in the README hasn't been updated, so it just came around again and just tried to reinstall again and again and again. 2. This correctly sets/maintains `argv[0]` for the re-exec'd process (which is what `sys.executable` gets set to) so that when we do `assert_pre_commit_installed` and try to find the pre-commit version, we don't invoke breeze again by mistake! (cherry picked from commit 67b958c4e3bd720abac3a81cf0348427740f3627) --- dev/breeze/src/airflow_breeze/utils/path_utils.py | 10 ++++++++-- dev/breeze/src/airflow_breeze/utils/reinstall.py | 5 ++++- 2 files changed, 12 insertions(+), 3 deletions(-) diff --git a/dev/breeze/src/airflow_breeze/utils/path_utils.py b/dev/breeze/src/airflow_breeze/utils/path_utils.py index 28c0f4579634e..7138d24c36974 100644 --- a/dev/breeze/src/airflow_breeze/utils/path_utils.py +++ b/dev/breeze/src/airflow_breeze/utils/path_utils.py @@ -57,11 +57,17 @@ def in_self_upgrade() -> bool: def in_help() -> bool: - return "--help" in sys.argv + return "--help" in sys.argv or "-h" in sys.argv def skip_upgrade_check(): - return in_self_upgrade() or in_autocomplete() or in_help() or hasattr(sys, "_called_from_test") + return ( + in_self_upgrade() + or in_autocomplete() + or in_help() + or hasattr(sys, "_called_from_test") + or os.environ.get("SKIP_UPGRADE_CHECK") + ) def skip_group_output(): diff --git a/dev/breeze/src/airflow_breeze/utils/reinstall.py b/dev/breeze/src/airflow_breeze/utils/reinstall.py index 9fb59f3ae1bb5..6dd459784d42e 100644 --- a/dev/breeze/src/airflow_breeze/utils/reinstall.py +++ b/dev/breeze/src/airflow_breeze/utils/reinstall.py @@ -37,7 +37,10 @@ def reinstall_breeze(breeze_sources: Path, re_run: bool = True): get_console().print(f"\n[info]Reinstalling Breeze from {breeze_sources}\n") subprocess.check_call(["pipx", "install", "-e", str(breeze_sources), "--force"]) if re_run: - os.execl(sys.executable, "breeze", *sys.argv) + # Make sure we don't loop forever if the metadata hash hasn't been updated yet (else it is tricky to + # run pre-commit checks via breeze!) + os.environ["SKIP_UPGRADE_CHECK"] = "1" + os.execl(sys.executable, sys.executable, *sys.argv) get_console().print(f"\n[info]Breeze has been reinstalled from {breeze_sources}. Exiting now.[/]\n\n") sys.exit(0) From 007414a8f665a5e1c40772e0bba7aafea175c955 Mon Sep 17 00:00:00 2001 From: Ash Berlin-Taylor Date: Mon, 28 Nov 2022 16:27:20 +0000 Subject: [PATCH 114/158] Make breeze SVG output more stable (#27960) Rich 12.6.0 introduces a new feature that lets us control the prefix used for class names/ids in the generated SVG -- which should mean that after this change is written a single word change to help text will not cause the _entire_ SVG to be re-generated (cherry picked from commit b5084bcef9c4b0304a1643f891e1b8c80c8b0621) --- dev/breeze/README.md | 2 +- dev/breeze/setup.cfg | 2 +- .../airflow_breeze/commands/setup_commands.py | 46 +--- .../src/airflow_breeze/utils/recording.py | 5 +- images/breeze/output_build-docs.svg | 260 +++++++++--------- images/breeze/output_ci-image_pull.svg | 152 +++++----- images/breeze/output_ci-image_verify.svg | 96 +++---- images/breeze/output_ci_fix-ownership.svg | 74 ++--- images/breeze/output_ci_free-space.svg | 68 ++--- images/breeze/output_ci_get-workflow-info.svg | 72 ++--- images/breeze/output_ci_resource-check.svg | 62 ++--- images/breeze/output_ci_selective-check.svg | 108 ++++---- images/breeze/output_cleanup.svg | 80 +++--- images/breeze/output_compile-www-assets.svg | 78 +++--- images/breeze/output_exec.svg | 62 ++--- images/breeze/output_k8s.svg | 146 +++++----- images/breeze/output_k8s_build-k8s-image.svg | 132 ++++----- .../breeze/output_k8s_configure-cluster.svg | 152 +++++----- images/breeze/output_k8s_create-cluster.svg | 156 +++++------ images/breeze/output_k8s_delete-cluster.svg | 96 +++---- images/breeze/output_k8s_deploy-airflow.svg | 168 +++++------ images/breeze/output_k8s_k9s.svg | 92 +++---- images/breeze/output_k8s_logs.svg | 96 +++---- .../breeze/output_k8s_run-complete-tests.svg | 212 +++++++------- images/breeze/output_k8s_setup-env.svg | 74 ++--- images/breeze/output_k8s_shell.svg | 108 ++++---- images/breeze/output_k8s_status.svg | 100 +++---- images/breeze/output_k8s_tests.svg | 164 +++++------ images/breeze/output_k8s_upload-k8s-image.svg | 144 +++++----- images/breeze/output_prod-image_pull.svg | 152 +++++----- images/breeze/output_prod-image_verify.svg | 100 +++---- ...elease-management_generate-constraints.svg | 156 +++++------ ...ase-management_prepare-airflow-package.svg | 88 +++--- ...agement_prepare-provider-documentation.svg | 152 +++++----- ...e-management_prepare-provider-packages.svg | 152 +++++----- ...release-management_release-prod-images.svg | 124 ++++----- images/breeze/output_setup.svg | 82 +++--- images/breeze/output_setup_autocomplete.svg | 80 +++--- images/breeze/output_setup_config.svg | 108 ++++---- ...output_setup_regenerate-command-images.svg | 136 ++++----- images/breeze/output_setup_self-upgrade.svg | 66 ++--- images/breeze/output_setup_version.svg | 62 ++--- images/breeze/output_stop.svg | 74 ++--- images/breeze/output_testing.svg | 74 ++--- .../output_testing_docker-compose-tests.svg | 96 +++---- images/breeze/output_testing_helm-tests.svg | 100 +++---- images/breeze/output_testing_tests.svg | 212 +++++++------- 47 files changed, 2498 insertions(+), 2523 deletions(-) diff --git a/dev/breeze/README.md b/dev/breeze/README.md index 67ed31c85ac6a..6872f4378abaa 100644 --- a/dev/breeze/README.md +++ b/dev/breeze/README.md @@ -52,6 +52,6 @@ PLEASE DO NOT MODIFY THE HASH BELOW! IT IS AUTOMATICALLY UPDATED BY PRE-COMMIT. --------------------------------------------------------------------------------------------------------- -Package config hash: 3097f9e408686a68b2736d6c6d96866f9eb41525b83cda3b8ed29a13bd38fc8da9a6bc3b74444467054f31927107e6a640504b9e011bd1603dda5d7605a4267e +Package config hash: 99e484ad56c10cbba1755bb3a994f55d4acacc477ea73e23bbddd1e2f95f477f7fe873b0f8f20873a1b3d78e173d713660e59ab3c6f292ef836489043740c061 --------------------------------------------------------------------------------------------------------- diff --git a/dev/breeze/setup.cfg b/dev/breeze/setup.cfg index 7db0782695ebd..eaddc76e17261 100644 --- a/dev/breeze/setup.cfg +++ b/dev/breeze/setup.cfg @@ -64,7 +64,7 @@ install_requires = pytest-xdist pyyaml requests - rich>=12.4.4 + rich>=12.6.0 rich-click>=1.5 [options.packages.find] diff --git a/dev/breeze/src/airflow_breeze/commands/setup_commands.py b/dev/breeze/src/airflow_breeze/commands/setup_commands.py index 905ccf4719612..7cbe2d2804970 100644 --- a/dev/breeze/src/airflow_breeze/commands/setup_commands.py +++ b/dev/breeze/src/airflow_breeze/commands/setup_commands.py @@ -406,8 +406,6 @@ def regenerate_help_images_for_all_commands(commands: tuple[str, ...], check_onl env = os.environ.copy() env["AIRFLOW_SOURCES_ROOT"] = str(AIRFLOW_SOURCES_ROOT) env["RECORD_BREEZE_WIDTH"] = SCREENSHOT_WIDTH - env["RECORD_BREEZE_TITLE"] = "Breeze commands" - env["RECORD_BREEZE_OUTPUT_FILE"] = str(BREEZE_IMAGES_DIR / "output-commands.svg") env["TERM"] = "xterm-256color" env["PYTHONPATH"] = str(BREEZE_SOURCES_DIR) new_hash_text_dump = PREAMBLE + get_command_hash_export() @@ -450,12 +448,9 @@ def regenerate_help_images_for_all_commands(commands: tuple[str, ...], check_onl console.print(f"[bright_blue]Unchanged command: {hash_command}") regenerate_all_commands = True if regenerate_all_commands: - env = os.environ.copy() - env["AIRFLOW_SOURCES_ROOT"] = str(AIRFLOW_SOURCES_ROOT) - env["RECORD_BREEZE_WIDTH"] = SCREENSHOT_WIDTH env["RECORD_BREEZE_TITLE"] = "Breeze commands" env["RECORD_BREEZE_OUTPUT_FILE"] = str(BREEZE_IMAGES_DIR / "output-commands.svg") - env["TERM"] = "xterm-256color" + env["RECORD_BREEZE_UNIQUE_ID"] = "breeze-help" run_command( ["breeze", "--help"], env=env, @@ -463,36 +458,15 @@ def regenerate_help_images_for_all_commands(commands: tuple[str, ...], check_onl for command in commands_list: if command == "main": continue - if ":" not in command: - env = os.environ.copy() - env["AIRFLOW_SOURCES_ROOT"] = str(AIRFLOW_SOURCES_ROOT) - env["RECORD_BREEZE_WIDTH"] = SCREENSHOT_WIDTH - env["RECORD_BREEZE_TITLE"] = f"Command: {command}" - env["RECORD_BREEZE_OUTPUT_FILE"] = str(BREEZE_IMAGES_DIR / f"output_{command}.svg") - env["TERM"] = "xterm-256color" - run_command( - ["breeze", command, "--help"], - env=env, - ) - else: - split_command = command.split(":") - env = os.environ.copy() - env["AIRFLOW_SOURCES_ROOT"] = str(AIRFLOW_SOURCES_ROOT) - env["RECORD_BREEZE_WIDTH"] = SCREENSHOT_WIDTH - env["RECORD_BREEZE_TITLE"] = f"Command: {split_command[0]} {split_command[1]}" - env["RECORD_BREEZE_OUTPUT_FILE"] = str( - BREEZE_IMAGES_DIR / f"output_{split_command[0]}_{split_command[1]}.svg" - ) - env["TERM"] = "xterm-256color" - run_command( - [ - "breeze", - split_command[0], - split_command[1], - "--help", - ], - env=env, - ) + + subcommands = command.split(":") + env["RECORD_BREEZE_TITLE"] = f"Command: {' '.join(subcommands)}" + env["RECORD_BREEZE_OUTPUT_FILE"] = str(BREEZE_IMAGES_DIR / f"output_{'_'.join(subcommands)}.svg") + env["RECORD_BREEZE_UNIQUE_ID"] = f"breeze-{'-'.join(subcommands)}" + run_command( + ["breeze", *subcommands, "--help"], + env=env, + ) if regenerate_all_commands: COMMAND_HASH_FILE_PATH.write_text(new_hash_text_dump) get_console().print(f"\n[info]New hash of breeze commands written in {COMMAND_HASH_FILE_PATH}\n") diff --git a/dev/breeze/src/airflow_breeze/utils/recording.py b/dev/breeze/src/airflow_breeze/utils/recording.py index f53ef8ce92922..7125b3d7252c5 100644 --- a/dev/breeze/src/airflow_breeze/utils/recording.py +++ b/dev/breeze/src/airflow_breeze/utils/recording.py @@ -34,7 +34,7 @@ def generating_command_images() -> bool: return "RECORD_BREEZE_TITLE" in os.environ or "regenerate-command-images" in sys.argv -def enable_recording_of_help_output(path: str, title: str | None, width: str | None): +def enable_recording_of_help_output(path: str, title: str | None, width: str | None, unique_id: str | None): import rich_click as click if not title: @@ -46,7 +46,7 @@ def enable_recording_of_help_output(path: str, title: str | None, width: str | N def save_ouput_as_svg(): if help_console: - help_console.save_svg(path=path, title=title) + help_console.save_svg(path=path, title=title, unique_id=unique_id) class RecordingConsole(rich.console.Console): def __init__(self, **kwargs): @@ -72,6 +72,7 @@ def __init__(self, **kwargs): path=output_file, title=os.environ.get("RECORD_BREEZE_TITLE"), width=os.environ.get("RECORD_BREEZE_WIDTH"), + unique_id=os.environ.get("RECORD_BREEZE_UNIQUE_ID"), ) else: try: diff --git a/images/breeze/output_build-docs.svg b/images/breeze/output_build-docs.svg index 2f4c6c07c8d21..693781f961f9c 100644 --- a/images/breeze/output_build-docs.svg +++ b/images/breeze/output_build-docs.svg @@ -19,277 +19,277 @@ font-weight: 700; } - .terminal-2641230012-matrix { + .breeze-build-docs-matrix { font-family: Fira Code, monospace; font-size: 20px; line-height: 24.4px; font-variant-east-asian: full-width; } - .terminal-2641230012-title { + .breeze-build-docs-title { font-size: 18px; font-weight: bold; font-family: arial; } - .terminal-2641230012-r1 { fill: #c5c8c6;font-weight: bold } -.terminal-2641230012-r2 { fill: #c5c8c6 } -.terminal-2641230012-r3 { fill: #d0b344;font-weight: bold } -.terminal-2641230012-r4 { fill: #68a0b3;font-weight: bold } -.terminal-2641230012-r5 { fill: #868887 } -.terminal-2641230012-r6 { fill: #98a84b;font-weight: bold } -.terminal-2641230012-r7 { fill: #8d7b39 } + .breeze-build-docs-r1 { fill: #c5c8c6;font-weight: bold } +.breeze-build-docs-r2 { fill: #c5c8c6 } +.breeze-build-docs-r3 { fill: #d0b344;font-weight: bold } +.breeze-build-docs-r4 { fill: #68a0b3;font-weight: bold } +.breeze-build-docs-r5 { fill: #868887 } +.breeze-build-docs-r6 { fill: #98a84b;font-weight: bold } +.breeze-build-docs-r7 { fill: #8d7b39 } - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - Command: build-docs + Command: build-docs - + - - -Usage: breeze build-docs [OPTIONS] - -Build documentation in the container. - -╭─ Doc flags ──────────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---docs-only-dOnly build documentation. ---spellcheck-only-sOnly run spell checking. ---clean-buildClean inventories of Inter-Sphinx documentation and generated APIs and sphinx artifacts     -before the build - useful for a clean build.                                                ---for-productionBuilds documentation for official release i.e. all links point to stable version. Implies   ---clean-build ---package-filterList of packages to consider.                                                               -(apache-airflow | apache-airflow-providers-airbyte | apache-airflow-providers-alibaba |     -apache-airflow-providers-amazon | apache-airflow-providers-apache-beam |                    -apache-airflow-providers-apache-cassandra | apache-airflow-providers-apache-drill |         -apache-airflow-providers-apache-druid | apache-airflow-providers-apache-hdfs |              -apache-airflow-providers-apache-hive | apache-airflow-providers-apache-kylin |              -apache-airflow-providers-apache-livy | apache-airflow-providers-apache-pig |                -apache-airflow-providers-apache-pinot | apache-airflow-providers-apache-spark |             -apache-airflow-providers-apache-sqoop | apache-airflow-providers-arangodb |                 -apache-airflow-providers-asana | apache-airflow-providers-atlassian-jira |                  -apache-airflow-providers-celery | apache-airflow-providers-cloudant |                       -apache-airflow-providers-cncf-kubernetes | apache-airflow-providers-common-sql |            -apache-airflow-providers-databricks | apache-airflow-providers-datadog |                    -apache-airflow-providers-dbt-cloud | apache-airflow-providers-dingding |                    -apache-airflow-providers-discord | apache-airflow-providers-docker |                        -apache-airflow-providers-elasticsearch | apache-airflow-providers-exasol |                  -apache-airflow-providers-facebook | apache-airflow-providers-ftp |                          -apache-airflow-providers-github | apache-airflow-providers-google |                         -apache-airflow-providers-grpc | apache-airflow-providers-hashicorp |                        -apache-airflow-providers-http | apache-airflow-providers-imap |                             -apache-airflow-providers-influxdb | apache-airflow-providers-jdbc |                         -apache-airflow-providers-jenkins | apache-airflow-providers-microsoft-azure |               -apache-airflow-providers-microsoft-mssql | apache-airflow-providers-microsoft-psrp |        -apache-airflow-providers-microsoft-winrm | apache-airflow-providers-mongo |                 -apache-airflow-providers-mysql | apache-airflow-providers-neo4j |                           -apache-airflow-providers-odbc | apache-airflow-providers-openfaas |                         -apache-airflow-providers-opsgenie | apache-airflow-providers-oracle |                       -apache-airflow-providers-pagerduty | apache-airflow-providers-papermill |                   -apache-airflow-providers-plexus | apache-airflow-providers-postgres |                       -apache-airflow-providers-presto | apache-airflow-providers-qubole |                         -apache-airflow-providers-redis | apache-airflow-providers-salesforce |                      -apache-airflow-providers-samba | apache-airflow-providers-segment |                         -apache-airflow-providers-sendgrid | apache-airflow-providers-sftp |                         -apache-airflow-providers-singularity | apache-airflow-providers-slack |                     -apache-airflow-providers-snowflake | apache-airflow-providers-sqlite |                      -apache-airflow-providers-ssh | apache-airflow-providers-tableau |                           -apache-airflow-providers-tabular | apache-airflow-providers-telegram |                      -apache-airflow-providers-trino | apache-airflow-providers-vertica |                         -apache-airflow-providers-yandex | apache-airflow-providers-zendesk | docker-stack |         -helm-chart)                                                                                 ---github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---verbose-vPrint verbose information about performed steps. ---dry-run-DIf dry-run is set, commands are only printed, not executed. ---help-hShow this message and exit. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + +Usage: breeze build-docs [OPTIONS] + +Build documentation in the container. + +╭─ Doc flags ──────────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--docs-only-dOnly build documentation. +--spellcheck-only-sOnly run spell checking. +--clean-buildClean inventories of Inter-Sphinx documentation and generated APIs and sphinx artifacts     +before the build - useful for a clean build.                                                +--for-productionBuilds documentation for official release i.e. all links point to stable version. Implies   +--clean-build +--package-filterList of packages to consider.                                                               +(apache-airflow | apache-airflow-providers-airbyte | apache-airflow-providers-alibaba |     +apache-airflow-providers-amazon | apache-airflow-providers-apache-beam |                    +apache-airflow-providers-apache-cassandra | apache-airflow-providers-apache-drill |         +apache-airflow-providers-apache-druid | apache-airflow-providers-apache-hdfs |              +apache-airflow-providers-apache-hive | apache-airflow-providers-apache-kylin |              +apache-airflow-providers-apache-livy | apache-airflow-providers-apache-pig |                +apache-airflow-providers-apache-pinot | apache-airflow-providers-apache-spark |             +apache-airflow-providers-apache-sqoop | apache-airflow-providers-arangodb |                 +apache-airflow-providers-asana | apache-airflow-providers-atlassian-jira |                  +apache-airflow-providers-celery | apache-airflow-providers-cloudant |                       +apache-airflow-providers-cncf-kubernetes | apache-airflow-providers-common-sql |            +apache-airflow-providers-databricks | apache-airflow-providers-datadog |                    +apache-airflow-providers-dbt-cloud | apache-airflow-providers-dingding |                    +apache-airflow-providers-discord | apache-airflow-providers-docker |                        +apache-airflow-providers-elasticsearch | apache-airflow-providers-exasol |                  +apache-airflow-providers-facebook | apache-airflow-providers-ftp |                          +apache-airflow-providers-github | apache-airflow-providers-google |                         +apache-airflow-providers-grpc | apache-airflow-providers-hashicorp |                        +apache-airflow-providers-http | apache-airflow-providers-imap |                             +apache-airflow-providers-influxdb | apache-airflow-providers-jdbc |                         +apache-airflow-providers-jenkins | apache-airflow-providers-microsoft-azure |               +apache-airflow-providers-microsoft-mssql | apache-airflow-providers-microsoft-psrp |        +apache-airflow-providers-microsoft-winrm | apache-airflow-providers-mongo |                 +apache-airflow-providers-mysql | apache-airflow-providers-neo4j |                           +apache-airflow-providers-odbc | apache-airflow-providers-openfaas |                         +apache-airflow-providers-opsgenie | apache-airflow-providers-oracle |                       +apache-airflow-providers-pagerduty | apache-airflow-providers-papermill |                   +apache-airflow-providers-plexus | apache-airflow-providers-postgres |                       +apache-airflow-providers-presto | apache-airflow-providers-qubole |                         +apache-airflow-providers-redis | apache-airflow-providers-salesforce |                      +apache-airflow-providers-samba | apache-airflow-providers-segment |                         +apache-airflow-providers-sendgrid | apache-airflow-providers-sftp |                         +apache-airflow-providers-singularity | apache-airflow-providers-slack |                     +apache-airflow-providers-snowflake | apache-airflow-providers-sqlite |                      +apache-airflow-providers-ssh | apache-airflow-providers-tableau |                           +apache-airflow-providers-tabular | apache-airflow-providers-telegram |                      +apache-airflow-providers-trino | apache-airflow-providers-vertica |                         +apache-airflow-providers-yandex | apache-airflow-providers-zendesk | docker-stack |         +helm-chart)                                                                                 +--github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ diff --git a/images/breeze/output_ci-image_pull.svg b/images/breeze/output_ci-image_pull.svg index 0474ae1195dcc..d0e5c8d141651 100644 --- a/images/breeze/output_ci-image_pull.svg +++ b/images/breeze/output_ci-image_pull.svg @@ -19,169 +19,169 @@ font-weight: 700; } - .terminal-4226770614-matrix { + .breeze-ci-image-pull-matrix { font-family: Fira Code, monospace; font-size: 20px; line-height: 24.4px; font-variant-east-asian: full-width; } - .terminal-4226770614-title { + .breeze-ci-image-pull-title { font-size: 18px; font-weight: bold; font-family: arial; } - .terminal-4226770614-r1 { fill: #c5c8c6;font-weight: bold } -.terminal-4226770614-r2 { fill: #c5c8c6 } -.terminal-4226770614-r3 { fill: #d0b344;font-weight: bold } -.terminal-4226770614-r4 { fill: #868887 } -.terminal-4226770614-r5 { fill: #68a0b3;font-weight: bold } -.terminal-4226770614-r6 { fill: #98a84b;font-weight: bold } -.terminal-4226770614-r7 { fill: #8d7b39 } + .breeze-ci-image-pull-r1 { fill: #c5c8c6;font-weight: bold } +.breeze-ci-image-pull-r2 { fill: #c5c8c6 } +.breeze-ci-image-pull-r3 { fill: #d0b344;font-weight: bold } +.breeze-ci-image-pull-r4 { fill: #68a0b3;font-weight: bold } +.breeze-ci-image-pull-r5 { fill: #868887 } +.breeze-ci-image-pull-r6 { fill: #98a84b;font-weight: bold } +.breeze-ci-image-pull-r7 { fill: #8d7b39 } - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - Command: ci-image pull + Command: ci-image pull - + - - -Usage: breeze ci-image pull [OPTIONS] [EXTRA_PYTEST_ARGS]... - -Pull and optionally verify CI images - possibly in parallel for all Python versions. - -╭─ Pull image flags ───────────────────────────────────────────────────────────────────────────────────────────────────╮ ---image-tag-tTag of the image which is used to pull the image.(TEXT)[default: latest] ---python-pPython major/minor version used in Airflow image for images.(>3.7< | 3.8 | 3.9 | 3.10) -[default: 3.7]                                               ---github-tokenThe token used to authenticate to GitHub.(TEXT) ---verifyVerify image. ---wait-for-imageWait until image is available. ---tag-as-latestTags the image as latest and update checksum of all files after pulling. Useful when you    -build or pull image with --image-tag.                                                       ---github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Parallel running ───────────────────────────────────────────────────────────────────────────────────────────────────╮ ---run-in-parallelRun the operation in parallel on all or selected subset of Python versions. ---parallelismMaximum number of processes to use while running the operation in parallel. -(INTEGER RANGE)                                                             -[default: 4; 1<=x<=8]                                                       ---python-versionsSpace separated list of python versions used for build with multiple versions.(TEXT) -[default: 3.7 3.8 3.9 3.10]                                                    ---skip-cleanupSkip cleanup of temporary files created during parallel run. ---debug-resourcesWhether to show resource information while running in parallel. ---include-success-outputsWhether to include outputs of successful parallel runs (skipped by default). -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---verbose-vPrint verbose information about performed steps. ---dry-run-DIf dry-run is set, commands are only printed, not executed. ---help-hShow this message and exit. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + +Usage: breeze ci-image pull [OPTIONS] [EXTRA_PYTEST_ARGS]... + +Pull and optionally verify CI images - possibly in parallel for all Python versions. + +╭─ Pull image flags ───────────────────────────────────────────────────────────────────────────────────────────────────╮ +--image-tag-tTag of the image which is used to pull the image.(TEXT)[default: latest] +--python-pPython major/minor version used in Airflow image for images.(>3.7< | 3.8 | 3.9 | 3.10) +[default: 3.7]                                               +--github-tokenThe token used to authenticate to GitHub.(TEXT) +--verifyVerify image. +--wait-for-imageWait until image is available. +--tag-as-latestTags the image as latest and update checksum of all files after pulling. Useful when you    +build or pull image with --image-tag.                                                       +--github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Parallel running ───────────────────────────────────────────────────────────────────────────────────────────────────╮ +--run-in-parallelRun the operation in parallel on all or selected subset of Python versions. +--parallelismMaximum number of processes to use while running the operation in parallel. +(INTEGER RANGE)                                                             +[default: 4; 1<=x<=8]                                                       +--python-versionsSpace separated list of python versions used for build with multiple versions.(TEXT) +[default: 3.7 3.8 3.9 3.10]                                                    +--skip-cleanupSkip cleanup of temporary files created during parallel run. +--debug-resourcesWhether to show resource information while running in parallel. +--include-success-outputsWhether to include outputs of successful parallel runs (skipped by default). +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ diff --git a/images/breeze/output_ci-image_verify.svg b/images/breeze/output_ci-image_verify.svg index 00945e992b3e5..6c18349eed7f2 100644 --- a/images/breeze/output_ci-image_verify.svg +++ b/images/breeze/output_ci-image_verify.svg @@ -19,113 +19,113 @@ font-weight: 700; } - .terminal-1202260089-matrix { + .breeze-ci-image-verify-matrix { font-family: Fira Code, monospace; font-size: 20px; line-height: 24.4px; font-variant-east-asian: full-width; } - .terminal-1202260089-title { + .breeze-ci-image-verify-title { font-size: 18px; font-weight: bold; font-family: arial; } - .terminal-1202260089-r1 { fill: #c5c8c6;font-weight: bold } -.terminal-1202260089-r2 { fill: #c5c8c6 } -.terminal-1202260089-r3 { fill: #d0b344;font-weight: bold } -.terminal-1202260089-r4 { fill: #868887 } -.terminal-1202260089-r5 { fill: #68a0b3;font-weight: bold } -.terminal-1202260089-r6 { fill: #98a84b;font-weight: bold } -.terminal-1202260089-r7 { fill: #8d7b39 } + .breeze-ci-image-verify-r1 { fill: #c5c8c6;font-weight: bold } +.breeze-ci-image-verify-r2 { fill: #c5c8c6 } +.breeze-ci-image-verify-r3 { fill: #d0b344;font-weight: bold } +.breeze-ci-image-verify-r4 { fill: #68a0b3;font-weight: bold } +.breeze-ci-image-verify-r5 { fill: #868887 } +.breeze-ci-image-verify-r6 { fill: #98a84b;font-weight: bold } +.breeze-ci-image-verify-r7 { fill: #8d7b39 } - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - Command: ci-image verify + Command: ci-image verify - + - - -Usage: breeze ci-image verify [OPTIONS] [EXTRA_PYTEST_ARGS]... - -Verify CI image. - -╭─ Verify image flags ─────────────────────────────────────────────────────────────────────────────────────────────────╮ ---image-name-nName of the image to verify (overrides --python and --image-tag).(TEXT) ---python-pPython major/minor version used in Airflow image for images.(>3.7< | 3.8 | 3.9 | 3.10) -[default: 3.7]                                               ---image-tag-tTag of the image when verifying it.(TEXT)[default: latest] ---pullPull image is missing before attempting to verify it. ---github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---verbose-vPrint verbose information about performed steps. ---dry-run-DIf dry-run is set, commands are only printed, not executed. ---help-hShow this message and exit. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + +Usage: breeze ci-image verify [OPTIONS] [EXTRA_PYTEST_ARGS]... + +Verify CI image. + +╭─ Verify image flags ─────────────────────────────────────────────────────────────────────────────────────────────────╮ +--image-name-nName of the image to verify (overrides --python and --image-tag).(TEXT) +--python-pPython major/minor version used in Airflow image for images.(>3.7< | 3.8 | 3.9 | 3.10) +[default: 3.7]                                               +--image-tag-tTag of the image when verifying it.(TEXT)[default: latest] +--pullPull image is missing before attempting to verify it. +--github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ diff --git a/images/breeze/output_ci_fix-ownership.svg b/images/breeze/output_ci_fix-ownership.svg index 4b6b67df2f1f5..4ad92f41782ba 100644 --- a/images/breeze/output_ci_fix-ownership.svg +++ b/images/breeze/output_ci_fix-ownership.svg @@ -19,92 +19,92 @@ font-weight: 700; } - .terminal-530343897-matrix { + .breeze-ci-fix-ownership-matrix { font-family: Fira Code, monospace; font-size: 20px; line-height: 24.4px; font-variant-east-asian: full-width; } - .terminal-530343897-title { + .breeze-ci-fix-ownership-title { font-size: 18px; font-weight: bold; font-family: arial; } - .terminal-530343897-r1 { fill: #c5c8c6;font-weight: bold } -.terminal-530343897-r2 { fill: #c5c8c6 } -.terminal-530343897-r3 { fill: #d0b344;font-weight: bold } -.terminal-530343897-r4 { fill: #868887 } -.terminal-530343897-r5 { fill: #68a0b3;font-weight: bold } -.terminal-530343897-r6 { fill: #98a84b;font-weight: bold } + .breeze-ci-fix-ownership-r1 { fill: #c5c8c6;font-weight: bold } +.breeze-ci-fix-ownership-r2 { fill: #c5c8c6 } +.breeze-ci-fix-ownership-r3 { fill: #d0b344;font-weight: bold } +.breeze-ci-fix-ownership-r4 { fill: #68a0b3;font-weight: bold } +.breeze-ci-fix-ownership-r5 { fill: #868887 } +.breeze-ci-fix-ownership-r6 { fill: #98a84b;font-weight: bold } - + - + - + - + - + - + - + - + - + - + - + - + - + - Command: ci fix-ownership + Command: ci fix-ownership - + - - -Usage: breeze ci fix-ownership [OPTIONS] - -Fix ownership of source files to be same as host user. - -╭─ Fix ownership flags ────────────────────────────────────────────────────────────────────────────────────────────────╮ ---use-sudoUse sudo instead of docker image to fix the ownership. You need to be a `sudoer` to run it -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---verbose-vPrint verbose information about performed steps. ---dry-run-DIf dry-run is set, commands are only printed, not executed. ---help-hShow this message and exit. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + +Usage: breeze ci fix-ownership [OPTIONS] + +Fix ownership of source files to be same as host user. + +╭─ Fix ownership flags ────────────────────────────────────────────────────────────────────────────────────────────────╮ +--use-sudoUse sudo instead of docker image to fix the ownership. You need to be a `sudoer` to run it +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ diff --git a/images/breeze/output_ci_free-space.svg b/images/breeze/output_ci_free-space.svg index d6337eaac8f8d..d3210c1aacb4c 100644 --- a/images/breeze/output_ci_free-space.svg +++ b/images/breeze/output_ci_free-space.svg @@ -19,85 +19,85 @@ font-weight: 700; } - .terminal-2980552758-matrix { + .breeze-ci-free-space-matrix { font-family: Fira Code, monospace; font-size: 20px; line-height: 24.4px; font-variant-east-asian: full-width; } - .terminal-2980552758-title { + .breeze-ci-free-space-title { font-size: 18px; font-weight: bold; font-family: arial; } - .terminal-2980552758-r1 { fill: #c5c8c6;font-weight: bold } -.terminal-2980552758-r2 { fill: #c5c8c6 } -.terminal-2980552758-r3 { fill: #d0b344;font-weight: bold } -.terminal-2980552758-r4 { fill: #868887 } -.terminal-2980552758-r5 { fill: #68a0b3;font-weight: bold } -.terminal-2980552758-r6 { fill: #98a84b;font-weight: bold } -.terminal-2980552758-r7 { fill: #8d7b39 } + .breeze-ci-free-space-r1 { fill: #c5c8c6;font-weight: bold } +.breeze-ci-free-space-r2 { fill: #c5c8c6 } +.breeze-ci-free-space-r3 { fill: #d0b344;font-weight: bold } +.breeze-ci-free-space-r4 { fill: #68a0b3;font-weight: bold } +.breeze-ci-free-space-r5 { fill: #868887 } +.breeze-ci-free-space-r6 { fill: #98a84b;font-weight: bold } +.breeze-ci-free-space-r7 { fill: #8d7b39 } - + - + - + - + - + - + - + - + - + - + - + - Command: ci free-space + Command: ci free-space - + - - -Usage: breeze ci free-space [OPTIONS] - -Free space for jobs run in CI. - -╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---verbose-vPrint verbose information about performed steps. ---dry-run-DIf dry-run is set, commands are only printed, not executed. ---answer-aForce answer to questions.(y | n | q | yes | no | quit) ---help-hShow this message and exit. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + +Usage: breeze ci free-space [OPTIONS] + +Free space for jobs run in CI. + +╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--answer-aForce answer to questions.(y | n | q | yes | no | quit) +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ diff --git a/images/breeze/output_ci_get-workflow-info.svg b/images/breeze/output_ci_get-workflow-info.svg index 2a4e1973abf27..d0fd6bc59fb29 100644 --- a/images/breeze/output_ci_get-workflow-info.svg +++ b/images/breeze/output_ci_get-workflow-info.svg @@ -19,89 +19,89 @@ font-weight: 700; } - .terminal-1038436741-matrix { + .breeze-ci-get-workflow-info-matrix { font-family: Fira Code, monospace; font-size: 20px; line-height: 24.4px; font-variant-east-asian: full-width; } - .terminal-1038436741-title { + .breeze-ci-get-workflow-info-title { font-size: 18px; font-weight: bold; font-family: arial; } - .terminal-1038436741-r1 { fill: #c5c8c6;font-weight: bold } -.terminal-1038436741-r2 { fill: #c5c8c6 } -.terminal-1038436741-r3 { fill: #d0b344;font-weight: bold } -.terminal-1038436741-r4 { fill: #868887 } -.terminal-1038436741-r5 { fill: #68a0b3;font-weight: bold } -.terminal-1038436741-r6 { fill: #8d7b39 } -.terminal-1038436741-r7 { fill: #98a84b;font-weight: bold } + .breeze-ci-get-workflow-info-r1 { fill: #c5c8c6;font-weight: bold } +.breeze-ci-get-workflow-info-r2 { fill: #c5c8c6 } +.breeze-ci-get-workflow-info-r3 { fill: #d0b344;font-weight: bold } +.breeze-ci-get-workflow-info-r4 { fill: #68a0b3;font-weight: bold } +.breeze-ci-get-workflow-info-r5 { fill: #868887 } +.breeze-ci-get-workflow-info-r6 { fill: #8d7b39 } +.breeze-ci-get-workflow-info-r7 { fill: #98a84b;font-weight: bold } - + - + - + - + - + - + - + - + - + - + - + - + - Command: ci get-workflow-info + Command: ci get-workflow-info - + - - -Usage: breeze ci get-workflow-info [OPTIONS] - -Retrieve information about current workflow in the CIand produce github actions output extracted from it. - -╭─ Get workflow info flags ────────────────────────────────────────────────────────────────────────────────────────────╮ ---github-contextJSON-formatted github context(TEXT) ---github-context-inputfile input (might be `-`) with JSON-formatted github context(FILENAME) -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---help-hShow this message and exit. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + +Usage: breeze ci get-workflow-info [OPTIONS] + +Retrieve information about current workflow in the CIand produce github actions output extracted from it. + +╭─ Get workflow info flags ────────────────────────────────────────────────────────────────────────────────────────────╮ +--github-contextJSON-formatted github context(TEXT) +--github-context-inputfile input (might be `-`) with JSON-formatted github context(FILENAME) +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ diff --git a/images/breeze/output_ci_resource-check.svg b/images/breeze/output_ci_resource-check.svg index 2f4c13a6bbc2e..4afc5dd570fda 100644 --- a/images/breeze/output_ci_resource-check.svg +++ b/images/breeze/output_ci_resource-check.svg @@ -19,80 +19,80 @@ font-weight: 700; } - .terminal-481855223-matrix { + .breeze-ci-resource-check-matrix { font-family: Fira Code, monospace; font-size: 20px; line-height: 24.4px; font-variant-east-asian: full-width; } - .terminal-481855223-title { + .breeze-ci-resource-check-title { font-size: 18px; font-weight: bold; font-family: arial; } - .terminal-481855223-r1 { fill: #c5c8c6;font-weight: bold } -.terminal-481855223-r2 { fill: #c5c8c6 } -.terminal-481855223-r3 { fill: #d0b344;font-weight: bold } -.terminal-481855223-r4 { fill: #868887 } -.terminal-481855223-r5 { fill: #68a0b3;font-weight: bold } -.terminal-481855223-r6 { fill: #98a84b;font-weight: bold } + .breeze-ci-resource-check-r1 { fill: #c5c8c6;font-weight: bold } +.breeze-ci-resource-check-r2 { fill: #c5c8c6 } +.breeze-ci-resource-check-r3 { fill: #d0b344;font-weight: bold } +.breeze-ci-resource-check-r4 { fill: #68a0b3;font-weight: bold } +.breeze-ci-resource-check-r5 { fill: #868887 } +.breeze-ci-resource-check-r6 { fill: #98a84b;font-weight: bold } - + - + - + - + - + - + - + - + - + - + - Command: ci resource-check + Command: ci resource-check - + - - -Usage: breeze ci resource-check [OPTIONS] - -Check if available docker resources are enough. - -╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---verbose-vPrint verbose information about performed steps. ---dry-run-DIf dry-run is set, commands are only printed, not executed. ---help-hShow this message and exit. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + +Usage: breeze ci resource-check [OPTIONS] + +Check if available docker resources are enough. + +╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ diff --git a/images/breeze/output_ci_selective-check.svg b/images/breeze/output_ci_selective-check.svg index 2f117e63d068f..0d1250f1e20cf 100644 --- a/images/breeze/output_ci_selective-check.svg +++ b/images/breeze/output_ci_selective-check.svg @@ -19,125 +19,125 @@ font-weight: 700; } - .terminal-2396090381-matrix { + .breeze-ci-selective-check-matrix { font-family: Fira Code, monospace; font-size: 20px; line-height: 24.4px; font-variant-east-asian: full-width; } - .terminal-2396090381-title { + .breeze-ci-selective-check-title { font-size: 18px; font-weight: bold; font-family: arial; } - .terminal-2396090381-r1 { fill: #c5c8c6;font-weight: bold } -.terminal-2396090381-r2 { fill: #c5c8c6 } -.terminal-2396090381-r3 { fill: #d0b344;font-weight: bold } -.terminal-2396090381-r4 { fill: #868887 } -.terminal-2396090381-r5 { fill: #68a0b3;font-weight: bold } -.terminal-2396090381-r6 { fill: #8d7b39 } -.terminal-2396090381-r7 { fill: #98a84b;font-weight: bold } + .breeze-ci-selective-check-r1 { fill: #c5c8c6;font-weight: bold } +.breeze-ci-selective-check-r2 { fill: #c5c8c6 } +.breeze-ci-selective-check-r3 { fill: #d0b344;font-weight: bold } +.breeze-ci-selective-check-r4 { fill: #68a0b3;font-weight: bold } +.breeze-ci-selective-check-r5 { fill: #868887 } +.breeze-ci-selective-check-r6 { fill: #8d7b39 } +.breeze-ci-selective-check-r7 { fill: #98a84b;font-weight: bold } - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - Command: ci selective-check + Command: ci selective-check - + - - -Usage: breeze ci selective-check [OPTIONS] - -Checks what kind of tests should be run for an incoming commit. - -╭─ Selective check flags ──────────────────────────────────────────────────────────────────────────────────────────────╮ ---commit-refCommit-ish reference to the commit that should be checked(TEXT) ---pr-labelsPython array formatted PR labels assigned to the PR(TEXT) ---default-branchBranch against which the PR should be run(TEXT)[default: main] ---default-constraints-branchConstraints Branch against which the PR should be run(TEXT) -[default: constraints-main]                           ---github-event-nameName of the GitHub event that triggered the check                                    -(pull_request | pull_request_review | pull_request_target | pull_request_workflow |  -push | schedule | workflow_dispatch | workflow_run)                                  -[default: pull_request]                                                              -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---verbose-vPrint verbose information about performed steps. ---dry-run-DIf dry-run is set, commands are only printed, not executed. ---help-hShow this message and exit. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + +Usage: breeze ci selective-check [OPTIONS] + +Checks what kind of tests should be run for an incoming commit. + +╭─ Selective check flags ──────────────────────────────────────────────────────────────────────────────────────────────╮ +--commit-refCommit-ish reference to the commit that should be checked(TEXT) +--pr-labelsPython array formatted PR labels assigned to the PR(TEXT) +--default-branchBranch against which the PR should be run(TEXT)[default: main] +--default-constraints-branchConstraints Branch against which the PR should be run(TEXT) +[default: constraints-main]                           +--github-event-nameName of the GitHub event that triggered the check                                    +(pull_request | pull_request_review | pull_request_target | pull_request_workflow |  +push | schedule | workflow_dispatch | workflow_run)                                  +[default: pull_request]                                                              +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ diff --git a/images/breeze/output_cleanup.svg b/images/breeze/output_cleanup.svg index e6bfc347cf67a..ea7865374f822 100644 --- a/images/breeze/output_cleanup.svg +++ b/images/breeze/output_cleanup.svg @@ -19,97 +19,97 @@ font-weight: 700; } - .terminal-3676080220-matrix { + .breeze-cleanup-matrix { font-family: Fira Code, monospace; font-size: 20px; line-height: 24.4px; font-variant-east-asian: full-width; } - .terminal-3676080220-title { + .breeze-cleanup-title { font-size: 18px; font-weight: bold; font-family: arial; } - .terminal-3676080220-r1 { fill: #c5c8c6;font-weight: bold } -.terminal-3676080220-r2 { fill: #c5c8c6 } -.terminal-3676080220-r3 { fill: #d0b344;font-weight: bold } -.terminal-3676080220-r4 { fill: #868887 } -.terminal-3676080220-r5 { fill: #68a0b3;font-weight: bold } -.terminal-3676080220-r6 { fill: #98a84b;font-weight: bold } -.terminal-3676080220-r7 { fill: #8d7b39 } + .breeze-cleanup-r1 { fill: #c5c8c6;font-weight: bold } +.breeze-cleanup-r2 { fill: #c5c8c6 } +.breeze-cleanup-r3 { fill: #d0b344;font-weight: bold } +.breeze-cleanup-r4 { fill: #68a0b3;font-weight: bold } +.breeze-cleanup-r5 { fill: #868887 } +.breeze-cleanup-r6 { fill: #98a84b;font-weight: bold } +.breeze-cleanup-r7 { fill: #8d7b39 } - + - + - + - + - + - + - + - + - + - + - + - + - + - + - Command: cleanup + Command: cleanup - + - - -Usage: breeze cleanup [OPTIONS] - -Cleans the cache of parameters, docker cache and optionally built CI/PROD images. - -╭─ Cleanup flags ──────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---allAlso remove currently downloaded Breeze images. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---verbose-vPrint verbose information about performed steps. ---dry-run-DIf dry-run is set, commands are only printed, not executed. ---answer-aForce answer to questions.(y | n | q | yes | no | quit) ---help-hShow this message and exit. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + +Usage: breeze cleanup [OPTIONS] + +Cleans the cache of parameters, docker cache and optionally built CI/PROD images. + +╭─ Cleanup flags ──────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--allAlso remove currently downloaded Breeze images. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--answer-aForce answer to questions.(y | n | q | yes | no | quit) +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ diff --git a/images/breeze/output_compile-www-assets.svg b/images/breeze/output_compile-www-assets.svg index 26ba96de9c537..ab1607d439c70 100644 --- a/images/breeze/output_compile-www-assets.svg +++ b/images/breeze/output_compile-www-assets.svg @@ -19,96 +19,96 @@ font-weight: 700; } - .terminal-3087694031-matrix { + .breeze-compile-www-assets-matrix { font-family: Fira Code, monospace; font-size: 20px; line-height: 24.4px; font-variant-east-asian: full-width; } - .terminal-3087694031-title { + .breeze-compile-www-assets-title { font-size: 18px; font-weight: bold; font-family: arial; } - .terminal-3087694031-r1 { fill: #c5c8c6;font-weight: bold } -.terminal-3087694031-r2 { fill: #c5c8c6 } -.terminal-3087694031-r3 { fill: #d0b344;font-weight: bold } -.terminal-3087694031-r4 { fill: #868887 } -.terminal-3087694031-r5 { fill: #68a0b3;font-weight: bold } -.terminal-3087694031-r6 { fill: #98a84b;font-weight: bold } + .breeze-compile-www-assets-r1 { fill: #c5c8c6;font-weight: bold } +.breeze-compile-www-assets-r2 { fill: #c5c8c6 } +.breeze-compile-www-assets-r3 { fill: #d0b344;font-weight: bold } +.breeze-compile-www-assets-r4 { fill: #68a0b3;font-weight: bold } +.breeze-compile-www-assets-r5 { fill: #868887 } +.breeze-compile-www-assets-r6 { fill: #98a84b;font-weight: bold } - + - + - + - + - + - + - + - + - + - + - + - + - + - + - Command: compile-www-assets + Command: compile-www-assets - + - - -Usage: breeze compile-www-assets [OPTIONS] - -Compiles www assets. - -╭─ Compile www assets flag ────────────────────────────────────────────────────────────────────────────────────────────╮ ---devRun development version of assets compilation - it will not quit and automatically recompile assets         -on-the-fly when they are changed.                                                                           -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---verbose-vPrint verbose information about performed steps. ---dry-run-DIf dry-run is set, commands are only printed, not executed. ---help-hShow this message and exit. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + +Usage: breeze compile-www-assets [OPTIONS] + +Compiles www assets. + +╭─ Compile www assets flag ────────────────────────────────────────────────────────────────────────────────────────────╮ +--devRun development version of assets compilation - it will not quit and automatically recompile assets         +on-the-fly when they are changed.                                                                           +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ diff --git a/images/breeze/output_exec.svg b/images/breeze/output_exec.svg index 2541c71f65db0..36e2b55c4fb10 100644 --- a/images/breeze/output_exec.svg +++ b/images/breeze/output_exec.svg @@ -19,80 +19,80 @@ font-weight: 700; } - .terminal-1229358110-matrix { + .breeze-exec-matrix { font-family: Fira Code, monospace; font-size: 20px; line-height: 24.4px; font-variant-east-asian: full-width; } - .terminal-1229358110-title { + .breeze-exec-title { font-size: 18px; font-weight: bold; font-family: arial; } - .terminal-1229358110-r1 { fill: #c5c8c6;font-weight: bold } -.terminal-1229358110-r2 { fill: #c5c8c6 } -.terminal-1229358110-r3 { fill: #d0b344;font-weight: bold } -.terminal-1229358110-r4 { fill: #868887 } -.terminal-1229358110-r5 { fill: #68a0b3;font-weight: bold } -.terminal-1229358110-r6 { fill: #98a84b;font-weight: bold } + .breeze-exec-r1 { fill: #c5c8c6;font-weight: bold } +.breeze-exec-r2 { fill: #c5c8c6 } +.breeze-exec-r3 { fill: #d0b344;font-weight: bold } +.breeze-exec-r4 { fill: #68a0b3;font-weight: bold } +.breeze-exec-r5 { fill: #868887 } +.breeze-exec-r6 { fill: #98a84b;font-weight: bold } - + - + - + - + - + - + - + - + - + - + - Command: exec + Command: exec - + - - -Usage: breeze exec [OPTIONS] [EXEC_ARGS]... - -Joins the interactive shell of running airflow container. - -╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---verbose-vPrint verbose information about performed steps. ---dry-run-DIf dry-run is set, commands are only printed, not executed. ---help-hShow this message and exit. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + +Usage: breeze exec [OPTIONS] [EXEC_ARGS]... + +Joins the interactive shell of running airflow container. + +╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ diff --git a/images/breeze/output_k8s.svg b/images/breeze/output_k8s.svg index d9e05970f4039..5ba36f4e52999 100644 --- a/images/breeze/output_k8s.svg +++ b/images/breeze/output_k8s.svg @@ -19,164 +19,164 @@ font-weight: 700; } - .terminal-781963109-matrix { + .breeze-k8s-matrix { font-family: Fira Code, monospace; font-size: 20px; line-height: 24.4px; font-variant-east-asian: full-width; } - .terminal-781963109-title { + .breeze-k8s-title { font-size: 18px; font-weight: bold; font-family: arial; } - .terminal-781963109-r1 { fill: #c5c8c6;font-weight: bold } -.terminal-781963109-r2 { fill: #c5c8c6 } -.terminal-781963109-r3 { fill: #d0b344;font-weight: bold } -.terminal-781963109-r4 { fill: #68a0b3;font-weight: bold } -.terminal-781963109-r5 { fill: #868887 } -.terminal-781963109-r6 { fill: #98a84b;font-weight: bold } + .breeze-k8s-r1 { fill: #c5c8c6;font-weight: bold } +.breeze-k8s-r2 { fill: #c5c8c6 } +.breeze-k8s-r3 { fill: #d0b344;font-weight: bold } +.breeze-k8s-r4 { fill: #68a0b3;font-weight: bold } +.breeze-k8s-r5 { fill: #868887 } +.breeze-k8s-r6 { fill: #98a84b;font-weight: bold } - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - Command: k8s + Command: k8s - + - - -Usage: breeze k8s [OPTIONSCOMMAND [ARGS]... - -Tools that developers use to run Kubernetes tests - -╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---help-hShow this message and exit. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ K8S cluster management commands ────────────────────────────────────────────────────────────────────────────────────╮ -setup-env        Setup shared Kubernetes virtual environment and tools.                                            -create-cluster   Create a KinD Cluster for Python and Kubernetes version specified (optionally create all clusters -in parallel).                                                                                     -configure-clusterConfigures cluster for airflow deployment - creates namespaces and test resources (optionally for -all clusters in parallel).                                                                        -build-k8s-image  Build k8s-ready airflow image (optionally all images in parallel).                                -upload-k8s-image Upload k8s-ready airflow image to the KinD cluster (optionally to all clusters in parallel)       -deploy-airflow   Deploy airflow image to the current KinD cluster (or all clusters).                               -delete-cluster   Delete the current KinD Cluster (optionally all clusters).                                        -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ K8S inspection commands ────────────────────────────────────────────────────────────────────────────────────────────╮ -status  Check status of the current cluster and airflow deployed to it (optionally all clusters).                  -logs    Dump k8s logs to ${TMP_DIR}/kind_logs_<cluster_name> directory (optionally all clusters).                  -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ K8S testing commands ───────────────────────────────────────────────────────────────────────────────────────────────╮ -tests             Run tests against the current KinD cluster (optionally for all clusters in parallel).            -run-complete-testsRun complete k8s tests consisting of: creating cluster, building and uploading image, deploying  -airflow, running tests and deleting clusters (optionally for all clusters in parallel).          -shell             Run shell environment for the current KinD cluster.                                              -k9s               Run k9s tool. You can pass any k9s args as extra args.                                           -logs              Dump k8s logs to ${TMP_DIR}/kind_logs_<cluster_name> directory (optionally all clusters).        -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + +Usage: breeze k8s [OPTIONSCOMMAND [ARGS]... + +Tools that developers use to run Kubernetes tests + +╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ K8S cluster management commands ────────────────────────────────────────────────────────────────────────────────────╮ +setup-env        Setup shared Kubernetes virtual environment and tools.                                            +create-cluster   Create a KinD Cluster for Python and Kubernetes version specified (optionally create all clusters +in parallel).                                                                                     +configure-clusterConfigures cluster for airflow deployment - creates namespaces and test resources (optionally for +all clusters in parallel).                                                                        +build-k8s-image  Build k8s-ready airflow image (optionally all images in parallel).                                +upload-k8s-image Upload k8s-ready airflow image to the KinD cluster (optionally to all clusters in parallel)       +deploy-airflow   Deploy airflow image to the current KinD cluster (or all clusters).                               +delete-cluster   Delete the current KinD Cluster (optionally all clusters).                                        +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ K8S inspection commands ────────────────────────────────────────────────────────────────────────────────────────────╮ +status  Check status of the current cluster and airflow deployed to it (optionally all clusters).                  +logs    Dump k8s logs to ${TMP_DIR}/kind_logs_<cluster_name> directory (optionally all clusters).                  +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ K8S testing commands ───────────────────────────────────────────────────────────────────────────────────────────────╮ +tests             Run tests against the current KinD cluster (optionally for all clusters in parallel).            +run-complete-testsRun complete k8s tests consisting of: creating cluster, building and uploading image, deploying  +airflow, running tests and deleting clusters (optionally for all clusters in parallel).          +shell             Run shell environment for the current KinD cluster.                                              +k9s               Run k9s tool. You can pass any k9s args as extra args.                                           +logs              Dump k8s logs to ${TMP_DIR}/kind_logs_<cluster_name> directory (optionally all clusters).        +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ diff --git a/images/breeze/output_k8s_build-k8s-image.svg b/images/breeze/output_k8s_build-k8s-image.svg index d34d962cc977d..c607491c2c8b0 100644 --- a/images/breeze/output_k8s_build-k8s-image.svg +++ b/images/breeze/output_k8s_build-k8s-image.svg @@ -19,149 +19,149 @@ font-weight: 700; } - .terminal-925219226-matrix { + .breeze-k8s-build-k8s-image-matrix { font-family: Fira Code, monospace; font-size: 20px; line-height: 24.4px; font-variant-east-asian: full-width; } - .terminal-925219226-title { + .breeze-k8s-build-k8s-image-title { font-size: 18px; font-weight: bold; font-family: arial; } - .terminal-925219226-r1 { fill: #c5c8c6;font-weight: bold } -.terminal-925219226-r2 { fill: #c5c8c6 } -.terminal-925219226-r3 { fill: #d0b344;font-weight: bold } -.terminal-925219226-r4 { fill: #868887 } -.terminal-925219226-r5 { fill: #68a0b3;font-weight: bold } -.terminal-925219226-r6 { fill: #98a84b;font-weight: bold } -.terminal-925219226-r7 { fill: #8d7b39 } + .breeze-k8s-build-k8s-image-r1 { fill: #c5c8c6;font-weight: bold } +.breeze-k8s-build-k8s-image-r2 { fill: #c5c8c6 } +.breeze-k8s-build-k8s-image-r3 { fill: #d0b344;font-weight: bold } +.breeze-k8s-build-k8s-image-r4 { fill: #68a0b3;font-weight: bold } +.breeze-k8s-build-k8s-image-r5 { fill: #868887 } +.breeze-k8s-build-k8s-image-r6 { fill: #98a84b;font-weight: bold } +.breeze-k8s-build-k8s-image-r7 { fill: #8d7b39 } - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - Command: k8s build-k8s-image + Command: k8s build-k8s-image - + - - -Usage: breeze k8s build-k8s-image [OPTIONS] - -Build k8s-ready airflow image (optionally all images in parallel). - -╭─ Build image flags ──────────────────────────────────────────────────────────────────────────────────────────────────╮ ---python-pPython major/minor version used in Airflow image for images.(>3.7< | 3.8 | 3.9 | 3.10) -[default: 3.7]                                               ---rebuild-base-imageRebuilds base Airflow image before building K8S image. ---image-tag-tImage tag used to build K8S image from.(TEXT)[default: latest] -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Parallel options ───────────────────────────────────────────────────────────────────────────────────────────────────╮ ---run-in-parallelRun the operation in parallel on all or selected subset of Python versions. ---parallelismMaximum number of processes to use while running the operation in parallel. -(INTEGER RANGE)                                                             -[default: 4; 1<=x<=8]                                                       ---python-versionsSpace separated list of python versions used for build with multiple versions.(TEXT) -[default: 3.7 3.8 3.9 3.10]                                                    ---skip-cleanupSkip cleanup of temporary files created during parallel run. ---debug-resourcesWhether to show resource information while running in parallel. ---include-success-outputsWhether to include outputs of successful parallel runs (skipped by default). -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---verbose-vPrint verbose information about performed steps. ---dry-run-DIf dry-run is set, commands are only printed, not executed. ---help-hShow this message and exit. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + +Usage: breeze k8s build-k8s-image [OPTIONS] + +Build k8s-ready airflow image (optionally all images in parallel). + +╭─ Build image flags ──────────────────────────────────────────────────────────────────────────────────────────────────╮ +--python-pPython major/minor version used in Airflow image for images.(>3.7< | 3.8 | 3.9 | 3.10) +[default: 3.7]                                               +--rebuild-base-imageRebuilds base Airflow image before building K8S image. +--image-tag-tImage tag used to build K8S image from.(TEXT)[default: latest] +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Parallel options ───────────────────────────────────────────────────────────────────────────────────────────────────╮ +--run-in-parallelRun the operation in parallel on all or selected subset of Python versions. +--parallelismMaximum number of processes to use while running the operation in parallel. +(INTEGER RANGE)                                                             +[default: 4; 1<=x<=8]                                                       +--python-versionsSpace separated list of python versions used for build with multiple versions.(TEXT) +[default: 3.7 3.8 3.9 3.10]                                                    +--skip-cleanupSkip cleanup of temporary files created during parallel run. +--debug-resourcesWhether to show resource information while running in parallel. +--include-success-outputsWhether to include outputs of successful parallel runs (skipped by default). +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ diff --git a/images/breeze/output_k8s_configure-cluster.svg b/images/breeze/output_k8s_configure-cluster.svg index 5e7c5df363a8e..c20e517e6acea 100644 --- a/images/breeze/output_k8s_configure-cluster.svg +++ b/images/breeze/output_k8s_configure-cluster.svg @@ -19,169 +19,169 @@ font-weight: 700; } - .terminal-2773104650-matrix { + .breeze-k8s-configure-cluster-matrix { font-family: Fira Code, monospace; font-size: 20px; line-height: 24.4px; font-variant-east-asian: full-width; } - .terminal-2773104650-title { + .breeze-k8s-configure-cluster-title { font-size: 18px; font-weight: bold; font-family: arial; } - .terminal-2773104650-r1 { fill: #c5c8c6;font-weight: bold } -.terminal-2773104650-r2 { fill: #c5c8c6 } -.terminal-2773104650-r3 { fill: #d0b344;font-weight: bold } -.terminal-2773104650-r4 { fill: #68a0b3;font-weight: bold } -.terminal-2773104650-r5 { fill: #868887 } -.terminal-2773104650-r6 { fill: #98a84b;font-weight: bold } -.terminal-2773104650-r7 { fill: #8d7b39 } + .breeze-k8s-configure-cluster-r1 { fill: #c5c8c6;font-weight: bold } +.breeze-k8s-configure-cluster-r2 { fill: #c5c8c6 } +.breeze-k8s-configure-cluster-r3 { fill: #d0b344;font-weight: bold } +.breeze-k8s-configure-cluster-r4 { fill: #68a0b3;font-weight: bold } +.breeze-k8s-configure-cluster-r5 { fill: #868887 } +.breeze-k8s-configure-cluster-r6 { fill: #98a84b;font-weight: bold } +.breeze-k8s-configure-cluster-r7 { fill: #8d7b39 } - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - Command: k8s configure-cluster + Command: k8s configure-cluster - + - - -Usage: breeze k8s configure-cluster [OPTIONS] - -Configures cluster for airflow deployment - creates namespaces and test resources (optionally for all clusters in  -parallel). - -╭─ Configure cluster flags ────────────────────────────────────────────────────────────────────────────────────────────╮ ---python-pPython major/minor version used in Airflow image for images.(>3.7< | 3.8 | 3.9 | 3.10) -[default: 3.7]                                               ---kubernetes-versionKubernetes version used to create the KinD cluster of. -(>v1.25.3< | v1.24.7 | v1.23.13 | v1.22.15 | v1.21.14) -[default: v1.25.3]                                     -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Parallel options ───────────────────────────────────────────────────────────────────────────────────────────────────╮ ---run-in-parallelRun the operation in parallel on all or selected subset of Python versions. ---parallelismMaximum number of processes to use while running the operation in parallel for cluster  -operations.                                                                             -(INTEGER RANGE)                                                                         -[default: 2; 1<=x<=4]                                                                   ---python-versionsSpace separated list of python versions used for build with multiple versions.(TEXT) -[default: 3.7 3.8 3.9 3.10]                                                    ---kubernetes-versionsKubernetes versions used to run in parallel (space separated).(TEXT) -[default: v1.25.3 v1.24.7 v1.23.13 v1.22.15 v1.21.14]          ---skip-cleanupSkip cleanup of temporary files created during parallel run. ---debug-resourcesWhether to show resource information while running in parallel. ---include-success-outputsWhether to include outputs of successful parallel runs (skipped by default). -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---verbose-vPrint verbose information about performed steps. ---dry-run-DIf dry-run is set, commands are only printed, not executed. ---help-hShow this message and exit. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + +Usage: breeze k8s configure-cluster [OPTIONS] + +Configures cluster for airflow deployment - creates namespaces and test resources (optionally for all clusters in  +parallel). + +╭─ Configure cluster flags ────────────────────────────────────────────────────────────────────────────────────────────╮ +--python-pPython major/minor version used in Airflow image for images.(>3.7< | 3.8 | 3.9 | 3.10) +[default: 3.7]                                               +--kubernetes-versionKubernetes version used to create the KinD cluster of. +(>v1.25.3< | v1.24.7 | v1.23.13 | v1.22.15 | v1.21.14) +[default: v1.25.3]                                     +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Parallel options ───────────────────────────────────────────────────────────────────────────────────────────────────╮ +--run-in-parallelRun the operation in parallel on all or selected subset of Python versions. +--parallelismMaximum number of processes to use while running the operation in parallel for cluster  +operations.                                                                             +(INTEGER RANGE)                                                                         +[default: 2; 1<=x<=4]                                                                   +--python-versionsSpace separated list of python versions used for build with multiple versions.(TEXT) +[default: 3.7 3.8 3.9 3.10]                                                    +--kubernetes-versionsKubernetes versions used to run in parallel (space separated).(TEXT) +[default: v1.25.3 v1.24.7 v1.23.13 v1.22.15 v1.21.14]          +--skip-cleanupSkip cleanup of temporary files created during parallel run. +--debug-resourcesWhether to show resource information while running in parallel. +--include-success-outputsWhether to include outputs of successful parallel runs (skipped by default). +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ diff --git a/images/breeze/output_k8s_create-cluster.svg b/images/breeze/output_k8s_create-cluster.svg index 9f435540c44eb..75df60100c67d 100644 --- a/images/breeze/output_k8s_create-cluster.svg +++ b/images/breeze/output_k8s_create-cluster.svg @@ -19,173 +19,173 @@ font-weight: 700; } - .terminal-308107195-matrix { + .breeze-k8s-create-cluster-matrix { font-family: Fira Code, monospace; font-size: 20px; line-height: 24.4px; font-variant-east-asian: full-width; } - .terminal-308107195-title { + .breeze-k8s-create-cluster-title { font-size: 18px; font-weight: bold; font-family: arial; } - .terminal-308107195-r1 { fill: #c5c8c6;font-weight: bold } -.terminal-308107195-r2 { fill: #c5c8c6 } -.terminal-308107195-r3 { fill: #d0b344;font-weight: bold } -.terminal-308107195-r4 { fill: #68a0b3;font-weight: bold } -.terminal-308107195-r5 { fill: #868887 } -.terminal-308107195-r6 { fill: #98a84b;font-weight: bold } -.terminal-308107195-r7 { fill: #8d7b39 } + .breeze-k8s-create-cluster-r1 { fill: #c5c8c6;font-weight: bold } +.breeze-k8s-create-cluster-r2 { fill: #c5c8c6 } +.breeze-k8s-create-cluster-r3 { fill: #d0b344;font-weight: bold } +.breeze-k8s-create-cluster-r4 { fill: #68a0b3;font-weight: bold } +.breeze-k8s-create-cluster-r5 { fill: #868887 } +.breeze-k8s-create-cluster-r6 { fill: #98a84b;font-weight: bold } +.breeze-k8s-create-cluster-r7 { fill: #8d7b39 } - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - Command: k8s create-cluster + Command: k8s create-cluster - + - - -Usage: breeze k8s create-cluster [OPTIONS] - -Create a KinD Cluster for Python and Kubernetes version specified (optionally create all clusters in parallel). - -╭─ K8S cluster creation flags ─────────────────────────────────────────────────────────────────────────────────────────╮ ---python-pPython major/minor version used in Airflow image for images. -(>3.7< | 3.8 | 3.9 | 3.10)                                   -[default: 3.7]                                               ---kubernetes-versionKubernetes version used to create the KinD cluster of. -(>v1.25.3< | v1.24.7 | v1.23.13 | v1.22.15 | v1.21.14) -[default: v1.25.3]                                     ---force-recreate-clusterForce recreation of the cluster even if it is already created. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Parallel options ───────────────────────────────────────────────────────────────────────────────────────────────────╮ ---run-in-parallelRun the operation in parallel on all or selected subset of Python versions. ---parallelismMaximum number of processes to use while running the operation in parallel for cluster  -operations.                                                                             -(INTEGER RANGE)                                                                         -[default: 2; 1<=x<=4]                                                                   ---python-versionsSpace separated list of python versions used for build with multiple versions.(TEXT) -[default: 3.7 3.8 3.9 3.10]                                                    ---kubernetes-versionsKubernetes versions used to run in parallel (space separated).(TEXT) -[default: v1.25.3 v1.24.7 v1.23.13 v1.22.15 v1.21.14]          ---skip-cleanupSkip cleanup of temporary files created during parallel run. ---debug-resourcesWhether to show resource information while running in parallel. ---include-success-outputsWhether to include outputs of successful parallel runs (skipped by default). -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---verbose-vPrint verbose information about performed steps. ---dry-run-DIf dry-run is set, commands are only printed, not executed. ---help-hShow this message and exit. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + +Usage: breeze k8s create-cluster [OPTIONS] + +Create a KinD Cluster for Python and Kubernetes version specified (optionally create all clusters in parallel). + +╭─ K8S cluster creation flags ─────────────────────────────────────────────────────────────────────────────────────────╮ +--python-pPython major/minor version used in Airflow image for images. +(>3.7< | 3.8 | 3.9 | 3.10)                                   +[default: 3.7]                                               +--kubernetes-versionKubernetes version used to create the KinD cluster of. +(>v1.25.3< | v1.24.7 | v1.23.13 | v1.22.15 | v1.21.14) +[default: v1.25.3]                                     +--force-recreate-clusterForce recreation of the cluster even if it is already created. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Parallel options ───────────────────────────────────────────────────────────────────────────────────────────────────╮ +--run-in-parallelRun the operation in parallel on all or selected subset of Python versions. +--parallelismMaximum number of processes to use while running the operation in parallel for cluster  +operations.                                                                             +(INTEGER RANGE)                                                                         +[default: 2; 1<=x<=4]                                                                   +--python-versionsSpace separated list of python versions used for build with multiple versions.(TEXT) +[default: 3.7 3.8 3.9 3.10]                                                    +--kubernetes-versionsKubernetes versions used to run in parallel (space separated).(TEXT) +[default: v1.25.3 v1.24.7 v1.23.13 v1.22.15 v1.21.14]          +--skip-cleanupSkip cleanup of temporary files created during parallel run. +--debug-resourcesWhether to show resource information while running in parallel. +--include-success-outputsWhether to include outputs of successful parallel runs (skipped by default). +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ diff --git a/images/breeze/output_k8s_delete-cluster.svg b/images/breeze/output_k8s_delete-cluster.svg index bbf2b9935f4dc..8c3030e532d47 100644 --- a/images/breeze/output_k8s_delete-cluster.svg +++ b/images/breeze/output_k8s_delete-cluster.svg @@ -19,113 +19,113 @@ font-weight: 700; } - .terminal-2211177044-matrix { + .breeze-k8s-delete-cluster-matrix { font-family: Fira Code, monospace; font-size: 20px; line-height: 24.4px; font-variant-east-asian: full-width; } - .terminal-2211177044-title { + .breeze-k8s-delete-cluster-title { font-size: 18px; font-weight: bold; font-family: arial; } - .terminal-2211177044-r1 { fill: #c5c8c6;font-weight: bold } -.terminal-2211177044-r2 { fill: #c5c8c6 } -.terminal-2211177044-r3 { fill: #d0b344;font-weight: bold } -.terminal-2211177044-r4 { fill: #68a0b3;font-weight: bold } -.terminal-2211177044-r5 { fill: #868887 } -.terminal-2211177044-r6 { fill: #98a84b;font-weight: bold } -.terminal-2211177044-r7 { fill: #8d7b39 } + .breeze-k8s-delete-cluster-r1 { fill: #c5c8c6;font-weight: bold } +.breeze-k8s-delete-cluster-r2 { fill: #c5c8c6 } +.breeze-k8s-delete-cluster-r3 { fill: #d0b344;font-weight: bold } +.breeze-k8s-delete-cluster-r4 { fill: #68a0b3;font-weight: bold } +.breeze-k8s-delete-cluster-r5 { fill: #868887 } +.breeze-k8s-delete-cluster-r6 { fill: #98a84b;font-weight: bold } +.breeze-k8s-delete-cluster-r7 { fill: #8d7b39 } - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - Command: k8s delete-cluster + Command: k8s delete-cluster - + - - -Usage: breeze k8s delete-cluster [OPTIONS] - -Delete the current KinD Cluster (optionally all clusters). - -╭─ K8S cluster delete flags ───────────────────────────────────────────────────────────────────────────────────────────╮ ---python-pPython major/minor version used in Airflow image for images.(>3.7< | 3.8 | 3.9 | 3.10) -[default: 3.7]                                               ---kubernetes-versionKubernetes version used to create the KinD cluster of. -(>v1.25.3< | v1.24.7 | v1.23.13 | v1.22.15 | v1.21.14) -[default: v1.25.3]                                     ---allApply it to all created clusters -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---verbose-vPrint verbose information about performed steps. ---dry-run-DIf dry-run is set, commands are only printed, not executed. ---help-hShow this message and exit. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + +Usage: breeze k8s delete-cluster [OPTIONS] + +Delete the current KinD Cluster (optionally all clusters). + +╭─ K8S cluster delete flags ───────────────────────────────────────────────────────────────────────────────────────────╮ +--python-pPython major/minor version used in Airflow image for images.(>3.7< | 3.8 | 3.9 | 3.10) +[default: 3.7]                                               +--kubernetes-versionKubernetes version used to create the KinD cluster of. +(>v1.25.3< | v1.24.7 | v1.23.13 | v1.22.15 | v1.21.14) +[default: v1.25.3]                                     +--allApply it to all created clusters +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ diff --git a/images/breeze/output_k8s_deploy-airflow.svg b/images/breeze/output_k8s_deploy-airflow.svg index b84dadd4e705c..3c40384e05806 100644 --- a/images/breeze/output_k8s_deploy-airflow.svg +++ b/images/breeze/output_k8s_deploy-airflow.svg @@ -19,185 +19,185 @@ font-weight: 700; } - .terminal-2688862534-matrix { + .breeze-k8s-deploy-airflow-matrix { font-family: Fira Code, monospace; font-size: 20px; line-height: 24.4px; font-variant-east-asian: full-width; } - .terminal-2688862534-title { + .breeze-k8s-deploy-airflow-title { font-size: 18px; font-weight: bold; font-family: arial; } - .terminal-2688862534-r1 { fill: #c5c8c6;font-weight: bold } -.terminal-2688862534-r2 { fill: #c5c8c6 } -.terminal-2688862534-r3 { fill: #d0b344;font-weight: bold } -.terminal-2688862534-r4 { fill: #68a0b3;font-weight: bold } -.terminal-2688862534-r5 { fill: #868887 } -.terminal-2688862534-r6 { fill: #98a84b;font-weight: bold } -.terminal-2688862534-r7 { fill: #8d7b39 } + .breeze-k8s-deploy-airflow-r1 { fill: #c5c8c6;font-weight: bold } +.breeze-k8s-deploy-airflow-r2 { fill: #c5c8c6 } +.breeze-k8s-deploy-airflow-r3 { fill: #d0b344;font-weight: bold } +.breeze-k8s-deploy-airflow-r4 { fill: #68a0b3;font-weight: bold } +.breeze-k8s-deploy-airflow-r5 { fill: #868887 } +.breeze-k8s-deploy-airflow-r6 { fill: #98a84b;font-weight: bold } +.breeze-k8s-deploy-airflow-r7 { fill: #8d7b39 } - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - Command: k8s deploy-airflow + Command: k8s deploy-airflow - + - - -Usage: breeze k8s deploy-airflow [OPTIONS] [EXTRA_OPTIONS]... - -Deploy airflow image to the current KinD cluster (or all clusters). - -╭─ Airflow deploy flags ───────────────────────────────────────────────────────────────────────────────────────────────╮ ---python-pPython major/minor version used in Airflow image for images.(>3.7< | 3.8 | 3.9 | 3.10) -[default: 3.7]                                               ---kubernetes-versionKubernetes version used to create the KinD cluster of. -(>v1.25.3< | v1.24.7 | v1.23.13 | v1.22.15 | v1.21.14) -[default: v1.25.3]                                     ---executorExecutor to use for a kubernetes cluster.                                          -(>KubernetesExecutor< | CeleryExecutor | LocalExecutor | CeleryKubernetesExecutor) -[default: KubernetesExecutor]                                                      ---upgradeUpgrade Helm Chart rather than installing it. ---wait-time-in-secondsWait for Airflow webserver for specified number of seconds.(INTEGER RANGE) -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Parallel options ───────────────────────────────────────────────────────────────────────────────────────────────────╮ ---run-in-parallelRun the operation in parallel on all or selected subset of Python versions. ---parallelismMaximum number of processes to use while running the operation in parallel for cluster  -operations.                                                                             -(INTEGER RANGE)                                                                         -[default: 2; 1<=x<=4]                                                                   ---python-versionsSpace separated list of python versions used for build with multiple versions.(TEXT) -[default: 3.7 3.8 3.9 3.10]                                                    ---kubernetes-versionsKubernetes versions used to run in parallel (space separated).(TEXT) -[default: v1.25.3 v1.24.7 v1.23.13 v1.22.15 v1.21.14]          ---skip-cleanupSkip cleanup of temporary files created during parallel run. ---debug-resourcesWhether to show resource information while running in parallel. ---include-success-outputsWhether to include outputs of successful parallel runs (skipped by default). -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---verbose-vPrint verbose information about performed steps. ---dry-run-DIf dry-run is set, commands are only printed, not executed. ---help-hShow this message and exit. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + +Usage: breeze k8s deploy-airflow [OPTIONS] [EXTRA_OPTIONS]... + +Deploy airflow image to the current KinD cluster (or all clusters). + +╭─ Airflow deploy flags ───────────────────────────────────────────────────────────────────────────────────────────────╮ +--python-pPython major/minor version used in Airflow image for images.(>3.7< | 3.8 | 3.9 | 3.10) +[default: 3.7]                                               +--kubernetes-versionKubernetes version used to create the KinD cluster of. +(>v1.25.3< | v1.24.7 | v1.23.13 | v1.22.15 | v1.21.14) +[default: v1.25.3]                                     +--executorExecutor to use for a kubernetes cluster.                                          +(>KubernetesExecutor< | CeleryExecutor | LocalExecutor | CeleryKubernetesExecutor) +[default: KubernetesExecutor]                                                      +--upgradeUpgrade Helm Chart rather than installing it. +--wait-time-in-secondsWait for Airflow webserver for specified number of seconds.(INTEGER RANGE) +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Parallel options ───────────────────────────────────────────────────────────────────────────────────────────────────╮ +--run-in-parallelRun the operation in parallel on all or selected subset of Python versions. +--parallelismMaximum number of processes to use while running the operation in parallel for cluster  +operations.                                                                             +(INTEGER RANGE)                                                                         +[default: 2; 1<=x<=4]                                                                   +--python-versionsSpace separated list of python versions used for build with multiple versions.(TEXT) +[default: 3.7 3.8 3.9 3.10]                                                    +--kubernetes-versionsKubernetes versions used to run in parallel (space separated).(TEXT) +[default: v1.25.3 v1.24.7 v1.23.13 v1.22.15 v1.21.14]          +--skip-cleanupSkip cleanup of temporary files created during parallel run. +--debug-resourcesWhether to show resource information while running in parallel. +--include-success-outputsWhether to include outputs of successful parallel runs (skipped by default). +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ diff --git a/images/breeze/output_k8s_k9s.svg b/images/breeze/output_k8s_k9s.svg index 381bb073de25a..772cb71cc0476 100644 --- a/images/breeze/output_k8s_k9s.svg +++ b/images/breeze/output_k8s_k9s.svg @@ -19,109 +19,109 @@ font-weight: 700; } - .terminal-4047163396-matrix { + .breeze-k8s-k9s-matrix { font-family: Fira Code, monospace; font-size: 20px; line-height: 24.4px; font-variant-east-asian: full-width; } - .terminal-4047163396-title { + .breeze-k8s-k9s-title { font-size: 18px; font-weight: bold; font-family: arial; } - .terminal-4047163396-r1 { fill: #c5c8c6;font-weight: bold } -.terminal-4047163396-r2 { fill: #c5c8c6 } -.terminal-4047163396-r3 { fill: #d0b344;font-weight: bold } -.terminal-4047163396-r4 { fill: #68a0b3;font-weight: bold } -.terminal-4047163396-r5 { fill: #868887 } -.terminal-4047163396-r6 { fill: #98a84b;font-weight: bold } -.terminal-4047163396-r7 { fill: #8d7b39 } + .breeze-k8s-k9s-r1 { fill: #c5c8c6;font-weight: bold } +.breeze-k8s-k9s-r2 { fill: #c5c8c6 } +.breeze-k8s-k9s-r3 { fill: #d0b344;font-weight: bold } +.breeze-k8s-k9s-r4 { fill: #68a0b3;font-weight: bold } +.breeze-k8s-k9s-r5 { fill: #868887 } +.breeze-k8s-k9s-r6 { fill: #98a84b;font-weight: bold } +.breeze-k8s-k9s-r7 { fill: #8d7b39 } - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - Command: k8s k9s + Command: k8s k9s - + - - -Usage: breeze k8s k9s [OPTIONS] [K9S_ARGS]... - -Run k9s tool. You can pass any k9s args as extra args. - -╭─ K8S k9s flags ──────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---python-pPython major/minor version used in Airflow image for images.(>3.7< | 3.8 | 3.9 | 3.10) -[default: 3.7]                                               ---kubernetes-versionKubernetes version used to create the KinD cluster of. -(>v1.25.3< | v1.24.7 | v1.23.13 | v1.22.15 | v1.21.14) -[default: v1.25.3]                                     -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---verbose-vPrint verbose information about performed steps. ---dry-run-DIf dry-run is set, commands are only printed, not executed. ---help-hShow this message and exit. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + +Usage: breeze k8s k9s [OPTIONS] [K9S_ARGS]... + +Run k9s tool. You can pass any k9s args as extra args. + +╭─ K8S k9s flags ──────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--python-pPython major/minor version used in Airflow image for images.(>3.7< | 3.8 | 3.9 | 3.10) +[default: 3.7]                                               +--kubernetes-versionKubernetes version used to create the KinD cluster of. +(>v1.25.3< | v1.24.7 | v1.23.13 | v1.22.15 | v1.21.14) +[default: v1.25.3]                                     +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ diff --git a/images/breeze/output_k8s_logs.svg b/images/breeze/output_k8s_logs.svg index e607df69d52c4..3b48f4f148b1b 100644 --- a/images/breeze/output_k8s_logs.svg +++ b/images/breeze/output_k8s_logs.svg @@ -19,113 +19,113 @@ font-weight: 700; } - .terminal-722465535-matrix { + .breeze-k8s-logs-matrix { font-family: Fira Code, monospace; font-size: 20px; line-height: 24.4px; font-variant-east-asian: full-width; } - .terminal-722465535-title { + .breeze-k8s-logs-title { font-size: 18px; font-weight: bold; font-family: arial; } - .terminal-722465535-r1 { fill: #c5c8c6;font-weight: bold } -.terminal-722465535-r2 { fill: #c5c8c6 } -.terminal-722465535-r3 { fill: #d0b344;font-weight: bold } -.terminal-722465535-r4 { fill: #68a0b3;font-weight: bold } -.terminal-722465535-r5 { fill: #868887 } -.terminal-722465535-r6 { fill: #98a84b;font-weight: bold } -.terminal-722465535-r7 { fill: #8d7b39 } + .breeze-k8s-logs-r1 { fill: #c5c8c6;font-weight: bold } +.breeze-k8s-logs-r2 { fill: #c5c8c6 } +.breeze-k8s-logs-r3 { fill: #d0b344;font-weight: bold } +.breeze-k8s-logs-r4 { fill: #68a0b3;font-weight: bold } +.breeze-k8s-logs-r5 { fill: #868887 } +.breeze-k8s-logs-r6 { fill: #98a84b;font-weight: bold } +.breeze-k8s-logs-r7 { fill: #8d7b39 } - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - Command: k8s logs + Command: k8s logs - + - - -Usage: breeze k8s logs [OPTIONS] - -Dump k8s logs to ${TMP_DIR}/kind_logs_<cluster_name> directory (optionally all clusters). - -╭─ K8S logs flags ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---python-pPython major/minor version used in Airflow image for images.(>3.7< | 3.8 | 3.9 | 3.10) -[default: 3.7]                                               ---kubernetes-versionKubernetes version used to create the KinD cluster of. -(>v1.25.3< | v1.24.7 | v1.23.13 | v1.22.15 | v1.21.14) -[default: v1.25.3]                                     ---allApply it to all created clusters -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---verbose-vPrint verbose information about performed steps. ---dry-run-DIf dry-run is set, commands are only printed, not executed. ---help-hShow this message and exit. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + +Usage: breeze k8s logs [OPTIONS] + +Dump k8s logs to ${TMP_DIR}/kind_logs_<cluster_name> directory (optionally all clusters). + +╭─ K8S logs flags ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--python-pPython major/minor version used in Airflow image for images.(>3.7< | 3.8 | 3.9 | 3.10) +[default: 3.7]                                               +--kubernetes-versionKubernetes version used to create the KinD cluster of. +(>v1.25.3< | v1.24.7 | v1.23.13 | v1.22.15 | v1.21.14) +[default: v1.25.3]                                     +--allApply it to all created clusters +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ diff --git a/images/breeze/output_k8s_run-complete-tests.svg b/images/breeze/output_k8s_run-complete-tests.svg index ecd60284f3ff3..69086cf166733 100644 --- a/images/breeze/output_k8s_run-complete-tests.svg +++ b/images/breeze/output_k8s_run-complete-tests.svg @@ -19,229 +19,229 @@ font-weight: 700; } - .terminal-394306920-matrix { + .breeze-k8s-run-complete-tests-matrix { font-family: Fira Code, monospace; font-size: 20px; line-height: 24.4px; font-variant-east-asian: full-width; } - .terminal-394306920-title { + .breeze-k8s-run-complete-tests-title { font-size: 18px; font-weight: bold; font-family: arial; } - .terminal-394306920-r1 { fill: #c5c8c6;font-weight: bold } -.terminal-394306920-r2 { fill: #c5c8c6 } -.terminal-394306920-r3 { fill: #d0b344;font-weight: bold } -.terminal-394306920-r4 { fill: #68a0b3;font-weight: bold } -.terminal-394306920-r5 { fill: #868887 } -.terminal-394306920-r6 { fill: #8d7b39 } -.terminal-394306920-r7 { fill: #98a84b;font-weight: bold } + .breeze-k8s-run-complete-tests-r1 { fill: #c5c8c6;font-weight: bold } +.breeze-k8s-run-complete-tests-r2 { fill: #c5c8c6 } +.breeze-k8s-run-complete-tests-r3 { fill: #d0b344;font-weight: bold } +.breeze-k8s-run-complete-tests-r4 { fill: #68a0b3;font-weight: bold } +.breeze-k8s-run-complete-tests-r5 { fill: #868887 } +.breeze-k8s-run-complete-tests-r6 { fill: #8d7b39 } +.breeze-k8s-run-complete-tests-r7 { fill: #98a84b;font-weight: bold } - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - Command: k8s run-complete-tests + Command: k8s run-complete-tests - + - - -Usage: breeze k8s run-complete-tests [OPTIONS] [TEST_ARGS]... - -Run complete k8s tests consisting of: creating cluster, building and uploading image, deploying airflow, running tests -and deleting clusters (optionally for all clusters in parallel). - -╭─ K8S cluster creation flags ─────────────────────────────────────────────────────────────────────────────────────────╮ ---force-recreate-clusterForce recreation of the cluster even if it is already created. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Airflow deploy flags ───────────────────────────────────────────────────────────────────────────────────────────────╮ ---upgradeUpgrade Helm Chart rather than installing it. ---wait-time-in-secondsWait for Airflow webserver for specified number of seconds.(INTEGER RANGE) -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Build image flags ──────────────────────────────────────────────────────────────────────────────────────────────────╮ ---rebuild-base-imageRebuilds base Airflow image before building K8S image. ---image-tag-tImage tag used to build K8S image from.(TEXT)[default: latest] -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ K8S tests flags ────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---python-pPython major/minor version used in Airflow image for images.(>3.7< | 3.8 | 3.9 | 3.10) -[default: 3.7]                                               ---kubernetes-versionKubernetes version used to create the KinD cluster of. -(>v1.25.3< | v1.24.7 | v1.23.13 | v1.22.15 | v1.21.14) -[default: v1.25.3]                                     ---executorExecutor to use for a kubernetes cluster.                                          -(>KubernetesExecutor< | CeleryExecutor | LocalExecutor | CeleryKubernetesExecutor) -[default: KubernetesExecutor]                                                      ---force-venv-setupForce recreation of the virtualenv. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Parallel options ───────────────────────────────────────────────────────────────────────────────────────────────────╮ ---run-in-parallelRun the operation in parallel on all or selected subset of Python versions. ---parallelismMaximum number of processes to use while running the operation in parallel for cluster  -operations.                                                                             -(INTEGER RANGE)                                                                         -[default: 2; 1<=x<=4]                                                                   ---python-versionsSpace separated list of python versions used for build with multiple versions.(TEXT) -[default: 3.7 3.8 3.9 3.10]                                                    ---kubernetes-versionsKubernetes versions used to run in parallel (space separated).(TEXT) -[default: v1.25.3 v1.24.7 v1.23.13 v1.22.15 v1.21.14]          ---skip-cleanupSkip cleanup of temporary files created during parallel run. ---debug-resourcesWhether to show resource information while running in parallel. ---include-success-outputsWhether to include outputs of successful parallel runs (skipped by default). -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---verbose-vPrint verbose information about performed steps. ---dry-run-DIf dry-run is set, commands are only printed, not executed. ---help-hShow this message and exit. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + +Usage: breeze k8s run-complete-tests [OPTIONS] [TEST_ARGS]... + +Run complete k8s tests consisting of: creating cluster, building and uploading image, deploying airflow, running tests +and deleting clusters (optionally for all clusters in parallel). + +╭─ K8S cluster creation flags ─────────────────────────────────────────────────────────────────────────────────────────╮ +--force-recreate-clusterForce recreation of the cluster even if it is already created. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Airflow deploy flags ───────────────────────────────────────────────────────────────────────────────────────────────╮ +--upgradeUpgrade Helm Chart rather than installing it. +--wait-time-in-secondsWait for Airflow webserver for specified number of seconds.(INTEGER RANGE) +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Build image flags ──────────────────────────────────────────────────────────────────────────────────────────────────╮ +--rebuild-base-imageRebuilds base Airflow image before building K8S image. +--image-tag-tImage tag used to build K8S image from.(TEXT)[default: latest] +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ K8S tests flags ────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--python-pPython major/minor version used in Airflow image for images.(>3.7< | 3.8 | 3.9 | 3.10) +[default: 3.7]                                               +--kubernetes-versionKubernetes version used to create the KinD cluster of. +(>v1.25.3< | v1.24.7 | v1.23.13 | v1.22.15 | v1.21.14) +[default: v1.25.3]                                     +--executorExecutor to use for a kubernetes cluster.                                          +(>KubernetesExecutor< | CeleryExecutor | LocalExecutor | CeleryKubernetesExecutor) +[default: KubernetesExecutor]                                                      +--force-venv-setupForce recreation of the virtualenv. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Parallel options ───────────────────────────────────────────────────────────────────────────────────────────────────╮ +--run-in-parallelRun the operation in parallel on all or selected subset of Python versions. +--parallelismMaximum number of processes to use while running the operation in parallel for cluster  +operations.                                                                             +(INTEGER RANGE)                                                                         +[default: 2; 1<=x<=4]                                                                   +--python-versionsSpace separated list of python versions used for build with multiple versions.(TEXT) +[default: 3.7 3.8 3.9 3.10]                                                    +--kubernetes-versionsKubernetes versions used to run in parallel (space separated).(TEXT) +[default: v1.25.3 v1.24.7 v1.23.13 v1.22.15 v1.21.14]          +--skip-cleanupSkip cleanup of temporary files created during parallel run. +--debug-resourcesWhether to show resource information while running in parallel. +--include-success-outputsWhether to include outputs of successful parallel runs (skipped by default). +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ diff --git a/images/breeze/output_k8s_setup-env.svg b/images/breeze/output_k8s_setup-env.svg index ce2841fe3e1fd..3f440cb7551c1 100644 --- a/images/breeze/output_k8s_setup-env.svg +++ b/images/breeze/output_k8s_setup-env.svg @@ -19,92 +19,92 @@ font-weight: 700; } - .terminal-1615749758-matrix { + .breeze-k8s-setup-env-matrix { font-family: Fira Code, monospace; font-size: 20px; line-height: 24.4px; font-variant-east-asian: full-width; } - .terminal-1615749758-title { + .breeze-k8s-setup-env-title { font-size: 18px; font-weight: bold; font-family: arial; } - .terminal-1615749758-r1 { fill: #c5c8c6;font-weight: bold } -.terminal-1615749758-r2 { fill: #c5c8c6 } -.terminal-1615749758-r3 { fill: #d0b344;font-weight: bold } -.terminal-1615749758-r4 { fill: #868887 } -.terminal-1615749758-r5 { fill: #68a0b3;font-weight: bold } -.terminal-1615749758-r6 { fill: #98a84b;font-weight: bold } + .breeze-k8s-setup-env-r1 { fill: #c5c8c6;font-weight: bold } +.breeze-k8s-setup-env-r2 { fill: #c5c8c6 } +.breeze-k8s-setup-env-r3 { fill: #d0b344;font-weight: bold } +.breeze-k8s-setup-env-r4 { fill: #68a0b3;font-weight: bold } +.breeze-k8s-setup-env-r5 { fill: #868887 } +.breeze-k8s-setup-env-r6 { fill: #98a84b;font-weight: bold } - + - + - + - + - + - + - + - + - + - + - + - + - + - Command: k8s setup-env + Command: k8s setup-env - + - - -Usage: breeze k8s setup-env [OPTIONS] - -Setup shared Kubernetes virtual environment and tools. - -╭─ K8S setup flags ────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---force-venv-setupForce recreation of the virtualenv. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---verbose-vPrint verbose information about performed steps. ---dry-run-DIf dry-run is set, commands are only printed, not executed. ---help-hShow this message and exit. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + +Usage: breeze k8s setup-env [OPTIONS] + +Setup shared Kubernetes virtual environment and tools. + +╭─ K8S setup flags ────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--force-venv-setupForce recreation of the virtualenv. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ diff --git a/images/breeze/output_k8s_shell.svg b/images/breeze/output_k8s_shell.svg index 6deb7b071b343..f9b6df7f36daf 100644 --- a/images/breeze/output_k8s_shell.svg +++ b/images/breeze/output_k8s_shell.svg @@ -19,125 +19,125 @@ font-weight: 700; } - .terminal-2720092559-matrix { + .breeze-k8s-shell-matrix { font-family: Fira Code, monospace; font-size: 20px; line-height: 24.4px; font-variant-east-asian: full-width; } - .terminal-2720092559-title { + .breeze-k8s-shell-title { font-size: 18px; font-weight: bold; font-family: arial; } - .terminal-2720092559-r1 { fill: #c5c8c6;font-weight: bold } -.terminal-2720092559-r2 { fill: #c5c8c6 } -.terminal-2720092559-r3 { fill: #d0b344;font-weight: bold } -.terminal-2720092559-r4 { fill: #68a0b3;font-weight: bold } -.terminal-2720092559-r5 { fill: #868887 } -.terminal-2720092559-r6 { fill: #98a84b;font-weight: bold } -.terminal-2720092559-r7 { fill: #8d7b39 } + .breeze-k8s-shell-r1 { fill: #c5c8c6;font-weight: bold } +.breeze-k8s-shell-r2 { fill: #c5c8c6 } +.breeze-k8s-shell-r3 { fill: #d0b344;font-weight: bold } +.breeze-k8s-shell-r4 { fill: #68a0b3;font-weight: bold } +.breeze-k8s-shell-r5 { fill: #868887 } +.breeze-k8s-shell-r6 { fill: #98a84b;font-weight: bold } +.breeze-k8s-shell-r7 { fill: #8d7b39 } - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - Command: k8s shell + Command: k8s shell - + - - -Usage: breeze k8s shell [OPTIONS] [SHELL_ARGS]... - -Run shell environment for the current KinD cluster. - -╭─ K8S shell flags ────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---python-pPython major/minor version used in Airflow image for images.(>3.7< | 3.8 | 3.9 | 3.10) -[default: 3.7]                                               ---kubernetes-versionKubernetes version used to create the KinD cluster of. -(>v1.25.3< | v1.24.7 | v1.23.13 | v1.22.15 | v1.21.14) -[default: v1.25.3]                                     ---executorExecutor to use for a kubernetes cluster.                                          -(>KubernetesExecutor< | CeleryExecutor | LocalExecutor | CeleryKubernetesExecutor) -[default: KubernetesExecutor]                                                      ---force-venv-setupForce recreation of the virtualenv. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---verbose-vPrint verbose information about performed steps. ---dry-run-DIf dry-run is set, commands are only printed, not executed. ---help-hShow this message and exit. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + +Usage: breeze k8s shell [OPTIONS] [SHELL_ARGS]... + +Run shell environment for the current KinD cluster. + +╭─ K8S shell flags ────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--python-pPython major/minor version used in Airflow image for images.(>3.7< | 3.8 | 3.9 | 3.10) +[default: 3.7]                                               +--kubernetes-versionKubernetes version used to create the KinD cluster of. +(>v1.25.3< | v1.24.7 | v1.23.13 | v1.22.15 | v1.21.14) +[default: v1.25.3]                                     +--executorExecutor to use for a kubernetes cluster.                                          +(>KubernetesExecutor< | CeleryExecutor | LocalExecutor | CeleryKubernetesExecutor) +[default: KubernetesExecutor]                                                      +--force-venv-setupForce recreation of the virtualenv. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ diff --git a/images/breeze/output_k8s_status.svg b/images/breeze/output_k8s_status.svg index 6cb60c8ea3184..fe4c7c736c848 100644 --- a/images/breeze/output_k8s_status.svg +++ b/images/breeze/output_k8s_status.svg @@ -19,117 +19,117 @@ font-weight: 700; } - .terminal-3305639447-matrix { + .breeze-k8s-status-matrix { font-family: Fira Code, monospace; font-size: 20px; line-height: 24.4px; font-variant-east-asian: full-width; } - .terminal-3305639447-title { + .breeze-k8s-status-title { font-size: 18px; font-weight: bold; font-family: arial; } - .terminal-3305639447-r1 { fill: #c5c8c6;font-weight: bold } -.terminal-3305639447-r2 { fill: #c5c8c6 } -.terminal-3305639447-r3 { fill: #d0b344;font-weight: bold } -.terminal-3305639447-r4 { fill: #68a0b3;font-weight: bold } -.terminal-3305639447-r5 { fill: #868887 } -.terminal-3305639447-r6 { fill: #98a84b;font-weight: bold } -.terminal-3305639447-r7 { fill: #8d7b39 } + .breeze-k8s-status-r1 { fill: #c5c8c6;font-weight: bold } +.breeze-k8s-status-r2 { fill: #c5c8c6 } +.breeze-k8s-status-r3 { fill: #d0b344;font-weight: bold } +.breeze-k8s-status-r4 { fill: #68a0b3;font-weight: bold } +.breeze-k8s-status-r5 { fill: #868887 } +.breeze-k8s-status-r6 { fill: #98a84b;font-weight: bold } +.breeze-k8s-status-r7 { fill: #8d7b39 } - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - Command: k8s status + Command: k8s status - + - - -Usage: breeze k8s status [OPTIONS] - -Check status of the current cluster and airflow deployed to it (optionally all clusters). - -╭─ K8S cluster status flags ───────────────────────────────────────────────────────────────────────────────────────────╮ ---python-pPython major/minor version used in Airflow image for images.(>3.7< | 3.8 | 3.9 | 3.10) -[default: 3.7]                                               ---kubernetes-versionKubernetes version used to create the KinD cluster of. -(>v1.25.3< | v1.24.7 | v1.23.13 | v1.22.15 | v1.21.14) -[default: v1.25.3]                                     ---wait-time-in-secondsWait for Airflow webserver for specified number of seconds.(INTEGER RANGE) ---allApply it to all created clusters -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---verbose-vPrint verbose information about performed steps. ---dry-run-DIf dry-run is set, commands are only printed, not executed. ---help-hShow this message and exit. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + +Usage: breeze k8s status [OPTIONS] + +Check status of the current cluster and airflow deployed to it (optionally all clusters). + +╭─ K8S cluster status flags ───────────────────────────────────────────────────────────────────────────────────────────╮ +--python-pPython major/minor version used in Airflow image for images.(>3.7< | 3.8 | 3.9 | 3.10) +[default: 3.7]                                               +--kubernetes-versionKubernetes version used to create the KinD cluster of. +(>v1.25.3< | v1.24.7 | v1.23.13 | v1.22.15 | v1.21.14) +[default: v1.25.3]                                     +--wait-time-in-secondsWait for Airflow webserver for specified number of seconds.(INTEGER RANGE) +--allApply it to all created clusters +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ diff --git a/images/breeze/output_k8s_tests.svg b/images/breeze/output_k8s_tests.svg index b0f9829cb3e73..22a86166addf6 100644 --- a/images/breeze/output_k8s_tests.svg +++ b/images/breeze/output_k8s_tests.svg @@ -19,181 +19,181 @@ font-weight: 700; } - .terminal-3589318315-matrix { + .breeze-k8s-tests-matrix { font-family: Fira Code, monospace; font-size: 20px; line-height: 24.4px; font-variant-east-asian: full-width; } - .terminal-3589318315-title { + .breeze-k8s-tests-title { font-size: 18px; font-weight: bold; font-family: arial; } - .terminal-3589318315-r1 { fill: #c5c8c6;font-weight: bold } -.terminal-3589318315-r2 { fill: #c5c8c6 } -.terminal-3589318315-r3 { fill: #d0b344;font-weight: bold } -.terminal-3589318315-r4 { fill: #68a0b3;font-weight: bold } -.terminal-3589318315-r5 { fill: #868887 } -.terminal-3589318315-r6 { fill: #98a84b;font-weight: bold } -.terminal-3589318315-r7 { fill: #8d7b39 } + .breeze-k8s-tests-r1 { fill: #c5c8c6;font-weight: bold } +.breeze-k8s-tests-r2 { fill: #c5c8c6 } +.breeze-k8s-tests-r3 { fill: #d0b344;font-weight: bold } +.breeze-k8s-tests-r4 { fill: #68a0b3;font-weight: bold } +.breeze-k8s-tests-r5 { fill: #868887 } +.breeze-k8s-tests-r6 { fill: #98a84b;font-weight: bold } +.breeze-k8s-tests-r7 { fill: #8d7b39 } - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - Command: k8s tests + Command: k8s tests - + - - -Usage: breeze k8s tests [OPTIONS] [TEST_ARGS]... - -Run tests against the current KinD cluster (optionally for all clusters in parallel). - -╭─ K8S tests flags ────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---python-pPython major/minor version used in Airflow image for images.(>3.7< | 3.8 | 3.9 | 3.10) -[default: 3.7]                                               ---kubernetes-versionKubernetes version used to create the KinD cluster of. -(>v1.25.3< | v1.24.7 | v1.23.13 | v1.22.15 | v1.21.14) -[default: v1.25.3]                                     ---executorExecutor to use for a kubernetes cluster.                                          -(>KubernetesExecutor< | CeleryExecutor | LocalExecutor | CeleryKubernetesExecutor) -[default: KubernetesExecutor]                                                      ---force-venv-setupForce recreation of the virtualenv. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Parallel options ───────────────────────────────────────────────────────────────────────────────────────────────────╮ ---run-in-parallelRun the operation in parallel on all or selected subset of Python versions. ---parallelismMaximum number of processes to use while running the operation in parallel for cluster  -operations.                                                                             -(INTEGER RANGE)                                                                         -[default: 2; 1<=x<=4]                                                                   ---python-versionsSpace separated list of python versions used for build with multiple versions.(TEXT) -[default: 3.7 3.8 3.9 3.10]                                                    ---kubernetes-versionsKubernetes versions used to run in parallel (space separated).(TEXT) -[default: v1.25.3 v1.24.7 v1.23.13 v1.22.15 v1.21.14]          ---skip-cleanupSkip cleanup of temporary files created during parallel run. ---debug-resourcesWhether to show resource information while running in parallel. ---include-success-outputsWhether to include outputs of successful parallel runs (skipped by default). -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---verbose-vPrint verbose information about performed steps. ---dry-run-DIf dry-run is set, commands are only printed, not executed. ---help-hShow this message and exit. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + +Usage: breeze k8s tests [OPTIONS] [TEST_ARGS]... + +Run tests against the current KinD cluster (optionally for all clusters in parallel). + +╭─ K8S tests flags ────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--python-pPython major/minor version used in Airflow image for images.(>3.7< | 3.8 | 3.9 | 3.10) +[default: 3.7]                                               +--kubernetes-versionKubernetes version used to create the KinD cluster of. +(>v1.25.3< | v1.24.7 | v1.23.13 | v1.22.15 | v1.21.14) +[default: v1.25.3]                                     +--executorExecutor to use for a kubernetes cluster.                                          +(>KubernetesExecutor< | CeleryExecutor | LocalExecutor | CeleryKubernetesExecutor) +[default: KubernetesExecutor]                                                      +--force-venv-setupForce recreation of the virtualenv. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Parallel options ───────────────────────────────────────────────────────────────────────────────────────────────────╮ +--run-in-parallelRun the operation in parallel on all or selected subset of Python versions. +--parallelismMaximum number of processes to use while running the operation in parallel for cluster  +operations.                                                                             +(INTEGER RANGE)                                                                         +[default: 2; 1<=x<=4]                                                                   +--python-versionsSpace separated list of python versions used for build with multiple versions.(TEXT) +[default: 3.7 3.8 3.9 3.10]                                                    +--kubernetes-versionsKubernetes versions used to run in parallel (space separated).(TEXT) +[default: v1.25.3 v1.24.7 v1.23.13 v1.22.15 v1.21.14]          +--skip-cleanupSkip cleanup of temporary files created during parallel run. +--debug-resourcesWhether to show resource information while running in parallel. +--include-success-outputsWhether to include outputs of successful parallel runs (skipped by default). +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ diff --git a/images/breeze/output_k8s_upload-k8s-image.svg b/images/breeze/output_k8s_upload-k8s-image.svg index 5aeb6922939cc..5ce479c4a188f 100644 --- a/images/breeze/output_k8s_upload-k8s-image.svg +++ b/images/breeze/output_k8s_upload-k8s-image.svg @@ -19,161 +19,161 @@ font-weight: 700; } - .terminal-915737816-matrix { + .breeze-k8s-upload-k8s-image-matrix { font-family: Fira Code, monospace; font-size: 20px; line-height: 24.4px; font-variant-east-asian: full-width; } - .terminal-915737816-title { + .breeze-k8s-upload-k8s-image-title { font-size: 18px; font-weight: bold; font-family: arial; } - .terminal-915737816-r1 { fill: #c5c8c6;font-weight: bold } -.terminal-915737816-r2 { fill: #c5c8c6 } -.terminal-915737816-r3 { fill: #d0b344;font-weight: bold } -.terminal-915737816-r4 { fill: #68a0b3;font-weight: bold } -.terminal-915737816-r5 { fill: #868887 } -.terminal-915737816-r6 { fill: #98a84b;font-weight: bold } -.terminal-915737816-r7 { fill: #8d7b39 } + .breeze-k8s-upload-k8s-image-r1 { fill: #c5c8c6;font-weight: bold } +.breeze-k8s-upload-k8s-image-r2 { fill: #c5c8c6 } +.breeze-k8s-upload-k8s-image-r3 { fill: #d0b344;font-weight: bold } +.breeze-k8s-upload-k8s-image-r4 { fill: #68a0b3;font-weight: bold } +.breeze-k8s-upload-k8s-image-r5 { fill: #868887 } +.breeze-k8s-upload-k8s-image-r6 { fill: #98a84b;font-weight: bold } +.breeze-k8s-upload-k8s-image-r7 { fill: #8d7b39 } - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - Command: k8s upload-k8s-image + Command: k8s upload-k8s-image - + - - -Usage: breeze k8s upload-k8s-image [OPTIONS] - -Upload k8s-ready airflow image to the KinD cluster (optionally to all clusters in parallel) - -╭─ Upload image flags ─────────────────────────────────────────────────────────────────────────────────────────────────╮ ---python-pPython major/minor version used in Airflow image for images.(>3.7< | 3.8 | 3.9 | 3.10) -[default: 3.7]                                               ---kubernetes-versionKubernetes version used to create the KinD cluster of. -(>v1.25.3< | v1.24.7 | v1.23.13 | v1.22.15 | v1.21.14) -[default: v1.25.3]                                     -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Parallel options ───────────────────────────────────────────────────────────────────────────────────────────────────╮ ---run-in-parallelRun the operation in parallel on all or selected subset of Python versions. ---parallelismMaximum number of processes to use while running the operation in parallel. -(INTEGER RANGE)                                                             -[default: 4; 1<=x<=8]                                                       ---python-versionsSpace separated list of python versions used for build with multiple versions.(TEXT) -[default: 3.7 3.8 3.9 3.10]                                                    ---kubernetes-versionsKubernetes versions used to run in parallel (space separated).(TEXT) -[default: v1.25.3 v1.24.7 v1.23.13 v1.22.15 v1.21.14]          ---skip-cleanupSkip cleanup of temporary files created during parallel run. ---debug-resourcesWhether to show resource information while running in parallel. ---include-success-outputsWhether to include outputs of successful parallel runs (skipped by default). -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---verbose-vPrint verbose information about performed steps. ---dry-run-DIf dry-run is set, commands are only printed, not executed. ---help-hShow this message and exit. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + +Usage: breeze k8s upload-k8s-image [OPTIONS] + +Upload k8s-ready airflow image to the KinD cluster (optionally to all clusters in parallel) + +╭─ Upload image flags ─────────────────────────────────────────────────────────────────────────────────────────────────╮ +--python-pPython major/minor version used in Airflow image for images.(>3.7< | 3.8 | 3.9 | 3.10) +[default: 3.7]                                               +--kubernetes-versionKubernetes version used to create the KinD cluster of. +(>v1.25.3< | v1.24.7 | v1.23.13 | v1.22.15 | v1.21.14) +[default: v1.25.3]                                     +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Parallel options ───────────────────────────────────────────────────────────────────────────────────────────────────╮ +--run-in-parallelRun the operation in parallel on all or selected subset of Python versions. +--parallelismMaximum number of processes to use while running the operation in parallel. +(INTEGER RANGE)                                                             +[default: 4; 1<=x<=8]                                                       +--python-versionsSpace separated list of python versions used for build with multiple versions.(TEXT) +[default: 3.7 3.8 3.9 3.10]                                                    +--kubernetes-versionsKubernetes versions used to run in parallel (space separated).(TEXT) +[default: v1.25.3 v1.24.7 v1.23.13 v1.22.15 v1.21.14]          +--skip-cleanupSkip cleanup of temporary files created during parallel run. +--debug-resourcesWhether to show resource information while running in parallel. +--include-success-outputsWhether to include outputs of successful parallel runs (skipped by default). +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ diff --git a/images/breeze/output_prod-image_pull.svg b/images/breeze/output_prod-image_pull.svg index 5a0090360e7ae..a4a8028029481 100644 --- a/images/breeze/output_prod-image_pull.svg +++ b/images/breeze/output_prod-image_pull.svg @@ -19,169 +19,169 @@ font-weight: 700; } - .terminal-3280759523-matrix { + .breeze-prod-image-pull-matrix { font-family: Fira Code, monospace; font-size: 20px; line-height: 24.4px; font-variant-east-asian: full-width; } - .terminal-3280759523-title { + .breeze-prod-image-pull-title { font-size: 18px; font-weight: bold; font-family: arial; } - .terminal-3280759523-r1 { fill: #c5c8c6;font-weight: bold } -.terminal-3280759523-r2 { fill: #c5c8c6 } -.terminal-3280759523-r3 { fill: #d0b344;font-weight: bold } -.terminal-3280759523-r4 { fill: #868887 } -.terminal-3280759523-r5 { fill: #68a0b3;font-weight: bold } -.terminal-3280759523-r6 { fill: #98a84b;font-weight: bold } -.terminal-3280759523-r7 { fill: #8d7b39 } + .breeze-prod-image-pull-r1 { fill: #c5c8c6;font-weight: bold } +.breeze-prod-image-pull-r2 { fill: #c5c8c6 } +.breeze-prod-image-pull-r3 { fill: #d0b344;font-weight: bold } +.breeze-prod-image-pull-r4 { fill: #68a0b3;font-weight: bold } +.breeze-prod-image-pull-r5 { fill: #868887 } +.breeze-prod-image-pull-r6 { fill: #98a84b;font-weight: bold } +.breeze-prod-image-pull-r7 { fill: #8d7b39 } - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - Command: prod-image pull + Command: prod-image pull - + - - -Usage: breeze prod-image pull [OPTIONS] [EXTRA_PYTEST_ARGS]... - -Pull and optionally verify Production images - possibly in parallel for all Python versions. - -╭─ Pull image flags ───────────────────────────────────────────────────────────────────────────────────────────────────╮ ---image-tag-tTag of the image which is used to pull the image.(TEXT)[default: latest] ---python-pPython major/minor version used in Airflow image for images.(>3.7< | 3.8 | 3.9 | 3.10) -[default: 3.7]                                               ---github-tokenThe token used to authenticate to GitHub.(TEXT) ---verifyVerify image. ---wait-for-imageWait until image is available. ---tag-as-latestTags the image as latest and update checksum of all files after pulling. Useful when you    -build or pull image with --image-tag.                                                       ---github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Parallel running ───────────────────────────────────────────────────────────────────────────────────────────────────╮ ---run-in-parallelRun the operation in parallel on all or selected subset of Python versions. ---parallelismMaximum number of processes to use while running the operation in parallel. -(INTEGER RANGE)                                                             -[default: 4; 1<=x<=8]                                                       ---python-versionsSpace separated list of python versions used for build with multiple versions.(TEXT) -[default: 3.7 3.8 3.9 3.10]                                                    ---skip-cleanupSkip cleanup of temporary files created during parallel run. ---debug-resourcesWhether to show resource information while running in parallel. ---include-success-outputsWhether to include outputs of successful parallel runs (skipped by default). -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---verbose-vPrint verbose information about performed steps. ---dry-run-DIf dry-run is set, commands are only printed, not executed. ---help-hShow this message and exit. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + +Usage: breeze prod-image pull [OPTIONS] [EXTRA_PYTEST_ARGS]... + +Pull and optionally verify Production images - possibly in parallel for all Python versions. + +╭─ Pull image flags ───────────────────────────────────────────────────────────────────────────────────────────────────╮ +--image-tag-tTag of the image which is used to pull the image.(TEXT)[default: latest] +--python-pPython major/minor version used in Airflow image for images.(>3.7< | 3.8 | 3.9 | 3.10) +[default: 3.7]                                               +--github-tokenThe token used to authenticate to GitHub.(TEXT) +--verifyVerify image. +--wait-for-imageWait until image is available. +--tag-as-latestTags the image as latest and update checksum of all files after pulling. Useful when you    +build or pull image with --image-tag.                                                       +--github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Parallel running ───────────────────────────────────────────────────────────────────────────────────────────────────╮ +--run-in-parallelRun the operation in parallel on all or selected subset of Python versions. +--parallelismMaximum number of processes to use while running the operation in parallel. +(INTEGER RANGE)                                                             +[default: 4; 1<=x<=8]                                                       +--python-versionsSpace separated list of python versions used for build with multiple versions.(TEXT) +[default: 3.7 3.8 3.9 3.10]                                                    +--skip-cleanupSkip cleanup of temporary files created during parallel run. +--debug-resourcesWhether to show resource information while running in parallel. +--include-success-outputsWhether to include outputs of successful parallel runs (skipped by default). +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ diff --git a/images/breeze/output_prod-image_verify.svg b/images/breeze/output_prod-image_verify.svg index 9a4fbc1fcea07..9c01cdbdb6fb5 100644 --- a/images/breeze/output_prod-image_verify.svg +++ b/images/breeze/output_prod-image_verify.svg @@ -19,117 +19,117 @@ font-weight: 700; } - .terminal-6369683-matrix { + .breeze-prod-image-verify-matrix { font-family: Fira Code, monospace; font-size: 20px; line-height: 24.4px; font-variant-east-asian: full-width; } - .terminal-6369683-title { + .breeze-prod-image-verify-title { font-size: 18px; font-weight: bold; font-family: arial; } - .terminal-6369683-r1 { fill: #c5c8c6;font-weight: bold } -.terminal-6369683-r2 { fill: #c5c8c6 } -.terminal-6369683-r3 { fill: #d0b344;font-weight: bold } -.terminal-6369683-r4 { fill: #868887 } -.terminal-6369683-r5 { fill: #68a0b3;font-weight: bold } -.terminal-6369683-r6 { fill: #98a84b;font-weight: bold } -.terminal-6369683-r7 { fill: #8d7b39 } + .breeze-prod-image-verify-r1 { fill: #c5c8c6;font-weight: bold } +.breeze-prod-image-verify-r2 { fill: #c5c8c6 } +.breeze-prod-image-verify-r3 { fill: #d0b344;font-weight: bold } +.breeze-prod-image-verify-r4 { fill: #68a0b3;font-weight: bold } +.breeze-prod-image-verify-r5 { fill: #868887 } +.breeze-prod-image-verify-r6 { fill: #98a84b;font-weight: bold } +.breeze-prod-image-verify-r7 { fill: #8d7b39 } - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - Command: prod-image verify + Command: prod-image verify - + - - -Usage: breeze prod-image verify [OPTIONS] [EXTRA_PYTEST_ARGS]... - -Verify Production image. - -╭─ Verify image flags ─────────────────────────────────────────────────────────────────────────────────────────────────╮ ---image-name-nName of the image to verify (overrides --python and --image-tag).(TEXT) ---python-pPython major/minor version used in Airflow image for images.(>3.7< | 3.8 | 3.9 | 3.10) -[default: 3.7]                                               ---slim-imageThe image to verify is slim and non-slim tests should be skipped. ---image-tag-tTag of the image when verifying it.(TEXT)[default: latest] ---pullPull image is missing before attempting to verify it. ---github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---verbose-vPrint verbose information about performed steps. ---dry-run-DIf dry-run is set, commands are only printed, not executed. ---help-hShow this message and exit. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + +Usage: breeze prod-image verify [OPTIONS] [EXTRA_PYTEST_ARGS]... + +Verify Production image. + +╭─ Verify image flags ─────────────────────────────────────────────────────────────────────────────────────────────────╮ +--image-name-nName of the image to verify (overrides --python and --image-tag).(TEXT) +--python-pPython major/minor version used in Airflow image for images.(>3.7< | 3.8 | 3.9 | 3.10) +[default: 3.7]                                               +--slim-imageThe image to verify is slim and non-slim tests should be skipped. +--image-tag-tTag of the image when verifying it.(TEXT)[default: latest] +--pullPull image is missing before attempting to verify it. +--github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ diff --git a/images/breeze/output_release-management_generate-constraints.svg b/images/breeze/output_release-management_generate-constraints.svg index 35e0a01600614..f5b6bd70235c4 100644 --- a/images/breeze/output_release-management_generate-constraints.svg +++ b/images/breeze/output_release-management_generate-constraints.svg @@ -19,173 +19,173 @@ font-weight: 700; } - .terminal-3695932747-matrix { + .breeze-release-management-generate-constraints-matrix { font-family: Fira Code, monospace; font-size: 20px; line-height: 24.4px; font-variant-east-asian: full-width; } - .terminal-3695932747-title { + .breeze-release-management-generate-constraints-title { font-size: 18px; font-weight: bold; font-family: arial; } - .terminal-3695932747-r1 { fill: #c5c8c6;font-weight: bold } -.terminal-3695932747-r2 { fill: #c5c8c6 } -.terminal-3695932747-r3 { fill: #d0b344;font-weight: bold } -.terminal-3695932747-r4 { fill: #868887 } -.terminal-3695932747-r5 { fill: #68a0b3;font-weight: bold } -.terminal-3695932747-r6 { fill: #98a84b;font-weight: bold } -.terminal-3695932747-r7 { fill: #8d7b39 } + .breeze-release-management-generate-constraints-r1 { fill: #c5c8c6;font-weight: bold } +.breeze-release-management-generate-constraints-r2 { fill: #c5c8c6 } +.breeze-release-management-generate-constraints-r3 { fill: #d0b344;font-weight: bold } +.breeze-release-management-generate-constraints-r4 { fill: #68a0b3;font-weight: bold } +.breeze-release-management-generate-constraints-r5 { fill: #868887 } +.breeze-release-management-generate-constraints-r6 { fill: #98a84b;font-weight: bold } +.breeze-release-management-generate-constraints-r7 { fill: #8d7b39 } - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - Command: release-management generate-constraints + Command: release-management generate-constraints - + - - -Usage: breeze release-management generate-constraints [OPTIONS] - -Generates pinned constraint files with all extras from setup.py in parallel. - -╭─ Generate constraints flags ─────────────────────────────────────────────────────────────────────────────────────────╮ ---image-tag-tTag of the image which is used to run the image (implies --mount-sources=skip). -(TEXT)                                                                          -[default: latest]                                                               ---python-pPython major/minor version used in Airflow image for images. -(>3.7< | 3.8 | 3.9 | 3.10)                                   -[default: 3.7]                                               ---airflow-constraints-modeMode of constraints for CI image building.                              -(constraints-source-providers | constraints | constraints-no-providers) -[default: constraints-source-providers]                                 ---debugDrop user in shell instead of running the command. Useful for debugging. ---github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Parallel running ───────────────────────────────────────────────────────────────────────────────────────────────────╮ ---run-in-parallelRun the operation in parallel on all or selected subset of Python versions. ---parallelismMaximum number of processes to use while running the operation in parallel.(INTEGER RANGE) -[default: 4; 1<=x<=8]                                                       ---python-versionsSpace separated list of python versions used for build with multiple versions.(TEXT) -[default: 3.7 3.8 3.9 3.10]                                                    ---skip-cleanupSkip cleanup of temporary files created during parallel run. ---debug-resourcesWhether to show resource information while running in parallel. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---verbose-vPrint verbose information about performed steps. ---dry-run-DIf dry-run is set, commands are only printed, not executed. ---answer-aForce answer to questions.(y | n | q | yes | no | quit) ---help-hShow this message and exit. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + +Usage: breeze release-management generate-constraints [OPTIONS] + +Generates pinned constraint files with all extras from setup.py in parallel. + +╭─ Generate constraints flags ─────────────────────────────────────────────────────────────────────────────────────────╮ +--image-tag-tTag of the image which is used to run the image (implies --mount-sources=skip). +(TEXT)                                                                          +[default: latest]                                                               +--python-pPython major/minor version used in Airflow image for images. +(>3.7< | 3.8 | 3.9 | 3.10)                                   +[default: 3.7]                                               +--airflow-constraints-modeMode of constraints for CI image building.                              +(constraints-source-providers | constraints | constraints-no-providers) +[default: constraints-source-providers]                                 +--debugDrop user in shell instead of running the command. Useful for debugging. +--github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Parallel running ───────────────────────────────────────────────────────────────────────────────────────────────────╮ +--run-in-parallelRun the operation in parallel on all or selected subset of Python versions. +--parallelismMaximum number of processes to use while running the operation in parallel.(INTEGER RANGE) +[default: 4; 1<=x<=8]                                                       +--python-versionsSpace separated list of python versions used for build with multiple versions.(TEXT) +[default: 3.7 3.8 3.9 3.10]                                                    +--skip-cleanupSkip cleanup of temporary files created during parallel run. +--debug-resourcesWhether to show resource information while running in parallel. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--answer-aForce answer to questions.(y | n | q | yes | no | quit) +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ diff --git a/images/breeze/output_release-management_prepare-airflow-package.svg b/images/breeze/output_release-management_prepare-airflow-package.svg index b7126cac0c5e3..11e6d3d5064b8 100644 --- a/images/breeze/output_release-management_prepare-airflow-package.svg +++ b/images/breeze/output_release-management_prepare-airflow-package.svg @@ -19,105 +19,105 @@ font-weight: 700; } - .terminal-276230213-matrix { + .breeze-release-management-prepare-airflow-package-matrix { font-family: Fira Code, monospace; font-size: 20px; line-height: 24.4px; font-variant-east-asian: full-width; } - .terminal-276230213-title { + .breeze-release-management-prepare-airflow-package-title { font-size: 18px; font-weight: bold; font-family: arial; } - .terminal-276230213-r1 { fill: #c5c8c6;font-weight: bold } -.terminal-276230213-r2 { fill: #c5c8c6 } -.terminal-276230213-r3 { fill: #d0b344;font-weight: bold } -.terminal-276230213-r4 { fill: #868887 } -.terminal-276230213-r5 { fill: #68a0b3;font-weight: bold } -.terminal-276230213-r6 { fill: #8d7b39 } -.terminal-276230213-r7 { fill: #98a84b;font-weight: bold } + .breeze-release-management-prepare-airflow-package-r1 { fill: #c5c8c6;font-weight: bold } +.breeze-release-management-prepare-airflow-package-r2 { fill: #c5c8c6 } +.breeze-release-management-prepare-airflow-package-r3 { fill: #d0b344;font-weight: bold } +.breeze-release-management-prepare-airflow-package-r4 { fill: #68a0b3;font-weight: bold } +.breeze-release-management-prepare-airflow-package-r5 { fill: #868887 } +.breeze-release-management-prepare-airflow-package-r6 { fill: #8d7b39 } +.breeze-release-management-prepare-airflow-package-r7 { fill: #98a84b;font-weight: bold } - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - Command: release-management prepare-airflow-package + Command: release-management prepare-airflow-package - + - - -Usage: breeze release-management prepare-airflow-package [OPTIONS] - -Prepare sdist/whl package of Airflow. - -╭─ Package flags ──────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---package-formatFormat of packages.(wheel | sdist | both)[default: wheel] ---version-suffix-for-pypiVersion suffix used for PyPI packages (alpha, beta, rc1, etc.).(TEXT) ---debugDrop user in shell instead of running the command. Useful for debugging. ---github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---verbose-vPrint verbose information about performed steps. ---dry-run-DIf dry-run is set, commands are only printed, not executed. ---help-hShow this message and exit. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + +Usage: breeze release-management prepare-airflow-package [OPTIONS] + +Prepare sdist/whl package of Airflow. + +╭─ Package flags ──────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--package-formatFormat of packages.(wheel | sdist | both)[default: wheel] +--version-suffix-for-pypiVersion suffix used for PyPI packages (alpha, beta, rc1, etc.).(TEXT) +--debugDrop user in shell instead of running the command. Useful for debugging. +--github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ diff --git a/images/breeze/output_release-management_prepare-provider-documentation.svg b/images/breeze/output_release-management_prepare-provider-documentation.svg index e3c575a127a42..86dbeba6f7c5f 100644 --- a/images/breeze/output_release-management_prepare-provider-documentation.svg +++ b/images/breeze/output_release-management_prepare-provider-documentation.svg @@ -19,169 +19,169 @@ font-weight: 700; } - .terminal-3926816982-matrix { + .breeze-release-management-prepare-provider-documentation-matrix { font-family: Fira Code, monospace; font-size: 20px; line-height: 24.4px; font-variant-east-asian: full-width; } - .terminal-3926816982-title { + .breeze-release-management-prepare-provider-documentation-title { font-size: 18px; font-weight: bold; font-family: arial; } - .terminal-3926816982-r1 { fill: #c5c8c6;font-weight: bold } -.terminal-3926816982-r2 { fill: #c5c8c6 } -.terminal-3926816982-r3 { fill: #d0b344;font-weight: bold } -.terminal-3926816982-r4 { fill: #868887 } -.terminal-3926816982-r5 { fill: #68a0b3;font-weight: bold } -.terminal-3926816982-r6 { fill: #98a84b;font-weight: bold } -.terminal-3926816982-r7 { fill: #8d7b39 } + .breeze-release-management-prepare-provider-documentation-r1 { fill: #c5c8c6;font-weight: bold } +.breeze-release-management-prepare-provider-documentation-r2 { fill: #c5c8c6 } +.breeze-release-management-prepare-provider-documentation-r3 { fill: #d0b344;font-weight: bold } +.breeze-release-management-prepare-provider-documentation-r4 { fill: #68a0b3;font-weight: bold } +.breeze-release-management-prepare-provider-documentation-r5 { fill: #868887 } +.breeze-release-management-prepare-provider-documentation-r6 { fill: #98a84b;font-weight: bold } +.breeze-release-management-prepare-provider-documentation-r7 { fill: #8d7b39 } - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - Command: release-management prepare-provider-documentation + Command: release-management prepare-provider-documentation - + - - -Usage: breeze release-management prepare-provider-documentation [OPTIONS] [airbyte | alibaba | amazon | apache.beam | -                                                                apache.cassandra | apache.drill | apache.druid | -                                                                apache.hdfs | apache.hive | apache.kylin | apache.livy -                                                                | apache.pig | apache.pinot | apache.spark | -                                                                apache.sqoop | arangodb | asana | atlassian.jira | -                                                                celery | cloudant | cncf.kubernetes | common.sql | -                                                                databricks | datadog | dbt.cloud | dingding | discord -                                                                | docker | elasticsearch | exasol | facebook | ftp | -                                                                github | google | grpc | hashicorp | http | imap | -                                                                influxdb | jdbc | jenkins | microsoft.azure | -                                                                microsoft.mssql | microsoft.psrp | microsoft.winrm | -                                                                mongo | mysql | neo4j | odbc | openfaas | opsgenie | -                                                                oracle | pagerduty | papermill | plexus | postgres | -                                                                presto | qubole | redis | salesforce | samba | segment -                                                                | sendgrid | sftp | singularity | slack | snowflake | -                                                                sqlite | ssh | tableau | tabular | telegram | trino | -                                                                vertica | yandex | zendesk]... - -Prepare CHANGELOG, README and COMMITS information for providers. - -╭─ Provider documentation preparation flags ───────────────────────────────────────────────────────────────────────────╮ ---debugDrop user in shell instead of running the command. Useful for debugging. ---github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---base-branch(TEXT) ---verbose-vPrint verbose information about performed steps. ---dry-run-DIf dry-run is set, commands are only printed, not executed. ---answer-aForce answer to questions.(y | n | q | yes | no | quit) ---help-hShow this message and exit. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + +Usage: breeze release-management prepare-provider-documentation [OPTIONS] [airbyte | alibaba | amazon | apache.beam | +                                                                apache.cassandra | apache.drill | apache.druid | +                                                                apache.hdfs | apache.hive | apache.kylin | apache.livy +                                                                | apache.pig | apache.pinot | apache.spark | +                                                                apache.sqoop | arangodb | asana | atlassian.jira | +                                                                celery | cloudant | cncf.kubernetes | common.sql | +                                                                databricks | datadog | dbt.cloud | dingding | discord +                                                                | docker | elasticsearch | exasol | facebook | ftp | +                                                                github | google | grpc | hashicorp | http | imap | +                                                                influxdb | jdbc | jenkins | microsoft.azure | +                                                                microsoft.mssql | microsoft.psrp | microsoft.winrm | +                                                                mongo | mysql | neo4j | odbc | openfaas | opsgenie | +                                                                oracle | pagerduty | papermill | plexus | postgres | +                                                                presto | qubole | redis | salesforce | samba | segment +                                                                | sendgrid | sftp | singularity | slack | snowflake | +                                                                sqlite | ssh | tableau | tabular | telegram | trino | +                                                                vertica | yandex | zendesk]... + +Prepare CHANGELOGREADME and COMMITS information for providers. + +╭─ Provider documentation preparation flags ───────────────────────────────────────────────────────────────────────────╮ +--debugDrop user in shell instead of running the command. Useful for debugging. +--github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--base-branch(TEXT) +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--answer-aForce answer to questions.(y | n | q | yes | no | quit) +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ diff --git a/images/breeze/output_release-management_prepare-provider-packages.svg b/images/breeze/output_release-management_prepare-provider-packages.svg index 8db498e25121e..f09d7f0c2aa58 100644 --- a/images/breeze/output_release-management_prepare-provider-packages.svg +++ b/images/breeze/output_release-management_prepare-provider-packages.svg @@ -19,169 +19,169 @@ font-weight: 700; } - .terminal-3787835496-matrix { + .breeze-release-management-prepare-provider-packages-matrix { font-family: Fira Code, monospace; font-size: 20px; line-height: 24.4px; font-variant-east-asian: full-width; } - .terminal-3787835496-title { + .breeze-release-management-prepare-provider-packages-title { font-size: 18px; font-weight: bold; font-family: arial; } - .terminal-3787835496-r1 { fill: #c5c8c6;font-weight: bold } -.terminal-3787835496-r2 { fill: #c5c8c6 } -.terminal-3787835496-r3 { fill: #d0b344;font-weight: bold } -.terminal-3787835496-r4 { fill: #868887 } -.terminal-3787835496-r5 { fill: #68a0b3;font-weight: bold } -.terminal-3787835496-r6 { fill: #8d7b39 } -.terminal-3787835496-r7 { fill: #98a84b;font-weight: bold } + .breeze-release-management-prepare-provider-packages-r1 { fill: #c5c8c6;font-weight: bold } +.breeze-release-management-prepare-provider-packages-r2 { fill: #c5c8c6 } +.breeze-release-management-prepare-provider-packages-r3 { fill: #d0b344;font-weight: bold } +.breeze-release-management-prepare-provider-packages-r4 { fill: #68a0b3;font-weight: bold } +.breeze-release-management-prepare-provider-packages-r5 { fill: #868887 } +.breeze-release-management-prepare-provider-packages-r6 { fill: #8d7b39 } +.breeze-release-management-prepare-provider-packages-r7 { fill: #98a84b;font-weight: bold } - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - Command: release-management prepare-provider-packages + Command: release-management prepare-provider-packages - + - - -Usage: breeze release-management prepare-provider-packages [OPTIONS] [airbyte | alibaba | amazon | apache.beam | -                                                           apache.cassandra | apache.drill | apache.druid | -                                                           apache.hdfs | apache.hive | apache.kylin | apache.livy | -                                                           apache.pig | apache.pinot | apache.spark | apache.sqoop | -                                                           arangodb | asana | atlassian.jira | celery | cloudant | -                                                           cncf.kubernetes | common.sql | databricks | datadog | -                                                           dbt.cloud | dingding | discord | docker | elasticsearch | -                                                           exasol | facebook | ftp | github | google | grpc | -                                                           hashicorp | http | imap | influxdb | jdbc | jenkins | -                                                           microsoft.azure | microsoft.mssql | microsoft.psrp | -                                                           microsoft.winrm | mongo | mysql | neo4j | odbc | openfaas | -                                                           opsgenie | oracle | pagerduty | papermill | plexus | -                                                           postgres | presto | qubole | redis | salesforce | samba | -                                                           segment | sendgrid | sftp | singularity | slack | snowflake -                                                           | sqlite | ssh | tableau | tabular | telegram | trino | -                                                           vertica | yandex | zendesk]... - -Prepare sdist/whl packages of Airflow Providers. - -╭─ Package flags ──────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---package-formatFormat of packages.(wheel | sdist | both)[default: wheel] ---version-suffix-for-pypiVersion suffix used for PyPI packages (alpha, beta, rc1, etc.).(TEXT) ---package-list-fileRead list of packages from text file (one package per line).(FILENAME) ---debugDrop user in shell instead of running the command. Useful for debugging. ---github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---verbose-vPrint verbose information about performed steps. ---dry-run-DIf dry-run is set, commands are only printed, not executed. ---help-hShow this message and exit. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + +Usage: breeze release-management prepare-provider-packages [OPTIONS] [airbyte | alibaba | amazon | apache.beam | +                                                           apache.cassandra | apache.drill | apache.druid | +                                                           apache.hdfs | apache.hive | apache.kylin | apache.livy | +                                                           apache.pig | apache.pinot | apache.spark | apache.sqoop | +                                                           arangodb | asana | atlassian.jira | celery | cloudant | +                                                           cncf.kubernetes | common.sql | databricks | datadog | +                                                           dbt.cloud | dingding | discord | docker | elasticsearch | +                                                           exasol | facebook | ftp | github | google | grpc | +                                                           hashicorp | http | imap | influxdb | jdbc | jenkins | +                                                           microsoft.azure | microsoft.mssql | microsoft.psrp | +                                                           microsoft.winrm | mongo | mysql | neo4j | odbc | openfaas | +                                                           opsgenie | oracle | pagerduty | papermill | plexus | +                                                           postgres | presto | qubole | redis | salesforce | samba | +                                                           segment | sendgrid | sftp | singularity | slack | snowflake +                                                           | sqlite | ssh | tableau | tabular | telegram | trino | +                                                           vertica | yandex | zendesk]... + +Prepare sdist/whl packages of Airflow Providers. + +╭─ Package flags ──────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--package-formatFormat of packages.(wheel | sdist | both)[default: wheel] +--version-suffix-for-pypiVersion suffix used for PyPI packages (alpha, beta, rc1, etc.).(TEXT) +--package-list-fileRead list of packages from text file (one package per line).(FILENAME) +--debugDrop user in shell instead of running the command. Useful for debugging. +--github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ diff --git a/images/breeze/output_release-management_release-prod-images.svg b/images/breeze/output_release-management_release-prod-images.svg index ef945e9fd0ed2..dba5274a626d1 100644 --- a/images/breeze/output_release-management_release-prod-images.svg +++ b/images/breeze/output_release-management_release-prod-images.svg @@ -19,139 +19,139 @@ font-weight: 700; } - .terminal-2418532981-matrix { + .breeze-release-management-release-prod-images-matrix { font-family: Fira Code, monospace; font-size: 20px; line-height: 24.4px; font-variant-east-asian: full-width; } - .terminal-2418532981-title { + .breeze-release-management-release-prod-images-title { font-size: 18px; font-weight: bold; font-family: arial; } - .terminal-2418532981-r1 { fill: #c5c8c6;font-weight: bold } -.terminal-2418532981-r2 { fill: #c5c8c6 } -.terminal-2418532981-r3 { fill: #d0b344;font-weight: bold } -.terminal-2418532981-r4 { fill: #868887 } -.terminal-2418532981-r5 { fill: #cc555a } -.terminal-2418532981-r6 { fill: #68a0b3;font-weight: bold } -.terminal-2418532981-r7 { fill: #8d7b39 } -.terminal-2418532981-r8 { fill: #8a4346 } -.terminal-2418532981-r9 { fill: #98a84b;font-weight: bold } + .breeze-release-management-release-prod-images-r1 { fill: #c5c8c6;font-weight: bold } +.breeze-release-management-release-prod-images-r2 { fill: #c5c8c6 } +.breeze-release-management-release-prod-images-r3 { fill: #d0b344;font-weight: bold } +.breeze-release-management-release-prod-images-r4 { fill: #68a0b3;font-weight: bold } +.breeze-release-management-release-prod-images-r5 { fill: #868887 } +.breeze-release-management-release-prod-images-r6 { fill: #cc555a } +.breeze-release-management-release-prod-images-r7 { fill: #8d7b39 } +.breeze-release-management-release-prod-images-r8 { fill: #8a4346 } +.breeze-release-management-release-prod-images-r9 { fill: #98a84b;font-weight: bold } - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - Command: release-management release-prod-images + Command: release-management release-prod-images - + - - -Usage: breeze release-management release-prod-images [OPTIONS] - -Release production images to DockerHub (needs DockerHub permissions). - -╭─ Release PROD IMAGE flags ───────────────────────────────────────────────────────────────────────────────────────────╮ -*--airflow-versionAirflow version to release (2.3.0, 2.3.0rc1 etc.)(TEXT)[required] ---dockerhub-repoDockerHub repository for the images(TEXT)[default: apache/airflow] ---slim-imagesWhether to prepare slim images instead of the regular ones. ---limit-pythonSpecific python to build slim images for (if not specified - the images are built for all    -available python versions)                                                                   -(3.7 | 3.8 | 3.9 | 3.10)                                                                     ---limit-platformSpecific platform to build images for (if not specified, multiplatform images will be built. -(linux/amd64 | linux/arm64 | linux/amd64,linux/arm64)                                        -[default: linux/amd64,linux/arm64]                                                           ---skip-latestWhether to skip publishing the latest images (so that 'latest' images are not updated). This -should only be used if you release image for previous branches. Automatically set when       -rc/alpha/beta images are built.                                                              -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---verbose-vPrint verbose information about performed steps. ---dry-run-DIf dry-run is set, commands are only printed, not executed. ---help-hShow this message and exit. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + +Usage: breeze release-management release-prod-images [OPTIONS] + +Release production images to DockerHub (needs DockerHub permissions). + +╭─ Release PROD IMAGE flags ───────────────────────────────────────────────────────────────────────────────────────────╮ +*--airflow-versionAirflow version to release (2.3.02.3.0rc1 etc.)(TEXT)[required] +--dockerhub-repoDockerHub repository for the images(TEXT)[default: apache/airflow] +--slim-imagesWhether to prepare slim images instead of the regular ones. +--limit-pythonSpecific python to build slim images for (if not specified - the images are built for all    +available python versions)                                                                   +(3.7 | 3.8 | 3.9 | 3.10)                                                                     +--limit-platformSpecific platform to build images for (if not specified, multiplatform images will be built. +(linux/amd64 | linux/arm64 | linux/amd64,linux/arm64)                                        +[default: linux/amd64,linux/arm64]                                                           +--skip-latestWhether to skip publishing the latest images (so that 'latest' images are not updated). This +should only be used if you release image for previous branches. Automatically set when       +rc/alpha/beta images are built.                                                              +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ diff --git a/images/breeze/output_setup.svg b/images/breeze/output_setup.svg index 4ea9da3ed8a9e..206ba39938296 100644 --- a/images/breeze/output_setup.svg +++ b/images/breeze/output_setup.svg @@ -19,100 +19,100 @@ font-weight: 700; } - .terminal-563718598-matrix { + .breeze-setup-matrix { font-family: Fira Code, monospace; font-size: 20px; line-height: 24.4px; font-variant-east-asian: full-width; } - .terminal-563718598-title { + .breeze-setup-title { font-size: 18px; font-weight: bold; font-family: arial; } - .terminal-563718598-r1 { fill: #c5c8c6;font-weight: bold } -.terminal-563718598-r2 { fill: #c5c8c6 } -.terminal-563718598-r3 { fill: #d0b344;font-weight: bold } -.terminal-563718598-r4 { fill: #68a0b3;font-weight: bold } -.terminal-563718598-r5 { fill: #868887 } -.terminal-563718598-r6 { fill: #98a84b;font-weight: bold } + .breeze-setup-r1 { fill: #c5c8c6;font-weight: bold } +.breeze-setup-r2 { fill: #c5c8c6 } +.breeze-setup-r3 { fill: #d0b344;font-weight: bold } +.breeze-setup-r4 { fill: #68a0b3;font-weight: bold } +.breeze-setup-r5 { fill: #868887 } +.breeze-setup-r6 { fill: #98a84b;font-weight: bold } - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - Command: setup + Command: setup - + - - -Usage: breeze setup [OPTIONSCOMMAND [ARGS]... - -Tools that developers can use to configure Breeze - -╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---help-hShow this message and exit. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Commands ───────────────────────────────────────────────────────────────────────────────────────────────────────────╮ -autocomplete                    Enables autocompletion of breeze commands.                                         -config                          Show/update configuration (Python, Backend, Cheatsheet, ASCIIART).                 -regenerate-command-images       Regenerate breeze command images.                                                  -self-upgrade                    Self upgrade Breeze.                                                               -version                         Print information about version of apache-airflow-breeze.                          -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + +Usage: breeze setup [OPTIONSCOMMAND [ARGS]... + +Tools that developers can use to configure Breeze + +╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Commands ───────────────────────────────────────────────────────────────────────────────────────────────────────────╮ +autocomplete                    Enables autocompletion of breeze commands.                                         +config                          Show/update configuration (Python, Backend, Cheatsheet, ASCIIART).                 +regenerate-command-images       Regenerate breeze command images.                                                  +self-upgrade                    Self upgrade Breeze.                                                               +version                         Print information about version of apache-airflow-breeze.                          +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ diff --git a/images/breeze/output_setup_autocomplete.svg b/images/breeze/output_setup_autocomplete.svg index 74efd13f6c770..1bdb99cbad2be 100644 --- a/images/breeze/output_setup_autocomplete.svg +++ b/images/breeze/output_setup_autocomplete.svg @@ -19,97 +19,97 @@ font-weight: 700; } - .terminal-2355068599-matrix { + .breeze-setup-autocomplete-matrix { font-family: Fira Code, monospace; font-size: 20px; line-height: 24.4px; font-variant-east-asian: full-width; } - .terminal-2355068599-title { + .breeze-setup-autocomplete-title { font-size: 18px; font-weight: bold; font-family: arial; } - .terminal-2355068599-r1 { fill: #c5c8c6;font-weight: bold } -.terminal-2355068599-r2 { fill: #c5c8c6 } -.terminal-2355068599-r3 { fill: #d0b344;font-weight: bold } -.terminal-2355068599-r4 { fill: #868887 } -.terminal-2355068599-r5 { fill: #68a0b3;font-weight: bold } -.terminal-2355068599-r6 { fill: #98a84b;font-weight: bold } -.terminal-2355068599-r7 { fill: #8d7b39 } + .breeze-setup-autocomplete-r1 { fill: #c5c8c6;font-weight: bold } +.breeze-setup-autocomplete-r2 { fill: #c5c8c6 } +.breeze-setup-autocomplete-r3 { fill: #d0b344;font-weight: bold } +.breeze-setup-autocomplete-r4 { fill: #68a0b3;font-weight: bold } +.breeze-setup-autocomplete-r5 { fill: #868887 } +.breeze-setup-autocomplete-r6 { fill: #98a84b;font-weight: bold } +.breeze-setup-autocomplete-r7 { fill: #8d7b39 } - + - + - + - + - + - + - + - + - + - + - + - + - + - + - Command: setup autocomplete + Command: setup autocomplete - + - - -Usage: breeze setup autocomplete [OPTIONS] - -Enables autocompletion of breeze commands. - -╭─ Setup autocomplete flags ───────────────────────────────────────────────────────────────────────────────────────────╮ ---force-fForce autocomplete setup even if already setup before (overrides the setup). -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---verbose-vPrint verbose information about performed steps. ---dry-run-DIf dry-run is set, commands are only printed, not executed. ---answer-aForce answer to questions.(y | n | q | yes | no | quit) ---help-hShow this message and exit. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + +Usage: breeze setup autocomplete [OPTIONS] + +Enables autocompletion of breeze commands. + +╭─ Setup autocomplete flags ───────────────────────────────────────────────────────────────────────────────────────────╮ +--force-fForce autocomplete setup even if already setup before (overrides the setup). +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--answer-aForce answer to questions.(y | n | q | yes | no | quit) +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ diff --git a/images/breeze/output_setup_config.svg b/images/breeze/output_setup_config.svg index 6e3ea427439fb..d97ad0f3ab1da 100644 --- a/images/breeze/output_setup_config.svg +++ b/images/breeze/output_setup_config.svg @@ -19,125 +19,125 @@ font-weight: 700; } - .terminal-545217859-matrix { + .breeze-setup-config-matrix { font-family: Fira Code, monospace; font-size: 20px; line-height: 24.4px; font-variant-east-asian: full-width; } - .terminal-545217859-title { + .breeze-setup-config-title { font-size: 18px; font-weight: bold; font-family: arial; } - .terminal-545217859-r1 { fill: #c5c8c6;font-weight: bold } -.terminal-545217859-r2 { fill: #c5c8c6 } -.terminal-545217859-r3 { fill: #d0b344;font-weight: bold } -.terminal-545217859-r4 { fill: #68a0b3;font-weight: bold } -.terminal-545217859-r5 { fill: #868887 } -.terminal-545217859-r6 { fill: #98a84b;font-weight: bold } -.terminal-545217859-r7 { fill: #8d7b39 } + .breeze-setup-config-r1 { fill: #c5c8c6;font-weight: bold } +.breeze-setup-config-r2 { fill: #c5c8c6 } +.breeze-setup-config-r3 { fill: #d0b344;font-weight: bold } +.breeze-setup-config-r4 { fill: #68a0b3;font-weight: bold } +.breeze-setup-config-r5 { fill: #868887 } +.breeze-setup-config-r6 { fill: #98a84b;font-weight: bold } +.breeze-setup-config-r7 { fill: #8d7b39 } - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - Command: setup config + Command: setup config - + - - -Usage: breeze setup config [OPTIONS] - -Show/update configuration (Python, Backend, Cheatsheet, ASCIIART). - -╭─ Config flags ───────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---python-pPython major/minor version used in Airflow image for images. -(>3.7< | 3.8 | 3.9 | 3.10)                                   -[default: 3.7]                                               ---backend-bDatabase backend to use.(>sqlite< | mysql | postgres | mssql) -[default: sqlite]        ---postgres-version-PVersion of Postgres used.(>11< | 12 | 13 | 14 | 15)[default: 11] ---mysql-version-MVersion of MySQL used.(>5.7< | 8)[default: 5.7] ---mssql-version-SVersion of MsSQL used.(>2017-latest< | 2019-latest)[default: 2017-latest] ---cheatsheet/--no-cheatsheet-C/-cEnable/disable cheatsheet. ---asciiart/--no-asciiart-A/-aEnable/disable ASCIIart. ---colour/--no-colourEnable/disable Colour mode (useful for colour blind-friendly communication). -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---help-hShow this message and exit. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + +Usage: breeze setup config [OPTIONS] + +Show/update configuration (Python, Backend, Cheatsheet, ASCIIART). + +╭─ Config flags ───────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--python-pPython major/minor version used in Airflow image for images. +(>3.7< | 3.8 | 3.9 | 3.10)                                   +[default: 3.7]                                               +--backend-bDatabase backend to use.(>sqlite< | mysql | postgres | mssql) +[default: sqlite]        +--postgres-version-PVersion of Postgres used.(>11< | 12 | 13 | 14 | 15)[default: 11] +--mysql-version-MVersion of MySQL used.(>5.7< | 8)[default: 5.7] +--mssql-version-SVersion of MsSQL used.(>2017-latest< | 2019-latest)[default: 2017-latest] +--cheatsheet/--no-cheatsheet-C/-cEnable/disable cheatsheet. +--asciiart/--no-asciiart-A/-aEnable/disable ASCIIart. +--colour/--no-colourEnable/disable Colour mode (useful for colour blind-friendly communication). +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ diff --git a/images/breeze/output_setup_regenerate-command-images.svg b/images/breeze/output_setup_regenerate-command-images.svg index 5ce1112d9a94d..11cc8d3d0bdc9 100644 --- a/images/breeze/output_setup_regenerate-command-images.svg +++ b/images/breeze/output_setup_regenerate-command-images.svg @@ -19,153 +19,153 @@ font-weight: 700; } - .terminal-2816716479-matrix { + .breeze-setup-regenerate-command-images-matrix { font-family: Fira Code, monospace; font-size: 20px; line-height: 24.4px; font-variant-east-asian: full-width; } - .terminal-2816716479-title { + .breeze-setup-regenerate-command-images-title { font-size: 18px; font-weight: bold; font-family: arial; } - .terminal-2816716479-r1 { fill: #c5c8c6;font-weight: bold } -.terminal-2816716479-r2 { fill: #c5c8c6 } -.terminal-2816716479-r3 { fill: #d0b344;font-weight: bold } -.terminal-2816716479-r4 { fill: #868887 } -.terminal-2816716479-r5 { fill: #68a0b3;font-weight: bold } -.terminal-2816716479-r6 { fill: #8d7b39 } -.terminal-2816716479-r7 { fill: #98a84b;font-weight: bold } + .breeze-setup-regenerate-command-images-r1 { fill: #c5c8c6;font-weight: bold } +.breeze-setup-regenerate-command-images-r2 { fill: #c5c8c6 } +.breeze-setup-regenerate-command-images-r3 { fill: #d0b344;font-weight: bold } +.breeze-setup-regenerate-command-images-r4 { fill: #68a0b3;font-weight: bold } +.breeze-setup-regenerate-command-images-r5 { fill: #868887 } +.breeze-setup-regenerate-command-images-r6 { fill: #8d7b39 } +.breeze-setup-regenerate-command-images-r7 { fill: #98a84b;font-weight: bold } - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - Command: setup regenerate-command-images + Command: setup regenerate-command-images - + - - -Usage: breeze setup regenerate-command-images [OPTIONS] - -Regenerate breeze command images. - -╭─ Image regeneration option ──────────────────────────────────────────────────────────────────────────────────────────╮ ---forceForces regeneration of all images ---commandCommand(s) to regenerate images for (optional, might be repeated)                                    -(main | build-docs | ci:find-newer-dependencies | ci:fix-ownership | ci:free-space |                 -ci:get-workflow-info | ci:resource-check | ci:selective-check | ci | ci-image:build | ci-image:pull  -| ci-image:verify | ci-image | cleanup | compile-www-assets | exec | k8s:build-k8s-image |           -k8s:configure-cluster | k8s:create-cluster | k8s:delete-cluster | k8s:deploy-airflow | k8s:k9s |     -k8s:logs | k8s:run-complete-tests | k8s:setup-env | k8s:shell | k8s:status | k8s:tests |             -k8s:upload-k8s-image | k8s | prod-image:build | prod-image:pull | prod-image:verify | prod-image |   -release-management:generate-constraints | release-management:prepare-airflow-package |               -release-management:prepare-provider-documentation | release-management:prepare-provider-packages |   -release-management:release-prod-images | release-management:verify-provider-packages |               -release-management | setup:autocomplete | setup:config | setup:regenerate-command-images |           -setup:self-upgrade | setup:version | setup | shell | start-airflow | static-checks | stop |          -testing:docker-compose-tests | testing:helm-tests | testing:tests | testing)                         ---check-onlyOnly check if some images need to be regenerated. Return 0 if no need or 1 if needed. Cannot be used -together with --command flag or --force.                                                             -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---verbose-vPrint verbose information about performed steps. ---dry-run-DIf dry-run is set, commands are only printed, not executed. ---help-hShow this message and exit. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + +Usage: breeze setup regenerate-command-images [OPTIONS] + +Regenerate breeze command images. + +╭─ Image regeneration option ──────────────────────────────────────────────────────────────────────────────────────────╮ +--forceForces regeneration of all images +--commandCommand(s) to regenerate images for (optional, might be repeated)                                    +(main | build-docs | ci:find-newer-dependencies | ci:fix-ownership | ci:free-space |                 +ci:get-workflow-info | ci:resource-check | ci:selective-check | ci | ci-image:build | ci-image:pull  +| ci-image:verify | ci-image | cleanup | compile-www-assets | exec | k8s:build-k8s-image |           +k8s:configure-cluster | k8s:create-cluster | k8s:delete-cluster | k8s:deploy-airflow | k8s:k9s |     +k8s:logs | k8s:run-complete-tests | k8s:setup-env | k8s:shell | k8s:status | k8s:tests |             +k8s:upload-k8s-image | k8s | prod-image:build | prod-image:pull | prod-image:verify | prod-image |   +release-management:generate-constraints | release-management:prepare-airflow-package |               +release-management:prepare-provider-documentation | release-management:prepare-provider-packages |   +release-management:release-prod-images | release-management:verify-provider-packages |               +release-management | setup:autocomplete | setup:config | setup:regenerate-command-images |           +setup:self-upgrade | setup:version | setup | shell | start-airflow | static-checks | stop |          +testing:docker-compose-tests | testing:helm-tests | testing:tests | testing)                         +--check-onlyOnly check if some images need to be regenerated. Return 0 if no need or 1 if needed. Cannot be used +together with --command flag or --force.                                                             +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ diff --git a/images/breeze/output_setup_self-upgrade.svg b/images/breeze/output_setup_self-upgrade.svg index 5a188b25a48d9..930d6d8f95650 100644 --- a/images/breeze/output_setup_self-upgrade.svg +++ b/images/breeze/output_setup_self-upgrade.svg @@ -19,84 +19,84 @@ font-weight: 700; } - .terminal-2544310981-matrix { + .breeze-setup-self-upgrade-matrix { font-family: Fira Code, monospace; font-size: 20px; line-height: 24.4px; font-variant-east-asian: full-width; } - .terminal-2544310981-title { + .breeze-setup-self-upgrade-title { font-size: 18px; font-weight: bold; font-family: arial; } - .terminal-2544310981-r1 { fill: #c5c8c6;font-weight: bold } -.terminal-2544310981-r2 { fill: #c5c8c6 } -.terminal-2544310981-r3 { fill: #d0b344;font-weight: bold } -.terminal-2544310981-r4 { fill: #868887 } -.terminal-2544310981-r5 { fill: #68a0b3;font-weight: bold } -.terminal-2544310981-r6 { fill: #98a84b;font-weight: bold } + .breeze-setup-self-upgrade-r1 { fill: #c5c8c6;font-weight: bold } +.breeze-setup-self-upgrade-r2 { fill: #c5c8c6 } +.breeze-setup-self-upgrade-r3 { fill: #d0b344;font-weight: bold } +.breeze-setup-self-upgrade-r4 { fill: #68a0b3;font-weight: bold } +.breeze-setup-self-upgrade-r5 { fill: #868887 } +.breeze-setup-self-upgrade-r6 { fill: #98a84b;font-weight: bold } - + - + - + - + - + - + - + - + - + - + - + - Command: setup self-upgrade + Command: setup self-upgrade - + - - -Usage: breeze setup self-upgrade [OPTIONS] - -Self upgrade Breeze. - -╭─ Self-upgrade flags ─────────────────────────────────────────────────────────────────────────────────────────────────╮ ---use-current-airflow-sources-aUse current workdir Airflow sources for upgrade. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---help-hShow this message and exit. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + +Usage: breeze setup self-upgrade [OPTIONS] + +Self upgrade Breeze. + +╭─ Self-upgrade flags ─────────────────────────────────────────────────────────────────────────────────────────────────╮ +--use-current-airflow-sources-aUse current workdir Airflow sources for upgrade. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ diff --git a/images/breeze/output_setup_version.svg b/images/breeze/output_setup_version.svg index 0c4918f318f8a..70447cebc0dff 100644 --- a/images/breeze/output_setup_version.svg +++ b/images/breeze/output_setup_version.svg @@ -19,80 +19,80 @@ font-weight: 700; } - .terminal-2681505854-matrix { + .breeze-setup-version-matrix { font-family: Fira Code, monospace; font-size: 20px; line-height: 24.4px; font-variant-east-asian: full-width; } - .terminal-2681505854-title { + .breeze-setup-version-title { font-size: 18px; font-weight: bold; font-family: arial; } - .terminal-2681505854-r1 { fill: #c5c8c6;font-weight: bold } -.terminal-2681505854-r2 { fill: #c5c8c6 } -.terminal-2681505854-r3 { fill: #d0b344;font-weight: bold } -.terminal-2681505854-r4 { fill: #868887 } -.terminal-2681505854-r5 { fill: #68a0b3;font-weight: bold } -.terminal-2681505854-r6 { fill: #98a84b;font-weight: bold } + .breeze-setup-version-r1 { fill: #c5c8c6;font-weight: bold } +.breeze-setup-version-r2 { fill: #c5c8c6 } +.breeze-setup-version-r3 { fill: #d0b344;font-weight: bold } +.breeze-setup-version-r4 { fill: #68a0b3;font-weight: bold } +.breeze-setup-version-r5 { fill: #868887 } +.breeze-setup-version-r6 { fill: #98a84b;font-weight: bold } - + - + - + - + - + - + - + - + - + - + - Command: setup version + Command: setup version - + - - -Usage: breeze setup version [OPTIONS] - -Print information about version of apache-airflow-breeze. - -╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---verbose-vPrint verbose information about performed steps. ---dry-run-DIf dry-run is set, commands are only printed, not executed. ---help-hShow this message and exit. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + +Usage: breeze setup version [OPTIONS] + +Print information about version of apache-airflow-breeze. + +╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ diff --git a/images/breeze/output_stop.svg b/images/breeze/output_stop.svg index 54785bed03598..eee71a18885b4 100644 --- a/images/breeze/output_stop.svg +++ b/images/breeze/output_stop.svg @@ -19,92 +19,92 @@ font-weight: 700; } - .terminal-893805321-matrix { + .breeze-stop-matrix { font-family: Fira Code, monospace; font-size: 20px; line-height: 24.4px; font-variant-east-asian: full-width; } - .terminal-893805321-title { + .breeze-stop-title { font-size: 18px; font-weight: bold; font-family: arial; } - .terminal-893805321-r1 { fill: #c5c8c6;font-weight: bold } -.terminal-893805321-r2 { fill: #c5c8c6 } -.terminal-893805321-r3 { fill: #d0b344;font-weight: bold } -.terminal-893805321-r4 { fill: #868887 } -.terminal-893805321-r5 { fill: #68a0b3;font-weight: bold } -.terminal-893805321-r6 { fill: #98a84b;font-weight: bold } + .breeze-stop-r1 { fill: #c5c8c6;font-weight: bold } +.breeze-stop-r2 { fill: #c5c8c6 } +.breeze-stop-r3 { fill: #d0b344;font-weight: bold } +.breeze-stop-r4 { fill: #68a0b3;font-weight: bold } +.breeze-stop-r5 { fill: #868887 } +.breeze-stop-r6 { fill: #98a84b;font-weight: bold } - + - + - + - + - + - + - + - + - + - + - + - + - + - Command: stop + Command: stop - + - - -Usage: breeze stop [OPTIONS] - -Stop running breeze environment. - -╭─ Stop flags ─────────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---preserve-volumes-pSkip removing volumes when stopping Breeze. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---verbose-vPrint verbose information about performed steps. ---dry-run-DIf dry-run is set, commands are only printed, not executed. ---help-hShow this message and exit. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + +Usage: breeze stop [OPTIONS] + +Stop running breeze environment. + +╭─ Stop flags ─────────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--preserve-volumes-pSkip removing volumes when stopping Breeze. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ diff --git a/images/breeze/output_testing.svg b/images/breeze/output_testing.svg index ed5d5d1ccb92f..bedaaf4ac7086 100644 --- a/images/breeze/output_testing.svg +++ b/images/breeze/output_testing.svg @@ -19,92 +19,92 @@ font-weight: 700; } - .terminal-3208273230-matrix { + .breeze-testing-matrix { font-family: Fira Code, monospace; font-size: 20px; line-height: 24.4px; font-variant-east-asian: full-width; } - .terminal-3208273230-title { + .breeze-testing-title { font-size: 18px; font-weight: bold; font-family: arial; } - .terminal-3208273230-r1 { fill: #c5c8c6;font-weight: bold } -.terminal-3208273230-r2 { fill: #c5c8c6 } -.terminal-3208273230-r3 { fill: #d0b344;font-weight: bold } -.terminal-3208273230-r4 { fill: #68a0b3;font-weight: bold } -.terminal-3208273230-r5 { fill: #868887 } -.terminal-3208273230-r6 { fill: #98a84b;font-weight: bold } + .breeze-testing-r1 { fill: #c5c8c6;font-weight: bold } +.breeze-testing-r2 { fill: #c5c8c6 } +.breeze-testing-r3 { fill: #d0b344;font-weight: bold } +.breeze-testing-r4 { fill: #68a0b3;font-weight: bold } +.breeze-testing-r5 { fill: #868887 } +.breeze-testing-r6 { fill: #98a84b;font-weight: bold } - + - + - + - + - + - + - + - + - + - + - + - + - + - Command: testing + Command: testing - + - - -Usage: breeze testing [OPTIONSCOMMAND [ARGS]... - -Tools that developers can use to run tests - -╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---help-hShow this message and exit. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Testing ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ -tests                                    Run the specified unit test targets.                                      -helm-tests                               Run Helm chart tests.                                                     -docker-compose-tests                     Run docker-compose tests.                                                 -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + +Usage: breeze testing [OPTIONSCOMMAND [ARGS]... + +Tools that developers can use to run tests + +╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Testing ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ +tests                                    Run the specified unit test targets.                                      +helm-tests                               Run Helm chart tests.                                                     +docker-compose-tests                     Run docker-compose tests.                                                 +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ diff --git a/images/breeze/output_testing_docker-compose-tests.svg b/images/breeze/output_testing_docker-compose-tests.svg index 78c8b50b68c0a..61fc891ab7404 100644 --- a/images/breeze/output_testing_docker-compose-tests.svg +++ b/images/breeze/output_testing_docker-compose-tests.svg @@ -19,113 +19,113 @@ font-weight: 700; } - .terminal-1722025249-matrix { + .breeze-testing-docker-compose-tests-matrix { font-family: Fira Code, monospace; font-size: 20px; line-height: 24.4px; font-variant-east-asian: full-width; } - .terminal-1722025249-title { + .breeze-testing-docker-compose-tests-title { font-size: 18px; font-weight: bold; font-family: arial; } - .terminal-1722025249-r1 { fill: #c5c8c6;font-weight: bold } -.terminal-1722025249-r2 { fill: #c5c8c6 } -.terminal-1722025249-r3 { fill: #d0b344;font-weight: bold } -.terminal-1722025249-r4 { fill: #868887 } -.terminal-1722025249-r5 { fill: #68a0b3;font-weight: bold } -.terminal-1722025249-r6 { fill: #98a84b;font-weight: bold } -.terminal-1722025249-r7 { fill: #8d7b39 } + .breeze-testing-docker-compose-tests-r1 { fill: #c5c8c6;font-weight: bold } +.breeze-testing-docker-compose-tests-r2 { fill: #c5c8c6 } +.breeze-testing-docker-compose-tests-r3 { fill: #d0b344;font-weight: bold } +.breeze-testing-docker-compose-tests-r4 { fill: #68a0b3;font-weight: bold } +.breeze-testing-docker-compose-tests-r5 { fill: #868887 } +.breeze-testing-docker-compose-tests-r6 { fill: #98a84b;font-weight: bold } +.breeze-testing-docker-compose-tests-r7 { fill: #8d7b39 } - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - Command: testing docker-compose-tests + Command: testing docker-compose-tests - + - - -Usage: breeze testing docker-compose-tests [OPTIONS] [EXTRA_PYTEST_ARGS]... - -Run docker-compose tests. - -╭─ Docker-compose tests flag ──────────────────────────────────────────────────────────────────────────────────────────╮ ---image-name-nName of the image to verify (overrides --python and --image-tag).(TEXT) ---image-tag-tTag of the image which is used to run the image (implies --mount-sources=skip).(TEXT) -[default: latest]                                                               ---python-pPython major/minor version used in Airflow image for images.(>3.7< | 3.8 | 3.9 | 3.10) -[default: 3.7]                                               ---github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---verbose-vPrint verbose information about performed steps. ---dry-run-DIf dry-run is set, commands are only printed, not executed. ---help-hShow this message and exit. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + +Usage: breeze testing docker-compose-tests [OPTIONS] [EXTRA_PYTEST_ARGS]... + +Run docker-compose tests. + +╭─ Docker-compose tests flag ──────────────────────────────────────────────────────────────────────────────────────────╮ +--image-name-nName of the image to verify (overrides --python and --image-tag).(TEXT) +--image-tag-tTag of the image which is used to run the image (implies --mount-sources=skip).(TEXT) +[default: latest]                                                               +--python-pPython major/minor version used in Airflow image for images.(>3.7< | 3.8 | 3.9 | 3.10) +[default: 3.7]                                               +--github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ diff --git a/images/breeze/output_testing_helm-tests.svg b/images/breeze/output_testing_helm-tests.svg index ee5c7a3269f69..9983efe5eb0cb 100644 --- a/images/breeze/output_testing_helm-tests.svg +++ b/images/breeze/output_testing_helm-tests.svg @@ -19,117 +19,117 @@ font-weight: 700; } - .terminal-1053162087-matrix { + .breeze-testing-helm-tests-matrix { font-family: Fira Code, monospace; font-size: 20px; line-height: 24.4px; font-variant-east-asian: full-width; } - .terminal-1053162087-title { + .breeze-testing-helm-tests-title { font-size: 18px; font-weight: bold; font-family: arial; } - .terminal-1053162087-r1 { fill: #c5c8c6;font-weight: bold } -.terminal-1053162087-r2 { fill: #c5c8c6 } -.terminal-1053162087-r3 { fill: #d0b344;font-weight: bold } -.terminal-1053162087-r4 { fill: #868887 } -.terminal-1053162087-r5 { fill: #68a0b3;font-weight: bold } -.terminal-1053162087-r6 { fill: #98a84b;font-weight: bold } -.terminal-1053162087-r7 { fill: #8d7b39 } + .breeze-testing-helm-tests-r1 { fill: #c5c8c6;font-weight: bold } +.breeze-testing-helm-tests-r2 { fill: #c5c8c6 } +.breeze-testing-helm-tests-r3 { fill: #d0b344;font-weight: bold } +.breeze-testing-helm-tests-r4 { fill: #68a0b3;font-weight: bold } +.breeze-testing-helm-tests-r5 { fill: #868887 } +.breeze-testing-helm-tests-r6 { fill: #98a84b;font-weight: bold } +.breeze-testing-helm-tests-r7 { fill: #8d7b39 } - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - Command: testing helm-tests + Command: testing helm-tests - + - - -Usage: breeze testing helm-tests [OPTIONS] [EXTRA_PYTEST_ARGS]... - -Run Helm chart tests. - -╭─ Advanced flag for helms-tests command ──────────────────────────────────────────────────────────────────────────────╮ ---image-tag-tTag of the image which is used to run the image (implies --mount-sources=skip).(TEXT) -[default: latest]                                                               ---mount-sourcesChoose scope of local sources that should be mounted, skipped, or removed (default =        -selected).                                                                                  -(selected | all | skip | remove)                                                            -[default: selected]                                                                         ---github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---verbose-vPrint verbose information about performed steps. ---dry-run-DIf dry-run is set, commands are only printed, not executed. ---help-hShow this message and exit. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + +Usage: breeze testing helm-tests [OPTIONS] [EXTRA_PYTEST_ARGS]... + +Run Helm chart tests. + +╭─ Advanced flag for helms-tests command ──────────────────────────────────────────────────────────────────────────────╮ +--image-tag-tTag of the image which is used to run the image (implies --mount-sources=skip).(TEXT) +[default: latest]                                                               +--mount-sourcesChoose scope of local sources that should be mounted, skipped, or removed (default =        +selected).                                                                                  +(selected | all | skip | remove)                                                            +[default: selected]                                                                         +--github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ diff --git a/images/breeze/output_testing_tests.svg b/images/breeze/output_testing_tests.svg index 4c30bc457656d..1b23af7398977 100644 --- a/images/breeze/output_testing_tests.svg +++ b/images/breeze/output_testing_tests.svg @@ -19,229 +19,229 @@ font-weight: 700; } - .terminal-96960050-matrix { + .breeze-testing-tests-matrix { font-family: Fira Code, monospace; font-size: 20px; line-height: 24.4px; font-variant-east-asian: full-width; } - .terminal-96960050-title { + .breeze-testing-tests-title { font-size: 18px; font-weight: bold; font-family: arial; } - .terminal-96960050-r1 { fill: #c5c8c6;font-weight: bold } -.terminal-96960050-r2 { fill: #c5c8c6 } -.terminal-96960050-r3 { fill: #d0b344;font-weight: bold } -.terminal-96960050-r4 { fill: #68a0b3;font-weight: bold } -.terminal-96960050-r5 { fill: #868887 } -.terminal-96960050-r6 { fill: #8d7b39 } -.terminal-96960050-r7 { fill: #98a84b;font-weight: bold } + .breeze-testing-tests-r1 { fill: #c5c8c6;font-weight: bold } +.breeze-testing-tests-r2 { fill: #c5c8c6 } +.breeze-testing-tests-r3 { fill: #d0b344;font-weight: bold } +.breeze-testing-tests-r4 { fill: #68a0b3;font-weight: bold } +.breeze-testing-tests-r5 { fill: #868887 } +.breeze-testing-tests-r6 { fill: #8d7b39 } +.breeze-testing-tests-r7 { fill: #98a84b;font-weight: bold } - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - Command: testing tests + Command: testing tests - + - - -Usage: breeze testing tests [OPTIONS] [EXTRA_PYTEST_ARGS]... - -Run the specified unit test targets. - -╭─ Basic flag for tests command ───────────────────────────────────────────────────────────────────────────────────────╮ ---integrationIntegration(s) to enable when running (can be more than one).                               -(cassandra | kerberos | mongo | openldap | pinot | rabbitmq | redis | statsd | trino | all) ---test-typeType of test to run. Note that with Providers, you can also specify which provider tests     -should be run - for example --test-type "Providers[airbyte,http]"                            -(All | API | Always | CLI | Core | Integration | Other | Providers | WWW | Helm | Postgres | -MySQL | Quarantine)                                                                          ---test-timeoutTest timeout. Set the pytest setup, execution and teardown timeouts to this value -(INTEGER RANGE)                                                                   -[default: 60; x>=0]                                                               ---db-reset-dReset DB when entering the container. ---backend-bDatabase backend to use.(>sqlite< | mysql | postgres | mssql)[default: sqlite] ---python-pPython major/minor version used in Airflow image for images.(>3.7< | 3.8 | 3.9 | 3.10) -[default: 3.7]                                               ---postgres-version-PVersion of Postgres used.(>11< | 12 | 13 | 14 | 15)[default: 11] ---mysql-version-MVersion of MySQL used.(>5.7< | 8)[default: 5.7] ---mssql-version-SVersion of MsSQL used.(>2017-latest< | 2019-latest)[default: 2017-latest] -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Options for parallel test commands ─────────────────────────────────────────────────────────────────────────────────╮ ---run-in-parallelRun the operation in parallel on all or selected subset of Python versions. ---parallelismMaximum number of processes to use while running the operation in parallel. -(INTEGER RANGE)                                                             -[default: 4; 1<=x<=8]                                                       ---test-typesSpace separated list of test types used for testing in parallel.(TEXT) -[default: API Always CLI Core Integration Other Providers WWW]   ---skip-cleanupSkip cleanup of temporary files created during parallel run. ---debug-resourcesWhether to show resource information while running in parallel. ---include-success-outputsWhether to include outputs of successful parallel runs (skipped by default). ---full-tests-neededWhether full set of tests is run. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Advanced flag for tests command ────────────────────────────────────────────────────────────────────────────────────╮ ---image-tag-tTag of the image which is used to run the image (implies --mount-sources=skip).(TEXT) -[default: latest]                                                               ---mount-sourcesChoose scope of local sources that should be mounted, skipped, or removed (default = selected). -(selected | all | skip | remove)                                                                -[default: selected]                                                                             -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---verbose-vPrint verbose information about performed steps. ---dry-run-DIf dry-run is set, commands are only printed, not executed. ---help-hShow this message and exit. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + +Usage: breeze testing tests [OPTIONS] [EXTRA_PYTEST_ARGS]... + +Run the specified unit test targets. + +╭─ Basic flag for tests command ───────────────────────────────────────────────────────────────────────────────────────╮ +--integrationIntegration(s) to enable when running (can be more than one).                               +(cassandra | kerberos | mongo | openldap | pinot | rabbitmq | redis | statsd | trino | all) +--test-typeType of test to run. Note that with Providers, you can also specify which provider tests     +should be run - for example --test-type "Providers[airbyte,http]"                            +(All | API | Always | CLI | Core | Integration | Other | Providers | WWW | Helm | Postgres | +MySQL | Quarantine)                                                                          +--test-timeoutTest timeout. Set the pytest setup, execution and teardown timeouts to this value +(INTEGER RANGE)                                                                   +[default: 60; x>=0]                                                               +--db-reset-dReset DB when entering the container. +--backend-bDatabase backend to use.(>sqlite< | mysql | postgres | mssql)[default: sqlite] +--python-pPython major/minor version used in Airflow image for images.(>3.7< | 3.8 | 3.9 | 3.10) +[default: 3.7]                                               +--postgres-version-PVersion of Postgres used.(>11< | 12 | 13 | 14 | 15)[default: 11] +--mysql-version-MVersion of MySQL used.(>5.7< | 8)[default: 5.7] +--mssql-version-SVersion of MsSQL used.(>2017-latest< | 2019-latest)[default: 2017-latest] +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Options for parallel test commands ─────────────────────────────────────────────────────────────────────────────────╮ +--run-in-parallelRun the operation in parallel on all or selected subset of Python versions. +--parallelismMaximum number of processes to use while running the operation in parallel. +(INTEGER RANGE)                                                             +[default: 4; 1<=x<=8]                                                       +--test-typesSpace separated list of test types used for testing in parallel.(TEXT) +[default: API Always CLI Core Integration Other Providers WWW]   +--skip-cleanupSkip cleanup of temporary files created during parallel run. +--debug-resourcesWhether to show resource information while running in parallel. +--include-success-outputsWhether to include outputs of successful parallel runs (skipped by default). +--full-tests-neededWhether full set of tests is run. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Advanced flag for tests command ────────────────────────────────────────────────────────────────────────────────────╮ +--image-tag-tTag of the image which is used to run the image (implies --mount-sources=skip).(TEXT) +[default: latest]                                                               +--mount-sourcesChoose scope of local sources that should be mounted, skipped, or removed (default = selected). +(selected | all | skip | remove)                                                                +[default: selected]                                                                             +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ From 8ad86bff98b4b46558c37a55ffdd5357ebaafd37 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Rapha=C3=ABl=20Vandon?= <114772123+vandonr-amz@users.noreply.github.com> Date: Wed, 30 Nov 2022 08:52:05 +0100 Subject: [PATCH 115/158] better warning messages when running breeze in rosetta (#27874) (cherry picked from commit 527fbce462429fc9836837378f801eed4e9d194f) --- .../src/airflow_breeze/commands/main_command.py | 14 ++++++++++---- 1 file changed, 10 insertions(+), 4 deletions(-) diff --git a/dev/breeze/src/airflow_breeze/commands/main_command.py b/dev/breeze/src/airflow_breeze/commands/main_command.py index 13faf884f2be7..b217ae1b22a79 100644 --- a/dev/breeze/src/airflow_breeze/commands/main_command.py +++ b/dev/breeze/src/airflow_breeze/commands/main_command.py @@ -135,14 +135,17 @@ def check_for_python_emulation(): "[warning]You likely installed your Python wrongly and you should " "remove it and reinstall from scratch[/]\n" ) - from inputimeout import inputimeout + from inputimeout import TimeoutOccurred, inputimeout user_status = inputimeout( - prompt="Are you REALLY sure you want to continue? (press y otherwise we exit in 20s) ", + prompt="Are you REALLY sure you want to continue? (answer with y otherwise we exit in 20s)\n", timeout=20, ) if not user_status.upper() in ["Y", "YES"]: sys.exit(1) + except TimeoutOccurred: + get_console().print("\nNo answer, exiting...") + sys.exit(1) except subprocess.CalledProcessError: pass except PermissionError: @@ -178,14 +181,17 @@ def check_for_rosetta_environment(): "If you have mixed Intel/ARM binaries installed you should likely nuke and " "reinstall your development environment (including brew and Python) from scratch!\n\n" ) - from inputimeout import inputimeout + from inputimeout import TimeoutOccurred, inputimeout user_status = inputimeout( - prompt="Are you REALLY sure you want to continue? (press y otherwise we exit in 20s) ", + prompt="Are you REALLY sure you want to continue? (answer with y otherwise we exit in 20s)\n", timeout=20, ) if not user_status.upper() in ["Y", "YES"]: sys.exit(1) + except TimeoutOccurred: + get_console().print("\nNo answer, exiting...") + sys.exit(1) except subprocess.CalledProcessError: pass except PermissionError: From 4f0ae0df62641a35ca2c1505d90329985830d14d Mon Sep 17 00:00:00 2001 From: Tomek Urbaszek Date: Mon, 5 Dec 2022 23:00:51 +0100 Subject: [PATCH 116/158] Fix UnboundLocalError in breeze (#28126) * Fix UnboundLocalError in breeze I got the following error after installing breeze: `UnboundLocalError: local variable 'TimeoutOccurred' referenced before assignment` It seems that changing the import place fixes the issue. (cherry picked from commit 82af388de2032d1c44f0c11a1759fcb96b245d0d) --- dev/breeze/src/airflow_breeze/commands/main_command.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/dev/breeze/src/airflow_breeze/commands/main_command.py b/dev/breeze/src/airflow_breeze/commands/main_command.py index b217ae1b22a79..620068b181a10 100644 --- a/dev/breeze/src/airflow_breeze/commands/main_command.py +++ b/dev/breeze/src/airflow_breeze/commands/main_command.py @@ -155,6 +155,9 @@ def check_for_python_emulation(): def check_for_rosetta_environment(): if sys.platform != "darwin": return + + from inputimeout import TimeoutOccurred, inputimeout + try: runs_in_rosetta = subprocess.check_output( ["sysctl", "-n", "sysctl.proc_translated"], @@ -181,7 +184,6 @@ def check_for_rosetta_environment(): "If you have mixed Intel/ARM binaries installed you should likely nuke and " "reinstall your development environment (including brew and Python) from scratch!\n\n" ) - from inputimeout import TimeoutOccurred, inputimeout user_status = inputimeout( prompt="Are you REALLY sure you want to continue? (answer with y otherwise we exit in 20s)\n", From 268fe7c03f43080bcf336d323bd589660e8e9336 Mon Sep 17 00:00:00 2001 From: Jarek Potiuk Date: Mon, 5 Dec 2022 23:42:22 +0100 Subject: [PATCH 117/158] Check if ghcr.io is reachable before network-bound breeze operations (#28137) Unfortunately, ghcr.io behaves nasty when token you logged in with to it expired. It refuses to pull the images, even if they are public. This PR adds extra check for all network-bound commands that require ghcr.io access. (cherry picked from commit c4224e28fc94219cde2c15b5e7993cf76772ad7a) --- .../commands/ci_image_commands.py | 4 +++ .../commands/production_image_commands.py | 4 +++ .../commands/release_management_commands.py | 4 +++ .../utils/docker_command_utils.py | 28 +++++++++++++++++++ 4 files changed, 40 insertions(+) diff --git a/dev/breeze/src/airflow_breeze/commands/ci_image_commands.py b/dev/breeze/src/airflow_breeze/commands/ci_image_commands.py index 09d8eca7325ad..83c436deb902f 100644 --- a/dev/breeze/src/airflow_breeze/commands/ci_image_commands.py +++ b/dev/breeze/src/airflow_breeze/commands/ci_image_commands.py @@ -76,6 +76,7 @@ from airflow_breeze.utils.console import Output, get_console from airflow_breeze.utils.docker_command_utils import ( build_cache, + check_remote_ghcr_io_commands, make_sure_builder_configured, perform_environment_checks, prepare_docker_build_command, @@ -216,6 +217,7 @@ def run_build(ci_image_params: BuildCiParams) -> None: sys.exit(return_code) perform_environment_checks() + check_remote_ghcr_io_commands() parameters_passed = filter_out_none(**kwargs) parameters_passed["force_build"] = True fix_group_permissions() @@ -277,6 +279,7 @@ def pull( ): """Pull and optionally verify CI images - possibly in parallel for all Python versions.""" perform_environment_checks() + check_remote_ghcr_io_commands() if run_in_parallel: python_version_list = get_python_version_list(python_versions) ci_image_params_list = [ @@ -344,6 +347,7 @@ def verify( build_params = BuildCiParams(python=python, image_tag=image_tag, github_repository=github_repository) image_name = build_params.airflow_image_name_with_tag if pull: + check_remote_ghcr_io_commands() command_to_run = ["docker", "pull", image_name] run_command(command_to_run, check=True) get_console().print(f"[info]Verifying CI image: {image_name}[/]") diff --git a/dev/breeze/src/airflow_breeze/commands/production_image_commands.py b/dev/breeze/src/airflow_breeze/commands/production_image_commands.py index 4e29a961d6e21..8cc058741b6f8 100644 --- a/dev/breeze/src/airflow_breeze/commands/production_image_commands.py +++ b/dev/breeze/src/airflow_breeze/commands/production_image_commands.py @@ -78,6 +78,7 @@ from airflow_breeze.utils.custom_param_types import BetterChoice from airflow_breeze.utils.docker_command_utils import ( build_cache, + check_remote_ghcr_io_commands, make_sure_builder_configured, perform_environment_checks, prepare_docker_build_command, @@ -245,6 +246,7 @@ def run_build(prod_image_params: BuildProdParams) -> None: sys.exit(return_code) perform_environment_checks() + check_remote_ghcr_io_commands() parameters_passed = filter_out_none(**kwargs) fix_group_permissions() @@ -305,6 +307,7 @@ def pull_prod_image( ): """Pull and optionally verify Production images - possibly in parallel for all Python versions.""" perform_environment_checks() + check_remote_ghcr_io_commands() if run_in_parallel: python_version_list = get_python_version_list(python_versions) prod_image_params_list = [ @@ -381,6 +384,7 @@ def verify( ) image_name = build_params.airflow_image_name_with_tag if pull: + check_remote_ghcr_io_commands() command_to_run = ["docker", "pull", image_name] run_command(command_to_run, check=True) get_console().print(f"[info]Verifying PROD image: {image_name}[/]") diff --git a/dev/breeze/src/airflow_breeze/commands/release_management_commands.py b/dev/breeze/src/airflow_breeze/commands/release_management_commands.py index e7ccfa945f511..b319ebaa78217 100644 --- a/dev/breeze/src/airflow_breeze/commands/release_management_commands.py +++ b/dev/breeze/src/airflow_breeze/commands/release_management_commands.py @@ -64,6 +64,7 @@ from airflow_breeze.utils.console import Output, get_console from airflow_breeze.utils.custom_param_types import BetterChoice from airflow_breeze.utils.docker_command_utils import ( + check_remote_ghcr_io_commands, get_env_variables_for_docker_commands, get_extra_docker_flags, perform_environment_checks, @@ -216,6 +217,7 @@ def prepare_provider_documentation( packages: list[str], ): perform_environment_checks() + check_remote_ghcr_io_commands() cleanup_python_generated_files() shell_params = ShellParams( mount_sources=MOUNT_ALL, @@ -383,6 +385,7 @@ def generate_constraints( github_repository: str, ): perform_environment_checks() + check_remote_ghcr_io_commands() cleanup_python_generated_files() if debug and run_in_parallel: get_console().print("\n[error]Cannot run --debug and --run-in-parallel at the same time[/]\n") @@ -568,6 +571,7 @@ def release_prod_images( skip_latest: bool, ): perform_environment_checks() + check_remote_ghcr_io_commands() rebuild_or_pull_ci_image_if_needed(command_params=ShellParams(python=DEFAULT_PYTHON_MAJOR_MINOR_VERSION)) if not match(r"^\d*\.\d*\.\d*$", airflow_version): get_console().print( diff --git a/dev/breeze/src/airflow_breeze/utils/docker_command_utils.py b/dev/breeze/src/airflow_breeze/utils/docker_command_utils.py index b9e30dd21bc40..7a6a9afa6b7d4 100644 --- a/dev/breeze/src/airflow_breeze/utils/docker_command_utils.py +++ b/dev/breeze/src/airflow_breeze/utils/docker_command_utils.py @@ -262,6 +262,34 @@ def check_docker_version(): ) +def check_remote_ghcr_io_commands(): + """ + Checks if you have permissions to pull an empty image from ghcr.io. Unfortunately, GitHub packages + treat expired login as "no-access" even on public repos. We need to detect that situation and suggest + user to log-out. + :return: + """ + response = run_command( + ["docker", "pull", "ghcr.io/apache/airflow-hello-world"], + no_output_dump_on_exception=True, + text=False, + capture_output=True, + check=False, + ) + if response.returncode != 0: + if "no such host" in response.stderr.decode("utf-8"): + get_console().print( + "[error]\nYou seem to be offline. This command requires access to network.[/]\n" + ) + sys.exit(2) + get_console().print( + "[error]\nYou seem to have expired permissions on ghcr.io.[/]\n" + "[warning]Please logout. Run this command:[/]\n\n" + " docker logout ghcr.io\n\n" + ) + sys.exit(1) + + DOCKER_COMPOSE_COMMAND = ["docker-compose"] From 1fa8f439b42bda8a8b40d9e1b3b7882f8a5b3fa7 Mon Sep 17 00:00:00 2001 From: Jarek Potiuk Date: Tue, 6 Dec 2022 18:58:07 +0100 Subject: [PATCH 118/158] Summarize all warnings from all builds (#28151) (cherry picked from commit 1bb594e99cf0c363eed9736260dcb6201aa010e6) --- .github/workflows/ci.yml | 33 +++++++++++++++++++++++++++++++++ 1 file changed, 33 insertions(+) diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index 6814d69f2ce63..431edb171a24c 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -1006,6 +1006,39 @@ jobs: with: directory: "./coverage-files" + summarize-warnings: + timeout-minutes: 15 + name: "Summarize warnings" + runs-on: "${{needs.build-info.outputs.runs-on}}" + needs: + - build-info + - tests-postgres + - tests-sqlite + - tests-mysql + - tests-mssql + - tests-quarantined + env: + RUNS_ON: "${{needs.build-info.outputs.runs-on}}" + steps: + - name: Cleanup repo + run: docker run -v "${GITHUB_WORKSPACE}:/workspace" -u 0:0 bash -c "rm -rf /workspace/*" + - name: "Checkout ${{ github.ref }} ( ${{ github.sha }} )" + uses: actions/checkout@v3 + with: + persist-credentials: false + submodules: recursive + - name: "Download all artifacts from the current build" + uses: actions/download-artifact@v3 + with: + path: ./artifacts + - name: "Summarize all warnings" + run: | + ls -R ./artifacts/ + cat ./artifacts/test-warnings*/* | sort | uniq + echo + echo Total number of unique warnings $(cat ./artifacts/test-warnings*/* | sort | uniq | wc -l) + + wait-for-prod-images: timeout-minutes: 120 name: "Wait for PROD images" From 39094095c6a4c5a9f0065794c413017c89b6db24 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Tue, 6 Dec 2022 12:38:07 -0800 Subject: [PATCH 119/158] Don't check rosetta on i386 (#28169) (cherry picked from commit 2d86a123da8deea139164094b9bdd31cb063817d) --- dev/breeze/src/airflow_breeze/commands/main_command.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dev/breeze/src/airflow_breeze/commands/main_command.py b/dev/breeze/src/airflow_breeze/commands/main_command.py index 620068b181a10..3761c4dabc5ad 100644 --- a/dev/breeze/src/airflow_breeze/commands/main_command.py +++ b/dev/breeze/src/airflow_breeze/commands/main_command.py @@ -153,7 +153,7 @@ def check_for_python_emulation(): def check_for_rosetta_environment(): - if sys.platform != "darwin": + if sys.platform != "darwin" or platform.processor() == "i386": return from inputimeout import TimeoutOccurred, inputimeout From 2fed1585e8ccd5ef08c8a83ed6c6111afcc7c021 Mon Sep 17 00:00:00 2001 From: Jarek Potiuk Date: Thu, 8 Dec 2022 09:40:46 +0100 Subject: [PATCH 120/158] Force higher parallelism for waiting for images in CI (#28209) Default parallelism for public runners is 2 because they have 2 CPUS. However image pulling is mostly about I/O (CPU is only really used when images are extracted and when tests are run - a bit). With parallelism = 2 the 4 images are serializing (first 2 images are pulled and tested and then 2 remaining ones) By setting parallelism to 6 we are allowing all 4 images to run in parallel and we are safe for 3.11 when it is out to also run in parallel). That should save ~2 minutes for image pulling. (cherry picked from commit 1ed01b58752650985d67127acfb19705ca0c967f) --- .github/workflows/ci.yml | 4 ++++ dev/breeze/src/airflow_breeze/utils/common_options.py | 2 +- 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index 431edb171a24c..b983c4da61e5a 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -511,6 +511,8 @@ jobs: env: RUNS_ON: "${{needs.build-info.outputs.runs-on}}" BACKEND: sqlite + # Force more parallelism for pull even on public images + PARALLELISM: 6 steps: - name: Cleanup repo run: docker run -v "${GITHUB_WORKSPACE}:/workspace" -u 0:0 bash -c "rm -rf /workspace/*" @@ -1049,6 +1051,8 @@ jobs: RUNS_ON: "${{needs.build-info.outputs.runs-on}}" BACKEND: sqlite PYTHON_MAJOR_MINOR_VERSION: "${{needs.build-info.outputs.default-python-version}}" + # Force more parallelism for pull even on public images + PARALLELISM: 6 steps: - name: Cleanup repo run: docker run -v "${GITHUB_WORKSPACE}:/workspace" -u 0:0 bash -c "rm -rf /workspace/*" diff --git a/dev/breeze/src/airflow_breeze/utils/common_options.py b/dev/breeze/src/airflow_breeze/utils/common_options.py index 82bbde6f7e003..9caf98b007dd1 100644 --- a/dev/breeze/src/airflow_breeze/utils/common_options.py +++ b/dev/breeze/src/airflow_breeze/utils/common_options.py @@ -439,7 +439,7 @@ def _set_default_from_parent(ctx: click.core.Context, option: click.core.Option, option_parallelism = click.option( "--parallelism", help="Maximum number of processes to use while running the operation in parallel.", - type=click.IntRange(1, mp.cpu_count() * 2 if not generating_command_images() else 8), + type=click.IntRange(1, mp.cpu_count() * 3 if not generating_command_images() else 8), default=mp.cpu_count() if not generating_command_images() else 4, envvar="PARALLELISM", show_default=True, From 56d76517a7254fb73564032c40a872f9d9bd1e1f Mon Sep 17 00:00:00 2001 From: Jarek Potiuk Date: Fri, 9 Dec 2022 11:10:51 +0100 Subject: [PATCH 121/158] Improve caching for pre-commits in CI (#28240) Configuration of caching for pre-commits in CI has been broken: * full pre-commit cache had `pre-commit-` instead of `pre-commit-full-` * basic checks never run in "main" so the cache had not been stored in the main branch - thus pre-commits for the basic checks were never cached. However it is quite OK for pre-commit basic to use the full pre-commit package cache (cherry picked from commit 395a34b960c73118a732d371e93aeab8dcd76275) --- .github/workflows/ci.yml | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index b983c4da61e5a..b540d1ba5fc28 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -556,7 +556,7 @@ jobs: with: path: ~/.cache/pre-commit-full # yamllint disable-line rule:line-length - key: "pre-commit-${{steps.host-python-version.outputs.host-python-version}}-${{ hashFiles('.pre-commit-config.yaml') }}" + key: "pre-commit-full-${{steps.host-python-version.outputs.host-python-version}}-${{ hashFiles('.pre-commit-config.yaml') }}" restore-keys: | pre-commit-full-${{steps.host-python-version.outputs.host-python-version}} pre-commit-full @@ -604,9 +604,13 @@ jobs: path: ~/.cache/pre-commit # yamllint disable-line rule:line-length key: "pre-commit-basic-${{steps.host-python-version.outputs.host-python-version}}-${{ hashFiles('.pre-commit-config.yaml') }}" - restore-keys: | - pre-commit-basic-${{steps.host-python-version.outputs.host-python-version}} - pre-commit-basic- + restore-keys: "\ + pre-commit-full-${{steps.host-python-version.outputs.host-python-version}}-\ + ${{ hashFiles('.pre-commit-config.yaml') }}\n + pre-commit-basic-${{steps.host-python-version.outputs.host-python-version}}\n + pre-commit-full-${{steps.host-python-version.outputs.host-python-version}}\n + pre-commit-basic-\n + pre-commit-full-" - name: Fetch incoming commit ${{ github.sha }} with its parent uses: actions/checkout@v3 with: From 9081910301f7b7dd76c308fb6a94522e070e8bb4 Mon Sep 17 00:00:00 2001 From: Jarek Potiuk Date: Sat, 10 Dec 2022 23:57:56 +0100 Subject: [PATCH 122/158] Proper Python Host output from composite tasks in CI (#28254) When we separated actions into composite workflows in #27369 and the #27371 we missed the fact that there was a step passing the host python version to static checks - to make sure we start with a fresh cache every time new python version is changed (this caused problems in the past) The "host-python-version" was therefore empty and not really used in the cache key determining. This has no bad effect when Python version does not change but when there is an upgrade of Python, the symbolic links in stored cache get broken, and next time new Python version is used in the AMIs, we might get them broken, so better to fix it now. (cherry picked from commit 0db5e46ec1814ba34d7d56b9617ad0096559c777) --- .github/actions/breeze/action.yml | 4 ++++ .../actions/prepare_breeze_and_image/action.yml | 5 +++++ .github/workflows/ci.yml | 16 +++++++++------- 3 files changed, 18 insertions(+), 7 deletions(-) diff --git a/.github/actions/breeze/action.yml b/.github/actions/breeze/action.yml index bd8853d5551e9..4db1164e2fe9b 100644 --- a/.github/actions/breeze/action.yml +++ b/.github/actions/breeze/action.yml @@ -18,6 +18,10 @@ --- name: 'Setup Breeze' description: 'Sets up Python and Breeze' +outputs: + host-python-version: + description: Python version used in host + value: ${{ steps.host-python-version.outputs.host-python-version }} runs: using: "composite" steps: diff --git a/.github/actions/prepare_breeze_and_image/action.yml b/.github/actions/prepare_breeze_and_image/action.yml index ed943446fe484..61967e1e52bb2 100644 --- a/.github/actions/prepare_breeze_and_image/action.yml +++ b/.github/actions/prepare_breeze_and_image/action.yml @@ -22,11 +22,16 @@ inputs: pull-image-type: description: 'Which image to pull' default: CI +outputs: + host-python-version: + description: Python version used in host + value: ${{ steps.breeze.outputs.host-python-version }} runs: using: "composite" steps: - name: "Install Breeze" uses: ./.github/actions/breeze + id: breeze - name: Pull CI image ${{ env.PYTHON_MAJOR_MINOR_VERSION }}:${{ env.IMAGE_TAG }} shell: bash run: breeze ci-image pull --tag-as-latest diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index b540d1ba5fc28..6347ad182f66b 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -551,14 +551,15 @@ jobs: - name: > Prepare breeze & CI image: ${{needs.build-info.outputs.default-python-version}}:${{env.IMAGE_TAG}} uses: ./.github/actions/prepare_breeze_and_image + id: breeze - name: Cache pre-commit envs uses: actions/cache@v3 with: - path: ~/.cache/pre-commit-full + path: ~/.cache/pre-commit # yamllint disable-line rule:line-length - key: "pre-commit-full-${{steps.host-python-version.outputs.host-python-version}}-${{ hashFiles('.pre-commit-config.yaml') }}" + key: "pre-commit-full-${{steps.breeze.outputs.host-python-version}}-${{ hashFiles('.pre-commit-config.yaml') }}" restore-keys: | - pre-commit-full-${{steps.host-python-version.outputs.host-python-version}} + pre-commit-full-${{steps.breeze.outputs.host-python-version}} pre-commit-full - name: "Static checks" run: breeze static-checks --all-files --show-diff-on-failure --color always @@ -598,17 +599,18 @@ jobs: cache-dependency-path: ./dev/breeze/setup* - name: "Install Breeze" uses: ./.github/actions/breeze + id: breeze - name: Cache pre-commit envs uses: actions/cache@v3 with: path: ~/.cache/pre-commit # yamllint disable-line rule:line-length - key: "pre-commit-basic-${{steps.host-python-version.outputs.host-python-version}}-${{ hashFiles('.pre-commit-config.yaml') }}" + key: "pre-commit-basic-${{steps.breeze.outputs.host-python-version}}-${{ hashFiles('.pre-commit-config.yaml') }}" restore-keys: "\ - pre-commit-full-${{steps.host-python-version.outputs.host-python-version}}-\ + pre-commit-full-${{steps.breeze.outputs.host-python-version}}-\ ${{ hashFiles('.pre-commit-config.yaml') }}\n - pre-commit-basic-${{steps.host-python-version.outputs.host-python-version}}\n - pre-commit-full-${{steps.host-python-version.outputs.host-python-version}}\n + pre-commit-basic-${{steps.breeze.outputs.host-python-version}}\n + pre-commit-full-${{steps.breeze.outputs.host-python-version}}\n pre-commit-basic-\n pre-commit-full-" - name: Fetch incoming commit ${{ github.sha }} with its parent From 8073efc057a7fb945d11729040a1bcf5f3120f17 Mon Sep 17 00:00:00 2001 From: Jarek Potiuk Date: Mon, 12 Dec 2022 23:20:24 +0100 Subject: [PATCH 123/158] Make sure the key for k8s environment changes with requirements in CI (#28317) Previously, caching key for k8s environment was static - it did not take into account that some of the file changes should invalidate the cache. This PR changes it so that it changes wheneve one of the contributing factors change: * requirements * airflow setup files * airflow pyproject.toml * generated provider dependencies Changes in any of those should cause recreation of the venv. (cherry picked from commit 37e754bad350d43b40ea503c3e3f84c684226290) --- .github/workflows/ci.yml | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index 6347ad182f66b..5434c46ba6913 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -1141,7 +1141,9 @@ jobs: uses: actions/cache@v3 with: path: ".build/.k8s-env" - key: "k8s-env" + key: "\ + k8s-env-${{ hashFiles('scripts/ci/kubernetes/k8s_requirements.txt','setup.cfg',\ + 'setup.py','pyproject.toml','generated/provider_dependencies.json') }}" - name: Run complete K8S tests ${{needs.build-info.outputs.kubernetes-combos}} run: breeze k8s run-complete-tests --run-in-parallel --upgrade env: From 86eb97cd38916d3770551a958e8dce2eeb762aba Mon Sep 17 00:00:00 2001 From: Jarek Potiuk Date: Tue, 13 Dec 2022 00:00:25 +0100 Subject: [PATCH 124/158] Selective checks for API did not match API test specification (#28319) The selective checks for the API had "^tests/api" as matching regexp, where we used "tests/api", "tests/api_connexion" to trigger the tests. This means that changes to "tests/api_internal" were not treated as "Other" as they were included in "^tests/api". This PR fixes it. (cherry picked from commit 17131b3340245e68d4721bf9f57771cb3f2d04a1) --- dev/breeze/src/airflow_breeze/utils/selective_checks.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dev/breeze/src/airflow_breeze/utils/selective_checks.py b/dev/breeze/src/airflow_breeze/utils/selective_checks.py index 63232bbbf5fca..294ed7ed0ecb2 100644 --- a/dev/breeze/src/airflow_breeze/utils/selective_checks.py +++ b/dev/breeze/src/airflow_breeze/utils/selective_checks.py @@ -104,7 +104,8 @@ def __hash__(self): r"^airflow/.*\.lock", ], FileGroupForCi.API_TEST_FILES: [ - r"^airflow/api", + r"^airflow/api/", + r"^airflow/api_connexion/", ], FileGroupForCi.API_CODEGEN_FILES: [ r"^airflow/api_connexion/openapi/v1\.yaml", From 5607ba9d4141d489f6c3c651c17d21530a176f47 Mon Sep 17 00:00:00 2001 From: Jarek Potiuk Date: Tue, 13 Dec 2022 10:01:44 +0100 Subject: [PATCH 125/158] Remove postgres from K8S venv (#28323) While postgres has always been in the k8s venv used for K8S tests, it seems it is not needed there at all. Not sure if it was ever needed in fact (maybe some super old historical version of those tests needed it). This removes postgres from the requirements for the venv. (cherry picked from commit db5995aa72f69b63fbd0c4f98b75c0a6436d3b40) --- scripts/ci/kubernetes/k8s_requirements.txt | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/scripts/ci/kubernetes/k8s_requirements.txt b/scripts/ci/kubernetes/k8s_requirements.txt index 9c965e9c3f790..afd9d897f7347 100644 --- a/scripts/ci/kubernetes/k8s_requirements.txt +++ b/scripts/ci/kubernetes/k8s_requirements.txt @@ -1,7 +1,6 @@ -.[cncf.kubernetes,postgres] +.[cncf.kubernetes] pytest pytest-cov pytest-instafail pytest-timeouts pytest-xdist -freezegun From 2c7ef41a93957fed0402f9f60eb9c2cde76229ac Mon Sep 17 00:00:00 2001 From: Jarek Potiuk Date: Tue, 13 Dec 2022 14:49:33 +0100 Subject: [PATCH 126/158] Fix output of error in K8s venv installation (#28322) (cherry picked from commit 6f57d4283341d76330ba0dea3db79c3eca9055b1) --- dev/breeze/src/airflow_breeze/utils/kubernetes_utils.py | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/dev/breeze/src/airflow_breeze/utils/kubernetes_utils.py b/dev/breeze/src/airflow_breeze/utils/kubernetes_utils.py index 3258be65a2758..2b84090008152 100644 --- a/dev/breeze/src/airflow_breeze/utils/kubernetes_utils.py +++ b/dev/breeze/src/airflow_breeze/utils/kubernetes_utils.py @@ -309,14 +309,10 @@ def _install_packages_in_k8s_virtualenv(with_constraints: bool): f"constraints-{sys.version_info.major}.{sys.version_info.minor}.txt", ] ) - install_packages_result = run_command( - install_command, - check=False, - capture_output=True, - ) + install_packages_result = run_command(install_command, check=False, capture_output=True, text=True) if install_packages_result.returncode != 0: get_console().print( - f"[error]Error when updating pip to {PIP_VERSION}:[/]\n" + f"[error]Error when installing packages from : {K8S_REQUIREMENTS.resolve()}[/]\n" f"{install_packages_result.stdout}\n{install_packages_result.stderr}" ) return install_packages_result From 71d3060ded2e8b28cf7bdca62973409cc6e8f39f Mon Sep 17 00:00:00 2001 From: Jarek Potiuk Date: Sun, 18 Dec 2022 17:40:24 +0100 Subject: [PATCH 127/158] Fix output buffering for `breeze testing test` (#28433) Since we disabled warnings and redirected them to the output file, we lost continuous output of the progress of tests when they were run as `breeze testing tests` command. The whole summary of the tests appeared only at the end of tests. This turned out to be Python output buffering in "filter_out_warning" class. It is fixed by flushing sys.stdout after every printed line now. (cherry picked from commit efaac217e95bda75ac26f7692e79e1eb33025dcf) --- scripts/in_container/filter_out_warnings.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/scripts/in_container/filter_out_warnings.py b/scripts/in_container/filter_out_warnings.py index df27eda7e6876..ebd5b0aa7cf20 100644 --- a/scripts/in_container/filter_out_warnings.py +++ b/scripts/in_container/filter_out_warnings.py @@ -19,6 +19,7 @@ from __future__ import annotations import fileinput +import sys suppress = False @@ -29,3 +30,4 @@ suppress = False if not suppress: print(line, end="") + sys.stdout.flush() From b8efa4dafe471c472b2b0851d7c3b65e7cc43529 Mon Sep 17 00:00:00 2001 From: Jarek Potiuk Date: Wed, 21 Dec 2022 11:09:42 +0100 Subject: [PATCH 128/158] Fix redirection for selective chekcs for all workflows (#28518) The #28514 added traceback handling for selective checks but the redirection had not been fixed in few other workflows where it was used. (cherry picked from commit 4615d1768cbe0b1287000bcaa2925a5c5fb80f1d) --- .github/workflows/build-images.yml | 2 +- .github/workflows/codeql-analysis.yml | 2 +- .github/workflows/release_dockerhub_image.yml | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/.github/workflows/build-images.yml b/.github/workflows/build-images.yml index 6b50bd9e1b78f..e5f71f15dd22c 100644 --- a/.github/workflows/build-images.yml +++ b/.github/workflows/build-images.yml @@ -151,7 +151,7 @@ jobs: env: PR_LABELS: "${{ steps.get-latest-pr-labels.outputs.pull-request-labels }}" COMMIT_REF: "${{ env.TARGET_COMMIT_SHA }}" - run: breeze ci selective-check 2>> ${GITHUB_OUTPUT} + run: breeze ci selective-check >> ${GITHUB_OUTPUT} - name: env run: printenv env: diff --git a/.github/workflows/codeql-analysis.yml b/.github/workflows/codeql-analysis.yml index c195a18f174df..bb190302315ce 100644 --- a/.github/workflows/codeql-analysis.yml +++ b/.github/workflows/codeql-analysis.yml @@ -49,7 +49,7 @@ jobs: id: selective-checks env: COMMIT_REF: "${{ github.sha }}" - run: breeze ci selective-check 2>> ${GITHUB_OUTPUT} + run: breeze ci selective-check >> ${GITHUB_OUTPUT} analyze: name: Analyze diff --git a/.github/workflows/release_dockerhub_image.yml b/.github/workflows/release_dockerhub_image.yml index a9f59b20dcaa6..2463943d18fbc 100644 --- a/.github/workflows/release_dockerhub_image.yml +++ b/.github/workflows/release_dockerhub_image.yml @@ -57,7 +57,7 @@ jobs: uses: ./.github/actions/breeze - name: Selective checks id: selective-checks - run: breeze ci selective-check 2>> ${GITHUB_OUTPUT} + run: breeze ci selective-check >> ${GITHUB_OUTPUT} release-images: timeout-minutes: 120 name: "Release images: ${{ github.event.inputs.airflowVersion }}, ${{ matrix.python-version }}" From 5eb90ef2be46933de8574eb6f2101b8678991f40 Mon Sep 17 00:00:00 2001 From: Jarek Potiuk Date: Wed, 21 Dec 2022 11:17:45 +0100 Subject: [PATCH 129/158] Add VERBOSE equal to false for all just changed selective checks (#28519) Follow-up change after #28518 (cherry picked from commit 61ef8ca239771a4d1644a308e898fdfdaa3daf8a) --- .github/workflows/build-images.yml | 1 + .github/workflows/codeql-analysis.yml | 1 + .github/workflows/release_dockerhub_image.yml | 2 ++ 3 files changed, 4 insertions(+) diff --git a/.github/workflows/build-images.yml b/.github/workflows/build-images.yml index e5f71f15dd22c..b6934b655538c 100644 --- a/.github/workflows/build-images.yml +++ b/.github/workflows/build-images.yml @@ -151,6 +151,7 @@ jobs: env: PR_LABELS: "${{ steps.get-latest-pr-labels.outputs.pull-request-labels }}" COMMIT_REF: "${{ env.TARGET_COMMIT_SHA }}" + VERBOSE: "false" run: breeze ci selective-check >> ${GITHUB_OUTPUT} - name: env run: printenv diff --git a/.github/workflows/codeql-analysis.yml b/.github/workflows/codeql-analysis.yml index bb190302315ce..858b8a5d6b5e1 100644 --- a/.github/workflows/codeql-analysis.yml +++ b/.github/workflows/codeql-analysis.yml @@ -49,6 +49,7 @@ jobs: id: selective-checks env: COMMIT_REF: "${{ github.sha }}" + VERBOSE: "false" run: breeze ci selective-check >> ${GITHUB_OUTPUT} analyze: diff --git a/.github/workflows/release_dockerhub_image.yml b/.github/workflows/release_dockerhub_image.yml index 2463943d18fbc..65527e2c9b6a0 100644 --- a/.github/workflows/release_dockerhub_image.yml +++ b/.github/workflows/release_dockerhub_image.yml @@ -57,6 +57,8 @@ jobs: uses: ./.github/actions/breeze - name: Selective checks id: selective-checks + env: + VERBOSE: "false" run: breeze ci selective-check >> ${GITHUB_OUTPUT} release-images: timeout-minutes: 120 From f59105e7494bc88a93b5ba35b32258a063530d00 Mon Sep 17 00:00:00 2001 From: Andrey Anshin Date: Thu, 22 Dec 2022 14:00:13 +0400 Subject: [PATCH 130/158] Re-enable `pymsql` on ARM as it now builds cleanly (#28530) (cherry picked from commit 78238d51ee4b0fdba28bb8f24e0a0402c2ef2922) --- airflow/providers/microsoft/mssql/provider.yaml | 2 +- dev/breeze/src/airflow_breeze/commands/developer_commands.py | 2 +- generated/provider_dependencies.json | 2 +- scripts/in_container/run_provider_yaml_files_check.py | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/airflow/providers/microsoft/mssql/provider.yaml b/airflow/providers/microsoft/mssql/provider.yaml index ee015253cadf3..228f411cb26dc 100644 --- a/airflow/providers/microsoft/mssql/provider.yaml +++ b/airflow/providers/microsoft/mssql/provider.yaml @@ -40,7 +40,7 @@ versions: dependencies: - apache-airflow>=2.3.0 - apache-airflow-providers-common-sql>=1.3.1 - - pymssql>=2.1.5; platform_machine != "aarch64" + - pymssql>=2.1.5 integrations: - integration-name: Microsoft SQL Server (MSSQL) diff --git a/dev/breeze/src/airflow_breeze/commands/developer_commands.py b/dev/breeze/src/airflow_breeze/commands/developer_commands.py index ccb95dda21509..c9562ca12b6ce 100644 --- a/dev/breeze/src/airflow_breeze/commands/developer_commands.py +++ b/dev/breeze/src/airflow_breeze/commands/developer_commands.py @@ -549,7 +549,7 @@ def enter_shell(**kwargs) -> RunCommandResult: sys.exit(1) if shell_params.backend == "mssql": get_console().print("\n[error]MSSQL is not supported on ARM architecture[/]\n") - return 1 + sys.exit(1) command_result = run_command( cmd, env=env_variables, text=True, check=False, output_outside_the_group=True ) diff --git a/generated/provider_dependencies.json b/generated/provider_dependencies.json index 178af7b20d095..4572100e46709 100644 --- a/generated/provider_dependencies.json +++ b/generated/provider_dependencies.json @@ -456,7 +456,7 @@ "deps": [ "apache-airflow-providers-common-sql>=1.3.1", "apache-airflow>=2.3.0", - "pymssql>=2.1.5; platform_machine != \"aarch64\"" + "pymssql>=2.1.5" ], "cross-providers-deps": [ "common.sql" diff --git a/scripts/in_container/run_provider_yaml_files_check.py b/scripts/in_container/run_provider_yaml_files_check.py index c2cfe565aefd9..c8d3b1f6b31d8 100755 --- a/scripts/in_container/run_provider_yaml_files_check.py +++ b/scripts/in_container/run_provider_yaml_files_check.py @@ -165,7 +165,7 @@ def check_if_object_exist(object_name: str, resource_type: str, yaml_file_path: raise RuntimeError(f"Wrong enum {object_type}???") except Exception as e: if architecture == Architecture.ARM: - if "pymssql" in str(e) or "MySQLdb" in str(e): + if "MySQLdb" in str(e): console.print( f"[yellow]The imports fail on ARM: {object_name} in {resource_type} {e}, " f"but it is expected.[/]" From c8a91a5197338b0349cb95ea775eaf8bb2d342c4 Mon Sep 17 00:00:00 2001 From: Jarek Potiuk Date: Sat, 24 Dec 2022 14:48:14 +0100 Subject: [PATCH 131/158] Wait for asset compilation to finish before starting airflow in Breeze (#28575) Asset compilation is performed in background and especially if it is run for the first time it might take some time. At the same time database intialization is done so usually it is not a problem but it some cases database might be initialized faster and webserver would start without assets compiled leading to nasty output. This change adds waiting for the compilation - tmux will not split screens and run webserver before it is done. (cherry picked from commit 979a72a7a3bff4e6cb8132360408584f65f2b203) --- Dockerfile.ci | 30 +++++++++++ dev/breeze/README.md | 2 +- dev/breeze/setup.cfg | 1 + .../utils/docker_command_utils.py | 1 + .../src/airflow_breeze/utils/path_utils.py | 3 ++ .../src/airflow_breeze/utils/run_utils.py | 53 ++++++++++++++----- scripts/ci/docker-compose/local.yml | 3 ++ .../pre_commit_compile_www_assets.py | 4 +- .../pre_commit_compile_www_assets_dev.py | 2 +- scripts/docker/entrypoint_ci.sh | 30 +++++++++++ scripts/in_container/bin/run_tmux | 1 - 11 files changed, 113 insertions(+), 17 deletions(-) diff --git a/Dockerfile.ci b/Dockerfile.ci index 74123f9427a0a..96a0c9ba4b45c 100644 --- a/Dockerfile.ci +++ b/Dockerfile.ci @@ -598,6 +598,35 @@ export AIRFLOW_HOME=${AIRFLOW_HOME:=${HOME}} : "${AIRFLOW_SOURCES:?"ERROR: AIRFLOW_SOURCES not set !!!!"}" +function wait_for_asset_compilation() { + if [[ -f "${AIRFLOW_SOURCES}/.build/www/.asset_compile.lock" ]]; then + echo + echo "${COLOR_YELLOW}Waiting for asset compilation to complete in the background.${COLOR_RESET}" + echo + local counter=0 + while [[ -f "${AIRFLOW_SOURCES}/.build/www/.asset_compile.lock" ]]; do + echo "${COLOR_BLUE}Still waiting .....${COLOR_RESET}" + sleep 1 + ((counter=counter+1)) + if [[ ${counter} == "30" ]]; then + echo + echo "${COLOR_YELLOW}The asset compilation is taking too long.${COLOR_YELLOW}" + echo """ +If it does not complete soon, you might want to stop it and remove file lock: + * press Ctrl-C + * run 'rm ${AIRFLOW_SOURCES}/.build/www/.asset_compile.lock' +""" + fi + if [[ ${counter} == "60" ]]; then + echo + echo "${COLOR_RED}The asset compilation is taking too long. Exiting.${COLOR_RED}" + echo + exit 1 + fi + done + fi +} + if [[ ${SKIP_ENVIRONMENT_INITIALIZATION=} != "true" ]]; then if [[ $(uname -m) == "arm64" || $(uname -m) == "aarch64" ]]; then @@ -785,6 +814,7 @@ if [[ ${SKIP_ENVIRONMENT_INITIALIZATION=} != "true" ]]; then if [[ ${START_AIRFLOW:="false"} == "true" || ${START_AIRFLOW} == "True" ]]; then export AIRFLOW__DATABASE__LOAD_DEFAULT_CONNECTIONS=${LOAD_DEFAULT_CONNECTIONS} export AIRFLOW__CORE__LOAD_EXAMPLES=${LOAD_EXAMPLES} + wait_for_asset_compilation # shellcheck source=scripts/in_container/bin/run_tmux exec run_tmux fi diff --git a/dev/breeze/README.md b/dev/breeze/README.md index 6872f4378abaa..7dd12fbe68261 100644 --- a/dev/breeze/README.md +++ b/dev/breeze/README.md @@ -52,6 +52,6 @@ PLEASE DO NOT MODIFY THE HASH BELOW! IT IS AUTOMATICALLY UPDATED BY PRE-COMMIT. --------------------------------------------------------------------------------------------------------- -Package config hash: 99e484ad56c10cbba1755bb3a994f55d4acacc477ea73e23bbddd1e2f95f477f7fe873b0f8f20873a1b3d78e173d713660e59ab3c6f292ef836489043740c061 +Package config hash: 670c60fceb07f18c6fabce5e1382039bc9cc5773d339b54b6957088c484d548ee99e66d11b8eb6cf6872d7467147bcf8661249dc9e64824350edc3eddd57ed5d --------------------------------------------------------------------------------------------------------- diff --git a/dev/breeze/setup.cfg b/dev/breeze/setup.cfg index eaddc76e17261..69d420c5ff56f 100644 --- a/dev/breeze/setup.cfg +++ b/dev/breeze/setup.cfg @@ -55,6 +55,7 @@ packages = find: install_requires = cached_property>=1.5.0;python_version<="3.7" click + filelock inputimeout importlib-metadata>=4.4; python_version < "3.8" pendulum diff --git a/dev/breeze/src/airflow_breeze/utils/docker_command_utils.py b/dev/breeze/src/airflow_breeze/utils/docker_command_utils.py index 7a6a9afa6b7d4..55cb94d7d8821 100644 --- a/dev/breeze/src/airflow_breeze/utils/docker_command_utils.py +++ b/dev/breeze/src/airflow_breeze/utils/docker_command_utils.py @@ -74,6 +74,7 @@ VOLUMES_FOR_SELECTED_MOUNTS = [ (".bash_aliases", "/root/.bash_aliases"), (".bash_history", "/root/.bash_history"), + (".build", "/opt/airflow/.build"), (".coveragerc", "/opt/airflow/.coveragerc"), (".dockerignore", "/opt/airflow/.dockerignore"), (".flake8", "/opt/airflow/.flake8"), diff --git a/dev/breeze/src/airflow_breeze/utils/path_utils.py b/dev/breeze/src/airflow_breeze/utils/path_utils.py index 7138d24c36974..33142072dcaff 100644 --- a/dev/breeze/src/airflow_breeze/utils/path_utils.py +++ b/dev/breeze/src/airflow_breeze/utils/path_utils.py @@ -263,6 +263,9 @@ def find_airflow_sources_root_to_operate_on() -> Path: AIRFLOW_SOURCES_ROOT = find_airflow_sources_root_to_operate_on().resolve() BUILD_CACHE_DIR = AIRFLOW_SOURCES_ROOT / ".build" +WWW_CACHE_DIR = BUILD_CACHE_DIR / "www" +WWW_ASSET_COMPILE_LOCK = WWW_CACHE_DIR / ".asset_compile.lock" +WWW_ASSET_OUT_FILE = WWW_CACHE_DIR / "asset_compile.out" DAGS_DIR = AIRFLOW_SOURCES_ROOT / "dags" FILES_DIR = AIRFLOW_SOURCES_ROOT / "files" HOOKS_DIR = AIRFLOW_SOURCES_ROOT / "hooks" diff --git a/dev/breeze/src/airflow_breeze/utils/run_utils.py b/dev/breeze/src/airflow_breeze/utils/run_utils.py index 506aff693fe71..70cf89687eb8a 100644 --- a/dev/breeze/src/airflow_breeze/utils/run_utils.py +++ b/dev/breeze/src/airflow_breeze/utils/run_utils.py @@ -36,7 +36,7 @@ from airflow_breeze.global_constants import APACHE_AIRFLOW_GITHUB_REPOSITORY from airflow_breeze.utils.ci_group import ci_group from airflow_breeze.utils.console import Output, get_console -from airflow_breeze.utils.path_utils import AIRFLOW_SOURCES_ROOT +from airflow_breeze.utils.path_utils import AIRFLOW_SOURCES_ROOT, WWW_ASSET_COMPILE_LOCK, WWW_ASSET_OUT_FILE from airflow_breeze.utils.shared_options import get_dry_run, get_verbose RunCommandResult = Union[subprocess.CompletedProcess, subprocess.CalledProcessError] @@ -393,16 +393,43 @@ def get_ci_image_for_pre_commits() -> str: return airflow_image -def _run_compile_internally(command_to_execute: list[str]): +def _run_compile_internally(command_to_execute: list[str], dev: bool) -> RunCommandResult: + from filelock import SoftFileLock, Timeout + env = os.environ.copy() - compile_www_assets_result = run_command( - command_to_execute, - check=False, - no_output_dump_on_exception=True, - text=True, - env=env, - ) - return compile_www_assets_result + if dev: + return run_command( + command_to_execute, + check=False, + no_output_dump_on_exception=True, + text=True, + env=env, + ) + else: + WWW_ASSET_COMPILE_LOCK.parent.mkdir(parents=True, exist_ok=True) + try: + WWW_ASSET_COMPILE_LOCK.unlink() + except FileNotFoundError: + pass + try: + with SoftFileLock(WWW_ASSET_COMPILE_LOCK, timeout=5): + with open(WWW_ASSET_OUT_FILE, "w") as f: + return run_command( + command_to_execute, + check=False, + no_output_dump_on_exception=True, + text=True, + env=env, + stderr=subprocess.STDOUT, + stdout=f, + ) + except Timeout: + get_console().print("[error]Another asset compilation is running. Exiting[/]\n") + get_console().print("[warning]If you are sure there is no other compilation,[/]") + get_console().print("[warning]Remove the lock file and re-run compilation:[/]") + get_console().print(WWW_ASSET_COMPILE_LOCK) + get_console().print() + sys.exit(1) def run_compile_www_assets( @@ -425,9 +452,11 @@ def run_compile_www_assets( "manual", "compile-www-assets-dev" if dev else "compile-www-assets", "--all-files", + "--verbose", ] + get_console().print(f"[info] The output of the asset compilation is stored in: [/]{WWW_ASSET_OUT_FILE}\n") if run_in_background: - thread = Thread(daemon=True, target=_run_compile_internally, args=(command_to_execute,)) + thread = Thread(daemon=True, target=_run_compile_internally, args=(command_to_execute, dev)) thread.start() else: - return _run_compile_internally(command_to_execute) + return _run_compile_internally(command_to_execute, dev) diff --git a/scripts/ci/docker-compose/local.yml b/scripts/ci/docker-compose/local.yml index b56b44af0aea6..c03b7bcc96c4f 100644 --- a/scripts/ci/docker-compose/local.yml +++ b/scripts/ci/docker-compose/local.yml @@ -33,6 +33,9 @@ services: - type: bind source: ../../../.bash_history target: /root/.bash_history + - type: bind + source: ../../../.build + target: /opt/airflow/.build - type: bind source: ../../../.coveragerc target: /opt/airflow/.coveragerc diff --git a/scripts/ci/pre_commit/pre_commit_compile_www_assets.py b/scripts/ci/pre_commit/pre_commit_compile_www_assets.py index 26f0fb8dc3c2a..a2b4723b93a3c 100755 --- a/scripts/ci/pre_commit/pre_commit_compile_www_assets.py +++ b/scripts/ci/pre_commit/pre_commit_compile_www_assets.py @@ -26,7 +26,7 @@ from common_precommit_utils import get_directory_hash # isort: skip # noqa AIRFLOW_SOURCES_PATH = Path(__file__).parents[3].resolve() -WWW_HASH_FILE = AIRFLOW_SOURCES_PATH / ".build" / "www_dir_hash.txt" +WWW_HASH_FILE = AIRFLOW_SOURCES_PATH / ".build" / "www" / "hash.txt" if __name__ not in ("__main__", "__mp_main__"): raise SystemExit( @@ -42,8 +42,8 @@ if new_hash == old_hash: print("The WWW directory has not changed! Skip regeneration.") sys.exit(0) - WWW_HASH_FILE.write_text(new_hash) env = os.environ.copy() env["FORCE_COLOR"] = "true" subprocess.check_call(["yarn", "install", "--frozen-lockfile"], cwd=str(www_directory)) subprocess.check_call(["yarn", "run", "build"], cwd=str(www_directory), env=env) + WWW_HASH_FILE.write_text(new_hash) diff --git a/scripts/ci/pre_commit/pre_commit_compile_www_assets_dev.py b/scripts/ci/pre_commit/pre_commit_compile_www_assets_dev.py index bcd906eb158f2..778e8d67d1253 100755 --- a/scripts/ci/pre_commit/pre_commit_compile_www_assets_dev.py +++ b/scripts/ci/pre_commit/pre_commit_compile_www_assets_dev.py @@ -28,7 +28,7 @@ ) AIRFLOW_SOURCES_PATH = Path(__file__).parents[3].resolve() -WWW_HASH_FILE = AIRFLOW_SOURCES_PATH / ".build" / "www_dir_hash.txt" +WWW_HASH_FILE = AIRFLOW_SOURCES_PATH / ".build" / "www" / "hash.txt" if __name__ == "__main__": www_directory = Path("airflow") / "www" diff --git a/scripts/docker/entrypoint_ci.sh b/scripts/docker/entrypoint_ci.sh index ca147c61c031d..0301cf4538416 100755 --- a/scripts/docker/entrypoint_ci.sh +++ b/scripts/docker/entrypoint_ci.sh @@ -45,6 +45,35 @@ export AIRFLOW_HOME=${AIRFLOW_HOME:=${HOME}} : "${AIRFLOW_SOURCES:?"ERROR: AIRFLOW_SOURCES not set !!!!"}" +function wait_for_asset_compilation() { + if [[ -f "${AIRFLOW_SOURCES}/.build/www/.asset_compile.lock" ]]; then + echo + echo "${COLOR_YELLOW}Waiting for asset compilation to complete in the background.${COLOR_RESET}" + echo + local counter=0 + while [[ -f "${AIRFLOW_SOURCES}/.build/www/.asset_compile.lock" ]]; do + echo "${COLOR_BLUE}Still waiting .....${COLOR_RESET}" + sleep 1 + ((counter=counter+1)) + if [[ ${counter} == "30" ]]; then + echo + echo "${COLOR_YELLOW}The asset compilation is taking too long.${COLOR_YELLOW}" + echo """ +If it does not complete soon, you might want to stop it and remove file lock: + * press Ctrl-C + * run 'rm ${AIRFLOW_SOURCES}/.build/www/.asset_compile.lock' +""" + fi + if [[ ${counter} == "60" ]]; then + echo + echo "${COLOR_RED}The asset compilation is taking too long. Exiting.${COLOR_RED}" + echo + exit 1 + fi + done + fi +} + if [[ ${SKIP_ENVIRONMENT_INITIALIZATION=} != "true" ]]; then if [[ $(uname -m) == "arm64" || $(uname -m) == "aarch64" ]]; then @@ -232,6 +261,7 @@ if [[ ${SKIP_ENVIRONMENT_INITIALIZATION=} != "true" ]]; then if [[ ${START_AIRFLOW:="false"} == "true" || ${START_AIRFLOW} == "True" ]]; then export AIRFLOW__DATABASE__LOAD_DEFAULT_CONNECTIONS=${LOAD_DEFAULT_CONNECTIONS} export AIRFLOW__CORE__LOAD_EXAMPLES=${LOAD_EXAMPLES} + wait_for_asset_compilation # shellcheck source=scripts/in_container/bin/run_tmux exec run_tmux fi diff --git a/scripts/in_container/bin/run_tmux b/scripts/in_container/bin/run_tmux index 8811863b4c189..a877411d28a35 100755 --- a/scripts/in_container/bin/run_tmux +++ b/scripts/in_container/bin/run_tmux @@ -15,7 +15,6 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. - if [ ! -e /usr/local/bin/stop_airflow ]; then ln -s "/opt/airflow/scripts/in_container/stop_tmux_airflow.sh" /usr/local/bin/stop_airflow || true fi From f9e535b41b90b22e080273ece4812957ebb6fe0b Mon Sep 17 00:00:00 2001 From: Jarek Potiuk Date: Sat, 24 Dec 2022 22:26:57 +0100 Subject: [PATCH 132/158] Output of the "dev" asset compilation for breeze available in file (#28579) When you start airflow in `dev-mode`, the output of asset compilation (which run continuously) is now available in a file that you can run `tail -f` on to see the output. (cherry picked from commit 790b9d173afe0a0ae98d322f2b8986a30a76dd81) --- .../pre_commit_compile_www_assets.py | 4 ++-- .../pre_commit_compile_www_assets_dev.py | 24 +++++++++++++++---- 2 files changed, 22 insertions(+), 6 deletions(-) diff --git a/scripts/ci/pre_commit/pre_commit_compile_www_assets.py b/scripts/ci/pre_commit/pre_commit_compile_www_assets.py index a2b4723b93a3c..27975f6b8c8ed 100755 --- a/scripts/ci/pre_commit/pre_commit_compile_www_assets.py +++ b/scripts/ci/pre_commit/pre_commit_compile_www_assets.py @@ -44,6 +44,6 @@ sys.exit(0) env = os.environ.copy() env["FORCE_COLOR"] = "true" - subprocess.check_call(["yarn", "install", "--frozen-lockfile"], cwd=str(www_directory)) - subprocess.check_call(["yarn", "run", "build"], cwd=str(www_directory), env=env) + subprocess.check_call(["yarn", "install", "--frozen-lockfile"], cwd=os.fspath(www_directory)) + subprocess.check_call(["yarn", "run", "build"], cwd=os.fspath(www_directory), env=env) WWW_HASH_FILE.write_text(new_hash) diff --git a/scripts/ci/pre_commit/pre_commit_compile_www_assets_dev.py b/scripts/ci/pre_commit/pre_commit_compile_www_assets_dev.py index 778e8d67d1253..6d8bfe05cd462 100755 --- a/scripts/ci/pre_commit/pre_commit_compile_www_assets_dev.py +++ b/scripts/ci/pre_commit/pre_commit_compile_www_assets_dev.py @@ -28,14 +28,30 @@ ) AIRFLOW_SOURCES_PATH = Path(__file__).parents[3].resolve() -WWW_HASH_FILE = AIRFLOW_SOURCES_PATH / ".build" / "www" / "hash.txt" +WWW_CACHE_DIR = AIRFLOW_SOURCES_PATH / ".build" / "www" +WWW_HASH_FILE = WWW_CACHE_DIR / "hash.txt" +WWW_ASSET_OUT_FILE = WWW_CACHE_DIR / "asset_compile.out" if __name__ == "__main__": - www_directory = Path("airflow") / "www" + www_directory = AIRFLOW_SOURCES_PATH / "airflow" / "www" if WWW_HASH_FILE.exists(): # cleanup hash of www so that next compile-assets recompiles them WWW_HASH_FILE.unlink() env = os.environ.copy() env["FORCE_COLOR"] = "true" - subprocess.check_call(["yarn", "install", "--frozen-lockfile"], cwd=str(www_directory)) - subprocess.check_call(["yarn", "dev"], cwd=str(www_directory), env=env) + with open(WWW_ASSET_OUT_FILE, "w") as f: + subprocess.run( + ["yarn", "install", "--frozen-lockfile"], + cwd=os.fspath(www_directory), + check=True, + stdout=f, + stderr=subprocess.STDOUT, + ) + subprocess.run( + ["yarn", "dev"], + check=True, + cwd=os.fspath(www_directory), + env=env, + stdout=f, + stderr=subprocess.STDOUT, + ) From 0df6a336a26ca858e04b9a74b0c9185fae38f5e3 Mon Sep 17 00:00:00 2001 From: Jarek Potiuk Date: Thu, 5 Jan 2023 11:31:57 +0100 Subject: [PATCH 133/158] Allow to specify `pip` version via github URL in Docker/CI workflow (#28697) We've only supported to install `pip` from released packages with a version number, but since `pip` does not support RC candidates (as extensively discussed in https://github.com/pypa/pip/issues/10882) we cannot use the release versions to do that. We still want to help `pip` maintainers and be able to test the versions they release as early as possible, so we add support to install `pip` in our toolchain from a GitHub URL. That will allow us to test new `pip` version as soon as designated branch of `pip` will contain something resembling a release candidate ready for testing. (cherry picked from commit 6cbf9b62e374666693b93f79a36dcb46cc8245c4) --- Dockerfile | 48 +++++++++---------- Dockerfile.ci | 40 ++++++++-------- scripts/docker/common.sh | 12 +++++ .../docker/install_additional_dependencies.sh | 6 +-- scripts/docker/install_airflow.sh | 9 ++-- ...ll_airflow_dependencies_from_branch_tip.sh | 3 +- .../install_from_docker_context_files.sh | 8 ++-- scripts/docker/install_pip_version.sh | 10 +--- scripts/in_container/_in_container_utils.sh | 7 ++- .../run_prepare_airflow_packages.sh | 3 +- 10 files changed, 71 insertions(+), 75 deletions(-) diff --git a/Dockerfile b/Dockerfile index bc8502982c29a..3da82f2d2b15b 100644 --- a/Dockerfile +++ b/Dockerfile @@ -325,20 +325,12 @@ COPY <<"EOF" /install_pip_version.sh : "${AIRFLOW_PIP_VERSION:?Should be set}" -function install_pip_version() { - echo - echo "${COLOR_BLUE}Installing pip version ${AIRFLOW_PIP_VERSION}${COLOR_RESET}" - echo - pip install --disable-pip-version-check --no-cache-dir --upgrade "pip==${AIRFLOW_PIP_VERSION}" && - mkdir -p ${HOME}/.local/bin -} - common::get_colors common::get_airflow_version_specification common::override_pip_version_if_needed common::show_pip_version_and_location -install_pip_version +common::install_pip_version EOF # The content below is automatically copied from scripts/docker/install_airflow_dependencies_from_branch_tip.sh @@ -369,8 +361,7 @@ function install_airflow_dependencies_from_branch_tip() { ${ADDITIONAL_PIP_INSTALL_FLAGS} \ "https://github.com/${AIRFLOW_REPO}/archive/${AIRFLOW_BRANCH}.tar.gz#egg=apache-airflow[${AIRFLOW_EXTRAS}]" \ --constraint "${AIRFLOW_CONSTRAINTS_LOCATION}" || true - # make sure correct PIP version is used - pip install --disable-pip-version-check "pip==${AIRFLOW_PIP_VERSION}" 2>/dev/null + common::install_pip_version pip freeze | grep apache-airflow-providers | xargs pip uninstall --yes 2>/dev/null || true set +x echo @@ -445,6 +436,18 @@ function common::show_pip_version_and_location() { echo "pip on path: $(which pip)" echo "Using pip: $(pip --version)" } + +function common::install_pip_version() { + echo + echo "${COLOR_BLUE}Installing pip version ${AIRFLOW_PIP_VERSION}${COLOR_RESET}" + echo + if [[ ${AIRFLOW_PIP_VERSION} =~ .*https.* ]]; then + pip install --disable-pip-version-check --no-cache-dir "pip @ ${AIRFLOW_PIP_VERSION}" + else + pip install --disable-pip-version-check --no-cache-dir "pip==${AIRFLOW_PIP_VERSION}" + fi + mkdir -p "${HOME}/.local/bin" +} EOF # The content below is automatically copied from scripts/docker/pip @@ -531,8 +534,7 @@ function install_airflow_and_providers_from_docker_context_files(){ ${EAGER_UPGRADE_ADDITIONAL_REQUIREMENTS} set +x - # make sure correct PIP version is left installed - pip install --disable-pip-version-check "pip==${AIRFLOW_PIP_VERSION}" 2>/dev/null + common::install_pip_version pip check } @@ -549,9 +551,8 @@ function install_all_other_packages_from_docker_context_files() { set -x pip install ${ADDITIONAL_PIP_INSTALL_FLAGS} \ --root-user-action ignore --force-reinstall --no-deps --no-index ${reinstalling_other_packages} - # make sure correct PIP version is used - pip install --disable-pip-version-check "pip==${AIRFLOW_PIP_VERSION}" 2>/dev/null - set -x + common::install_pip_version + set +x fi } @@ -613,8 +614,7 @@ function install_airflow() { set +x fi - # make sure correct PIP version is used - pip install --disable-pip-version-check "pip==${AIRFLOW_PIP_VERSION}" 2>/dev/null + common::install_pip_version echo echo "${COLOR_BLUE}Running 'pip check'${COLOR_RESET}" echo @@ -628,15 +628,13 @@ function install_airflow() { ${ADDITIONAL_PIP_INSTALL_FLAGS} \ "${AIRFLOW_INSTALLATION_METHOD}[${AIRFLOW_EXTRAS}]${AIRFLOW_VERSION_SPECIFICATION}" \ --constraint "${AIRFLOW_CONSTRAINTS_LOCATION}" - # make sure correct PIP version is used - pip install --disable-pip-version-check "pip==${AIRFLOW_PIP_VERSION}" 2>/dev/null + common::install_pip_version # then upgrade if needed without using constraints to account for new limits in setup.py pip install --root-user-action ignore --upgrade --upgrade-strategy only-if-needed \ ${ADDITIONAL_PIP_INSTALL_FLAGS} \ ${AIRFLOW_INSTALL_EDITABLE_FLAG} \ "${AIRFLOW_INSTALLATION_METHOD}[${AIRFLOW_EXTRAS}]${AIRFLOW_VERSION_SPECIFICATION}" - # make sure correct PIP version is used - pip install --disable-pip-version-check "pip==${AIRFLOW_PIP_VERSION}" 2>/dev/null + common::install_pip_version set +x echo echo "${COLOR_BLUE}Running 'pip check'${COLOR_RESET}" @@ -675,8 +673,7 @@ function install_additional_dependencies() { pip install --root-user-action ignore --upgrade --upgrade-strategy eager \ ${ADDITIONAL_PIP_INSTALL_FLAGS} \ ${ADDITIONAL_PYTHON_DEPS} ${EAGER_UPGRADE_ADDITIONAL_REQUIREMENTS} - # make sure correct PIP version is used - pip install --disable-pip-version-check "pip==${AIRFLOW_PIP_VERSION}" 2>/dev/null + common::install_pip_version set +x echo echo "${COLOR_BLUE}Running 'pip check'${COLOR_RESET}" @@ -690,8 +687,7 @@ function install_additional_dependencies() { pip install --root-user-action ignore --upgrade --upgrade-strategy only-if-needed \ ${ADDITIONAL_PIP_INSTALL_FLAGS} \ ${ADDITIONAL_PYTHON_DEPS} - # make sure correct PIP version is used - pip install --disable-pip-version-check "pip==${AIRFLOW_PIP_VERSION}" 2>/dev/null + common::install_pip_version set +x echo echo "${COLOR_BLUE}Running 'pip check'${COLOR_RESET}" diff --git a/Dockerfile.ci b/Dockerfile.ci index 96a0c9ba4b45c..96172445fe181 100644 --- a/Dockerfile.ci +++ b/Dockerfile.ci @@ -285,20 +285,12 @@ COPY <<"EOF" /install_pip_version.sh : "${AIRFLOW_PIP_VERSION:?Should be set}" -function install_pip_version() { - echo - echo "${COLOR_BLUE}Installing pip version ${AIRFLOW_PIP_VERSION}${COLOR_RESET}" - echo - pip install --disable-pip-version-check --no-cache-dir --upgrade "pip==${AIRFLOW_PIP_VERSION}" && - mkdir -p ${HOME}/.local/bin -} - common::get_colors common::get_airflow_version_specification common::override_pip_version_if_needed common::show_pip_version_and_location -install_pip_version +common::install_pip_version EOF # The content below is automatically copied from scripts/docker/install_airflow_dependencies_from_branch_tip.sh @@ -329,8 +321,7 @@ function install_airflow_dependencies_from_branch_tip() { ${ADDITIONAL_PIP_INSTALL_FLAGS} \ "https://github.com/${AIRFLOW_REPO}/archive/${AIRFLOW_BRANCH}.tar.gz#egg=apache-airflow[${AIRFLOW_EXTRAS}]" \ --constraint "${AIRFLOW_CONSTRAINTS_LOCATION}" || true - # make sure correct PIP version is used - pip install --disable-pip-version-check "pip==${AIRFLOW_PIP_VERSION}" 2>/dev/null + common::install_pip_version pip freeze | grep apache-airflow-providers | xargs pip uninstall --yes 2>/dev/null || true set +x echo @@ -405,6 +396,18 @@ function common::show_pip_version_and_location() { echo "pip on path: $(which pip)" echo "Using pip: $(pip --version)" } + +function common::install_pip_version() { + echo + echo "${COLOR_BLUE}Installing pip version ${AIRFLOW_PIP_VERSION}${COLOR_RESET}" + echo + if [[ ${AIRFLOW_PIP_VERSION} =~ .*https.* ]]; then + pip install --disable-pip-version-check --no-cache-dir "pip @ ${AIRFLOW_PIP_VERSION}" + else + pip install --disable-pip-version-check --no-cache-dir "pip==${AIRFLOW_PIP_VERSION}" + fi + mkdir -p "${HOME}/.local/bin" +} EOF # The content below is automatically copied from scripts/docker/install_pipx_tools.sh @@ -480,8 +483,7 @@ function install_airflow() { set +x fi - # make sure correct PIP version is used - pip install --disable-pip-version-check "pip==${AIRFLOW_PIP_VERSION}" 2>/dev/null + common::install_pip_version echo echo "${COLOR_BLUE}Running 'pip check'${COLOR_RESET}" echo @@ -495,15 +497,13 @@ function install_airflow() { ${ADDITIONAL_PIP_INSTALL_FLAGS} \ "${AIRFLOW_INSTALLATION_METHOD}[${AIRFLOW_EXTRAS}]${AIRFLOW_VERSION_SPECIFICATION}" \ --constraint "${AIRFLOW_CONSTRAINTS_LOCATION}" - # make sure correct PIP version is used - pip install --disable-pip-version-check "pip==${AIRFLOW_PIP_VERSION}" 2>/dev/null + common::install_pip_version # then upgrade if needed without using constraints to account for new limits in setup.py pip install --root-user-action ignore --upgrade --upgrade-strategy only-if-needed \ ${ADDITIONAL_PIP_INSTALL_FLAGS} \ ${AIRFLOW_INSTALL_EDITABLE_FLAG} \ "${AIRFLOW_INSTALLATION_METHOD}[${AIRFLOW_EXTRAS}]${AIRFLOW_VERSION_SPECIFICATION}" - # make sure correct PIP version is used - pip install --disable-pip-version-check "pip==${AIRFLOW_PIP_VERSION}" 2>/dev/null + common::install_pip_version set +x echo echo "${COLOR_BLUE}Running 'pip check'${COLOR_RESET}" @@ -542,8 +542,7 @@ function install_additional_dependencies() { pip install --root-user-action ignore --upgrade --upgrade-strategy eager \ ${ADDITIONAL_PIP_INSTALL_FLAGS} \ ${ADDITIONAL_PYTHON_DEPS} ${EAGER_UPGRADE_ADDITIONAL_REQUIREMENTS} - # make sure correct PIP version is used - pip install --disable-pip-version-check "pip==${AIRFLOW_PIP_VERSION}" 2>/dev/null + common::install_pip_version set +x echo echo "${COLOR_BLUE}Running 'pip check'${COLOR_RESET}" @@ -557,8 +556,7 @@ function install_additional_dependencies() { pip install --root-user-action ignore --upgrade --upgrade-strategy only-if-needed \ ${ADDITIONAL_PIP_INSTALL_FLAGS} \ ${ADDITIONAL_PYTHON_DEPS} - # make sure correct PIP version is used - pip install --disable-pip-version-check "pip==${AIRFLOW_PIP_VERSION}" 2>/dev/null + common::install_pip_version set +x echo echo "${COLOR_BLUE}Running 'pip check'${COLOR_RESET}" diff --git a/scripts/docker/common.sh b/scripts/docker/common.sh index e7cfefaea79b9..fab02a3b4fb41 100644 --- a/scripts/docker/common.sh +++ b/scripts/docker/common.sh @@ -70,3 +70,15 @@ function common::show_pip_version_and_location() { echo "pip on path: $(which pip)" echo "Using pip: $(pip --version)" } + +function common::install_pip_version() { + echo + echo "${COLOR_BLUE}Installing pip version ${AIRFLOW_PIP_VERSION}${COLOR_RESET}" + echo + if [[ ${AIRFLOW_PIP_VERSION} =~ .*https.* ]]; then + pip install --disable-pip-version-check --no-cache-dir "pip @ ${AIRFLOW_PIP_VERSION}" + else + pip install --disable-pip-version-check --no-cache-dir "pip==${AIRFLOW_PIP_VERSION}" + fi + mkdir -p "${HOME}/.local/bin" +} diff --git a/scripts/docker/install_additional_dependencies.sh b/scripts/docker/install_additional_dependencies.sh index 9b42445d2b6cd..a745d576f363a 100644 --- a/scripts/docker/install_additional_dependencies.sh +++ b/scripts/docker/install_additional_dependencies.sh @@ -35,8 +35,7 @@ function install_additional_dependencies() { pip install --root-user-action ignore --upgrade --upgrade-strategy eager \ ${ADDITIONAL_PIP_INSTALL_FLAGS} \ ${ADDITIONAL_PYTHON_DEPS} ${EAGER_UPGRADE_ADDITIONAL_REQUIREMENTS} - # make sure correct PIP version is used - pip install --disable-pip-version-check "pip==${AIRFLOW_PIP_VERSION}" 2>/dev/null + common::install_pip_version set +x echo echo "${COLOR_BLUE}Running 'pip check'${COLOR_RESET}" @@ -50,8 +49,7 @@ function install_additional_dependencies() { pip install --root-user-action ignore --upgrade --upgrade-strategy only-if-needed \ ${ADDITIONAL_PIP_INSTALL_FLAGS} \ ${ADDITIONAL_PYTHON_DEPS} - # make sure correct PIP version is used - pip install --disable-pip-version-check "pip==${AIRFLOW_PIP_VERSION}" 2>/dev/null + common::install_pip_version set +x echo echo "${COLOR_BLUE}Running 'pip check'${COLOR_RESET}" diff --git a/scripts/docker/install_airflow.sh b/scripts/docker/install_airflow.sh index 3ee5b168aa64b..265d2089c3e60 100644 --- a/scripts/docker/install_airflow.sh +++ b/scripts/docker/install_airflow.sh @@ -69,8 +69,7 @@ function install_airflow() { set +x fi - # make sure correct PIP version is used - pip install --disable-pip-version-check "pip==${AIRFLOW_PIP_VERSION}" 2>/dev/null + common::install_pip_version echo echo "${COLOR_BLUE}Running 'pip check'${COLOR_RESET}" echo @@ -84,15 +83,13 @@ function install_airflow() { ${ADDITIONAL_PIP_INSTALL_FLAGS} \ "${AIRFLOW_INSTALLATION_METHOD}[${AIRFLOW_EXTRAS}]${AIRFLOW_VERSION_SPECIFICATION}" \ --constraint "${AIRFLOW_CONSTRAINTS_LOCATION}" - # make sure correct PIP version is used - pip install --disable-pip-version-check "pip==${AIRFLOW_PIP_VERSION}" 2>/dev/null + common::install_pip_version # then upgrade if needed without using constraints to account for new limits in setup.py pip install --root-user-action ignore --upgrade --upgrade-strategy only-if-needed \ ${ADDITIONAL_PIP_INSTALL_FLAGS} \ ${AIRFLOW_INSTALL_EDITABLE_FLAG} \ "${AIRFLOW_INSTALLATION_METHOD}[${AIRFLOW_EXTRAS}]${AIRFLOW_VERSION_SPECIFICATION}" - # make sure correct PIP version is used - pip install --disable-pip-version-check "pip==${AIRFLOW_PIP_VERSION}" 2>/dev/null + common::install_pip_version set +x echo echo "${COLOR_BLUE}Running 'pip check'${COLOR_RESET}" diff --git a/scripts/docker/install_airflow_dependencies_from_branch_tip.sh b/scripts/docker/install_airflow_dependencies_from_branch_tip.sh index 60f82efa492a9..2b58d03950e5b 100644 --- a/scripts/docker/install_airflow_dependencies_from_branch_tip.sh +++ b/scripts/docker/install_airflow_dependencies_from_branch_tip.sh @@ -52,8 +52,7 @@ function install_airflow_dependencies_from_branch_tip() { ${ADDITIONAL_PIP_INSTALL_FLAGS} \ "https://github.com/${AIRFLOW_REPO}/archive/${AIRFLOW_BRANCH}.tar.gz#egg=apache-airflow[${AIRFLOW_EXTRAS}]" \ --constraint "${AIRFLOW_CONSTRAINTS_LOCATION}" || true - # make sure correct PIP version is used - pip install --disable-pip-version-check "pip==${AIRFLOW_PIP_VERSION}" 2>/dev/null + common::install_pip_version pip freeze | grep apache-airflow-providers | xargs pip uninstall --yes 2>/dev/null || true set +x echo diff --git a/scripts/docker/install_from_docker_context_files.sh b/scripts/docker/install_from_docker_context_files.sh index e21814f40e9c2..250bbaed21a1d 100644 --- a/scripts/docker/install_from_docker_context_files.sh +++ b/scripts/docker/install_from_docker_context_files.sh @@ -85,8 +85,7 @@ function install_airflow_and_providers_from_docker_context_files(){ ${EAGER_UPGRADE_ADDITIONAL_REQUIREMENTS} set +x - # make sure correct PIP version is left installed - pip install --disable-pip-version-check "pip==${AIRFLOW_PIP_VERSION}" 2>/dev/null + common::install_pip_version pip check } @@ -107,9 +106,8 @@ function install_all_other_packages_from_docker_context_files() { set -x pip install ${ADDITIONAL_PIP_INSTALL_FLAGS} \ --root-user-action ignore --force-reinstall --no-deps --no-index ${reinstalling_other_packages} - # make sure correct PIP version is used - pip install --disable-pip-version-check "pip==${AIRFLOW_PIP_VERSION}" 2>/dev/null - set -x + common::install_pip_version + set +x fi } diff --git a/scripts/docker/install_pip_version.sh b/scripts/docker/install_pip_version.sh index 60d804cd10f09..efe4f23c2fb18 100644 --- a/scripts/docker/install_pip_version.sh +++ b/scripts/docker/install_pip_version.sh @@ -20,17 +20,9 @@ : "${AIRFLOW_PIP_VERSION:?Should be set}" -function install_pip_version() { - echo - echo "${COLOR_BLUE}Installing pip version ${AIRFLOW_PIP_VERSION}${COLOR_RESET}" - echo - pip install --disable-pip-version-check --no-cache-dir --upgrade "pip==${AIRFLOW_PIP_VERSION}" && - mkdir -p ${HOME}/.local/bin -} - common::get_colors common::get_airflow_version_specification common::override_pip_version_if_needed common::show_pip_version_and_location -install_pip_version +common::install_pip_version diff --git a/scripts/in_container/_in_container_utils.sh b/scripts/in_container/_in_container_utils.sh index 7d9221625fa4b..3240b2c90dacc 100644 --- a/scripts/in_container/_in_container_utils.sh +++ b/scripts/in_container/_in_container_utils.sh @@ -353,7 +353,12 @@ function setup_provider_packages() { function install_supported_pip_version() { - pip install --disable-pip-version-check "pip==${AIRFLOW_PIP_VERSION}" + if [[ ${AIRFLOW_PIP_VERSION} =~ .*https.* ]]; then + pip install --disable-pip-version-check "pip @ ${AIRFLOW_PIP_VERSION}" + else + pip install --disable-pip-version-check "pip==${AIRFLOW_PIP_VERSION}" + fi + } function filename_to_python_module() { diff --git a/scripts/in_container/run_prepare_airflow_packages.sh b/scripts/in_container/run_prepare_airflow_packages.sh index 0ba7871494194..68f9dc92be92a 100755 --- a/scripts/in_container/run_prepare_airflow_packages.sh +++ b/scripts/in_container/run_prepare_airflow_packages.sh @@ -34,7 +34,8 @@ function prepare_airflow_packages() { rm -rf -- *egg-info* rm -rf -- build - pip install --disable-pip-version-check "pip==${AIRFLOW_PIP_VERSION}" "wheel==${WHEEL_VERSION}" + install_supported_pip_version + pip install "wheel==${WHEEL_VERSION}" local packages=() From 031c260a838e18ded4446fa4cb9e200d9674f893 Mon Sep 17 00:00:00 2001 From: Jarek Potiuk Date: Sun, 4 Dec 2022 17:57:30 +0100 Subject: [PATCH 134/158] Add Andrey as allowed to use self-hosted runners (#28095) (cherry picked from commit 582df47de938938406e5bd4abf167c18fc3de16b) --- .github/workflows/ci.yml | 1 + COMMITTERS.rst | 1 + 2 files changed, 2 insertions(+) diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index 5434c46ba6913..87610294a6bc2 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -73,6 +73,7 @@ jobs: "BasPH", "Fokko", "KevinYang21", + "Taragolis", "XD-DENG", "aijamalnk", "alexvanboxel", diff --git a/COMMITTERS.rst b/COMMITTERS.rst index 5b8a942503b52..3d05f2ed197e0 100644 --- a/COMMITTERS.rst +++ b/COMMITTERS.rst @@ -187,3 +187,4 @@ To be able to merge PRs, committers have to integrate their GitHub ID with Apach 3. Merge your Apache and GitHub accounts using `GitBox (Apache Account Linking utility) `__. You should see 3 green checks in GitBox. 4. Wait at least 30 minutes for an email inviting you to Apache GitHub Organization and accept invitation. 5. After accepting the GitHub Invitation verify that you are a member of the `Airflow committers team on GitHub `__. +6. Ask in ``#internal-airflow-ci-cd`` channel to be `configured in self-hosted runners `_ by the CI maintainers From 93f9e1c8cffcdf85b5c58b1e6ec4972c461dd447 Mon Sep 17 00:00:00 2001 From: Jarek Potiuk Date: Wed, 21 Dec 2022 07:17:19 +0100 Subject: [PATCH 135/158] Fix selective checks handling error tracebacks in CI (#28514) Initially selective check was implemented in the way that it printed diagnostic output on stdout and the GITHUB_OUTPUT compatible set of outputs on stderr so that it could be redirected to the GITHUB_OUTPUT in its entirety. But this turned out to be a bad idea because when there was an error generated in selective-checks themselves, the traceback was printed in stderr and redirecting stderr to GITHUB_OUTPUT swallowed the traceback. This change reverses the behaviour: * diagnostic output is printed to stderr * GITHUB_OUTPUT compatible output is printed to stdout This way when traceback happens it is printed to stderr and is not swalleowed by redirection to GITHUB_OUTPUT (cherry picked from commit bc7feda66ed7bb2f2940fa90ef26ff90dd7a8c80) --- .github/workflows/ci.yml | 3 +- .../airflow_breeze/commands/ci_commands.py | 12 +++--- .../airflow_breeze/utils/github_actions.py | 4 +- .../airflow_breeze/utils/selective_checks.py | 38 +++++++++---------- 4 files changed, 29 insertions(+), 28 deletions(-) diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index 87610294a6bc2..e8c18eb21eb3e 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -222,7 +222,8 @@ jobs: env: PR_LABELS: "${{ steps.source-run-info.outputs.pr-labels }}" COMMIT_REF: "${{ github.sha }}" - run: breeze ci selective-check 2>> ${GITHUB_OUTPUT} + VERBOSE: "false" + run: breeze ci selective-check >> ${GITHUB_OUTPUT} - name: env run: printenv env: diff --git a/dev/breeze/src/airflow_breeze/commands/ci_commands.py b/dev/breeze/src/airflow_breeze/commands/ci_commands.py index 39de15a76a08a..e62875630be7f 100644 --- a/dev/breeze/src/airflow_breeze/commands/ci_commands.py +++ b/dev/breeze/src/airflow_breeze/commands/ci_commands.py @@ -50,7 +50,7 @@ option_verbose, ) from airflow_breeze.utils.confirm import Answer, user_confirm -from airflow_breeze.utils.console import get_console +from airflow_breeze.utils.console import get_console, get_stderr_console from airflow_breeze.utils.custom_param_types import BetterChoice from airflow_breeze.utils.docker_command_utils import ( check_docker_resources, @@ -179,14 +179,14 @@ def get_changed_files(commit_ref: str | None) -> tuple[str, ...]: ] result = run_command(cmd, check=False, capture_output=True, text=True) if result.returncode != 0: - get_console().print( + get_stderr_console().print( f"[warning] Error when running diff-tree command [/]\n{result.stdout}\n{result.stderr}" ) return () changed_files = tuple(result.stdout.splitlines()) if result.stdout else () - get_console().print("\n[info]Changed files:[/]\n") - get_console().print(changed_files) - get_console().print() + get_stderr_console().print("\n[info]Changed files:[/]\n") + get_stderr_console().print(changed_files) + get_stderr_console().print() return changed_files @@ -250,7 +250,7 @@ def selective_check( pr_labels=tuple(ast.literal_eval(pr_labels)) if pr_labels else (), github_event=github_event, ) - print(str(sc), file=sys.stderr) + print(str(sc), file=sys.stdout) @ci_group.command(name="find-newer-dependencies", help="Finds which dependencies are being upgraded.") diff --git a/dev/breeze/src/airflow_breeze/utils/github_actions.py b/dev/breeze/src/airflow_breeze/utils/github_actions.py index 6b8043aa7e726..1566bbe81c8d6 100644 --- a/dev/breeze/src/airflow_breeze/utils/github_actions.py +++ b/dev/breeze/src/airflow_breeze/utils/github_actions.py @@ -20,11 +20,11 @@ from rich.markup import escape -from airflow_breeze.utils.console import get_console +from airflow_breeze.utils.console import get_stderr_console def get_ga_output(name: str, value: Any) -> str: output_name = name.replace("_", "-") printed_value = str(value).lower() if isinstance(value, bool) else value - get_console().print(f"[info]{output_name}[/] = [green]{escape(str(printed_value))}[/]") + get_stderr_console().print(f"[info]{output_name}[/] = [green]{escape(str(printed_value))}[/]") return f"{output_name}={printed_value}" diff --git a/dev/breeze/src/airflow_breeze/utils/selective_checks.py b/dev/breeze/src/airflow_breeze/utils/selective_checks.py index 294ed7ed0ecb2..91f311e21f7bb 100644 --- a/dev/breeze/src/airflow_breeze/utils/selective_checks.py +++ b/dev/breeze/src/airflow_breeze/utils/selective_checks.py @@ -54,7 +54,7 @@ SelectiveUnitTestTypes, all_selective_test_types, ) -from airflow_breeze.utils.console import get_console +from airflow_breeze.utils.console import get_stderr_console FULL_TESTS_NEEDED_LABEL = "full tests needed" DEBUG_CI_RESOURCES_LABEL = "debug ci resources" @@ -295,16 +295,16 @@ def default_constraints_branch(self) -> str: @cached_property def full_tests_needed(self) -> bool: if not self._commit_ref: - get_console().print("[warning]Running everything as commit is missing[/]") + get_stderr_console().print("[warning]Running everything as commit is missing[/]") return True if self._github_event in [GithubEvents.PUSH, GithubEvents.SCHEDULE, GithubEvents.WORKFLOW_DISPATCH]: - get_console().print(f"[warning]Full tests needed because event is {self._github_event}[/]") + get_stderr_console().print(f"[warning]Full tests needed because event is {self._github_event}[/]") return True if len(self._matching_files(FileGroupForCi.ENVIRONMENT_FILES, CI_FILE_GROUP_MATCHES)) > 0: - get_console().print("[warning]Running everything because env files changed[/]") + get_stderr_console().print("[warning]Running everything because env files changed[/]") return True if FULL_TESTS_NEEDED_LABEL in self._pr_labels: - get_console().print( + get_stderr_console().print( "[warning]Full tests needed because " f"label '{FULL_TESTS_NEEDED_LABEL}' is in {self._pr_labels}[/]" ) @@ -434,24 +434,24 @@ def _matching_files(self, match_group: T, match_dict: dict[T, list[str]]) -> lis self._match_files_with_regexps(matched_files, regexps) count = len(matched_files) if count > 0: - get_console().print(f"[warning]{match_group} matched {count} files.[/]") - get_console().print(matched_files) + get_stderr_console().print(f"[warning]{match_group} matched {count} files.[/]") + get_stderr_console().print(matched_files) else: - get_console().print(f"[warning]{match_group} did not match any file.[/]") + get_stderr_console().print(f"[warning]{match_group} did not match any file.[/]") return matched_files def _should_be_run(self, source_area: FileGroupForCi) -> bool: if self.full_tests_needed: - get_console().print(f"[warning]{source_area} enabled because we are running everything[/]") + get_stderr_console().print(f"[warning]{source_area} enabled because we are running everything[/]") return True matched_files = self._matching_files(source_area, CI_FILE_GROUP_MATCHES) if len(matched_files) > 0: - get_console().print( + get_stderr_console().print( f"[warning]{source_area} enabled because it matched {len(matched_files)} changed files[/]" ) return True else: - get_console().print( + get_stderr_console().print( f"[warning]{source_area} disabled because it did not match any changed files[/]" ) return False @@ -503,7 +503,7 @@ def _select_test_type_if_matching( count = len(matched_files) if count > 0: test_types.add(test_type.value) - get_console().print(f"[warning]{test_type} added because it matched {count} files[/]") + get_stderr_console().print(f"[warning]{test_type} added because it matched {count} files[/]") return matched_files def _get_test_types_to_run(self) -> list[str]: @@ -528,11 +528,11 @@ def _get_test_types_to_run(self) -> list[str]: remaining_files = set(all_source_files) - set(matched_files) - set(kubernetes_files) count_remaining_files = len(remaining_files) if count_remaining_files > 0: - get_console().print( + get_stderr_console().print( f"[warning]We should run all tests. There are {count_remaining_files} changed " "files that seems to fall into Core/Other category[/]" ) - get_console().print(remaining_files) + get_stderr_console().print(remaining_files) candidate_test_types.update(all_selective_test_types()) else: if "Providers" in candidate_test_types: @@ -540,12 +540,12 @@ def _get_test_types_to_run(self) -> list[str]: if len(affected_providers) != 0: candidate_test_types.remove("Providers") candidate_test_types.add(f"Providers[{','.join(sorted(affected_providers))}]") - get_console().print( + get_stderr_console().print( "[warning]There are no core/other files. Only tests relevant to the changed files are run.[/]" ) sorted_candidate_test_types = list(sorted(candidate_test_types)) - get_console().print("[warning]Selected test type candidates to run:[/]") - get_console().print(sorted_candidate_test_types) + get_stderr_console().print("[warning]Selected test type candidates to run:[/]") + get_stderr_console().print(sorted_candidate_test_types) return sorted_candidate_test_types @cached_property @@ -560,13 +560,13 @@ def test_types(self) -> str: test_types_to_remove: set[str] = set() for test_type in current_test_types: if test_type.startswith("Providers"): - get_console().print( + get_stderr_console().print( f"[warning]Removing {test_type} because the target branch " f"is {self._default_branch} and not main[/]" ) test_types_to_remove.add(test_type) if "Integration" in current_test_types: - get_console().print( + get_stderr_console().print( "[warning]Removing 'Integration' because the target branch " f"is {self._default_branch} and not main[/]" ) From b7cf404fc259650017b574ffa87fea412bde0553 Mon Sep 17 00:00:00 2001 From: Victor Chiapaikeo Date: Wed, 21 Dec 2022 00:29:56 -0500 Subject: [PATCH 136/158] Remove deprecated AIPlatformConsoleLinkk from google/provider.yaml (#28449) (cherry picked from commit 7950fb9711384f8ac4609fc19f319edb17e296ef) --- airflow/providers/google/provider.yaml | 1 - 1 file changed, 1 deletion(-) diff --git a/airflow/providers/google/provider.yaml b/airflow/providers/google/provider.yaml index ab625070d2619..319c061432d93 100644 --- a/airflow/providers/google/provider.yaml +++ b/airflow/providers/google/provider.yaml @@ -964,7 +964,6 @@ connection-types: extra-links: - airflow.providers.google.cloud.operators.bigquery.BigQueryConsoleLink - airflow.providers.google.cloud.operators.bigquery.BigQueryConsoleIndexableLink - - airflow.providers.google.cloud.operators.mlengine.AIPlatformConsoleLink - airflow.providers.google.cloud.links.dataform.DataformRepositoryLink - airflow.providers.google.cloud.links.dataform.DataformWorkspaceLink - airflow.providers.google.cloud.links.dataform.DataformWorkflowInvocationLink From 8dd14977f2af27f9f867c7126347a516b4026536 Mon Sep 17 00:00:00 2001 From: eladkal <45845474+eladkal@users.noreply.github.com> Date: Tue, 13 Dec 2022 23:43:16 +0200 Subject: [PATCH 137/158] Fix tag_providers.sh (#28345) to support both https/ssh (cherry picked from commit 7f9727fc45d8b52b390e3afcffa62c4be36dc2a4) --- dev/provider_packages/tag_providers.sh | 2 +- docs/exts/docs_build/spelling_checks.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dev/provider_packages/tag_providers.sh b/dev/provider_packages/tag_providers.sh index d06c7e10e6ca6..9a115c3ea07c4 100755 --- a/dev/provider_packages/tag_providers.sh +++ b/dev/provider_packages/tag_providers.sh @@ -20,7 +20,7 @@ AIRFLOW_SOURCES="$(cd "$(dirname "${BASH_SOURCE[0]}")/../../" && pwd)" # Check common named remotes for the upstream repo for remote in origin apache; do - git remote get-url --push "$remote" 2>/dev/null | grep -q git@github.com:apache/airflow && break + git remote get-url --push "$remote" 2>/dev/null | grep -q apache/airflow.git && break unset remote done diff --git a/docs/exts/docs_build/spelling_checks.py b/docs/exts/docs_build/spelling_checks.py index f89bfa50dc587..bbaa9fa5dde79 100644 --- a/docs/exts/docs_build/spelling_checks.py +++ b/docs/exts/docs_build/spelling_checks.py @@ -21,10 +21,10 @@ from functools import total_ordering from typing import NamedTuple -from docs.exts.docs_build.code_utils import CONSOLE_WIDTH from rich.console import Console from airflow.utils.code_utils import prepare_code_snippet +from docs.exts.docs_build.code_utils import CONSOLE_WIDTH CURRENT_DIR = os.path.abspath(os.path.join(os.path.dirname(__file__))) DOCS_DIR = os.path.abspath(os.path.join(CURRENT_DIR, os.pardir, os.pardir)) From d7db157adf1ad1fc687ce6e203d71fab7f495f25 Mon Sep 17 00:00:00 2001 From: Jarek Potiuk Date: Sun, 18 Dec 2022 17:04:48 +0100 Subject: [PATCH 138/158] fIx isort problems introduced by recent isort release (#28434) The recent isort changed their mind on sorting the imports. This change follows the change and bumps isort to latest released version (isort has no install_requires on its own so bumping min version has no effect on other dependencies) This change adds a number of isort:skip_file, isort:off, isort:skips in order to handle a very annoying bug in isort, that no matter how much you try, it sometimes treat "known first party" packages differently - depending on how many files it processes at a time. We should be able to restore it after this bug is fixed: https://github.com/PyCQA/isort/issues/2045 This change also updates the common.sql API to skip them from isort for the very same reason (depending on how many files are modified, the isort order might change. (cherry picked from commit f115b207bc844c10569b2df6fc9acfa32a3c7f41) --- .pre-commit-config.yaml | 10 +++++----- docker_tests/test_docker_compose_quick_start.py | 4 ++++ docker_tests/test_examples_of_prod_image_building.py | 3 +++ docker_tests/test_prod_image.py | 3 +++ docs/build_docs.py | 6 ++++++ docs/exts/docs_build/dev_index_generator.py | 3 +++ docs/exts/docs_build/errors.py | 4 +++- docs/publish_docs.py | 3 +++ kubernetes_tests/test_kubernetes_executor.py | 2 +- kubernetes_tests/test_other_executors.py | 2 +- pyproject.toml | 1 + setup.py | 5 ++++- 12 files changed, 37 insertions(+), 9 deletions(-) diff --git a/.pre-commit-config.yaml b/.pre-commit-config.yaml index 810ac0fc1bb1e..0405cbd86174f 100644 --- a/.pre-commit-config.yaml +++ b/.pre-commit-config.yaml @@ -146,6 +146,11 @@ repos: - --fuzzy-match-generates-todo files: > \.cfg$|\.conf$|\.ini$|\.ldif$|\.properties$|\.readthedocs$|\.service$|\.tf$|Dockerfile.*$ + - repo: https://github.com/PyCQA/isort + rev: 5.11.2 + hooks: + - id: isort + name: Run isort to sort imports in Python files # Keep version of black in sync wit blacken-docs and pre-commit-hook-names - repo: https://github.com/psf/black rev: 22.12.0 @@ -233,11 +238,6 @@ repos: entry: yamllint -c yamllint-config.yml --strict types: [yaml] exclude: ^.*init_git_sync\.template\.yaml$|^.*airflow\.template\.yaml$|^chart/(?:templates|files)/.*\.yaml$|openapi/.*\.yaml$|^\.pre-commit-config\.yaml$|^airflow/_vendor/ - - repo: https://github.com/PyCQA/isort - rev: 5.10.1 - hooks: - - id: isort - name: Run isort to sort imports in Python files - repo: https://github.com/pycqa/pydocstyle rev: 6.1.1 hooks: diff --git a/docker_tests/test_docker_compose_quick_start.py b/docker_tests/test_docker_compose_quick_start.py index fd553ed1756eb..4754aac3299ab 100644 --- a/docker_tests/test_docker_compose_quick_start.py +++ b/docker_tests/test_docker_compose_quick_start.py @@ -27,10 +27,14 @@ from unittest import mock import requests + +# isort:off (needed to workaround isort bug) from docker_tests.command_utils import run_command from docker_tests.constants import SOURCE_ROOT from docker_tests.docker_tests_utils import docker_image +# isort:on (needed to workaround isort bug) + AIRFLOW_WWW_USER_USERNAME = os.environ.get("_AIRFLOW_WWW_USER_USERNAME", "airflow") AIRFLOW_WWW_USER_PASSWORD = os.environ.get("_AIRFLOW_WWW_USER_PASSWORD", "airflow") DAG_ID = "example_bash_operator" diff --git a/docker_tests/test_examples_of_prod_image_building.py b/docker_tests/test_examples_of_prod_image_building.py index 978473e64e4ec..858e4c1e8fc9c 100644 --- a/docker_tests/test_examples_of_prod_image_building.py +++ b/docker_tests/test_examples_of_prod_image_building.py @@ -25,9 +25,12 @@ import pytest import requests +# isort:off (needed to workaround isort bug) from docker_tests.command_utils import run_command from docker_tests.constants import SOURCE_ROOT +# isort:on (needed to workaround isort bug) + DOCKER_EXAMPLES_DIR = SOURCE_ROOT / "docs" / "docker-stack" / "docker-examples" diff --git a/docker_tests/test_prod_image.py b/docker_tests/test_prod_image.py index f1d7fbd0180a9..8ad2d89a9f567 100644 --- a/docker_tests/test_prod_image.py +++ b/docker_tests/test_prod_image.py @@ -24,6 +24,7 @@ import pytest +# isort:off (needed to workaround isort bug) from docker_tests.command_utils import run_command from docker_tests.constants import SOURCE_ROOT from docker_tests.docker_tests_utils import ( @@ -32,6 +33,8 @@ run_bash_in_docker, run_python_in_docker, ) + +# isort:on (needed to workaround isort bug) from setup import PREINSTALLED_PROVIDERS INSTALLED_PROVIDER_PATH = SOURCE_ROOT / "scripts" / "ci" / "installed_providers.txt" diff --git a/docs/build_docs.py b/docs/build_docs.py index cd6c83249d4f3..273858be7636a 100755 --- a/docs/build_docs.py +++ b/docs/build_docs.py @@ -15,6 +15,11 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. +""" +Builds documentation and runs spell checking + +# isort:skip_file (needed to workaround isort bug) +""" from __future__ import annotations import argparse @@ -33,6 +38,7 @@ from docs.exts.docs_build.github_action_utils import with_group from docs.exts.docs_build.package_filter import process_package_filters from docs.exts.docs_build.spelling_checks import SpellingError, display_spelling_error_summary + from rich.console import Console from tabulate import tabulate diff --git a/docs/exts/docs_build/dev_index_generator.py b/docs/exts/docs_build/dev_index_generator.py index f423ed20f7703..0aee91b8f4a94 100644 --- a/docs/exts/docs_build/dev_index_generator.py +++ b/docs/exts/docs_build/dev_index_generator.py @@ -23,8 +23,11 @@ import jinja2 +# isort:off (needed to workaround isort bug) from docs.exts.provider_yaml_utils import load_package_data +# isort:on (needed to workaround isort bug) + CURRENT_DIR = os.path.abspath(os.path.dirname(__file__)) DOCS_DIR = os.path.abspath(os.path.join(CURRENT_DIR, os.pardir, os.pardir)) BUILD_DIR = os.path.abspath(os.path.join(DOCS_DIR, "_build")) diff --git a/docs/exts/docs_build/errors.py b/docs/exts/docs_build/errors.py index 187b89c15aaf2..321379896ca87 100644 --- a/docs/exts/docs_build/errors.py +++ b/docs/exts/docs_build/errors.py @@ -23,7 +23,9 @@ from rich.console import Console from airflow.utils.code_utils import prepare_code_snippet -from docs.exts.docs_build.code_utils import CONSOLE_WIDTH + +from docs.exts.docs_build.code_utils import CONSOLE_WIDTH # isort:skip (needed to workaround isort bug) + CURRENT_DIR = os.path.abspath(os.path.join(os.path.dirname(__file__))) DOCS_DIR = os.path.abspath(os.path.join(CURRENT_DIR, os.pardir, os.pardir)) diff --git a/docs/publish_docs.py b/docs/publish_docs.py index bf337995043a9..9a17aa29ef48e 100755 --- a/docs/publish_docs.py +++ b/docs/publish_docs.py @@ -21,10 +21,13 @@ import argparse import os +# isort:off (needed to workaround isort bug) from exts.docs_build.docs_builder import AirflowDocsBuilder from exts.docs_build.package_filter import process_package_filters from exts.provider_yaml_utils import load_package_data +# isort:on (needed to workaround isort bug) + AIRFLOW_SITE_DIR = os.environ.get("AIRFLOW_SITE_DIRECTORY") diff --git a/kubernetes_tests/test_kubernetes_executor.py b/kubernetes_tests/test_kubernetes_executor.py index 7f40a976103fc..95afb25590f44 100644 --- a/kubernetes_tests/test_kubernetes_executor.py +++ b/kubernetes_tests/test_kubernetes_executor.py @@ -20,7 +20,7 @@ import pytest -from kubernetes_tests.test_base import EXECUTOR, TestBase +from kubernetes_tests.test_base import EXECUTOR, TestBase # isort:skip (needed to workaround isort bug) @pytest.mark.skipif(EXECUTOR != "KubernetesExecutor", reason="Only runs on KubernetesExecutor") diff --git a/kubernetes_tests/test_other_executors.py b/kubernetes_tests/test_other_executors.py index f517fdc82fd5c..42a8258871a2c 100644 --- a/kubernetes_tests/test_other_executors.py +++ b/kubernetes_tests/test_other_executors.py @@ -20,7 +20,7 @@ import pytest -from kubernetes_tests.test_base import EXECUTOR, TestBase +from kubernetes_tests.test_base import EXECUTOR, TestBase # isort:skip (needed to workaround isort bug) # These tests are here because only KubernetesExecutor can run the tests in diff --git a/pyproject.toml b/pyproject.toml index 39cfef27e47d4..a40b7d9179902 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -35,5 +35,6 @@ known_first_party = ["airflow", "airflow_breeze", "docker_tests", "docs", "kuber # The test_python.py is needed because adding __future__.annotations breaks runtime checks that are # needed for the test to work skip = ["build", ".tox", "venv", "tests/decorators/test_python.py"] +lines_between_types = 0 skip_glob = ["*.pyi"] profile = "black" diff --git a/setup.py b/setup.py index 2484b2076d0ec..c7af47ffcc55f 100644 --- a/setup.py +++ b/setup.py @@ -377,7 +377,10 @@ def write_version(filename: str = str(AIRFLOW_SOURCES_ROOT / "airflow" / "git_ve "freezegun", "gitpython", "ipdb", - "isort", + # make sure that we are using stable sorting order from 5.* version (some changes were introduced + # in 5.11.3. Black is not compatible yet, so we need to limit isort + # we can remove the limit when black and isort agree on the order + "isort==5.11.2", "jira", "jsondiff", "mongomock", From 39a5d2f52292b664b4cc0ab9c62e3af4cee42a8e Mon Sep 17 00:00:00 2001 From: John Thomas <49878111+TohnJhomas@users.noreply.github.com> Date: Fri, 16 Dec 2022 15:24:33 -0700 Subject: [PATCH 139/158] Restructure Docs (#27235) A big restructuring of our docs to make them easier to reason about - in a much more logical and consistent way. Likel other updates and content changes will be needed to make it even better. (cherry picked from commit 401fc57e8ba1dddb041e0d777bb0277a09f227db) --- .github/boring-cyborg.yml | 20 ++--- RELEASE_NOTES.rst | 4 +- airflow/hooks/subprocess.py | 2 +- .../providers/cncf/kubernetes/CHANGELOG.rst | 2 +- .../operators.rst | 4 +- .../operators/index.rst | 2 +- .../core-extensions/auth-backends.rst | 2 +- .../core-extensions/connections.rst | 2 +- .../core-extensions/logging.rst | 2 +- .../core-extensions/secrets-backends.rst | 2 +- .../cluster-policies.rst | 0 .../dag-serialization.rst | 2 +- .../administration-and-deployment/index.rst | 37 ++++++++ .../kubernetes.rst | 4 +- .../lineage.rst | 0 .../listeners.rst | 2 +- .../logging-monitoring/callbacks.rst | 2 +- .../logging-monitoring/check-health.rst | 0 .../logging-monitoring/errors.rst | 0 .../logging-monitoring/index.rst | 0 .../logging-architecture.rst | 6 +- .../logging-monitoring/logging-tasks.rst | 0 .../logging-monitoring/metrics.rst | 0 .../tracking-user-activity.rst | 2 +- .../modules_management.rst | 2 +- .../pools.rst | 0 .../priority-weight.rst | 0 .../production-deployment.rst | 12 +-- .../scheduler.rst | 8 +- .../security/access-control.rst | 2 +- .../security/api.rst | 0 .../security/flower.rst | 0 .../security/index.rst | 0 .../security/kerberos.rst | 0 .../security/secrets/fernet.rst | 0 .../security/secrets/index.rst | 4 +- .../secrets/mask-sensitive-values.rst | 0 .../secrets/secrets-backend/index.rst | 2 +- .../local-filesystem-secrets-backend.rst | 0 .../security/webserver.rst | 2 +- .../security/workload.rst | 0 .../connections.rst | 0 .../dagfile-processing.rst | 0 .../datasets.rst | 0 .../deferring.rst | 2 +- .../dynamic-task-mapping.rst | 0 .../authoring-and-scheduling/index.rst | 44 ++++++++++ .../plugins.rst | 6 +- .../timetable.rst | 2 +- .../timezone.rst | 4 +- docs/apache-airflow/best-practices.rst | 6 +- .../cli-and-env-variables-ref.rst | 2 +- .../{ => core-concepts}/dag-run.rst | 8 +- .../{concepts => core-concepts}/dags.rst | 22 ++--- .../{ => core-concepts}/executor/celery.rst | 4 +- .../executor/celery_kubernetes.rst | 0 .../{ => core-concepts}/executor/dask.rst | 2 +- .../{ => core-concepts}/executor/debug.rst | 2 +- .../{ => core-concepts}/executor/index.rst | 2 +- .../executor/kubernetes.rst | 10 +-- .../{ => core-concepts}/executor/local.rst | 0 .../executor/local_kubernetes.rst | 0 .../executor/sequential.rst | 0 .../{concepts => core-concepts}/index.rst | 17 +--- .../{concepts => core-concepts}/operators.rst | 0 .../{concepts => core-concepts}/overview.rst | 12 +-- .../{concepts => core-concepts}/params.rst | 0 .../{concepts => core-concepts}/sensors.rst | 2 +- .../{concepts => core-concepts}/taskflow.rst | 0 .../{concepts => core-concepts}/tasks.rst | 16 ++-- .../{concepts => core-concepts}/variables.rst | 0 .../{concepts => core-concepts}/xcoms.rst | 0 docs/apache-airflow/faq.rst | 2 +- docs/apache-airflow/howto/connection.rst | 8 +- docs/apache-airflow/howto/custom-operator.rst | 4 +- .../howto/custom-view-plugin.rst | 2 +- docs/apache-airflow/howto/customize-ui.rst | 2 +- .../howto/define_extra_link.rst | 2 +- .../howto/docker-compose/index.rst | 12 +-- .../howto/dynamic-dag-generation.rst | 4 +- docs/apache-airflow/howto/index.rst | 2 + docs/apache-airflow/howto/operator/index.rst | 2 +- docs/apache-airflow/howto/set-config.rst | 2 +- docs/apache-airflow/howto/set-up-database.rst | 4 +- docs/apache-airflow/howto/timetable.rst | 2 +- .../{ => howto}/upgrading-from-1-10/index.rst | 0 .../upgrading-from-1-10/upgrade-check.rst | 0 docs/apache-airflow/{ => howto}/usage-cli.rst | 8 +- docs/apache-airflow/howto/variable.rst | 4 +- docs/apache-airflow/index.rst | 22 ++--- docs/apache-airflow/installation/index.rst | 2 +- .../installation/prerequisites.rst | 2 +- .../apache-airflow/installation/upgrading.rst | 2 +- docs/apache-airflow/integration.rst | 22 ++--- docs/apache-airflow/migrations-ref.rst | 2 +- docs/apache-airflow/redirects.txt | 87 ++++++++++++++++--- docs/apache-airflow/start.rst | 2 +- docs/apache-airflow/templates-ref.rst | 2 +- docs/apache-airflow/tutorial/fundamentals.rst | 6 +- docs/apache-airflow/tutorial/taskflow.rst | 6 +- docs/docker-stack/build.rst | 2 +- 101 files changed, 318 insertions(+), 191 deletions(-) rename docs/apache-airflow/{concepts => administration-and-deployment}/cluster-policies.rst (100%) rename docs/apache-airflow/{ => administration-and-deployment}/dag-serialization.rst (99%) create mode 100644 docs/apache-airflow/administration-and-deployment/index.rst rename docs/apache-airflow/{ => administration-and-deployment}/kubernetes.rst (92%) rename docs/apache-airflow/{ => administration-and-deployment}/lineage.rst (100%) rename docs/apache-airflow/{ => administration-and-deployment}/listeners.rst (97%) rename docs/apache-airflow/{ => administration-and-deployment}/logging-monitoring/callbacks.rst (97%) rename docs/apache-airflow/{ => administration-and-deployment}/logging-monitoring/check-health.rst (100%) rename docs/apache-airflow/{ => administration-and-deployment}/logging-monitoring/errors.rst (100%) rename docs/apache-airflow/{ => administration-and-deployment}/logging-monitoring/index.rst (100%) rename docs/apache-airflow/{ => administration-and-deployment}/logging-monitoring/logging-architecture.rst (86%) rename docs/apache-airflow/{ => administration-and-deployment}/logging-monitoring/logging-tasks.rst (100%) rename docs/apache-airflow/{ => administration-and-deployment}/logging-monitoring/metrics.rst (100%) rename docs/apache-airflow/{ => administration-and-deployment}/logging-monitoring/tracking-user-activity.rst (98%) rename docs/apache-airflow/{ => administration-and-deployment}/modules_management.rst (99%) rename docs/apache-airflow/{concepts => administration-and-deployment}/pools.rst (100%) rename docs/apache-airflow/{concepts => administration-and-deployment}/priority-weight.rst (100%) rename docs/apache-airflow/{ => administration-and-deployment}/production-deployment.rst (96%) rename docs/apache-airflow/{concepts => administration-and-deployment}/scheduler.rst (98%) rename docs/apache-airflow/{ => administration-and-deployment}/security/access-control.rst (99%) rename docs/apache-airflow/{ => administration-and-deployment}/security/api.rst (100%) rename docs/apache-airflow/{ => administration-and-deployment}/security/flower.rst (100%) rename docs/apache-airflow/{ => administration-and-deployment}/security/index.rst (100%) rename docs/apache-airflow/{ => administration-and-deployment}/security/kerberos.rst (100%) rename docs/apache-airflow/{ => administration-and-deployment}/security/secrets/fernet.rst (100%) rename docs/apache-airflow/{ => administration-and-deployment}/security/secrets/index.rst (83%) rename docs/apache-airflow/{ => administration-and-deployment}/security/secrets/mask-sensitive-values.rst (100%) rename docs/apache-airflow/{ => administration-and-deployment}/security/secrets/secrets-backend/index.rst (98%) rename docs/apache-airflow/{ => administration-and-deployment}/security/secrets/secrets-backend/local-filesystem-secrets-backend.rst (100%) rename docs/apache-airflow/{ => administration-and-deployment}/security/webserver.rst (99%) rename docs/apache-airflow/{ => administration-and-deployment}/security/workload.rst (100%) rename docs/apache-airflow/{concepts => authoring-and-scheduling}/connections.rst (100%) rename docs/apache-airflow/{concepts => authoring-and-scheduling}/dagfile-processing.rst (100%) rename docs/apache-airflow/{concepts => authoring-and-scheduling}/datasets.rst (100%) rename docs/apache-airflow/{concepts => authoring-and-scheduling}/deferring.rst (95%) rename docs/apache-airflow/{concepts => authoring-and-scheduling}/dynamic-task-mapping.rst (100%) create mode 100644 docs/apache-airflow/authoring-and-scheduling/index.rst rename docs/apache-airflow/{ => authoring-and-scheduling}/plugins.rst (98%) rename docs/apache-airflow/{concepts => authoring-and-scheduling}/timetable.rst (99%) rename docs/apache-airflow/{ => authoring-and-scheduling}/timezone.rst (99%) rename docs/apache-airflow/{ => core-concepts}/dag-run.rst (98%) rename docs/apache-airflow/{concepts => core-concepts}/dags.rst (98%) rename docs/apache-airflow/{ => core-concepts}/executor/celery.rst (98%) rename docs/apache-airflow/{ => core-concepts}/executor/celery_kubernetes.rst (100%) rename docs/apache-airflow/{ => core-concepts}/executor/dask.rst (98%) rename docs/apache-airflow/{ => core-concepts}/executor/debug.rst (99%) rename docs/apache-airflow/{ => core-concepts}/executor/index.rst (92%) rename docs/apache-airflow/{ => core-concepts}/executor/kubernetes.rst (98%) rename docs/apache-airflow/{ => core-concepts}/executor/local.rst (100%) rename docs/apache-airflow/{ => core-concepts}/executor/local_kubernetes.rst (100%) rename docs/apache-airflow/{ => core-concepts}/executor/sequential.rst (100%) rename docs/apache-airflow/{concepts => core-concepts}/index.rst (85%) rename docs/apache-airflow/{concepts => core-concepts}/operators.rst (100%) rename docs/apache-airflow/{concepts => core-concepts}/overview.rst (89%) rename docs/apache-airflow/{concepts => core-concepts}/params.rst (100%) rename docs/apache-airflow/{concepts => core-concepts}/sensors.rst (97%) rename docs/apache-airflow/{concepts => core-concepts}/taskflow.rst (100%) rename docs/apache-airflow/{concepts => core-concepts}/tasks.rst (94%) rename docs/apache-airflow/{concepts => core-concepts}/variables.rst (100%) rename docs/apache-airflow/{concepts => core-concepts}/xcoms.rst (100%) rename docs/apache-airflow/{ => howto}/upgrading-from-1-10/index.rst (100%) rename docs/apache-airflow/{ => howto}/upgrading-from-1-10/upgrade-check.rst (100%) rename docs/apache-airflow/{ => howto}/usage-cli.rst (98%) diff --git a/.github/boring-cyborg.yml b/.github/boring-cyborg.yml index 90f80674ec538..7a444cf253378 100644 --- a/.github/boring-cyborg.yml +++ b/.github/boring-cyborg.yml @@ -67,8 +67,8 @@ labelPRBasedOnFilePath: - airflow/kubernetes_executor_templates/**/* - airflow/executors/kubernetes_executor.py - airflow/executors/celery_kubernetes_executor.py - - docs/apache-airflow/executor/kubernetes.rst - - docs/apache-airflow/executor/celery_kubernetes.rst + - docs/apache-airflow/core-concepts/executor/kubernetes.rst + - docs/apache-airflow/core-concepts/executor/celery_kubernetes.rst - docs/apache-airflow-providers-cncf-kubernetes/**/* - kubernetes_tests/**/* @@ -135,17 +135,17 @@ labelPRBasedOnFilePath: - airflow/cli/**/*.py - tests/cli/**/*.py - docs/apache-airflow/cli-and-env-variables-ref.rst - - docs/apache-airflow/usage-cli.rst + - docs/apache-airflow/howto/usage-cli.rst area:Lineage: - airflow/lineage/**/* - tests/lineage/**/* - - docs/apache-airflow/lineage.rst + - docs/apache-airflow/administration-and-deployment/lineage.rst area:Logging: - airflow/providers/**/log/* - airflow/utils/log/**/* - - docs/apache-airflow/logging-monitoring/logging-*.rst + - docs/apache-airflow/administration-and-deployment/logging-monitoring/logging-*.rst - tests/providers/**/log/* - tests/utils/log/**/* @@ -154,15 +154,15 @@ labelPRBasedOnFilePath: - airflow/plugins_manager.py - tests/cli/commands/test_plugins_command.py - tests/plugins/**/* - - docs/apache-airflow/plugins.rst + - docs/apache-airflow/authoring-and-scheduling/plugins.rst area:Scheduler/Executor: - airflow/executors/**/* - airflow/jobs/**/* - airflow/task/task_runner/**/* - airflow/dag_processing/**/* - - docs/apache-airflow/executor/**/* - - docs/apache-airflow/concepts/scheduler.rst + - docs/apache-airflow/core-concepts/executor/**/* + - docs/apache-airflow/administration-and-deployment/scheduler.rst - tests/executors/**/* - tests/jobs/**/* @@ -171,14 +171,14 @@ labelPRBasedOnFilePath: - airflow/providers/**/secrets/* - tests/secrets/**/* - tests/providers/**/secrets/* - - docs/apache-airflow/security/secrets/**/* + - docs/apache-airflow/administration-and-deployment/security/secrets/**/* area:Serialization: - airflow/serialization/**/* - airflow/models/serialized_dag.py - tests/serialization/**/* - tests/models/test_serialized_dag.py - - docs/apache-airflow/dag-serialization.rst + - docs/apache-airflow/administration-and-deployment/dag-serialization.rst area:core-operators: - airflow/operators/**/* diff --git a/RELEASE_NOTES.rst b/RELEASE_NOTES.rst index f124693fb41b8..e963142a69d64 100644 --- a/RELEASE_NOTES.rst +++ b/RELEASE_NOTES.rst @@ -429,14 +429,14 @@ If you have the producer and consumer in different files you do not need to use Datasets represent the abstract concept of a dataset, and (for now) do not have any direct read or write capability - in this release we are adding the foundational feature that we will build upon. -For more info on Datasets please see :doc:`/concepts/datasets`. +For more info on Datasets please see :doc:`/authoring-and-scheduling/datasets`. Expanded dynamic task mapping support """"""""""""""""""""""""""""""""""""" Dynamic task mapping now includes support for ``expand_kwargs``, ``zip`` and ``map``. -For more info on dynamic task mapping please see :doc:`/concepts/dynamic-task-mapping`. +For more info on dynamic task mapping please see :doc:`/authoring-and-scheduling/dynamic-task-mapping`. DAGS used in a context manager no longer need to be assigned to a module variable (#23592) """""""""""""""""""""""""""""""""""""""""""""""""""""""""""""""""""""""""""""""""""""""""" diff --git a/airflow/hooks/subprocess.py b/airflow/hooks/subprocess.py index b5a30b06aed1a..84479a3a40809 100644 --- a/airflow/hooks/subprocess.py +++ b/airflow/hooks/subprocess.py @@ -53,7 +53,7 @@ def run_command( environment in which ``command`` will be executed. If omitted, ``os.environ`` will be used. Note, that in case you have Sentry configured, original variables from the environment will also be passed to the subprocess with ``SUBPROCESS_`` prefix. See - :doc:`/logging-monitoring/errors` for details. + :doc:`/administration-and-deployment/logging-monitoring/errors` for details. :param output_encoding: encoding to use for decoding stdout :param cwd: Working directory to run the command in. If None (default), the command is run in a temporary directory. diff --git a/airflow/providers/cncf/kubernetes/CHANGELOG.rst b/airflow/providers/cncf/kubernetes/CHANGELOG.rst index b378c11fba518..f7b293fd1fd40 100644 --- a/airflow/providers/cncf/kubernetes/CHANGELOG.rst +++ b/airflow/providers/cncf/kubernetes/CHANGELOG.rst @@ -366,7 +366,7 @@ Notes on changes KubernetesPodOperator and PodLauncher Overview '''''''' -Generally speaking if you did not subclass ``KubernetesPodOperator`` and you didn't use the ``PodLauncher`` class directly, +Generally speaking if you did not subclass ``KubernetesPodOperator`` and you did not use the ``PodLauncher`` class directly, then you don't need to worry about this change. If however you have subclassed ``KubernetesPodOperator``, what follows are some notes on the changes in this release. diff --git a/docs/apache-airflow-providers-cncf-kubernetes/operators.rst b/docs/apache-airflow-providers-cncf-kubernetes/operators.rst index 57966e7fdd118..4d49ece5ebb09 100644 --- a/docs/apache-airflow-providers-cncf-kubernetes/operators.rst +++ b/docs/apache-airflow-providers-cncf-kubernetes/operators.rst @@ -17,7 +17,7 @@ -.. _howto/operator:KubernetesPodOperator: +.. _howto/operator:kubernetespodoperator: KubernetesPodOperator ===================== @@ -32,7 +32,7 @@ you to create and run Pods on a Kubernetes cluster. simplifies the Kubernetes authorization process. .. note:: - The :doc:`Kubernetes executor ` is **not** required to use this operator. + The :doc:`Kubernetes executor ` is **not** required to use this operator. How does this operator work? ^^^^^^^^^^^^^^^^^^^^^^^^^^^^ diff --git a/docs/apache-airflow-providers-microsoft-psrp/operators/index.rst b/docs/apache-airflow-providers-microsoft-psrp/operators/index.rst index ffa739dbc0295..176cf03967db5 100644 --- a/docs/apache-airflow-providers-microsoft-psrp/operators/index.rst +++ b/docs/apache-airflow-providers-microsoft-psrp/operators/index.rst @@ -81,7 +81,7 @@ mechanism for records that arrive on these streams using a job status polling mechanism. The success stream (i.e., stdout or shell output) is handled differently, as explained in the following: -When :doc:`XComs ` are enabled and when +When :doc:`XComs ` are enabled and when the operator is used with a native PowerShell cmdlet or script, the shell output is converted to JSON using the ``ConvertTo-Json`` cmdlet and then decoded on the client-side by the operator such that the diff --git a/docs/apache-airflow-providers/core-extensions/auth-backends.rst b/docs/apache-airflow-providers/core-extensions/auth-backends.rst index 325b0c2819a26..edb38bb882f42 100644 --- a/docs/apache-airflow-providers/core-extensions/auth-backends.rst +++ b/docs/apache-airflow-providers/core-extensions/auth-backends.rst @@ -26,7 +26,7 @@ capabilities. You can read more about those in `FAB security docs `_. You can also -take a look at Auth backends available in the core Airflow in :doc:`apache-airflow:security/webserver` +take a look at Auth backends available in the core Airflow in :doc:`apache-airflow:administration-and-deployment/security/webserver` or see those provided by the community-managed providers: .. airflow-auth-backends:: diff --git a/docs/apache-airflow-providers/core-extensions/connections.rst b/docs/apache-airflow-providers/core-extensions/connections.rst index 667e0af55f933..4861402d1d3ae 100644 --- a/docs/apache-airflow-providers/core-extensions/connections.rst +++ b/docs/apache-airflow-providers/core-extensions/connections.rst @@ -27,7 +27,7 @@ connection, when the connection is managed via Airflow UI. Those connections als that can be used to automatically create Airflow Hooks for specific connection types. The connection management is explained in -:doc:`apache-airflow:concepts/connections` and you can also see those +:doc:`apache-airflow:authoring-and-scheduling/connections` and you can also see those provided by the community-managed providers: .. airflow-connections:: diff --git a/docs/apache-airflow-providers/core-extensions/logging.rst b/docs/apache-airflow-providers/core-extensions/logging.rst index ca26263ad5d73..822cf54d2b2f8 100644 --- a/docs/apache-airflow-providers/core-extensions/logging.rst +++ b/docs/apache-airflow-providers/core-extensions/logging.rst @@ -20,7 +20,7 @@ Writing logs This is a summary of all Apache Airflow Community provided implementations of writing task logs exposed via community-managed providers. You can also see logging options available in the core Airflow in -:doc:`apache-airflow:logging-monitoring/logging-tasks` and here you can see those +:doc:`apache-airflow:administration-and-deployment/logging-monitoring/logging-tasks` and here you can see those provided by the community-managed providers: .. airflow-logging:: diff --git a/docs/apache-airflow-providers/core-extensions/secrets-backends.rst b/docs/apache-airflow-providers/core-extensions/secrets-backends.rst index 26ee3ce882a02..22fae4b8a6c47 100644 --- a/docs/apache-airflow-providers/core-extensions/secrets-backends.rst +++ b/docs/apache-airflow-providers/core-extensions/secrets-backends.rst @@ -28,7 +28,7 @@ via providers that implement secrets backends for services Airflow integrates wi You can also take a look at Secret backends available in the core Airflow in -:doc:`apache-airflow:security/secrets/secrets-backend/index` and here you can see the ones +:doc:`apache-airflow:administration-and-deployment/security/secrets/secrets-backend/index` and here you can see the ones provided by the community-managed providers: .. airflow-secrets-backends:: diff --git a/docs/apache-airflow/concepts/cluster-policies.rst b/docs/apache-airflow/administration-and-deployment/cluster-policies.rst similarity index 100% rename from docs/apache-airflow/concepts/cluster-policies.rst rename to docs/apache-airflow/administration-and-deployment/cluster-policies.rst diff --git a/docs/apache-airflow/dag-serialization.rst b/docs/apache-airflow/administration-and-deployment/dag-serialization.rst similarity index 99% rename from docs/apache-airflow/dag-serialization.rst rename to docs/apache-airflow/administration-and-deployment/dag-serialization.rst index fc5edb43ed870..9c4550544c436 100644 --- a/docs/apache-airflow/dag-serialization.rst +++ b/docs/apache-airflow/administration-and-deployment/dag-serialization.rst @@ -25,7 +25,7 @@ In order to make Airflow Webserver stateless, Airflow >=1.10.7 supports DAG Serialization and DB Persistence. From Airflow 2.0.0, the Scheduler also uses Serialized DAGs for consistency and makes scheduling decisions. -.. image:: img/dag_serialization.png +.. image:: ../img/dag_serialization.png Without DAG Serialization & persistence in DB, the Webserver and the Scheduler both need access to the DAG files. Both the Scheduler and Webserver parse the DAG files. diff --git a/docs/apache-airflow/administration-and-deployment/index.rst b/docs/apache-airflow/administration-and-deployment/index.rst new file mode 100644 index 0000000000000..1acb9d8e3a90b --- /dev/null +++ b/docs/apache-airflow/administration-and-deployment/index.rst @@ -0,0 +1,37 @@ + .. 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. + +Administration and Deployment +===================================== + +This section contains information about deploying DAGs into production and the administration of airflow deployments. + +.. toctree:: + :maxdepth: 2 + + production-deployment + security/index + logging-monitoring/index + kubernetes + lineage + listeners + dag-serialization + modules_management + scheduler + pools + cluster-policies + priority-weight diff --git a/docs/apache-airflow/kubernetes.rst b/docs/apache-airflow/administration-and-deployment/kubernetes.rst similarity index 92% rename from docs/apache-airflow/kubernetes.rst rename to docs/apache-airflow/administration-and-deployment/kubernetes.rst index b88a83ddab46f..961a93328bcc7 100644 --- a/docs/apache-airflow/kubernetes.rst +++ b/docs/apache-airflow/administration-and-deployment/kubernetes.rst @@ -32,13 +32,13 @@ We maintain :doc:`official Helm chart ` for Airflow that helps Kubernetes Executor ^^^^^^^^^^^^^^^^^^^ -The :doc:`Kubernetes Executor ` allows you to run all the Airflow tasks on +The :doc:`Kubernetes Executor ` allows you to run all the Airflow tasks on Kubernetes as separate Pods. KubernetesPodOperator ^^^^^^^^^^^^^^^^^^^^^ -The :ref:`KubernetesPodOperator ` allows you to create +The :ref:`KubernetesPodOperator ` allows you to create Pods on Kubernetes. Pod Mutation Hook diff --git a/docs/apache-airflow/lineage.rst b/docs/apache-airflow/administration-and-deployment/lineage.rst similarity index 100% rename from docs/apache-airflow/lineage.rst rename to docs/apache-airflow/administration-and-deployment/lineage.rst diff --git a/docs/apache-airflow/listeners.rst b/docs/apache-airflow/administration-and-deployment/listeners.rst similarity index 97% rename from docs/apache-airflow/listeners.rst rename to docs/apache-airflow/administration-and-deployment/listeners.rst index cee610d61eca2..2678e12e5ef66 100644 --- a/docs/apache-airflow/listeners.rst +++ b/docs/apache-airflow/administration-and-deployment/listeners.rst @@ -49,7 +49,7 @@ Their specification is defined as ``hookspec`` in ``airflow/listeners/spec`` dir Your implementation needs to accept the same named parameters as defined in hookspec, or Pluggy will complain about your plugin. On the other hand, you don't need to implement every method - it's perfectly fine to have a listener that implements just one method, or any subset of methods. -To include listener in your Airflow installation, include it as a part of an :doc:`Airflow Plugin ` +To include listener in your Airflow installation, include it as a part of an :doc:`Airflow Plugin ` Listener API is meant to be called across all dags, and all operators - in contrast to methods like ``on_success_callback``, ``pre_execute`` and related family which are meant to provide callbacks diff --git a/docs/apache-airflow/logging-monitoring/callbacks.rst b/docs/apache-airflow/administration-and-deployment/logging-monitoring/callbacks.rst similarity index 97% rename from docs/apache-airflow/logging-monitoring/callbacks.rst rename to docs/apache-airflow/administration-and-deployment/logging-monitoring/callbacks.rst index f0f014cf6f46f..96c07b7d02b8e 100644 --- a/docs/apache-airflow/logging-monitoring/callbacks.rst +++ b/docs/apache-airflow/administration-and-deployment/logging-monitoring/callbacks.rst @@ -26,7 +26,7 @@ For example, you may wish to alert when certain tasks have failed, or have the l .. note:: Callback functions are only invoked when the task state changes due to execution by a worker. - As such, task changes set by the command line interface (:doc:`CLI <../usage-cli>`) or user interface (:doc:`UI <../ui>`) do not + As such, task changes set by the command line interface (:doc:`CLI <../../howto/usage-cli>`) or user interface (:doc:`UI <../../ui>`) do not execute callback functions. Callback Types diff --git a/docs/apache-airflow/logging-monitoring/check-health.rst b/docs/apache-airflow/administration-and-deployment/logging-monitoring/check-health.rst similarity index 100% rename from docs/apache-airflow/logging-monitoring/check-health.rst rename to docs/apache-airflow/administration-and-deployment/logging-monitoring/check-health.rst diff --git a/docs/apache-airflow/logging-monitoring/errors.rst b/docs/apache-airflow/administration-and-deployment/logging-monitoring/errors.rst similarity index 100% rename from docs/apache-airflow/logging-monitoring/errors.rst rename to docs/apache-airflow/administration-and-deployment/logging-monitoring/errors.rst diff --git a/docs/apache-airflow/logging-monitoring/index.rst b/docs/apache-airflow/administration-and-deployment/logging-monitoring/index.rst similarity index 100% rename from docs/apache-airflow/logging-monitoring/index.rst rename to docs/apache-airflow/administration-and-deployment/logging-monitoring/index.rst diff --git a/docs/apache-airflow/logging-monitoring/logging-architecture.rst b/docs/apache-airflow/administration-and-deployment/logging-monitoring/logging-architecture.rst similarity index 86% rename from docs/apache-airflow/logging-monitoring/logging-architecture.rst rename to docs/apache-airflow/administration-and-deployment/logging-monitoring/logging-architecture.rst index 6a3fabec57917..07cd3f42cc2e7 100644 --- a/docs/apache-airflow/logging-monitoring/logging-architecture.rst +++ b/docs/apache-airflow/administration-and-deployment/logging-monitoring/logging-architecture.rst @@ -22,7 +22,7 @@ Logging and Monitoring architecture Airflow supports a variety of logging and monitoring mechanisms as shown below. -.. image:: ../img/arch-diag-logging.png +.. image:: ../../img/arch-diag-logging.png By default, Airflow supports logging into the local file system. These include logs from the Web server, the Scheduler, and the Workers running tasks. This is suitable for development environments and for quick debugging. @@ -33,9 +33,9 @@ The logging settings and options can be specified in the Airflow Configuration f For production deployments, we recommend using FluentD to capture logs and send it to destinations such as ElasticSearch or Splunk. .. note:: - For more information on configuring logging, see :doc:`/logging-monitoring/logging-tasks` + For more information on configuring logging, see :doc:`/administration-and-deployment/logging-monitoring/logging-tasks` Similarly, we recommend using StatsD for gathering metrics from Airflow and send them to destinations such as Prometheus. .. note:: - For more information on configuring metrics, see :doc:`/logging-monitoring/metrics` + For more information on configuring metrics, see :doc:`/administration-and-deployment/logging-monitoring/metrics` diff --git a/docs/apache-airflow/logging-monitoring/logging-tasks.rst b/docs/apache-airflow/administration-and-deployment/logging-monitoring/logging-tasks.rst similarity index 100% rename from docs/apache-airflow/logging-monitoring/logging-tasks.rst rename to docs/apache-airflow/administration-and-deployment/logging-monitoring/logging-tasks.rst diff --git a/docs/apache-airflow/logging-monitoring/metrics.rst b/docs/apache-airflow/administration-and-deployment/logging-monitoring/metrics.rst similarity index 100% rename from docs/apache-airflow/logging-monitoring/metrics.rst rename to docs/apache-airflow/administration-and-deployment/logging-monitoring/metrics.rst diff --git a/docs/apache-airflow/logging-monitoring/tracking-user-activity.rst b/docs/apache-airflow/administration-and-deployment/logging-monitoring/tracking-user-activity.rst similarity index 98% rename from docs/apache-airflow/logging-monitoring/tracking-user-activity.rst rename to docs/apache-airflow/administration-and-deployment/logging-monitoring/tracking-user-activity.rst index b302ab707746e..6401ea72195e0 100644 --- a/docs/apache-airflow/logging-monitoring/tracking-user-activity.rst +++ b/docs/apache-airflow/administration-and-deployment/logging-monitoring/tracking-user-activity.rst @@ -36,4 +36,4 @@ Edit ``airflow.cfg`` and set the ``webserver`` block to have an ``analytics_tool variables are set in ``airflow/www/templates/app.py``. .. note:: - For more information on setting the configuration, see :doc:`../howto/set-config` + For more information on setting the configuration, see :doc:`../../howto/set-config` diff --git a/docs/apache-airflow/modules_management.rst b/docs/apache-airflow/administration-and-deployment/modules_management.rst similarity index 99% rename from docs/apache-airflow/modules_management.rst rename to docs/apache-airflow/administration-and-deployment/modules_management.rst index 3d88bda57450e..9afd3da8e4673 100644 --- a/docs/apache-airflow/modules_management.rst +++ b/docs/apache-airflow/administration-and-deployment/modules_management.rst @@ -326,7 +326,7 @@ you might organize your versioning approach, control which versions of the share and deploy the code to all your instances and containers in controlled way - all by system admins/DevOps rather than by the DAG writers. It is usually suitable when you have a separate team that manages this shared code, but if you know your python ways you can also distribute your code this way in smaller -deployments. You can also install your :doc:`/plugins` and :doc:`apache-airflow-providers:index` as python +deployments. You can also install your :doc:`../authoring-and-scheduling/plugins` and :doc:`apache-airflow-providers:index` as python packages, so learning how to build your package is handy. Here is how to create your package: diff --git a/docs/apache-airflow/concepts/pools.rst b/docs/apache-airflow/administration-and-deployment/pools.rst similarity index 100% rename from docs/apache-airflow/concepts/pools.rst rename to docs/apache-airflow/administration-and-deployment/pools.rst diff --git a/docs/apache-airflow/concepts/priority-weight.rst b/docs/apache-airflow/administration-and-deployment/priority-weight.rst similarity index 100% rename from docs/apache-airflow/concepts/priority-weight.rst rename to docs/apache-airflow/administration-and-deployment/priority-weight.rst diff --git a/docs/apache-airflow/production-deployment.rst b/docs/apache-airflow/administration-and-deployment/production-deployment.rst similarity index 96% rename from docs/apache-airflow/production-deployment.rst rename to docs/apache-airflow/administration-and-deployment/production-deployment.rst index fc036c6ecac16..6b7e0e9a1f384 100644 --- a/docs/apache-airflow/production-deployment.rst +++ b/docs/apache-airflow/administration-and-deployment/production-deployment.rst @@ -27,7 +27,7 @@ Database backend Airflow comes with an ``SQLite`` backend by default. This allows the user to run Airflow without any external database. However, such a setup is meant to be used for testing purposes only; running the default setup in production can lead to data loss in multiple scenarios. If you want to run production-grade Airflow, -make sure you :doc:`configure the backend ` to be an external database +make sure you :doc:`configure the backend <../howto/set-up-database>` to be an external database such as PostgreSQL or MySQL. You can change the backend using the following config @@ -60,8 +60,8 @@ Airflow uses :class:`~airflow.executors.sequential_executor.SequentialExecutor` nature, the user is limited to executing at most one task at a time. ``Sequential Executor`` also pauses the scheduler when it runs a task, hence it is not recommended in a production setup. You should use the :class:`~airflow.executors.local_executor.LocalExecutor` for a single machine. -For a multi-node setup, you should use the :doc:`Kubernetes executor <../executor/kubernetes>` or -the :doc:`Celery executor <../executor/celery>`. +For a multi-node setup, you should use the :doc:`Kubernetes executor <../core-concepts/executor/kubernetes>` or +the :doc:`Celery executor <../core-concepts/executor/celery>`. Once you have configured the executor, it is necessary to make sure that every node in the cluster contains @@ -111,7 +111,7 @@ Airflow users occasionally report instances of the scheduler hanging without a t * `Scheduler gets stuck without a trace `_ * `Scheduler stopping frequently `_ -To mitigate these issues, make sure you have a :doc:`health check ` set up that will detect when your scheduler has not heartbeat in a while. +To mitigate these issues, make sure you have a :doc:`health check ` set up that will detect when your scheduler has not heartbeat in a while. .. _docker_image: @@ -154,7 +154,7 @@ the side-car container and read by the worker container. This concept is implemented in :doc:`the Helm Chart for Apache Airflow `. -.. spelling:: +.. spelling:word-list:: pypirc dockerignore @@ -222,7 +222,7 @@ you can exchange the Google Cloud Platform identity to the Amazon Web Service id which effectively means access to Amazon Web Service platform. For more information, see: :ref:`howto/connection:aws:gcp-federation` -.. spelling:: +.. spelling:word-list:: nsswitch cryptographic diff --git a/docs/apache-airflow/concepts/scheduler.rst b/docs/apache-airflow/administration-and-deployment/scheduler.rst similarity index 98% rename from docs/apache-airflow/concepts/scheduler.rst rename to docs/apache-airflow/administration-and-deployment/scheduler.rst index b07c5c16c9d22..7600e244f00a7 100644 --- a/docs/apache-airflow/concepts/scheduler.rst +++ b/docs/apache-airflow/administration-and-deployment/scheduler.rst @@ -31,7 +31,7 @@ Airflow production environment. To kick it off, all you need to do is execute the ``airflow scheduler`` command. It uses the configuration specified in ``airflow.cfg``. -The scheduler uses the configured :doc:`Executor ` to run tasks that are ready. +The scheduler uses the configured :doc:`Executor <../core-concepts/executor/index>` to run tasks that are ready. To start a scheduler, simply run the command: @@ -44,7 +44,7 @@ Your DAGs will start executing once the scheduler is running successfully. .. note:: The first DAG Run is created based on the minimum ``start_date`` for the tasks in your DAG. - Subsequent DAG Runs are created according to your DAG's :doc:`timetable `. + Subsequent DAG Runs are created according to your DAG's :doc:`timetable <../authoring-and-scheduling/timetable>`. For dags with a cron or timedelta schedule, scheduler won't trigger your tasks until the period it covers has ended e.g., A job with ``schedule`` set as ``@daily`` runs after the day @@ -57,7 +57,7 @@ In the UI, it appears as if Airflow is running your tasks a day **late** **Let's Repeat That**, the scheduler runs your job one ``schedule`` AFTER the start date, at the END of the interval. - You should refer to :doc:`/dag-run` for details on scheduling a DAG. + You should refer to :doc:`../core-concepts/dag-run` for details on scheduling a DAG. .. note:: The scheduler is designed for high throughput. This is an informed design decision to achieve scheduling @@ -73,7 +73,7 @@ DAG File Processing You can have the Airflow Scheduler be responsible for starting the process that turns the Python files contained in the DAGs folder into DAG objects that contain tasks to be scheduled. -Refer to :doc:`dagfile-processing` for details on how this can be achieved +Refer to :doc:`../authoring-and-scheduling/dagfile-processing` for details on how this can be achieved Triggering DAG with Future Date diff --git a/docs/apache-airflow/security/access-control.rst b/docs/apache-airflow/administration-and-deployment/security/access-control.rst similarity index 99% rename from docs/apache-airflow/security/access-control.rst rename to docs/apache-airflow/administration-and-deployment/security/access-control.rst index cdeb52ade439f..349df6a19bcc6 100644 --- a/docs/apache-airflow/security/access-control.rst +++ b/docs/apache-airflow/administration-and-deployment/security/access-control.rst @@ -22,7 +22,7 @@ Access Control of Airflow Webserver UI is handled by Flask AppBuilder (FAB). Please read its related `security document `_ regarding its security model. -.. spelling:: +.. spelling:word-list:: clearTaskInstances dagRuns dagSources diff --git a/docs/apache-airflow/security/api.rst b/docs/apache-airflow/administration-and-deployment/security/api.rst similarity index 100% rename from docs/apache-airflow/security/api.rst rename to docs/apache-airflow/administration-and-deployment/security/api.rst diff --git a/docs/apache-airflow/security/flower.rst b/docs/apache-airflow/administration-and-deployment/security/flower.rst similarity index 100% rename from docs/apache-airflow/security/flower.rst rename to docs/apache-airflow/administration-and-deployment/security/flower.rst diff --git a/docs/apache-airflow/security/index.rst b/docs/apache-airflow/administration-and-deployment/security/index.rst similarity index 100% rename from docs/apache-airflow/security/index.rst rename to docs/apache-airflow/administration-and-deployment/security/index.rst diff --git a/docs/apache-airflow/security/kerberos.rst b/docs/apache-airflow/administration-and-deployment/security/kerberos.rst similarity index 100% rename from docs/apache-airflow/security/kerberos.rst rename to docs/apache-airflow/administration-and-deployment/security/kerberos.rst diff --git a/docs/apache-airflow/security/secrets/fernet.rst b/docs/apache-airflow/administration-and-deployment/security/secrets/fernet.rst similarity index 100% rename from docs/apache-airflow/security/secrets/fernet.rst rename to docs/apache-airflow/administration-and-deployment/security/secrets/fernet.rst diff --git a/docs/apache-airflow/security/secrets/index.rst b/docs/apache-airflow/administration-and-deployment/security/secrets/index.rst similarity index 83% rename from docs/apache-airflow/security/secrets/index.rst rename to docs/apache-airflow/administration-and-deployment/security/secrets/index.rst index fb645a9c188cb..3fced1edfb5ee 100644 --- a/docs/apache-airflow/security/secrets/index.rst +++ b/docs/apache-airflow/administration-and-deployment/security/secrets/index.rst @@ -23,8 +23,8 @@ This guide provides ways to protect this data. The following are particularly protected: -* Variables. See the :doc:`Variables Concepts ` documentation for more information. -* Connections. See the :doc:`Connections Concepts ` documentation for more information. +* Variables. See the :doc:`Variables Concepts ` documentation for more information. +* Connections. See the :doc:`Connections Concepts ` documentation for more information. .. toctree:: diff --git a/docs/apache-airflow/security/secrets/mask-sensitive-values.rst b/docs/apache-airflow/administration-and-deployment/security/secrets/mask-sensitive-values.rst similarity index 100% rename from docs/apache-airflow/security/secrets/mask-sensitive-values.rst rename to docs/apache-airflow/administration-and-deployment/security/secrets/mask-sensitive-values.rst diff --git a/docs/apache-airflow/security/secrets/secrets-backend/index.rst b/docs/apache-airflow/administration-and-deployment/security/secrets/secrets-backend/index.rst similarity index 98% rename from docs/apache-airflow/security/secrets/secrets-backend/index.rst rename to docs/apache-airflow/administration-and-deployment/security/secrets/secrets-backend/index.rst index fa92e1d61e4b7..9d0f44ba447e0 100644 --- a/docs/apache-airflow/security/secrets/secrets-backend/index.rst +++ b/docs/apache-airflow/administration-and-deployment/security/secrets/secrets-backend/index.rst @@ -31,7 +31,7 @@ can also enable alternative secrets backend to retrieve Airflow connections or A If you use an alternative secrets backend, check inside your backend to view the values of your variables and connections. You can also get Airflow configurations with sensitive data from the Secrets Backend. -See :doc:`../../../howto/set-config` for more details. +See :doc:`/howto/set-config` for more details. Search path ^^^^^^^^^^^ diff --git a/docs/apache-airflow/security/secrets/secrets-backend/local-filesystem-secrets-backend.rst b/docs/apache-airflow/administration-and-deployment/security/secrets/secrets-backend/local-filesystem-secrets-backend.rst similarity index 100% rename from docs/apache-airflow/security/secrets/secrets-backend/local-filesystem-secrets-backend.rst rename to docs/apache-airflow/administration-and-deployment/security/secrets/secrets-backend/local-filesystem-secrets-backend.rst diff --git a/docs/apache-airflow/security/webserver.rst b/docs/apache-airflow/administration-and-deployment/security/webserver.rst similarity index 99% rename from docs/apache-airflow/security/webserver.rst rename to docs/apache-airflow/administration-and-deployment/security/webserver.rst index 5ff6abcb251cd..f08b84d32be56 100644 --- a/docs/apache-airflow/security/webserver.rst +++ b/docs/apache-airflow/administration-and-deployment/security/webserver.rst @@ -77,7 +77,7 @@ user will have by default: AUTH_ROLE_PUBLIC = 'Admin' -Be sure to checkout :doc:`/security/api` for securing the API. +Be sure to checkout :doc:`/administration-and-deployment/security/api` for securing the API. .. note:: diff --git a/docs/apache-airflow/security/workload.rst b/docs/apache-airflow/administration-and-deployment/security/workload.rst similarity index 100% rename from docs/apache-airflow/security/workload.rst rename to docs/apache-airflow/administration-and-deployment/security/workload.rst diff --git a/docs/apache-airflow/concepts/connections.rst b/docs/apache-airflow/authoring-and-scheduling/connections.rst similarity index 100% rename from docs/apache-airflow/concepts/connections.rst rename to docs/apache-airflow/authoring-and-scheduling/connections.rst diff --git a/docs/apache-airflow/concepts/dagfile-processing.rst b/docs/apache-airflow/authoring-and-scheduling/dagfile-processing.rst similarity index 100% rename from docs/apache-airflow/concepts/dagfile-processing.rst rename to docs/apache-airflow/authoring-and-scheduling/dagfile-processing.rst diff --git a/docs/apache-airflow/concepts/datasets.rst b/docs/apache-airflow/authoring-and-scheduling/datasets.rst similarity index 100% rename from docs/apache-airflow/concepts/datasets.rst rename to docs/apache-airflow/authoring-and-scheduling/datasets.rst diff --git a/docs/apache-airflow/concepts/deferring.rst b/docs/apache-airflow/authoring-and-scheduling/deferring.rst similarity index 95% rename from docs/apache-airflow/concepts/deferring.rst rename to docs/apache-airflow/authoring-and-scheduling/deferring.rst index ab87a029fce45..b16fbdde1db4b 100644 --- a/docs/apache-airflow/concepts/deferring.rst +++ b/docs/apache-airflow/authoring-and-scheduling/deferring.rst @@ -18,7 +18,7 @@ Deferrable Operators & Triggers =============================== -Standard :doc:`Operators ` and :doc:`Sensors ` take up a full *worker slot* for the entire time they are running, even if they are idle; for example, if you only have 100 worker slots available to run Tasks, and you have 100 DAGs waiting on a Sensor that's currently running but idle, then you *cannot run anything else* - even though your entire Airflow cluster is essentially idle. ``reschedule`` mode for Sensors solves some of this, allowing Sensors to only run at fixed intervals, but it is inflexible and only allows using time as the reason to resume, not anything else. +Standard :doc:`Operators ` and :doc:`Sensors <../core-concepts/sensors>` take up a full *worker slot* for the entire time they are running, even if they are idle; for example, if you only have 100 worker slots available to run Tasks, and you have 100 DAGs waiting on a Sensor that's currently running but idle, then you *cannot run anything else* - even though your entire Airflow cluster is essentially idle. ``reschedule`` mode for Sensors solves some of this, allowing Sensors to only run at fixed intervals, but it is inflexible and only allows using time as the reason to resume, not anything else. This is where *Deferrable Operators* come in. A deferrable operator is one that is written with the ability to suspend itself and free up the worker when it knows it has to wait, and hand off the job of resuming it to something called a *Trigger*. As a result, while it is suspended (deferred), it is not taking up a worker slot and your cluster will have a lot less resources wasted on idle Operators or Sensors. diff --git a/docs/apache-airflow/concepts/dynamic-task-mapping.rst b/docs/apache-airflow/authoring-and-scheduling/dynamic-task-mapping.rst similarity index 100% rename from docs/apache-airflow/concepts/dynamic-task-mapping.rst rename to docs/apache-airflow/authoring-and-scheduling/dynamic-task-mapping.rst diff --git a/docs/apache-airflow/authoring-and-scheduling/index.rst b/docs/apache-airflow/authoring-and-scheduling/index.rst new file mode 100644 index 0000000000000..db1ef3726cc67 --- /dev/null +++ b/docs/apache-airflow/authoring-and-scheduling/index.rst @@ -0,0 +1,44 @@ + .. 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. + +Authoring and Scheduling +========================= + +Here you can find detailed documentation about advanced authoring and scheduling airflow DAGs. +It's recommended that you first review the pages in :doc:`core concepts ` + +**Authoring** + +.. toctree:: + :maxdepth: 2 + + plugins + deferring + dagfile-processing + serializers + connections + dynamic-task-mapping + + +**Scheduling** + +.. toctree:: + :maxdepth: 2 + + timezone + datasets + timetable diff --git a/docs/apache-airflow/plugins.rst b/docs/apache-airflow/authoring-and-scheduling/plugins.rst similarity index 98% rename from docs/apache-airflow/plugins.rst rename to docs/apache-airflow/authoring-and-scheduling/plugins.rst index ae011caa12350..77d51f762df49 100644 --- a/docs/apache-airflow/plugins.rst +++ b/docs/apache-airflow/authoring-and-scheduling/plugins.rst @@ -33,7 +33,7 @@ This command dumps information about loaded plugins. .. versionchanged:: 2.0 Importing operators, sensors, hooks added in plugins via ``airflow.{operators,sensors,hooks}.`` is no longer supported, and these extensions should - just be imported as regular python modules. For more information, see: :doc:`/modules_management` and + just be imported as regular python modules. For more information, see: :doc:`../administration-and-deployment/modules_management` and :doc:`/howto/custom-operator` What for? @@ -93,7 +93,7 @@ config setting to True, resulting in launching a whole new python interpreter fo (Modules only imported by DAG files on the other hand do not suffer this problem, as DAG files are not loaded/parsed in any long-running Airflow process.) -.. _plugins:interface: +.. _plugins-interface: Interface --------- @@ -315,7 +315,7 @@ you should set ``reload_on_plugin_change`` option in ``[webserver]`` section to For more information on setting the configuration, see :doc:`/howto/set-config` .. note:: - See :doc:`modules_management` for details on how Python and Airflow manage modules. + See :doc:`../administration-and-deployment/modules_management` for details on how Python and Airflow manage modules. Troubleshooting --------------- diff --git a/docs/apache-airflow/concepts/timetable.rst b/docs/apache-airflow/authoring-and-scheduling/timetable.rst similarity index 99% rename from docs/apache-airflow/concepts/timetable.rst rename to docs/apache-airflow/authoring-and-scheduling/timetable.rst index 7b8ff9cc53cc6..f12e56fe30617 100644 --- a/docs/apache-airflow/concepts/timetable.rst +++ b/docs/apache-airflow/authoring-and-scheduling/timetable.rst @@ -132,7 +132,7 @@ trigger points, and triggers a DAG run at the end of each data interval. .. seealso:: `Differences between the two cron timetables`_ This can be selected by providing a string that is a valid cron expression to the ``schedule`` -parameter of a DAG as described in the :doc:`/concepts/dags` documentation. +parameter of a DAG as described in the :doc:`../core-concepts/dags` documentation. .. code-block:: python diff --git a/docs/apache-airflow/timezone.rst b/docs/apache-airflow/authoring-and-scheduling/timezone.rst similarity index 99% rename from docs/apache-airflow/timezone.rst rename to docs/apache-airflow/authoring-and-scheduling/timezone.rst index 9e5e65e3c521c..fac290442c8a0 100644 --- a/docs/apache-airflow/timezone.rst +++ b/docs/apache-airflow/authoring-and-scheduling/timezone.rst @@ -46,7 +46,7 @@ Web UI By default the Web UI will show times in UTC. It is possible to change the timezone shown by using the menu in the top right (click on the clock to activate it): -.. image:: img/ui-timezone-chooser.png +.. image:: ../img/ui-timezone-chooser.png "Local" is detected from the browser's timezone. The "Server" value comes from the ``default_timezone`` setting in the ``[core]`` section. @@ -123,7 +123,7 @@ it is therefore important to make sure this setting is equal on all Airflow node default_timezone = utc .. note:: - For more information on setting the configuration, see :doc:`howto/set-config` + For more information on setting the configuration, see :doc:`/howto/set-config` .. _timezone_aware_dags: diff --git a/docs/apache-airflow/best-practices.rst b/docs/apache-airflow/best-practices.rst index 6ac50611e0d3e..d3fdd8d4afc23 100644 --- a/docs/apache-airflow/best-practices.rst +++ b/docs/apache-airflow/best-practices.rst @@ -82,7 +82,7 @@ it difficult to check the logs of that Task from the Webserver. If that is not d Communication -------------- -Airflow executes tasks of a DAG on different servers in case you are using :doc:`Kubernetes executor <../executor/kubernetes>` or :doc:`Celery executor <../executor/celery>`. +Airflow executes tasks of a DAG on different servers in case you are using :doc:`Kubernetes executor ` or :doc:`Celery executor `. Therefore, you should not store any file or config in the local filesystem as the next task is likely to run on a different server without access to it — for example, a task that downloads the data file that the next task processes. In the case of :class:`Local executor `, storing a file on disk can make retries harder e.g., your task requires a config file that is deleted by another task in DAG. @@ -92,7 +92,7 @@ For example, if we have a task that stores processed data in S3 that task can pu and the downstream tasks can pull the path from XCom and use it to read the data. The tasks should also not store any authentication parameters such as passwords or token inside them. -Where at all possible, use :doc:`Connections ` to store data securely in Airflow backend and retrieve them using a unique connection id. +Where at all possible, use :doc:`Connections ` to store data securely in Airflow backend and retrieve them using a unique connection id. .. _best_practices/top_level_code: @@ -185,7 +185,7 @@ Avoiding excessive processing at the top level code described in the previous ch in case of dynamic DAG configuration, which can be configured essentially in one of those ways: * via `environment variables `_ (not to be mistaken - with the :doc:`Airflow Variables `) + with the :doc:`Airflow Variables `) * via externally provided, generated Python code, containing meta-data in the DAG folder * via externally provided, generated configuration meta-data file in the DAG folder diff --git a/docs/apache-airflow/cli-and-env-variables-ref.rst b/docs/apache-airflow/cli-and-env-variables-ref.rst index 5bcaec4afaaa9..75932335851f3 100644 --- a/docs/apache-airflow/cli-and-env-variables-ref.rst +++ b/docs/apache-airflow/cli-and-env-variables-ref.rst @@ -28,7 +28,7 @@ many types of operation on a DAG, starting services, and supporting development and testing. .. note:: - For more information on usage CLI, see :doc:`usage-cli` + For more information on usage CLI, see :doc:`/howto/usage-cli` .. contents:: Content :local: diff --git a/docs/apache-airflow/dag-run.rst b/docs/apache-airflow/core-concepts/dag-run.rst similarity index 98% rename from docs/apache-airflow/dag-run.rst rename to docs/apache-airflow/core-concepts/dag-run.rst index 641b12ec12038..fa4abb2b66839 100644 --- a/docs/apache-airflow/dag-run.rst +++ b/docs/apache-airflow/core-concepts/dag-run.rst @@ -45,7 +45,7 @@ Cron Presets You may set your DAG to run on a simple schedule by setting its ``schedule`` argument to either a `cron expression `_, a ``datetime.timedelta`` object, -or one of the following cron "presets". For more elaborate scheduling requirements, you can implement a :doc:`custom timetable ` +or one of the following cron "presets". For more elaborate scheduling requirements, you can implement a :doc:`custom timetable <../authoring-and-scheduling/timetable>` .. tip:: You can use an online editor for CRON expressions such as `Crontab guru `_ @@ -102,8 +102,8 @@ scheduled one interval after ``start_date``. .. tip:: If a cron expression or timedelta object is not enough to express your DAG's schedule, - logical date, or data interval, see :doc:`/concepts/timetable`. - For more information on ``logical date``, see :ref:`concepts:dag-run` and + logical date, or data interval, see :doc:`../authoring-and-scheduling/timetable`. + For more information on ``logical date``, see :ref:`concepts-dag-run` and :ref:`faq:what-does-execution-date-mean` Re-run DAG @@ -276,7 +276,7 @@ Using CLI Using UI ^^^^^^^^^^ -.. image:: img/example_passing_conf.png +.. image:: ../img/example_passing_conf.png To Keep in Mind '''''''''''''''' diff --git a/docs/apache-airflow/concepts/dags.rst b/docs/apache-airflow/core-concepts/dags.rst similarity index 98% rename from docs/apache-airflow/concepts/dags.rst rename to docs/apache-airflow/core-concepts/dags.rst index a7e0a75956b07..91bcde51cac13 100644 --- a/docs/apache-airflow/concepts/dags.rst +++ b/docs/apache-airflow/core-concepts/dags.rst @@ -15,7 +15,7 @@ specific language governing permissions and limitations under the License. -.. _concepts:dags: +.. _concepts-dags: DAGs ==== @@ -50,7 +50,7 @@ operators you use:: schedule="@daily", catchup=False) op = EmptyOperator(task_id="task", dag=my_dag) -Or, you can use the ``@dag`` decorator to :ref:`turn a function into a DAG generator `:: +Or, you can use the ``@dag`` decorator to :ref:`turn a function into a DAG generator `:: @dag(start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), schedule="@daily", catchup=False) @@ -106,7 +106,7 @@ Chain can also do *pairwise* dependencies for lists the same size (this is diffe chain(op1, [op2, op3], [op4, op5], op6) -.. _concepts:dag-loading: +.. _concepts-dag-loading: Loading DAGs ------------ @@ -135,7 +135,7 @@ While both DAG constructors get called when the file is accessed, only ``dag_1`` You can also provide an ``.airflowignore`` file inside your ``DAG_FOLDER``, or any of its subfolders, which describes patterns of files for the loader to ignore. It covers the directory it's in plus all subfolders underneath it. See :ref:`.airflowignore ` below for details of the file syntax. -.. _concepts:dag-run: +.. _concepts-dag-run: Running DAGs ------------ @@ -157,14 +157,14 @@ The ``schedule`` argument takes any value that is a valid `Crontab `. + For more information on ``schedule`` values, see :doc:`DAG Run `. If ``schedule`` is not enough to express the DAG's schedule, see :doc:`Timetables `. For more information on ``logical date``, see :ref:`data-interval` and :ref:`faq:what-does-execution-date-mean`. Every time you run a DAG, you are creating a new instance of that DAG which -Airflow calls a :doc:`DAG Run `. DAG Runs can run in parallel for the +Airflow calls a :doc:`DAG Run `. DAG Runs can run in parallel for the same DAG, and each has a defined data interval, which identifies the period of data the tasks should operate on. @@ -209,7 +209,7 @@ Note that every single Operator/Task must be assigned to a DAG in order to run. Otherwise, you must pass it into each Operator with ``dag=``. -.. _concepts:default-arguments: +.. _concepts-default-arguments: Default Arguments ----------------- @@ -230,7 +230,7 @@ Often, many Operators inside a DAG need the same set of default arguments (such print(op.retries) # 2 -.. _concepts:dag-decorator: +.. _concepts-dag-decorator: The DAG decorator ----------------- @@ -248,10 +248,10 @@ As well as being a new way of making DAGs cleanly, the decorator also sets up an .. note:: - Airflow will only load DAGs that :ref:`appear in the top level ` of a DAG file. This means you cannot just declare a function with ``@dag`` - you must also call it at least once in your DAG file and assign it to a top-level object, as you can see in the example above. + Airflow will only load DAGs that :ref:`appear in the top level ` of a DAG file. This means you cannot just declare a function with ``@dag`` - you must also call it at least once in your DAG file and assign it to a top-level object, as you can see in the example above. -.. _concepts:control-flow: +.. _concepts-control-flow: Control Flow ------------ @@ -624,7 +624,7 @@ We can combine all of the parallel ``task-*`` operators into a single SubDAG, so .. image:: /img/subdag_after.png -Note that SubDAG operators should contain a factory method that returns a DAG object. This will prevent the SubDAG from being treated like a separate DAG in the main UI - remember, if Airflow sees a DAG at the top level of a Python file, it will :ref:`load it as its own DAG `. For example: +Note that SubDAG operators should contain a factory method that returns a DAG object. This will prevent the SubDAG from being treated like a separate DAG in the main UI - remember, if Airflow sees a DAG at the top level of a Python file, it will :ref:`load it as its own DAG `. For example: .. exampleinclude:: /../../airflow/example_dags/subdags/subdag.py :language: python diff --git a/docs/apache-airflow/executor/celery.rst b/docs/apache-airflow/core-concepts/executor/celery.rst similarity index 98% rename from docs/apache-airflow/executor/celery.rst rename to docs/apache-airflow/core-concepts/executor/celery.rst index a904b2d1f48ef..e16cc7e3f873a 100644 --- a/docs/apache-airflow/executor/celery.rst +++ b/docs/apache-airflow/core-concepts/executor/celery.rst @@ -87,7 +87,7 @@ Some caveats: - Tasks can consume resources. Make sure your worker has enough resources to run ``worker_concurrency`` tasks - Queue names are limited to 256 characters, but each broker backend might have its own restrictions -See :doc:`../modules_management` for details on how Python and Airflow manage modules. +See :doc:`/administration-and-deployment/modules_management` for details on how Python and Airflow manage modules. Architecture ------------ @@ -163,7 +163,7 @@ The components communicate with each other in many places Task execution process ---------------------- -.. figure:: ../img/run_task_on_celery_executor.png +.. figure:: ../../img/run_task_on_celery_executor.png :scale: 50 % Sequence diagram - task execution process diff --git a/docs/apache-airflow/executor/celery_kubernetes.rst b/docs/apache-airflow/core-concepts/executor/celery_kubernetes.rst similarity index 100% rename from docs/apache-airflow/executor/celery_kubernetes.rst rename to docs/apache-airflow/core-concepts/executor/celery_kubernetes.rst diff --git a/docs/apache-airflow/executor/dask.rst b/docs/apache-airflow/core-concepts/executor/dask.rst similarity index 98% rename from docs/apache-airflow/executor/dask.rst rename to docs/apache-airflow/core-concepts/executor/dask.rst index 0d0a680a09b33..7551ffaf8b38e 100644 --- a/docs/apache-airflow/executor/dask.rst +++ b/docs/apache-airflow/core-concepts/executor/dask.rst @@ -44,7 +44,7 @@ Next start at least one Worker on any machine that can connect to the host: Edit your ``airflow.cfg`` to set your executor to :class:`airflow.executors.dask_executor.DaskExecutor` and provide the Dask Scheduler address in the ``[dask]`` section. For more information on setting the configuration, -see :doc:`../howto/set-config`. +see :doc:`../../howto/set-config`. Please note: diff --git a/docs/apache-airflow/executor/debug.rst b/docs/apache-airflow/core-concepts/executor/debug.rst similarity index 99% rename from docs/apache-airflow/executor/debug.rst rename to docs/apache-airflow/core-concepts/executor/debug.rst index 85568f52025f8..2d2ed6d2d71f6 100644 --- a/docs/apache-airflow/executor/debug.rst +++ b/docs/apache-airflow/core-concepts/executor/debug.rst @@ -81,7 +81,7 @@ blocking the execution of DAG. Additionally ``DebugExecutor`` can be used in a fail-fast mode that will make all other running or scheduled tasks fail immediately. To enable this option set ``AIRFLOW__DEBUG__FAIL_FAST=True`` or adjust ``fail_fast`` option in your ``airflow.cfg``. -For more information on setting the configuration, see :doc:`../howto/set-config`. +For more information on setting the configuration, see :doc:`../../howto/set-config`. **IDE setup steps:** diff --git a/docs/apache-airflow/executor/index.rst b/docs/apache-airflow/core-concepts/executor/index.rst similarity index 92% rename from docs/apache-airflow/executor/index.rst rename to docs/apache-airflow/core-concepts/executor/index.rst index fc82336a6d4d8..4ba8334c297e1 100644 --- a/docs/apache-airflow/executor/index.rst +++ b/docs/apache-airflow/core-concepts/executor/index.rst @@ -18,7 +18,7 @@ Executor ======== -Executors are the mechanism by which :doc:`task instances ` get run. They have a common API and are "pluggable", meaning you can swap executors based on your installation needs. +Executors are the mechanism by which :doc:`task instances ` get run. They have a common API and are "pluggable", meaning you can swap executors based on your installation needs. Airflow can only have one executor configured at a time; this is set by the ``executor`` option in the ``[core]`` section of :doc:`the configuration file `. diff --git a/docs/apache-airflow/executor/kubernetes.rst b/docs/apache-airflow/core-concepts/executor/kubernetes.rst similarity index 98% rename from docs/apache-airflow/executor/kubernetes.rst rename to docs/apache-airflow/core-concepts/executor/kubernetes.rst index 8eb2a81efbb7f..9af6c6dd044d2 100644 --- a/docs/apache-airflow/executor/kubernetes.rst +++ b/docs/apache-airflow/core-concepts/executor/kubernetes.rst @@ -30,12 +30,12 @@ KubernetesExecutor requires a non-sqlite database in the backend. When a DAG submits a task, the KubernetesExecutor requests a worker pod from the Kubernetes API. The worker pod then runs the task, reports the result, and terminates. -.. image:: ../img/arch-diag-kubernetes.png +.. image:: ../../img/arch-diag-kubernetes.png One example of an Airflow deployment running on a distributed set of five nodes in a Kubernetes cluster is shown below. -.. image:: ../img/arch-diag-kubernetes2.png +.. image:: ../../img/arch-diag-kubernetes2.png Consistent with the regular Airflow architecture, the Workers need access to the DAG files to execute the tasks within those DAGs and interact with the Metadata repository. Also, configuration information specific to the Kubernetes Executor, such as the worker namespace and image information, needs to be specified in the Airflow Configuration file. @@ -48,7 +48,7 @@ Additionally, the Kubernetes Executor enables specification of additional featur .. Airflow_Worker -> Kubernetes: Pod completes with state "Succeeded" and k8s records in ETCD .. Kubernetes -> Airflow_Scheduler: Airflow scheduler reads "Succeeded" from k8s watcher thread .. @enduml -.. image:: ../img/k8s-happy-path.png +.. image:: ../../img/k8s-happy-path.png Configuration ------------- @@ -96,7 +96,7 @@ With these requirements in mind, here are some examples of basic ``pod_template_ The examples below should work when using default Airflow configuration values. However, many custom configuration values need to be explicitly passed to the pod via this template too. This includes, but is not limited to, sql configuration, required Airflow connections, dag folder path and - logging settings. See :doc:`../configurations-ref` for details. + logging settings. See :doc:`../../configurations-ref` for details. Storing DAGs in the image: @@ -264,7 +264,7 @@ In the case where a worker dies before it can report its status to the backend D .. .. @enduml -.. image:: ../img/k8s-failed-pod.png +.. image:: ../../img/k8s-failed-pod.png A Kubernetes watcher is a thread that can subscribe to every change that occurs in Kubernetes' database. It is alerted when pods start, run, end, and fail. diff --git a/docs/apache-airflow/executor/local.rst b/docs/apache-airflow/core-concepts/executor/local.rst similarity index 100% rename from docs/apache-airflow/executor/local.rst rename to docs/apache-airflow/core-concepts/executor/local.rst diff --git a/docs/apache-airflow/executor/local_kubernetes.rst b/docs/apache-airflow/core-concepts/executor/local_kubernetes.rst similarity index 100% rename from docs/apache-airflow/executor/local_kubernetes.rst rename to docs/apache-airflow/core-concepts/executor/local_kubernetes.rst diff --git a/docs/apache-airflow/executor/sequential.rst b/docs/apache-airflow/core-concepts/executor/sequential.rst similarity index 100% rename from docs/apache-airflow/executor/sequential.rst rename to docs/apache-airflow/core-concepts/executor/sequential.rst diff --git a/docs/apache-airflow/concepts/index.rst b/docs/apache-airflow/core-concepts/index.rst similarity index 85% rename from docs/apache-airflow/concepts/index.rst rename to docs/apache-airflow/core-concepts/index.rst index 6663fc004b333..61918d5ec654f 100644 --- a/docs/apache-airflow/concepts/index.rst +++ b/docs/apache-airflow/core-concepts/index.rst @@ -15,8 +15,8 @@ specific language governing permissions and limitations under the License. -Concepts -======== +Core Concepts +============================= Here you can find detailed documentation about each one of Airflow's core concepts and how to use them, as well as a high-level :doc:`architectural overview `. @@ -34,20 +34,12 @@ Here you can find detailed documentation about each one of Airflow's core concep :maxdepth: 2 dags + dag-run tasks operators - dynamic-task-mapping sensors - datasets - deferring taskflow - ../executor/index - scheduler - dagfile-processing - pools - timetable - priority-weight - cluster-policies + executor/index **Communication** @@ -56,5 +48,4 @@ Here you can find detailed documentation about each one of Airflow's core concep xcoms variables - connections params diff --git a/docs/apache-airflow/concepts/operators.rst b/docs/apache-airflow/core-concepts/operators.rst similarity index 100% rename from docs/apache-airflow/concepts/operators.rst rename to docs/apache-airflow/core-concepts/operators.rst diff --git a/docs/apache-airflow/concepts/overview.rst b/docs/apache-airflow/core-concepts/overview.rst similarity index 89% rename from docs/apache-airflow/concepts/overview.rst rename to docs/apache-airflow/core-concepts/overview.rst index d1a8537e6fe19..547497b0cef68 100644 --- a/docs/apache-airflow/concepts/overview.rst +++ b/docs/apache-airflow/core-concepts/overview.rst @@ -20,16 +20,16 @@ Architecture Overview Airflow is a platform that lets you build and run *workflows*. A workflow is represented as a :doc:`DAG ` (a Directed Acyclic Graph), and contains individual pieces of work called :doc:`tasks`, arranged with dependencies and data flows taken into account. -.. image:: /img/edge_label_example.png +.. image:: ../img/edge_label_example.png :alt: An example Airflow DAG, rendered in Graph A DAG specifies the dependencies between Tasks, and the order in which to execute them and run retries; the Tasks themselves describe what to do, be it fetching data, running analysis, triggering other systems, or more. An Airflow installation generally consists of the following components: -* A :doc:`scheduler `, which handles both triggering scheduled workflows, and submitting :doc:`tasks` to the executor to run. +* A :doc:`scheduler <../administration-and-deployment/scheduler>`, which handles both triggering scheduled workflows, and submitting :doc:`tasks` to the executor to run. -* An :doc:`executor `, which handles running tasks. In the default Airflow installation, this runs everything *inside* the scheduler, but most production-suitable executors actually push task execution out to *workers*. +* An :doc:`executor `, which handles running tasks. In the default Airflow installation, this runs everything *inside* the scheduler, but most production-suitable executors actually push task execution out to *workers*. * A *webserver*, which presents a handy user interface to inspect, trigger and debug the behaviour of DAGs and tasks. @@ -37,7 +37,7 @@ An Airflow installation generally consists of the following components: * A *metadata database*, used by the scheduler, executor and webserver to store state. -.. image:: /img/arch-diag-basic.png +.. image:: ../img/arch-diag-basic.png Most executors will generally also introduce other components to let them talk to their workers - like a task queue - but you can still think of the executor and its workers as a single logical component in Airflow overall, handling the actual task execution. @@ -86,13 +86,13 @@ Airflow sends out Tasks to run on Workers as space becomes available, so there's As you build out your DAGs, they are likely to get very complex, so Airflow provides several mechanisms for making this more sustainable - :ref:`SubDAGs ` let you make "reusable" DAGs you can embed into other ones, and :ref:`concepts:taskgroups` let you visually group tasks in the UI. -There are also features for letting you easily pre-configure access to a central resource, like a datastore, in the form of :doc:`connections`, and for limiting concurrency, via :doc:`pools`. +There are also features for letting you easily pre-configure access to a central resource, like a datastore, in the form of :doc:`../authoring-and-scheduling/connections`, and for limiting concurrency, via :doc:`../administration-and-deployment/pools`. User interface -------------- Airflow comes with a user interface that lets you see what DAGs and their tasks are doing, trigger runs of DAGs, view logs, and do some limited debugging and resolution of problems with your DAGs. -.. image:: /img/dags.png +.. image:: ../img/dags.png It's generally the best way to see the status of your Airflow installation as a whole, as well as diving into individual DAGs to see their layout, the status of each task, and the logs from each task. diff --git a/docs/apache-airflow/concepts/params.rst b/docs/apache-airflow/core-concepts/params.rst similarity index 100% rename from docs/apache-airflow/concepts/params.rst rename to docs/apache-airflow/core-concepts/params.rst diff --git a/docs/apache-airflow/concepts/sensors.rst b/docs/apache-airflow/core-concepts/sensors.rst similarity index 97% rename from docs/apache-airflow/concepts/sensors.rst rename to docs/apache-airflow/core-concepts/sensors.rst index 265e5e5ca38e9..bbe61e68f91d3 100644 --- a/docs/apache-airflow/concepts/sensors.rst +++ b/docs/apache-airflow/core-concepts/sensors.rst @@ -29,4 +29,4 @@ The ``poke`` and ``reschedule`` modes can be configured directly when you instan Much like Operators, Airflow has a large set of pre-built Sensors you can use, both in core Airflow as well as via our *providers* system. -.. seealso:: :doc:`deferring` +.. seealso:: :doc:`../authoring-and-scheduling/deferring` diff --git a/docs/apache-airflow/concepts/taskflow.rst b/docs/apache-airflow/core-concepts/taskflow.rst similarity index 100% rename from docs/apache-airflow/concepts/taskflow.rst rename to docs/apache-airflow/core-concepts/taskflow.rst diff --git a/docs/apache-airflow/concepts/tasks.rst b/docs/apache-airflow/core-concepts/tasks.rst similarity index 94% rename from docs/apache-airflow/concepts/tasks.rst rename to docs/apache-airflow/core-concepts/tasks.rst index c3f9d1de3b50c..dd1cd536a17c0 100644 --- a/docs/apache-airflow/concepts/tasks.rst +++ b/docs/apache-airflow/core-concepts/tasks.rst @@ -53,7 +53,7 @@ Or the more explicit ``set_upstream`` and ``set_downstream`` methods:: These both do exactly the same thing, but in general we recommend you use the bitshift operators, as they are easier to read in most cases. -By default, a Task will run when all of its upstream (parent) tasks have succeeded, but there are many ways of modifying this behaviour to add branching, to only wait for some upstream tasks, or to change behaviour based on where the current run is in history. For more, see :ref:`concepts:control-flow`. +By default, a Task will run when all of its upstream (parent) tasks have succeeded, but there are many ways of modifying this behaviour to add branching, to only wait for some upstream tasks, or to change behaviour based on where the current run is in history. For more, see :ref:`concepts-control-flow`. Tasks don't pass information to each other by default, and run entirely independently. If you want to pass information from one Task to another, you should use :doc:`xcoms`. @@ -63,7 +63,7 @@ Tasks don't pass information to each other by default, and run entirely independ Task Instances -------------- -Much in the same way that a DAG is instantiated into a :ref:`DAG Run ` each time it runs, the tasks under a DAG are instantiated into *Task Instances*. +Much in the same way that a DAG is instantiated into a :ref:`DAG Run ` each time it runs, the tasks under a DAG are instantiated into *Task Instances*. An instance of a Task is a specific run of that task for a given DAG (and thus for a given data interval). They are also the representation of a Task that has *state*, representing what stage of the lifecycle it is in. @@ -83,7 +83,7 @@ The possible states for a Task Instance are: * ``upstream_failed``: An upstream task failed and the :ref:`Trigger Rule ` says we needed it * ``up_for_retry``: The task failed, but has retry attempts left and will be rescheduled. * ``up_for_reschedule``: The task is a :doc:`Sensor ` that is in ``reschedule`` mode -* ``deferred``: The task has been :doc:`deferred to a trigger ` +* ``deferred``: The task has been :doc:`deferred to a trigger <../authoring-and-scheduling/deferring>` * ``removed``: The task has vanished from the DAG since the run started .. image:: /img/task_lifecycle_diagram.png @@ -170,7 +170,7 @@ To read more about configuring the emails, see :doc:`/howto/email-config`. .. note:: - Manually-triggered tasks and tasks in event-driven DAGs will not be checked for an SLA miss. For more information on DAG ``schedule`` values see :doc:`DAG Run `. + Manually-triggered tasks and tasks in event-driven DAGs will not be checked for an SLA miss. For more information on DAG ``schedule`` values see :doc:`DAG Run `. .. _concepts:sla_miss_callback: @@ -182,7 +182,7 @@ The function signature of an ``sla_miss_callback`` requires 5 parameters. #. ``dag`` - * Parent :ref:`DAG ` Object for the :doc:`DAGRun ` in which tasks missed their + * Parent :ref:`DAG ` Object for the :doc:`DAGRun ` in which tasks missed their :ref:`SLA `. #. ``task_list`` @@ -192,7 +192,7 @@ The function signature of an ``sla_miss_callback`` requires 5 parameters. #. ``blocking_task_list`` - * Any task in the :doc:`DAGRun(s)` (with the same ``execution_date`` as a task that missed + * Any task in the :doc:`DAGRun(s)` (with the same ``execution_date`` as a task that missed :ref:`SLA `) that is not in a **SUCCESS** state at the time that the ``sla_miss_callback`` runs. i.e. 'running', 'failed'. These tasks are described as tasks that are blocking itself or another task from completing before its SLA window is complete. @@ -251,7 +251,7 @@ No system runs perfectly, and task instances are expected to die once in a while Executor Configuration ---------------------- -Some :doc:`Executors ` allow optional per-task configuration - such as the ``KubernetesExecutor``, which lets you set an image to run the task on. +Some :doc:`Executors ` allow optional per-task configuration - such as the ``KubernetesExecutor``, which lets you set an image to run the task on. This is achieved via the ``executor_config`` argument to a Task or Operator. Here's an example of setting the Docker image for a task that will run on the ``KubernetesExecutor``:: @@ -262,4 +262,4 @@ This is achieved via the ``executor_config`` argument to a Task or Operator. Her } ) -The settings you can pass into ``executor_config`` vary by executor, so read the :doc:`individual executor documentation ` in order to see what you can set. +The settings you can pass into ``executor_config`` vary by executor, so read the :doc:`individual executor documentation ` in order to see what you can set. diff --git a/docs/apache-airflow/concepts/variables.rst b/docs/apache-airflow/core-concepts/variables.rst similarity index 100% rename from docs/apache-airflow/concepts/variables.rst rename to docs/apache-airflow/core-concepts/variables.rst diff --git a/docs/apache-airflow/concepts/xcoms.rst b/docs/apache-airflow/core-concepts/xcoms.rst similarity index 100% rename from docs/apache-airflow/concepts/xcoms.rst rename to docs/apache-airflow/core-concepts/xcoms.rst diff --git a/docs/apache-airflow/faq.rst b/docs/apache-airflow/faq.rst index fd91cca4879b0..0c2146529cc8a 100644 --- a/docs/apache-airflow/faq.rst +++ b/docs/apache-airflow/faq.rst @@ -298,7 +298,7 @@ Note that ``ds`` (the YYYY-MM-DD form of ``data_interval_start``) refers to .. tip:: For more information on ``logical date``, see :ref:`data-interval` and - :ref:`concepts:dag-run`. + :ref:`concepts-dag-run`. How to create DAGs dynamically? diff --git a/docs/apache-airflow/howto/connection.rst b/docs/apache-airflow/howto/connection.rst index b230bf00761f4..d1ca89e09aab9 100644 --- a/docs/apache-airflow/howto/connection.rst +++ b/docs/apache-airflow/howto/connection.rst @@ -20,14 +20,14 @@ Managing Connections .. seealso:: - For an overview of hooks and connections, see :doc:`/concepts/connections`. + For an overview of hooks and connections, see :doc:`/authoring-and-scheduling/connections`. Airflow's :class:`~airflow.models.connection.Connection` object is used for storing credentials and other information necessary for connecting to external services. Connections may be defined in the following ways: - in :ref:`environment variables ` - - in an external :doc:`/security/secrets/secrets-backend/index` + - in an external :doc:`/administration-and-deployment/security/secrets/secrets-backend/index` - in the :ref:`Airflow metadata database ` (using the :ref:`CLI ` or :ref:`web UI `) @@ -84,7 +84,7 @@ See :ref:`Connection URI format ` for more details on how Storing connections in a Secrets Backend ---------------------------------------- -You can store Airflow connections in external secrets backends like HashiCorp Vault, AWS SSM Parameter Store, and other such services. For more details see :doc:`/security/secrets/secrets-backend/index`. +You can store Airflow connections in external secrets backends like HashiCorp Vault, AWS SSM Parameter Store, and other such services. For more details see :doc:`/administration-and-deployment/security/secrets/secrets-backend/index`. .. _connections-in-database: @@ -92,7 +92,7 @@ Storing connections in the database ----------------------------------- .. seealso:: - Connections can alternatively be stored in :ref:`environment variables ` or an :doc:`external secrets backend ` such as HashiCorp Vault, AWS SSM Parameter Store, etc. + Connections can alternatively be stored in :ref:`environment variables ` or an :doc:`external secrets backend ` such as HashiCorp Vault, AWS SSM Parameter Store, etc. When storing connections in the database, you may manage them using either the web UI or the Airflow CLI. diff --git a/docs/apache-airflow/howto/custom-operator.rst b/docs/apache-airflow/howto/custom-operator.rst index 3f0096c62107f..15f1e79e2c923 100644 --- a/docs/apache-airflow/howto/custom-operator.rst +++ b/docs/apache-airflow/howto/custom-operator.rst @@ -29,7 +29,7 @@ You can create any operator you want by extending the :class:`airflow.models.bas There are two methods that you need to override in a derived class: * Constructor - Define the parameters required for the operator. You only need to specify the arguments specific to your operator. - You can specify the ``default_args`` in the dag file. See :ref:`Default args ` for more details. + You can specify the ``default_args`` in the dag file. See :ref:`Default args ` for more details. * Execute - The code to execute when the runner calls the operator. The method contains the Airflow context as a parameter that can be used to read config values. @@ -63,7 +63,7 @@ Let's implement an example ``HelloOperator`` in a new file ``hello_operator.py`` is present in the :envvar:`PYTHONPATH` env. Airflow adds ``dags/``, ``plugins/``, and ``config/`` directories in the Airflow home to :envvar:`PYTHONPATH` by default. e.g., In our example, the file is placed in the ``custom_operator/`` directory. - See :doc:`../modules_management` for details on how Python and Airflow manage modules. + See :doc:`/administration-and-deployment/modules_management` for details on how Python and Airflow manage modules. You can now use the derived custom operator as follows: diff --git a/docs/apache-airflow/howto/custom-view-plugin.rst b/docs/apache-airflow/howto/custom-view-plugin.rst index e6ccad4562eed..a62525b81edf3 100644 --- a/docs/apache-airflow/howto/custom-view-plugin.rst +++ b/docs/apache-airflow/howto/custom-view-plugin.rst @@ -48,7 +48,7 @@ Custom view Registration ------------------------ A custom view with object reference to flask_appbuilder and Blueprint from flask -and be registered as a part of a :doc:`plugin `. The following is a +and be registered as a part of a :doc:`plugin `. The following is a skeleton for us to implement a new custom view: .. code-block:: python diff --git a/docs/apache-airflow/howto/customize-ui.rst b/docs/apache-airflow/howto/customize-ui.rst index 664502a5966e4..15b7c94b40397 100644 --- a/docs/apache-airflow/howto/customize-ui.rst +++ b/docs/apache-airflow/howto/customize-ui.rst @@ -79,7 +79,7 @@ After .. note:: - See :doc:`../modules_management` for details on how Python and Airflow manage modules. + See :doc:`/administration-and-deployment/modules_management` for details on how Python and Airflow manage modules. Customizing DAG UI Header and Airflow Page Titles ------------------------------------------------- diff --git a/docs/apache-airflow/howto/define_extra_link.rst b/docs/apache-airflow/howto/define_extra_link.rst index 1436c2ece7c18..8f2f918d45511 100644 --- a/docs/apache-airflow/howto/define_extra_link.rst +++ b/docs/apache-airflow/howto/define_extra_link.rst @@ -60,7 +60,7 @@ The following code shows how to add extra links to an operator via Plugins: You can also add a global operator extra link that will be available to all the operators through an Airflow plugin or through Airflow providers. You can learn more about it in the -:ref:`plugin interface ` and in :doc:`apache-airflow-providers:index`. +:ref:`plugin interface ` and in :doc:`apache-airflow-providers:index`. You can see all the extra links available via community-managed providers in :doc:`apache-airflow-providers:core-extensions/extra-links`. diff --git a/docs/apache-airflow/howto/docker-compose/index.rst b/docs/apache-airflow/howto/docker-compose/index.rst index cc7a67e2973cd..46494c1abe37d 100644 --- a/docs/apache-airflow/howto/docker-compose/index.rst +++ b/docs/apache-airflow/howto/docker-compose/index.rst @@ -20,7 +20,7 @@ Running Airflow in Docker ######################### -This quick-start guide will allow you to quickly get Airflow up and running with :doc:`CeleryExecutor ` in Docker. +This quick-start guide will allow you to quickly get Airflow up and running with :doc:`CeleryExecutor ` in Docker. .. caution:: This procedure can be useful for learning and exploration. However, adapting it for use in real-world situations can be complicated. Making changes to this procedure will require specialized expertise in Docker & Docker Compose, and the Airflow community may not be able to help you. @@ -64,7 +64,7 @@ Fetching ``docker-compose.yaml`` This file contains several service definitions: -- ``airflow-scheduler`` - The :doc:`scheduler ` monitors all tasks and DAGs, then triggers the +- ``airflow-scheduler`` - The :doc:`scheduler ` monitors all tasks and DAGs, then triggers the task instances once their dependencies are complete. - ``airflow-webserver`` - The webserver is available at ``http://localhost:8080``. - ``airflow-worker`` - The worker that executes the tasks given by the scheduler. @@ -76,13 +76,13 @@ Optionally, you can enable flower by adding ``--profile flower`` option, e.g. `` - ``flower`` - `The flower app `__ for monitoring the environment. It is available at ``http://localhost:5555``. -All these services allow you to run Airflow with :doc:`CeleryExecutor `. For more information, see :doc:`/concepts/overview`. +All these services allow you to run Airflow with :doc:`CeleryExecutor `. For more information, see :doc:`/core-concepts/overview`. Some directories in the container are mounted, which means that their contents are synchronized between your computer and the container. - ``./dags`` - you can put your DAG files here. - ``./logs`` - contains logs from task execution and scheduler. -- ``./plugins`` - you can put your :doc:`custom plugins ` here. +- ``./plugins`` - you can put your :doc:`custom plugins ` here. This file uses the latest Airflow image (`apache/airflow `__). If you need to install a new Python library or system library, you can :doc:`build your image `. @@ -183,14 +183,14 @@ Accessing the environment After starting Airflow, you can interact with it in 3 ways: -* by running :doc:`CLI commands `. +* by running :doc:`CLI commands `. * via a browser using :doc:`the web interface `. * using :doc:`the REST API `. Running the CLI commands ------------------------ -You can also run :doc:`CLI commands `, but you have to do it in one of the defined ``airflow-*`` services. For example, to run ``airflow info``, run the following command: +You can also run :doc:`CLI commands <../usage-cli>`, but you have to do it in one of the defined ``airflow-*`` services. For example, to run ``airflow info``, run the following command: .. code-block:: bash diff --git a/docs/apache-airflow/howto/dynamic-dag-generation.rst b/docs/apache-airflow/howto/dynamic-dag-generation.rst index 5f1067ac8f3d0..dbc4db9075afa 100644 --- a/docs/apache-airflow/howto/dynamic-dag-generation.rst +++ b/docs/apache-airflow/howto/dynamic-dag-generation.rst @@ -20,14 +20,14 @@ Dynamic DAG Generation ====================== -To have a task repeated based on the output/result of a previous task see :doc:`/concepts/dynamic-task-mapping`. +To have a task repeated based on the output/result of a previous task see :doc:`/authoring-and-scheduling/dynamic-task-mapping`. Dynamic DAGs with environment variables ....................................... If you want to use variables to configure your code, you should always use `environment variables `_ in your -top-level code rather than :doc:`Airflow Variables `. Using Airflow Variables +top-level code rather than :doc:`Airflow Variables `. Using Airflow Variables at top-level code creates a connection to metadata DB of Airflow to fetch the value, which can slow down parsing and place extra load on the DB. See the `Airflow Variables <_best_practices/airflow_variables>`_ on how to make best use of Airflow Variables in your DAGs using Jinja templates . diff --git a/docs/apache-airflow/howto/index.rst b/docs/apache-airflow/howto/index.rst index b784639bafe63..5d6b54fd774bb 100644 --- a/docs/apache-airflow/howto/index.rst +++ b/docs/apache-airflow/howto/index.rst @@ -29,6 +29,7 @@ configuring an Airflow environment. .. toctree:: :maxdepth: 2 + Using the CLI add-dag-tags add-owner-links set-config @@ -49,3 +50,4 @@ configuring an Airflow environment. email-config dynamic-dag-generation docker-compose/index + upgrading-from-1-10/index diff --git a/docs/apache-airflow/howto/operator/index.rst b/docs/apache-airflow/howto/operator/index.rst index f3b2ca0fd0f8c..e734e81b0ddeb 100644 --- a/docs/apache-airflow/howto/operator/index.rst +++ b/docs/apache-airflow/howto/operator/index.rst @@ -24,7 +24,7 @@ An operator represents a single, ideally idempotent, task. Operators determine what actually executes when your DAG runs. .. note:: - See the :doc:`Operators Concepts ` documentation and the + See the :doc:`Operators Concepts ` documentation and the :doc:`Operators API Reference ` for more information. diff --git a/docs/apache-airflow/howto/set-config.rst b/docs/apache-airflow/howto/set-config.rst index 906e436e4134b..0f7cf5850402e 100644 --- a/docs/apache-airflow/howto/set-config.rst +++ b/docs/apache-airflow/howto/set-config.rst @@ -118,7 +118,7 @@ the example below. For more information on configuration options, see :doc:`../configurations-ref` .. note:: - See :doc:`../modules_management` for details on how Python and Airflow manage modules. + See :doc:`/administration-and-deployment/modules_management` for details on how Python and Airflow manage modules. .. note:: Use the same configuration across all the Airflow components. While each component diff --git a/docs/apache-airflow/howto/set-up-database.rst b/docs/apache-airflow/howto/set-up-database.rst index 7fa9ce6d0bb2d..f9a6322d7a88d 100644 --- a/docs/apache-airflow/howto/set-up-database.rst +++ b/docs/apache-airflow/howto/set-up-database.rst @@ -260,7 +260,7 @@ For more information regarding setup of the PostgreSQL connection, see `PostgreS -.. spelling:: +.. spelling:word-list:: hba @@ -375,4 +375,4 @@ What's next? ------------ By default, Airflow uses ``SequentialExecutor``, which does not provide parallelism. You should consider -configuring a different :doc:`executor ` for better performance. +configuring a different :doc:`executor ` for better performance. diff --git a/docs/apache-airflow/howto/timetable.rst b/docs/apache-airflow/howto/timetable.rst index e4d70b942d644..c1019374f8d03 100644 --- a/docs/apache-airflow/howto/timetable.rst +++ b/docs/apache-airflow/howto/timetable.rst @@ -46,7 +46,7 @@ Timetable Registration ---------------------- A timetable must be a subclass of :class:`~airflow.timetables.base.Timetable`, -and be registered as a part of a :doc:`plugin `. The following is a +and be registered as a part of a :doc:`plugin `. The following is a skeleton for us to implement a new timetable: .. code-block:: python diff --git a/docs/apache-airflow/upgrading-from-1-10/index.rst b/docs/apache-airflow/howto/upgrading-from-1-10/index.rst similarity index 100% rename from docs/apache-airflow/upgrading-from-1-10/index.rst rename to docs/apache-airflow/howto/upgrading-from-1-10/index.rst diff --git a/docs/apache-airflow/upgrading-from-1-10/upgrade-check.rst b/docs/apache-airflow/howto/upgrading-from-1-10/upgrade-check.rst similarity index 100% rename from docs/apache-airflow/upgrading-from-1-10/upgrade-check.rst rename to docs/apache-airflow/howto/upgrading-from-1-10/upgrade-check.rst diff --git a/docs/apache-airflow/usage-cli.rst b/docs/apache-airflow/howto/usage-cli.rst similarity index 98% rename from docs/apache-airflow/usage-cli.rst rename to docs/apache-airflow/howto/usage-cli.rst index 14f88f03fc948..15d22b8e631d3 100644 --- a/docs/apache-airflow/usage-cli.rst +++ b/docs/apache-airflow/howto/usage-cli.rst @@ -22,7 +22,7 @@ Using the Command Line Interface This document is meant to give an overview of all common tasks while using the CLI. .. note:: - For more information on CLI commands, see :doc:`cli-and-env-variables-ref` + For more information on CLI commands, see :doc:`/cli-and-env-variables-ref` .. _cli-remote: @@ -50,7 +50,7 @@ For one-time activation of argcomplete for airflow only, use: eval "$(register-python-argcomplete airflow)" -.. image:: img/cli_completion.gif +.. image:: /img/cli_completion.gif If you're using ``zsh``, add the following to your ``.zshrc``: @@ -88,7 +88,7 @@ To save the ``example_complex`` DAG as a PNG file: This will save the following image as a file: -.. figure:: img/usage_cli_export.png +.. figure:: /img/usage_cli_export.png :width: 100% Example DAG representation @@ -158,7 +158,7 @@ want to display ``example_bash_operator`` DAG then you can use the following co You will see a similar result as in the screenshot below. -.. figure:: img/usage_cli_imgcat.png +.. figure:: /img/usage_cli_imgcat.png Preview of DAG in iTerm2 diff --git a/docs/apache-airflow/howto/variable.rst b/docs/apache-airflow/howto/variable.rst index 401dcb146b4aa..a9e0d82023f95 100644 --- a/docs/apache-airflow/howto/variable.rst +++ b/docs/apache-airflow/howto/variable.rst @@ -27,7 +27,7 @@ code or CLI. .. image:: ../img/variable_hidden.png -See the :doc:`Variables Concepts ` documentation for +See the :doc:`Variables Concepts ` documentation for more information. Storing Variables in Environment Variables @@ -73,4 +73,4 @@ It guarantees that without the encryption password, content cannot be manipulate without the key. For information on configuring Fernet, look at :ref:`security/fernet`. In addition to retrieving variables from environment variables or the metastore database, you can enable -a secrets backend to retrieve variables. For more details see :doc:`/security/secrets/secrets-backend/index`. +a secrets backend to retrieve variables. For more details see :doc:`/administration-and-deployment/security/secrets/secrets-backend/index`. diff --git a/docs/apache-airflow/index.rst b/docs/apache-airflow/index.rst index 0a66a9aa8265a..887479422b40e 100644 --- a/docs/apache-airflow/index.rst +++ b/docs/apache-airflow/index.rst @@ -130,29 +130,17 @@ so coding will always be required. license start installation/index - upgrading-from-1-10/index tutorial/index howto/index ui - concepts/index - executor/index - dag-run - plugins - security/index - logging-monitoring/index - timezone - Using the CLI + core-concepts/index + authoring-and-scheduling/index + administration-and-deployment/index integration - kubernetes - lineage - listeners - dag-serialization - modules_management - Release Policies - release_notes best-practices - production-deployment faq + Release Policies + release_notes privacy_notice .. toctree:: diff --git a/docs/apache-airflow/installation/index.rst b/docs/apache-airflow/installation/index.rst index c93361d9de0d6..7fd4381001131 100644 --- a/docs/apache-airflow/installation/index.rst +++ b/docs/apache-airflow/installation/index.rst @@ -51,7 +51,7 @@ also be kept updated when Airflow is upgraded. .. warning:: As of June 2021 Airflow 1.10 is end-of-life and is not going to receive any fixes even critical - security fixes. Follow the :doc:`/upgrading-from-1-10/index` to learn + security fixes. Follow the :doc:`/howto/upgrading-from-1-10/index` to learn how to upgrade the end-of-life 1.10 to Airflow 2. Using released sources diff --git a/docs/apache-airflow/installation/prerequisites.rst b/docs/apache-airflow/installation/prerequisites.rst index d0b0b0025669b..a537e5a92a841 100644 --- a/docs/apache-airflow/installation/prerequisites.rst +++ b/docs/apache-airflow/installation/prerequisites.rst @@ -46,7 +46,7 @@ wildly on the deployment options you have .. warning:: MySQL 5.x versions are unable to or have limitations with - running multiple schedulers -- please see: :doc:`/concepts/scheduler`. + running multiple schedulers -- please see: :doc:`/administration-and-deployment/scheduler`. .. warning:: SQLite is used in Airflow tests. Do not use it in production. We recommend diff --git a/docs/apache-airflow/installation/upgrading.rst b/docs/apache-airflow/installation/upgrading.rst index 072e266dca9fe..cb596cef429ec 100644 --- a/docs/apache-airflow/installation/upgrading.rst +++ b/docs/apache-airflow/installation/upgrading.rst @@ -54,7 +54,7 @@ How to upgrade In order to manually upgrade the database you should run the ``airflow db upgrade`` command in your environment. It can be run either in your virtual environment or in the containers that give -you access to Airflow ``CLI`` :doc:`/usage-cli` and the database. +you access to Airflow ``CLI`` :doc:`/howto/usage-cli` and the database. Offline SQL migration scripts ============================= diff --git a/docs/apache-airflow/integration.rst b/docs/apache-airflow/integration.rst index a628f95dec6a6..7ae1b683af8c3 100644 --- a/docs/apache-airflow/integration.rst +++ b/docs/apache-airflow/integration.rst @@ -20,18 +20,18 @@ Integration Airflow has a mechanism that allows you to expand its functionality and integrate with other systems. -* :doc:`API Authentication backends ` +* :doc:`API Authentication backends ` * :doc:`Email backends ` -* :doc:`Executor ` -* :doc:`Kerberos ` -* :doc:`Logging ` -* :doc:`Metrics (statsd) ` +* :doc:`Executor ` +* :doc:`Kerberos ` +* :doc:`Logging ` +* :doc:`Metrics (statsd) ` * :doc:`Operators and hooks ` -* :doc:`Plugins ` -* :doc:`Listeners ` -* :doc:`Secrets backends ` -* :doc:`Tracking systems ` -* :doc:`Web UI Authentication backends ` +* :doc:`Plugins ` +* :doc:`Listeners ` +* :doc:`Secrets backends ` +* :doc:`Tracking systems ` +* :doc:`Web UI Authentication backends ` -It also has integration with :doc:`Sentry ` service for error tracking. Other applications can also integrate using +It also has integration with :doc:`Sentry ` service for error tracking. Other applications can also integrate using the :doc:`REST API `. diff --git a/docs/apache-airflow/migrations-ref.rst b/docs/apache-airflow/migrations-ref.rst index 4328e3546a656..a4919749d4668 100644 --- a/docs/apache-airflow/migrations-ref.rst +++ b/docs/apache-airflow/migrations-ref.rst @@ -310,6 +310,6 @@ Here's the list of all the Database Migrations that are executed via when you ru .. End of auto-generated table -.. spelling:: +.. spelling:word-list:: branchpoint mergepoint diff --git a/docs/apache-airflow/redirects.txt b/docs/apache-airflow/redirects.txt index e8b47933a14d9..20e11a493a9cf 100644 --- a/docs/apache-airflow/redirects.txt +++ b/docs/apache-airflow/redirects.txt @@ -16,8 +16,8 @@ # under the License. # Security -howto/use-alternative-secrets-backend.rst security/secrets/secrets-backend/index.rst -security.rst security/index.rst +howto/use-alternative-secrets-backend.rst administration-and-deployment/security/secrets/secrets-backend/index.rst +security.rst administration-and-deployment/security/index.rst # Operators guides howto/operator/external.rst howto/operator/external_task_sensor.rst @@ -27,17 +27,17 @@ howto/customize-dag-ui-page-instance-name.rst howto/customize-ui.rst#customizing howto/customize-state-colors-ui.rst howto/customize-ui.rst#customizing-state-colours # Web UI -howto/add-new-role.rst security/access-control.rst +howto/add-new-role.rst administration-and-deployment/security/access-control.rst # Set up a database howto/initialize-database.rst howto/set-up-database.rst # Logging & Monitoring -howto/check-health.rst logging-monitoring/check-health.rst -errors.rst logging-monitoring/errors.rst -howto/write-logs.rst logging-monitoring/logging-tasks.rst -metrics.rst logging-monitoring/metrics.rst -howto/tracking-user-activity.rst logging-monitoring/tracking-user-activity.rst +howto/check-health.rst administration-and-deployment/logging-monitoring/check-health.rst +errors.rst administration-and-deployment/logging-monitoring/errors.rst +howto/write-logs.rst administration-and-deployment/logging-monitoring/logging-tasks.rst +metrics.rst administration-and-deployment/logging-monitoring/metrics.rst +howto/tracking-user-activity.rst administration-and-deployment/logging-monitoring/tracking-user-activity.rst # Quick start start/docker.rst howto/docker-compose/index.rst @@ -51,13 +51,13 @@ rest-api-ref.rst deprecated-rest-api-ref.rst macros-ref.rst templates-ref.rst # Concepts -concepts.rst concepts/index.rst -scheduler.rst concepts/scheduler.rst +concepts.rst core-concepts/index.rst +scheduler.rst administration-and-deployment/scheduler.rst # Installation installation.rst installation/index.rst upgrade-check.rst installation/upgrade-check.rst -upgrading-to-2.rst upgrading-from-1-10/index.rst +upgrading-to-2.rst howto/upgrading-from-1-10/index.rst # Release Notes changelog.rst release_notes.rst @@ -65,3 +65,68 @@ changelog.rst release_notes.rst # Tutorials tutorial.rst tutorial/index.rst tutorial_taskflow_api.rst tutorial/taskflow.rst + +## Docs Structure Refactor +# indexes +security/index.rst administration-and-deployment/security/index.rst +security/secrets/index.rst administration-and-deployment/security/secrets/index.rst +security/secrets/secrets-backend/index.rst administration-and-deployment/security/secrets/secrets-backend/index.rst +logging-monitoring/index.rst administration-and-deployment/logging-monitoring/index.rst +concepts/index.rst core-concepts/index.rst +executor/index.rst core-concepts/executor/index.rst +upgrading-from-1-10/index.rst howto/upgrading-from-1-10/index.rst + +listeners.rst administration-and-deployment/listeners.rst +kubernetes.rst administration-and-deployment/kubernetes.rst +executor/kubernetes.rst core-concepts/executor/kubernetes.rst +plugins.rst authoring-and-scheduling/plugins.rst +modules_management.rst administration-and-deployment/modules_management.rst +dag-serialization.rst administration-and-deployment/dag-serialization.rst +timezone.rst authoring-and-scheduling/timezone.rst +production-deployment.rst administration-and-deployment/production-deployment.rst +usage-cli.rst howto/usage-cli.rst +lineage.rst administration-and-deployment/lineage.rst +dag-run.rst core-concepts/dag-run.rst +concepts/taskflow.rst core-concepts/taskflow.rst +security/kerberos.rst administration-and-deployment/security/kerberos.rst +security/workload.rst administration-and-deployment/security/workload.rst +security/flower.rst administration-and-deployment/security/flower.rst +security/webserver.rst administration-and-deployment/security/webserver.rst +security/api.rst administration-and-deployment/security/api.rst +security/access-control.rst administration-and-deployment/security/access-control.rst +security/secrets/fernet.rst administration-and-deployment/security/secrets/fernet.rst +security/secrets/mask-sensitive-values.rst administration-and-deployment/security/secrets/mask-sensitive-values.rst +security/secrets/secrets-backend/local-filesystem-secrets-backend.rst administration-and-deployment/security/secrets/secrets-backend/local-filesystem-secrets-backend.rst +logging-monitoring/tracking-user-activity.rst administration-and-deployment/logging-monitoring/tracking-user-activity.rst +logging-monitoring/errors.rst administration-and-deployment/logging-monitoring/errors.rst +logging-monitoring/logging-architecture.rst administration-and-deployment/logging-monitoring/logging-architecture.rst +logging-monitoring/callbacks.rst administration-and-deployment/logging-monitoring/callbacks.rst +logging-monitoring/logging-tasks.rst administration-and-deployment/logging-monitoring/logging-tasks.rst +logging-monitoring/check-health.rst administration-and-deployment/logging-monitoring/check-health.rst +logging-monitoring/metrics.rst administration-and-deployment/logging-monitoring/metrics.rst +concepts/params.rst core-concepts/params.rst +concepts/dynamic-task-mapping.rst authoring-and-scheduling/dynamic-task-mapping.rst +concepts/sensors.rst core-concepts/sensors.rst +concepts/variables.rst core-concepts/variables.rst +concepts/overview.rst core-concepts/overview.rst +concepts/connections.rst authoring-and-scheduling/connections.rst +concepts/tasks.rst core-concepts/tasks.rst +concepts/timetable.rst authoring-and-scheduling/timetable.rst +concepts/operators.rst core-concepts/operators.rst +concepts/xcoms.rst core-concepts/xcoms.rst +concepts/scheduler.rst administration-and-deployment/scheduler.rst +concepts/pools.rst administration-and-deployment/pools.rst +concepts/priority-weight.rst administration-and-deployment/priority-weight.rst +concepts/dagfile-processing.rst authoring-and-scheduling/dagfile-processing.rst +concepts/deferring.rst authoring-and-scheduling/deferring.rst +concepts/datasets.rst authoring-and-scheduling/datasets.rst +concepts/cluster-policies.rst administration-and-deployment/cluster-policies.rst +concepts/dags.rst core-concepts/dags.rst +executor/local_kubernetes.rst core-concepts/executor/local_kubernetes.rst +executor/celery_kubernetes.rst core-concepts/executor/celery_kubernetes.rst +executor/dask.rst core-concepts/executor/dask.rst +executor/debug.rst core-concepts/executor/debug.rst +executor/celery.rst core-concepts/executor/celery.rst +executor/local.rst core-concepts/executor/local.rst +executor/sequential.rst core-concepts/executor/sequential.rst +upgrading-from-1-10/upgrade-check.rst howto/upgrading-from-1-10/upgrade-check.rst diff --git a/docs/apache-airflow/start.rst b/docs/apache-airflow/start.rst index 3e6a25e01c9d7..1c7625dbc2117 100644 --- a/docs/apache-airflow/start.rst +++ b/docs/apache-airflow/start.rst @@ -81,7 +81,7 @@ command line utilities. As you grow and deploy Airflow to production, you will also want to move away from the ``standalone`` command we use here to running the components -separately. You can read more in :doc:`/production-deployment`. +separately. You can read more in :doc:`/administration-and-deployment/production-deployment`. Here are a few commands that will trigger a few task instances. You should be able to see the status of the jobs change in the ``example_bash_operator`` DAG as you diff --git a/docs/apache-airflow/templates-ref.rst b/docs/apache-airflow/templates-ref.rst index c49077ef67f5f..a48b591b754c6 100644 --- a/docs/apache-airflow/templates-ref.rst +++ b/docs/apache-airflow/templates-ref.rst @@ -23,7 +23,7 @@ Templates reference Variables, macros and filters can be used in templates (see the :ref:`concepts:jinja-templating` section) The following come for free out of the box with Airflow. -Additional custom macros can be added globally through :doc:`/plugins`, or at a DAG level through the +Additional custom macros can be added globally through :doc:`authoring-and-scheduling/plugins`, or at a DAG level through the ``DAG.user_defined_macros`` argument. .. _templates:variables: diff --git a/docs/apache-airflow/tutorial/fundamentals.rst b/docs/apache-airflow/tutorial/fundamentals.rst index d2071a0682273..6e965292f5ddd 100644 --- a/docs/apache-airflow/tutorial/fundamentals.rst +++ b/docs/apache-airflow/tutorial/fundamentals.rst @@ -45,7 +45,7 @@ The actual tasks defined here will run in a different context from the context of this script. Different tasks run on different workers at different points in time, which means that this script cannot be used to cross communicate between tasks. Note that for this -purpose we have a more advanced feature called :doc:`/concepts/xcoms`. +purpose we have a more advanced feature called :doc:`/core-concepts/xcoms`. People sometimes think of the DAG definition file as a place where they can do some actual data processing - that is not the case at all! @@ -66,7 +66,7 @@ Airflow DAG object. Let's start by importing the libraries we will need. :end-before: [END import_module] -See :doc:`/modules_management` for details on how Python and Airflow manage modules. +See :doc:`/administration-and-deployment/modules_management` for details on how Python and Airflow manage modules. Default Arguments ----------------- @@ -400,4 +400,4 @@ Here are a few things you might want to do next: .. seealso:: - Continue to the next step of the tutorial: :doc:`/tutorial/taskflow` - - Skip to the the :doc:`/concepts/index` section for detailed explanation of Airflow concepts such as DAGs, Tasks, Operators, and more + - Skip to the the :doc:`/core-concepts/index` section for detailed explanation of Airflow concepts such as DAGs, Tasks, Operators, and more diff --git a/docs/apache-airflow/tutorial/taskflow.rst b/docs/apache-airflow/tutorial/taskflow.rst index d859257d73e18..c9b3f3e1287d2 100644 --- a/docs/apache-airflow/tutorial/taskflow.rst +++ b/docs/apache-airflow/tutorial/taskflow.rst @@ -458,7 +458,7 @@ To retrieve an XCom result for a key other than ``return_value``, you can use: listed as a ``template_field``. In the code example below, a :class:`~airflow.providers.http.operators.http.SimpleHttpOperator` result -is captured via :doc:`XComs `. This XCom result, which is the task output, is then passed +is captured via :doc:`XComs `. This XCom result, which is the task output, is then passed to a TaskFlow function which parses the response as JSON. .. code-block:: python @@ -587,5 +587,5 @@ You have seen how simple it is to write DAGs using the TaskFlow API paradigm wit .. seealso:: - Continue to the next step of the tutorial: :doc:`/tutorial/pipeline` - - Read the :doc:`Concepts section ` for detailed explanation of Airflow concepts such as DAGs, Tasks, Operators, and more - - View the section on the :doc:`TaskFlow API ` and the ``@task`` decorator. + - Read the :doc:`Concepts section ` for detailed explanation of Airflow concepts such as DAGs, Tasks, Operators, and more + - View the section on the :doc:`TaskFlow API ` and the ``@task`` decorator. diff --git a/docs/docker-stack/build.rst b/docs/docker-stack/build.rst index c0cdf671b8ed0..ea6ae91db586a 100644 --- a/docs/docker-stack/build.rst +++ b/docs/docker-stack/build.rst @@ -273,7 +273,7 @@ Naming conventions for the images: * The "latest" image is always the latest released stable version available. -.. spelling:: +.. spelling:word-list:: pythonN From c5d64cec781d75066b40dd34fea482323e7f21ec Mon Sep 17 00:00:00 2001 From: Jarek Potiuk Date: Thu, 22 Dec 2022 23:03:03 +0100 Subject: [PATCH 140/158] Cleanup and do housekeeping with plugin examples (#28537) This PR performs housekeeping of the plugin examples: * makes the examples independent of Hive being installed * adds "has_access" in the examples * removes the misleading "metastore" (which is hive metastore not Airflow Metastore as used in other places This way our example will be much easier to apply by anyone. (cherry picked from commit 66eb282b7d8746bcb5d90117479c35ae5ca0cfbc) --- .dockerignore | 1 - .../utils/docker_command_utils.py | 1 - dev/breeze/tests/test_commands.py | 8 +- .../authoring-and-scheduling/plugins.rst | 12 ++ .../apache-airflow/empty_plugin}/README.md | 4 +- .../empty_plugin/empty_plugin.py | 60 ++++++ .../templates/empty_plugin/index.html | 24 ++- .../howto/custom-view-plugin.rst | 72 ++----- metastore_browser/hive_metastore.py | 199 ------------------ .../templates/metastore_browser/base.html | 57 ----- .../templates/metastore_browser/db.html | 46 ---- .../templates/metastore_browser/table.html | 152 ------------- scripts/ci/docker-compose/local.yml | 3 - 13 files changed, 109 insertions(+), 530 deletions(-) rename {metastore_browser => docs/apache-airflow/empty_plugin}/README.md (90%) create mode 100644 docs/apache-airflow/empty_plugin/empty_plugin.py rename metastore_browser/templates/metastore_browser/dbs.html => docs/apache-airflow/empty_plugin/templates/empty_plugin/index.html (64%) delete mode 100644 metastore_browser/hive_metastore.py delete mode 100644 metastore_browser/templates/metastore_browser/base.html delete mode 100644 metastore_browser/templates/metastore_browser/db.html delete mode 100644 metastore_browser/templates/metastore_browser/table.html diff --git a/.dockerignore b/.dockerignore index 7a33552d0ee92..045b730630678 100644 --- a/.dockerignore +++ b/.dockerignore @@ -34,7 +34,6 @@ !chart !docs !licenses -!metastore_browser # Add those folders to the context so that they are available in the CI container !scripts/in_container diff --git a/dev/breeze/src/airflow_breeze/utils/docker_command_utils.py b/dev/breeze/src/airflow_breeze/utils/docker_command_utils.py index 55cb94d7d8821..7771811e86db1 100644 --- a/dev/breeze/src/airflow_breeze/utils/docker_command_utils.py +++ b/dev/breeze/src/airflow_breeze/utils/docker_command_utils.py @@ -105,7 +105,6 @@ ("kubernetes_tests", "/opt/airflow/kubernetes_tests"), ("docker_tests", "/opt/airflow/docker_tests"), ("chart", "/opt/airflow/chart"), - ("metastore_browser", "/opt/airflow/metastore_browser"), ] diff --git a/dev/breeze/tests/test_commands.py b/dev/breeze/tests/test_commands.py index 8b53228baee70..bbff4d70895de 100644 --- a/dev/breeze/tests/test_commands.py +++ b/dev/breeze/tests/test_commands.py @@ -27,23 +27,23 @@ def test_visuals(): def test_get_extra_docker_flags_all(): flags = get_extra_docker_flags(MOUNT_ALL) - assert "empty" not in "".join(flags) + assert "/empty," not in "".join(flags) assert len(flags) < 10 def test_get_extra_docker_flags_selected(): flags = get_extra_docker_flags(MOUNT_SELECTED) - assert "empty" not in "".join(flags) + assert "/empty," not in "".join(flags) assert len(flags) > 40 def test_get_extra_docker_flags_remove(): flags = get_extra_docker_flags(MOUNT_REMOVE) - assert "empty" in "".join(flags) + assert "/empty," in "".join(flags) assert len(flags) < 10 def test_get_extra_docker_flags_skip(): flags = get_extra_docker_flags(MOUNT_SKIP) - assert "empty" not in "".join(flags) + assert "/empty," not in "".join(flags) assert len(flags) < 10 diff --git a/docs/apache-airflow/authoring-and-scheduling/plugins.rst b/docs/apache-airflow/authoring-and-scheduling/plugins.rst index 77d51f762df49..bd74de93d09ad 100644 --- a/docs/apache-airflow/authoring-and-scheduling/plugins.rst +++ b/docs/apache-airflow/authoring-and-scheduling/plugins.rst @@ -168,6 +168,8 @@ definitions in Airflow. # This is the class you derive to create a plugin from airflow.plugins_manager import AirflowPlugin + from airflow.security import permissions + from airflow.www.auth import has_access from flask import Blueprint from flask_appbuilder import expose, BaseView as AppBuilderBaseView @@ -201,6 +203,11 @@ definitions in Airflow. default_view = "test" @expose("/") + @has_access( + [ + (permissions.ACTION_CAN_READ, permissions.RESOURCE_WEBSITE), + ] + ) def test(self): return self.render_template("test_plugin/test.html", content="Hello galaxy!") @@ -210,6 +217,11 @@ definitions in Airflow. default_view = "test" @expose("/") + @has_access( + [ + (permissions.ACTION_CAN_READ, permissions.RESOURCE_WEBSITE), + ] + ) def test(self): return self.render_template("test_plugin/test.html", content="Hello galaxy!") diff --git a/metastore_browser/README.md b/docs/apache-airflow/empty_plugin/README.md similarity index 90% rename from metastore_browser/README.md rename to docs/apache-airflow/empty_plugin/README.md index ddd414a2d334f..574c20ab10f89 100644 --- a/metastore_browser/README.md +++ b/docs/apache-airflow/empty_plugin/README.md @@ -17,11 +17,11 @@ under the License. --> -# Apache Hive metastore plugin +# Apache example plugin This is an example plugin for Apache Airflow. -This plugin allows you to view Apache Hive metastore from the web UI interface. +This plugin displays empty view. ## Installation diff --git a/docs/apache-airflow/empty_plugin/empty_plugin.py b/docs/apache-airflow/empty_plugin/empty_plugin.py new file mode 100644 index 0000000000000..abf7b6d8027ec --- /dev/null +++ b/docs/apache-airflow/empty_plugin/empty_plugin.py @@ -0,0 +1,60 @@ +# +# 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. +"""Plugins example""" +from __future__ import annotations + +from flask import Blueprint +from flask_appbuilder import BaseView, expose + +from airflow.plugins_manager import AirflowPlugin +from airflow.security import permissions +from airflow.www.auth import has_access + + +class EmptyPluginView(BaseView): + """Creating a Flask-AppBuilder View""" + + default_view = "index" + + @expose("/") + @has_access( + [ + (permissions.ACTION_CAN_READ, permissions.RESOURCE_WEBSITE), + ] + ) + def index(self): + """Create default view""" + return self.render_template("empty_plugin/index.html", name="Empty Plugin") + + +# Creating a flask blueprint +bp = Blueprint( + "Empty Plugin", + __name__, + template_folder="templates", + static_folder="static", + static_url_path="/static/empty_plugin", +) + + +class EmptyPlugin(AirflowPlugin): + """Defining the plugin class""" + + name = "Empty Plugin" + flask_blueprints = [bp] + appbuilder_views = [{"name": "Empty Plugin", "category": "Extra Views", "view": EmptyPluginView()}] diff --git a/metastore_browser/templates/metastore_browser/dbs.html b/docs/apache-airflow/empty_plugin/templates/empty_plugin/index.html similarity index 64% rename from metastore_browser/templates/metastore_browser/dbs.html rename to docs/apache-airflow/empty_plugin/templates/empty_plugin/index.html index 6a6e187a1af86..39489b172c28b 100644 --- a/metastore_browser/templates/metastore_browser/dbs.html +++ b/docs/apache-airflow/empty_plugin/templates/empty_plugin/index.html @@ -17,11 +17,23 @@ under the License. #} -{% extends 'metastore_browser/base.html' %} +{% extends base_template %} -{% block plugin_content %} -

    - Hive Databases -

    - {{ table }} +{% block head %} + {{ super() }} +{% endblock %} + +{% block body %} +
    +

    + {% block page_header %}{{ name }}{% endblock%} +

    +
    +
    +
    + {% block plugin_content %}{% endblock %} +{% endblock %} + +{% block tail %} + {{ super() }} {% endblock %} diff --git a/docs/apache-airflow/howto/custom-view-plugin.rst b/docs/apache-airflow/howto/custom-view-plugin.rst index a62525b81edf3..914890073bbdb 100644 --- a/docs/apache-airflow/howto/custom-view-plugin.rst +++ b/docs/apache-airflow/howto/custom-view-plugin.rst @@ -16,27 +16,22 @@ under the License. -Customize view of Apache Hive Metastore from Airflow web UI -=========================================================== +Customize view of Apache from Airflow web UI +============================================ Airflow has feature that allows to integrate a custom UI along with its core UI using the Plugin manager -This is an example plugin for Airflow that allows to create custom view of -Apache Hive metastore from the web UI of Airflow. Showing Metastore information -like list of the tables in database, finding the table object/ ddl information -for given table, retrieving partition information, retrieving data from the table, -retrieving table objects from Hive Metastore are some of the custom views shown -in this example. +This is an example plugin for Airflow that displays absolutely nothing. In this plugin, two object reference are derived from the base class ``airflow.plugins_manager.AirflowPlugin``. They are flask_blueprints and appbuilder_views Using flask_blueprints in Airflow plugin, the core application can be extended -to support the application that is customized to view Apache Hive Metastore. +to support the application that is customized to view Empty Plugin. In this object reference, the list of Blueprint object with the static template for -rendering the Metastore information is passed on. +rendering the information. Using appbuilder_views in Airflow plugin, a class that represents a concept is added and presented with views and methods to implement it. @@ -48,43 +43,16 @@ Custom view Registration ------------------------ A custom view with object reference to flask_appbuilder and Blueprint from flask -and be registered as a part of a :doc:`plugin `. The following is a -skeleton for us to implement a new custom view: +and be registered as a part of a :doc:`plugin `. -.. code-block:: python - - from airflow.plugins_manager import AirflowPlugin - from flask import Blueprint - from flask_appbuilder import BaseView - - - class MetastoreBrowserView(BaseView): - pass - - - # Creating a flask blueprint to integrate the templates and static folder - bp = Blueprint( - "metastore_browser", - __name__, - template_folder="templates", - static_folder="static", - static_url_path="/static/metastore_browser", - ) +The following is a skeleton for us to implement a new custom view: +.. exampleinclude:: ../empty_plugin/empty_plugin.py + :language: python - class MetastoreBrowserPlugin(AirflowPlugin): - name = "metastore_browser" - flask_blueprints = [bp] - appbuilder_views = [ - { - "category": "Plugins", # name of the tab in Airflow UI - "name": "Hive Metadata Browser", # name of link under the tab - "view": MetastoreBrowserView(), - } - ] ``Plugins`` specified in the ``category`` key of ``appbuilder_views`` dictionary is -the name of the tab in the navigation bar of the Airflow UI. ``Hive Metastore Browser`` +the name of the tab in the navigation bar of the Airflow UI. ``Empty Plugin`` is the name of the link under the tab ``Plugins``, which will launch the plugin We have to add Blueprint for generating the part of the application @@ -92,29 +60,15 @@ that needs to be rendered in Airflow web UI. We can define templates, static fil and this blueprint will be registered as part of the Airflow application when the plugin gets loaded. -Next, we can add code into ``MetastoreBrowserView`` with views and implementing -methods for each of those views. After the implementation, the custom view -created becomes part of the Airflow web UI. - -For reference, here's the plugin code within ``MetastoreBrowserView`` class that shows list of tables in the database: - -.. exampleinclude:: ../../../metastore_browser/hive_metastore.py - :language: python - :start-after: [START howto_customview_show_database_table] - :end-before: [END howto_customview_show_database_table] - The ``$AIRFLOW_HOME/plugins`` folder with custom view UI have the following folder structure. :: plugins - ├── hive_metastore.py + ├── empty_plugin.py ├── templates - | └── metastore_browser - | ├── base.html - | ├── db.html - | ├── dbs.html - │ └── table.html + | └── empty_plugin + | ├── index.html └── README.md The HTML files required to render the views built is added as part of the diff --git a/metastore_browser/hive_metastore.py b/metastore_browser/hive_metastore.py deleted file mode 100644 index 684893579792f..0000000000000 --- a/metastore_browser/hive_metastore.py +++ /dev/null @@ -1,199 +0,0 @@ -# -# 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. -"""Plugins metabrowser""" -from __future__ import annotations - -import json -from datetime import datetime - -import pandas as pd -from flask import Blueprint, request -from flask_appbuilder import BaseView, expose -from markupsafe import Markup - -from airflow.plugins_manager import AirflowPlugin -from airflow.providers.apache.hive.hooks.hive import HiveCliHook, HiveMetastoreHook -from airflow.providers.mysql.hooks.mysql import MySqlHook -from airflow.providers.presto.hooks.presto import PrestoHook -from airflow.www.decorators import gzipped - -METASTORE_CONN_ID = "metastore_default" -METASTORE_MYSQL_CONN_ID = "metastore_mysql" -PRESTO_CONN_ID = "presto_default" -HIVE_CLI_CONN_ID = "hive_default" -DEFAULT_DB = "default" -DB_ALLOW_LIST: list[str] = [] -DB_DENY_LIST: list[str] = ["tmp"] -TABLE_SELECTOR_LIMIT = 2000 - -# Keeping pandas from truncating long strings -pd.set_option("display.max_colwidth", -1) - - -class MetastoreBrowserView(BaseView): - """Creating a Flask-AppBuilder BaseView""" - - default_view = "index" - - @expose("/") - def index(self): - """Create default view""" - sql = """ - SELECT - a.name as db, db_location_uri as location, - count(1) as object_count, a.desc as description - FROM DBS a - JOIN TBLS b ON a.DB_ID = b.DB_ID - GROUP BY a.name, db_location_uri, a.desc - """ - hook = MySqlHook(METASTORE_MYSQL_CONN_ID) - df = hook.get_pandas_df(sql) - df.db = '' + df.db + "" - table = df.to_html( - classes="table table-striped table-bordered table-hover", - index=False, - escape=False, - na_rep="", - ) - return self.render_template("metastore_browser/dbs.html", table=Markup(table)) - - # [START howto_customview_table] - @expose("/table/") - def table(self): - """Create table view""" - table_name = request.args.get("table") - metastore = HiveMetastoreHook(METASTORE_CONN_ID) - table = metastore.get_table(table_name) - return self.render_template( - "metastore_browser/table.html", table=table, table_name=table_name, datetime=datetime, int=int - ) - - # [END howto_customview_table] - - # [START howto_customview_show_database_table] - @expose("/db/") - def db(self): - """Show tables in database""" - db = request.args.get("db") - metastore = HiveMetastoreHook(METASTORE_CONN_ID) - tables = sorted(metastore.get_tables(db=db), key=lambda x: x.tableName) - return self.render_template("metastore_browser/db.html", tables=tables, db=db) - - # [END howto_customview_show_database_table] - - # [START howto_customview_partitions_info] - @gzipped - @expose("/partitions/") - def partitions(self): - """Retrieve table partitions""" - schema, table = request.args.get("table").split(".") - sql = f""" - SELECT - a.PART_NAME, - a.CREATE_TIME, - c.LOCATION, - c.IS_COMPRESSED, - c.INPUT_FORMAT, - c.OUTPUT_FORMAT - FROM PARTITIONS a - JOIN TBLS b ON a.TBL_ID = b.TBL_ID - JOIN DBS d ON b.DB_ID = d.DB_ID - JOIN SDS c ON a.SD_ID = c.SD_ID - WHERE - b.TBL_NAME like '{table}' AND - d.NAME like '{schema}' - ORDER BY PART_NAME DESC - """ - hook = MySqlHook(METASTORE_MYSQL_CONN_ID) - df = hook.get_pandas_df(sql) - return df.to_html( - classes="table table-striped table-bordered table-hover", - index=False, - na_rep="", - ) - - # [END howto_customview_partitions_info] - - @gzipped - @expose("/objects/") - def objects(self): - """Retrieve objects from TBLS and DBS""" - where_clause = "" - if DB_ALLOW_LIST: - dbs = ",".join("'" + db + "'" for db in DB_ALLOW_LIST) - where_clause = f"AND b.name IN ({dbs})" - if DB_DENY_LIST: - dbs = ",".join("'" + db + "'" for db in DB_DENY_LIST) - where_clause = f"AND b.name NOT IN ({dbs})" - sql = f""" - SELECT CONCAT(b.NAME, '.', a.TBL_NAME), TBL_TYPE - FROM TBLS a - JOIN DBS b ON a.DB_ID = b.DB_ID - WHERE - a.TBL_NAME NOT LIKE '%tmp%' AND - a.TBL_NAME NOT LIKE '%temp%' AND - b.NAME NOT LIKE '%tmp%' AND - b.NAME NOT LIKE '%temp%' - {where_clause} - LIMIT {TABLE_SELECTOR_LIMIT}; - """ - hook = MySqlHook(METASTORE_MYSQL_CONN_ID) - data = [{"id": row[0], "text": row[0]} for row in hook.get_records(sql)] - return json.dumps(data) - - @gzipped - @expose("/data/") - def data(self): - """Retrieve data from table""" - table = request.args.get("table") - sql = f"SELECT * FROM {table} LIMIT 1000;" - hook = PrestoHook(PRESTO_CONN_ID) - df = hook.get_pandas_df(sql) - return df.to_html( - classes="table table-striped table-bordered table-hover", - index=False, - na_rep="", - ) - - @expose("/ddl/") - def ddl(self): - """Retrieve table ddl""" - table = request.args.get("table") - sql = f"SHOW CREATE TABLE {table};" - hook = HiveCliHook(HIVE_CLI_CONN_ID) - return hook.run_cli(sql) - - -# Creating a flask blueprint to integrate the templates and static folder -bp = Blueprint( - "metastore_browser", - __name__, - template_folder="templates", - static_folder="static", - static_url_path="/static/metastore_browser", -) - - -class MetastoreBrowserPlugin(AirflowPlugin): - """Defining the plugin class""" - - name = "metastore_browser" - flask_blueprints = [bp] - appbuilder_views = [ - {"name": "Hive Metadata Browser", "category": "Plugins", "view": MetastoreBrowserView()} - ] diff --git a/metastore_browser/templates/metastore_browser/base.html b/metastore_browser/templates/metastore_browser/base.html deleted file mode 100644 index 94ba0e10106c6..0000000000000 --- a/metastore_browser/templates/metastore_browser/base.html +++ /dev/null @@ -1,57 +0,0 @@ -{# - 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. -#} - -{% extends base_template %} - -{% block head %} - {{ super() }} - - -{% endblock %} - -{% block body %} -
    -

    - {% block page_header %}Hive Metastore Browser{% endblock%} -

    -
    -
    -
    - {% block plugin_content %}{% endblock %} -{% endblock %} - -{% block tail %} - {{ super() }} - - - - -{% endblock %} diff --git a/metastore_browser/templates/metastore_browser/db.html b/metastore_browser/templates/metastore_browser/db.html deleted file mode 100644 index 5f3b9565f41e7..0000000000000 --- a/metastore_browser/templates/metastore_browser/db.html +++ /dev/null @@ -1,46 +0,0 @@ -{# - 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. -#} - -{% extends 'metastore_browser/base.html' %} - -{% block plugin_content %} -

    - Database: {{ db }} -

    - - - - - - - - - {%for table in tables %} - - - - - {% endfor %} - -
    TableOwner
    - - {{ table.tableName }} - - {{ table.owner }}
    -{% endblock %} diff --git a/metastore_browser/templates/metastore_browser/table.html b/metastore_browser/templates/metastore_browser/table.html deleted file mode 100644 index c62615cf4dcf0..0000000000000 --- a/metastore_browser/templates/metastore_browser/table.html +++ /dev/null @@ -1,152 +0,0 @@ -{# - 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. -#} - -{% extends 'metastore_browser/base.html' %} - -{% block plugin_content %} -
    -

    - Table: {{ table.dbName }}.{{ table.tableName }} -

    -
    - -
    -
    - - - - - - - - - - {%for col in table.partitionKeys %} - - - - - - {% endfor %} - {%for col in table.sd.cols %} - - - - - - {% endfor %} - -
    ColumnTypeComment
    {{ col.name }} (PARTITION){{ col.type }}{{ col.comment or '' }}
    {{ col.name }}{{ col.type }}{{ col.comment or '' }}
    -
    -
    - -
    -
    - -
    -
    - - - - - - - - - {%for k, v in table.__dict__.items() %} - {% if v and k not in ('sd', 'partitionKeys', 'tableName', 'parameters') %} - - - {% if k.endswith('Time') %} - - {% else %} - - {% endif %} - - {% endif %} - {% endfor %} - -
    AttributeValue
    {{ k }}{{ datetime.fromtimestamp(int(v)).isoformat() }}{{ v }}
    -
    -
    - - - - - - - - - {%for k, v in table.parameters.items() %} - {% if v and k not in [] %} - - - {% if k.endswith('Time') %} - - {% else %} - - {% endif %} - - {% endif %} - {% endfor %} - -
    ParameterValue
    {{ k }}{{ datetime.fromtimestamp(int(v)).isoformat() }}{{ v }}
    -
    -
    -
    -        
    -      
    -
    -
    -{% endblock %} - -{% block tail %} -{{ super() }} - -{% endblock %} diff --git a/scripts/ci/docker-compose/local.yml b/scripts/ci/docker-compose/local.yml index c03b7bcc96c4f..45a18acceb6b0 100644 --- a/scripts/ci/docker-compose/local.yml +++ b/scripts/ci/docker-compose/local.yml @@ -126,7 +126,4 @@ services: - type: bind source: ../../../chart target: /opt/airflow/chart - - type: bind - source: ../../../metastore_browser - target: /opt/airflow/metastore_browser # END automatically generated volumes from VOLUMES_FOR_SELECTED_MOUNTS in docker_command_utils.py From fafce07bfb66bf8758afa2838e38c7bec794e050 Mon Sep 17 00:00:00 2001 From: nate contino Date: Mon, 19 Dec 2022 11:23:03 -0500 Subject: [PATCH 141/158] Remove extra H1 & improve formatting of Listeners docs page (#28450) * Remove extra H1 & improve formatting of Listeners docs page I noticed that the documentation has an unclickable "Usage" page in the TOC. A little digging later, I discovered that this page contains an extra H1, and since this page is in the top level of the TOC, all the H1s on this page show up in the left docs sidebar. Demoted the "Usage" section to an H2, and fixed the other headers on this page to use consistent underlining with most other docs pages in this repo. I also took the liberty of sprucing up the language on the page to follow docs best practices, like shorter, highly readable sentences, title case in section titles, and bulleted lists to draw attention to important collections. * Remove extra newlines from specification discussion * Remove single newlines from listener API discussion * Remove nonexistent DagRun events from listeners page (cherry picked from commit 672264b0af4874274bd130ba42a78e8e72f3d3ff) --- .../listeners.rst | 54 +++++++++---------- 1 file changed, 27 insertions(+), 27 deletions(-) diff --git a/docs/apache-airflow/administration-and-deployment/listeners.rst b/docs/apache-airflow/administration-and-deployment/listeners.rst index 2678e12e5ef66..c34689c53fa49 100644 --- a/docs/apache-airflow/administration-and-deployment/listeners.rst +++ b/docs/apache-airflow/administration-and-deployment/listeners.rst @@ -18,43 +18,43 @@ Listeners ========= -Airflow gives you an option to be notified of events happening in Airflow -by writing listeners. Listeners are powered by `pluggy `__ +You can write listeners to enable Airflow to notify you when events happen. +`Pluggy `__ powers these listeners. -Right now Airflow exposes few types of events. +Airflow supports notifications for the following events: -Lifecycle events -^^^^^^^^^^^^^^^^ -Those events - ``on_starting`` and ``before_stopping`` allow you to react to -lifecycle to an Airflow ``Job``, like ``SchedulerJob`` or ``BackfillJob``. +Lifecycle Events +---------------- -TaskInstance state change events -^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ -Those events - ``on_task_instance_running``, ``on_task_instance_success`` and ``on_task_instance_failed`` -once ``TaskInstance`` state changes to one of the respective states. This generally happens on ``LocalTaskJob``. +- ``on_starting`` +- ``before_stopping`` + +Lifecycle events allow you to react to start and stop events for an Airflow ``Job``, like ``SchedulerJob`` or ``BackfillJob``. + +TaskInstance State Change Events +-------------------------------- + +- ``on_task_instance_running`` +- ``on_task_instance_success`` +- ``on_task_instance_failed`` + +TaskInstance state change events occur when a ``TaskInstance`` changes state. +You can use these events to react to ``LocalTaskJob`` state changes. -DagRun state change events -^^^^^^^^^^^^^^^^^^^^^^^^^^ -Those events - ``on_dag_run_running``, ``on_dag_run_success`` and ``on_dag_run_failed`` -once ``DagRun`` state changes to one of the respective states. This generally happens on ``SchedulerJob`` or ``BackfillJob``. Usage -===== +----- + +To create a listener: -To create a listener you will need to derive the import -``airflow.listeners.hookimpl`` and implement the ``hookimpls`` for -events you want to be notified at. +- import ``airflow.listeners.hookimpl`` +- implement the ``hookimpls`` for events that you'd like to generate notifications -Their specification is defined as ``hookspec`` in ``airflow/listeners/spec`` directory. -Your implementation needs to accept the same named parameters as defined in hookspec, or Pluggy will complain about your plugin. -On the other hand, you don't need to implement every method - it's perfectly fine to have a listener that implements just one method, or any subset of methods. +Airflow defines the specification as `hookspec `__. Your implementation must accept the same named parameters as defined in hookspec. If you don't use the same parameters as hookspec, Pluggy throws an error when you try to use your plugin. But you don't need to implement every method. Many listeners only implement one method, or a subset of methods. -To include listener in your Airflow installation, include it as a part of an :doc:`Airflow Plugin ` +To include the listener in your Airflow installation, include it as a part of an :doc:`Airflow Plugin ` -Listener API is meant to be called across all dags, and all operators - in contrast to methods like -``on_success_callback``, ``pre_execute`` and related family which are meant to provide callbacks -for particular dag authors, or operator creators. There is no possibility to listen on events generated -by particular dag. +Listener API is meant to be called across all DAGs and all operators. You can't listen to events generated by specific DAGs. For that behavior, try methods like ``on_success_callback`` and ``pre_execute``. These provide callbacks for particular DAG authors or operator creators. |experimental| From c34ab42a627dd871f2f0698f190b87077c7f416c Mon Sep 17 00:00:00 2001 From: Jarek Potiuk Date: Sat, 31 Dec 2022 00:52:10 +0100 Subject: [PATCH 142/158] Slightly improve description of Dynamic DAG generation preamble (#28650) The description is more clear now what Dynamic DAG generation is vs. Dynamic Task Mapping and note is added to the users to pay attention about the stable sorting that should be applied when generating DAGS. Related: #27523 (cherry picked from commit 36d887b34edd9c23328df9d7aa72f27a2c9f487e) --- .../apache-airflow/howto/dynamic-dag-generation.rst | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/docs/apache-airflow/howto/dynamic-dag-generation.rst b/docs/apache-airflow/howto/dynamic-dag-generation.rst index dbc4db9075afa..abb8e6352064f 100644 --- a/docs/apache-airflow/howto/dynamic-dag-generation.rst +++ b/docs/apache-airflow/howto/dynamic-dag-generation.rst @@ -20,7 +20,18 @@ Dynamic DAG Generation ====================== -To have a task repeated based on the output/result of a previous task see :doc:`/authoring-and-scheduling/dynamic-task-mapping`. +This document describes creation of DAGs that have a structure generated dynamically, but where the number of +tasks in the DAG does not change between DAG Runs. If you want to implement a DAG where number of Tasks (or +Task Groups as of Airflow 2.6) can change based on the output/result of previous tasks, see +:doc:`/authoring-and-scheduling/dynamic-task-mapping`. + +.. note:: Consistent sequence of generating tasks and task groups + + In all cases where you generate DAGs dynamically, you should make sure that Tasks and Task Groups + are generated with consistent sequence every time the DAG is generated, otherwise you might end up with + Tasks and Task Groups changing their sequence in the Grid View every time you refresh the page. + This can be achieved for example by using a stable sorting mechanism in your Database queries or by using + ``sorted()`` function in Python. Dynamic DAGs with environment variables ....................................... From 1a044ed77db7f7430daa68f4388f5dda8cf2e4b5 Mon Sep 17 00:00:00 2001 From: Jarek Potiuk Date: Fri, 9 Dec 2022 16:25:21 +0100 Subject: [PATCH 143/158] Added --integration flag to "standard" set of flags for testing command (#28261) The --integration tag should be standard flag rather than left for the common options. This change moves the flag to the right group. Images were regenerated because Rich does not know that the commands changed just when the option was moved to another group (this is a rich-click configuration and we are generating hash of commands from rich's command definition. As result of it, some of the breeze's svg files are changed. There are still subtle differences (mainly about font specification) on Linux and Maci and possibly it depends on what fonts are installed on your system - so when you regenerate images, font definition changes. This should be no issue in general as those images have the same hash and for all practical purposes, they are unchanged. (cherry picked from commit d8a06581d4e6dac582b42dce0504a60a5ca4b96c) --- .../src/airflow_breeze/commands/testing_commands_config.py | 1 + images/breeze/output_static-checks.svg | 4 ++-- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/dev/breeze/src/airflow_breeze/commands/testing_commands_config.py b/dev/breeze/src/airflow_breeze/commands/testing_commands_config.py index 4bf151951a56e..737fdb5c26c23 100644 --- a/dev/breeze/src/airflow_breeze/commands/testing_commands_config.py +++ b/dev/breeze/src/airflow_breeze/commands/testing_commands_config.py @@ -34,6 +34,7 @@ "--postgres-version", "--mysql-version", "--mssql-version", + "--integration", ], }, { diff --git a/images/breeze/output_static-checks.svg b/images/breeze/output_static-checks.svg index acb8c2e3195a7..0418a91abd205 100644 --- a/images/breeze/output_static-checks.svg +++ b/images/breeze/output_static-checks.svg @@ -212,9 +212,9 @@
    - + - + Usage: breeze static-checks [OPTIONS] [PRECOMMIT_ARGS]... From 3c885de2361f30a3d5925313a2922233fd0a3b5f Mon Sep 17 00:00:00 2001 From: Jarek Potiuk Date: Mon, 5 Dec 2022 20:06:49 +0100 Subject: [PATCH 144/158] Add automated version replacement in example dag indexes (#28090) Despite earlier attempts to fix the links there were still many places where the links were pointing to wrong versions of examples. It has been Noticed in: https://github.com/apache/airflow/pull/27774#issuecomment-1321166935 This change addresses it in three ways: * updated tool to fix the linx in historically released documentation (applied already in https://github.com/apache/airflow-site/pull/706) * replaced hard-coded version with `|version|` everywhere and added sphinx extension to convert those to actual provider versions (though due to sphinx limitation it has to be done with post-processing of the generated .html rather than replacing during generation) * added pre-commit that detects any use of example_dags links to main/master/hard-coded version. (cherry picked from commit c8e348dcb0bae27e98d68545b59388c9f91fc382) --- .pre-commit-config.yaml | 13 + STATIC_CODE_CHECKS.rst | 406 +++++++++--------- .../src/airflow_breeze/pre_commit_ids.py | 1 + dev/example_dags/update_example_dags_paths.py | 24 +- .../index.rst | 2 +- .../index.rst | 2 +- .../example-dags.rst | 4 +- .../operators/athena.rst | 2 +- .../index.rst | 2 +- .../index.rst | 2 +- .../index.rst | 2 +- .../index.rst | 2 +- .../index.rst | 2 +- .../index.rst | 2 +- .../index.rst | 2 +- .../index.rst | 2 +- .../index.rst | 2 +- .../index.rst | 2 +- docs/apache-airflow-providers-asana/index.rst | 2 +- .../index.rst | 2 +- .../index.rst | 2 +- .../index.rst | 2 +- .../index.rst | 2 +- .../index.rst | 2 +- .../apache-airflow-providers-docker/index.rst | 2 +- .../index.rst | 2 +- .../apache-airflow-providers-github/index.rst | 2 +- .../example-dags.rst | 14 +- .../operators/cloud/index.rst | 2 +- docs/apache-airflow-providers-http/index.rst | 2 +- .../index.rst | 2 +- docs/apache-airflow-providers-jdbc/index.rst | 2 +- .../index.rst | 2 +- .../index.rst | 2 +- .../index.rst | 2 +- .../index.rst | 2 +- docs/apache-airflow-providers-mysql/index.rst | 2 +- docs/apache-airflow-providers-neo4j/index.rst | 2 +- .../index.rst | 2 +- .../index.rst | 2 +- .../apache-airflow-providers-plexus/index.rst | 2 +- .../index.rst | 2 +- .../apache-airflow-providers-presto/index.rst | 2 +- .../apache-airflow-providers-qubole/index.rst | 2 +- .../operators/index.rst | 2 +- .../index.rst | 2 +- .../index.rst | 2 +- docs/apache-airflow-providers-slack/index.rst | 2 +- .../index.rst | 2 +- .../apache-airflow-providers-sqlite/index.rst | 2 +- .../index.rst | 2 +- .../index.rst | 2 +- .../index.rst | 2 +- docs/apache-airflow-providers-trino/index.rst | 2 +- .../apache-airflow-providers-yandex/index.rst | 2 +- .../operators.rst | 2 +- .../index.rst | 2 +- docs/conf.py | 10 +- ...extra_provider_files_with_substitutions.py | 46 ++ images/breeze/output-commands-hash.txt | 2 +- images/breeze/output-commands.svg | 204 ++++----- images/breeze/output_static-checks.svg | 48 +-- 62 files changed, 470 insertions(+), 404 deletions(-) create mode 100644 docs/exts/extra_provider_files_with_substitutions.py diff --git a/.pre-commit-config.yaml b/.pre-commit-config.yaml index 0405cbd86174f..ed082b1944e37 100644 --- a/.pre-commit-config.yaml +++ b/.pre-commit-config.yaml @@ -402,6 +402,19 @@ repos: entry: "pydevd.*settrace\\(" pass_filenames: true files: \.py$ + - id: check-links-to-example-dags-do-not-use-hardcoded-versions + name: Check that example dags do not use hard-coded version numbers + description: The links to example dags should use |version| as version specification + language: pygrep + entry: > + (?i) + .*https://github.*[0-9]/tests/system/providers| + .*https://github.*/main/tests/system/providers| + .*https://github.*/master/tests/system/providers| + .*https://github.*/main/airflow/providers/.*/example_dags/| + .*https://github.*/master/airflow/providers/.*/example_dags/ + pass_filenames: true + files: ^docs/apache-airflow-providers-.*\.rst - id: check-safe-filter-usage-in-html language: pygrep name: Don't use safe in templates diff --git a/STATIC_CODE_CHECKS.rst b/STATIC_CODE_CHECKS.rst index b2b6081b5f976..b27d170d1c01e 100644 --- a/STATIC_CODE_CHECKS.rst +++ b/STATIC_CODE_CHECKS.rst @@ -129,208 +129,210 @@ require Breeze Docker image to be build locally. .. BEGIN AUTO-GENERATED STATIC CHECK LIST -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| ID | Description | Image | -+========================================================+==================================================================+=========+ -| black | Run black (python formatter) | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| blacken-docs | Run black on python code blocks in documentation files | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| check-airflow-config-yaml-consistent | Checks for consistency between config.yml and default_config.cfg | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| check-airflow-provider-compatibility | Check compatibility of Providers with Airflow | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| check-apache-license-rat | Check if licenses are OK for Apache | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| check-base-operator-partial-arguments | Check BaseOperator and partial() arguments | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| check-base-operator-usage | * Check BaseOperator[Link] core imports | | -| | * Check BaseOperator[Link] other imports | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| check-boring-cyborg-configuration | Checks for Boring Cyborg configuration consistency | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| check-breeze-top-dependencies-limited | Breeze should have small number of top-level dependencies | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| check-builtin-literals | Require literal syntax when initializing Python builtin types | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| check-changelog-has-no-duplicates | Check changelogs for duplicate entries | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| check-core-deprecation-classes | Verify using of dedicated Airflow deprecation classes in core | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| check-daysago-import-from-utils | Make sure days_ago is imported from airflow.utils.dates | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| check-decorated-operator-implements-custom-name | Check @task decorator implements custom_operator_name | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| check-docstring-param-types | Check that docstrings do not specify param types | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| check-example-dags-urls | Check that example dags url include provider versions | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| check-executables-have-shebangs | Check that executables have shebang | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| check-extra-packages-references | Checks setup extra packages | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| check-extras-order | Check order of extras in Dockerfile | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| check-for-inclusive-language | Check for language that we do not accept as community | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| check-hooks-apply | Check if all hooks apply to the repository | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| check-incorrect-use-of-LoggingMixin | Make sure LoggingMixin is not used alone | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| check-init-decorator-arguments | Check model __init__ and decorator arguments are in sync | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| check-lazy-logging | Check that all logging methods are lazy | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| check-merge-conflict | Check that merge conflicts are not being committed | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| check-newsfragments-are-valid | Check newsfragments are valid | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| check-no-providers-in-core-examples | No providers imports in core example DAGs | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| check-no-relative-imports | No relative imports | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| check-persist-credentials-disabled-in-github-workflows | Check that workflow files have persist-credentials disabled | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| check-pre-commit-information-consistent | Update information re pre-commit hooks and verify ids and names | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| check-provide-create-sessions-imports | Check provide_session and create_session imports | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| check-provider-yaml-valid | Validate provider.yaml files | * | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| check-providers-init-file-missing | Provider init file is missing | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| check-providers-subpackages-init-file-exist | Provider subpackage init files are there | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| check-pydevd-left-in-code | Check for pydevd debug statements accidentally left | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| check-revision-heads-map | Check that the REVISION_HEADS_MAP is up-to-date | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| check-safe-filter-usage-in-html | Don't use safe in templates | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| check-setup-order | Check order of dependencies in setup.cfg and setup.py | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| check-start-date-not-used-in-defaults | 'start_date' not to be defined in default_args in example_dags | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| check-system-tests-present | Check if system tests have required segments of code | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| check-system-tests-tocs | Check that system tests is properly added | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| check-xml | Check XML files with xmllint | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| codespell | Run codespell to check for common misspellings in files | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| compile-www-assets | Compile www assets | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| compile-www-assets-dev | Compile www assets in dev mode | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| create-missing-init-py-files-tests | Create missing init.py files in tests | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| debug-statements | Detect accidentally committed debug statements | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| detect-private-key | Detect if private key is added to the repository | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| doctoc | Add TOC for md and rst files | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| end-of-file-fixer | Make sure that there is an empty line at the end | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| fix-encoding-pragma | Remove encoding header from python files | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| flynt | Run flynt string format converter for Python | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| identity | Print input to the static check hooks for troubleshooting | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| insert-license | * Add license for all SQL files | | -| | * Add license for all rst files | | -| | * Add license for all CSS/JS/PUML/TS/TSX files | | -| | * Add license for all JINJA template files | | -| | * Add license for all shell files | | -| | * Add license for all Python files | | -| | * Add license for all XML files | | -| | * Add license for all YAML files | | -| | * Add license for all md files | | -| | * Add license for all other files | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| isort | Run isort to sort imports in Python files | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| lint-chart-schema | Lint chart/values.schema.json file | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| lint-css | stylelint | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| lint-dockerfile | Lint dockerfile | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| lint-helm-chart | Lint Helm Chart | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| lint-json-schema | * Lint JSON Schema files with JSON Schema | | -| | * Lint NodePort Service with JSON Schema | | -| | * Lint Docker compose files with JSON Schema | | -| | * Lint chart/values.schema.json file with JSON Schema | | -| | * Lint chart/values.yaml file with JSON Schema | | -| | * Lint airflow/config_templates/config.yml file with JSON Schema | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| lint-markdown | Run markdownlint | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| lint-openapi | * Lint OpenAPI using spectral | | -| | * Lint OpenAPI using openapi-spec-validator | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| mixed-line-ending | Detect if mixed line ending is used (\r vs. \r\n) | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| pretty-format-json | Format json files | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| pydocstyle | Run pydocstyle | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| python-no-log-warn | Check if there are no deprecate log warn | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| pyupgrade | Upgrade Python code automatically | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| replace-bad-characters | Replace bad characters | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| rst-backticks | Check if RST files use double backticks for code | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| run-flake8 | Run flake8 | * | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| run-mypy | * Run mypy for dev | * | -| | * Run mypy for core | | -| | * Run mypy for providers | | -| | * Run mypy for /docs/ folder | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| run-shellcheck | Check Shell scripts syntax correctness | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| static-check-autoflake | Remove all unused code | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| trailing-whitespace | Remove trailing whitespace at end of line | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| ts-compile-and-lint-javascript | TS types generation and ESLint against current UI files | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| update-breeze-cmd-output | Update output of breeze commands in BREEZE.rst | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| update-breeze-readme-config-hash | Update Breeze README.md with config files hash | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| update-er-diagram | Update ER diagram | * | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| update-extras | Update extras in documentation | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| update-in-the-wild-to-be-sorted | Sort INTHEWILD.md alphabetically | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| update-inlined-dockerfile-scripts | Inline Dockerfile and Dockerfile.ci scripts | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| update-local-yml-file | Update mounts in the local yml file | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| update-migration-references | Update migration ref doc | * | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| update-providers-dependencies | Update cross-dependencies for providers packages | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| update-spelling-wordlist-to-be-sorted | Sort alphabetically and uniquify spelling_wordlist.txt | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| update-supported-versions | Updates supported versions in documentation | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| update-vendored-in-k8s-json-schema | Vendor k8s definitions into values.schema.json | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| update-version | Update version to the latest version in the documentation | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| yamllint | Check YAML files with yamllint | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ -| yesqa | Remove unnecessary noqa statements | | -+--------------------------------------------------------+------------------------------------------------------------------+---------+ ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| ID | Description | Image | ++===========================================================+==================================================================+=========+ +| black | Run black (python formatter) | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| blacken-docs | Run black on python code blocks in documentation files | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| check-airflow-config-yaml-consistent | Checks for consistency between config.yml and default_config.cfg | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| check-airflow-provider-compatibility | Check compatibility of Providers with Airflow | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| check-apache-license-rat | Check if licenses are OK for Apache | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| check-base-operator-partial-arguments | Check BaseOperator and partial() arguments | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| check-base-operator-usage | * Check BaseOperator[Link] core imports | | +| | * Check BaseOperator[Link] other imports | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| check-boring-cyborg-configuration | Checks for Boring Cyborg configuration consistency | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| check-breeze-top-dependencies-limited | Breeze should have small number of top-level dependencies | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| check-builtin-literals | Require literal syntax when initializing Python builtin types | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| check-changelog-has-no-duplicates | Check changelogs for duplicate entries | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| check-core-deprecation-classes | Verify using of dedicated Airflow deprecation classes in core | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| check-daysago-import-from-utils | Make sure days_ago is imported from airflow.utils.dates | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| check-decorated-operator-implements-custom-name | Check @task decorator implements custom_operator_name | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| check-docstring-param-types | Check that docstrings do not specify param types | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| check-example-dags-urls | Check that example dags url include provider versions | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| check-executables-have-shebangs | Check that executables have shebang | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| check-extra-packages-references | Checks setup extra packages | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| check-extras-order | Check order of extras in Dockerfile | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| check-for-inclusive-language | Check for language that we do not accept as community | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| check-hooks-apply | Check if all hooks apply to the repository | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| check-incorrect-use-of-LoggingMixin | Make sure LoggingMixin is not used alone | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| check-init-decorator-arguments | Check model __init__ and decorator arguments are in sync | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| check-lazy-logging | Check that all logging methods are lazy | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| check-links-to-example-dags-do-not-use-hardcoded-versions | Check that example dags do not use hard-coded version numbers | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| check-merge-conflict | Check that merge conflicts are not being committed | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| check-newsfragments-are-valid | Check newsfragments are valid | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| check-no-providers-in-core-examples | No providers imports in core example DAGs | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| check-no-relative-imports | No relative imports | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| check-persist-credentials-disabled-in-github-workflows | Check that workflow files have persist-credentials disabled | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| check-pre-commit-information-consistent | Update information re pre-commit hooks and verify ids and names | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| check-provide-create-sessions-imports | Check provide_session and create_session imports | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| check-provider-yaml-valid | Validate provider.yaml files | * | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| check-providers-init-file-missing | Provider init file is missing | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| check-providers-subpackages-init-file-exist | Provider subpackage init files are there | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| check-pydevd-left-in-code | Check for pydevd debug statements accidentally left | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| check-revision-heads-map | Check that the REVISION_HEADS_MAP is up-to-date | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| check-safe-filter-usage-in-html | Don't use safe in templates | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| check-setup-order | Check order of dependencies in setup.cfg and setup.py | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| check-start-date-not-used-in-defaults | 'start_date' not to be defined in default_args in example_dags | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| check-system-tests-present | Check if system tests have required segments of code | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| check-system-tests-tocs | Check that system tests is properly added | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| check-xml | Check XML files with xmllint | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| codespell | Run codespell to check for common misspellings in files | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| compile-www-assets | Compile www assets | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| compile-www-assets-dev | Compile www assets in dev mode | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| create-missing-init-py-files-tests | Create missing init.py files in tests | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| debug-statements | Detect accidentally committed debug statements | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| detect-private-key | Detect if private key is added to the repository | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| doctoc | Add TOC for md and rst files | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| end-of-file-fixer | Make sure that there is an empty line at the end | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| fix-encoding-pragma | Remove encoding header from python files | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| flynt | Run flynt string format converter for Python | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| identity | Print input to the static check hooks for troubleshooting | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| insert-license | * Add license for all SQL files | | +| | * Add license for all rst files | | +| | * Add license for all CSS/JS/PUML/TS/TSX files | | +| | * Add license for all JINJA template files | | +| | * Add license for all shell files | | +| | * Add license for all Python files | | +| | * Add license for all XML files | | +| | * Add license for all YAML files | | +| | * Add license for all md files | | +| | * Add license for all other files | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| isort | Run isort to sort imports in Python files | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| lint-chart-schema | Lint chart/values.schema.json file | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| lint-css | stylelint | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| lint-dockerfile | Lint dockerfile | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| lint-helm-chart | Lint Helm Chart | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| lint-json-schema | * Lint JSON Schema files with JSON Schema | | +| | * Lint NodePort Service with JSON Schema | | +| | * Lint Docker compose files with JSON Schema | | +| | * Lint chart/values.schema.json file with JSON Schema | | +| | * Lint chart/values.yaml file with JSON Schema | | +| | * Lint airflow/config_templates/config.yml file with JSON Schema | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| lint-markdown | Run markdownlint | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| lint-openapi | * Lint OpenAPI using spectral | | +| | * Lint OpenAPI using openapi-spec-validator | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| mixed-line-ending | Detect if mixed line ending is used (\r vs. \r\n) | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| pretty-format-json | Format json files | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| pydocstyle | Run pydocstyle | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| python-no-log-warn | Check if there are no deprecate log warn | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| pyupgrade | Upgrade Python code automatically | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| replace-bad-characters | Replace bad characters | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| rst-backticks | Check if RST files use double backticks for code | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| run-flake8 | Run flake8 | * | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| run-mypy | * Run mypy for dev | * | +| | * Run mypy for core | | +| | * Run mypy for providers | | +| | * Run mypy for /docs/ folder | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| run-shellcheck | Check Shell scripts syntax correctness | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| static-check-autoflake | Remove all unused code | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| trailing-whitespace | Remove trailing whitespace at end of line | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| ts-compile-and-lint-javascript | TS types generation and ESLint against current UI files | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| update-breeze-cmd-output | Update output of breeze commands in BREEZE.rst | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| update-breeze-readme-config-hash | Update Breeze README.md with config files hash | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| update-er-diagram | Update ER diagram | * | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| update-extras | Update extras in documentation | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| update-in-the-wild-to-be-sorted | Sort INTHEWILD.md alphabetically | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| update-inlined-dockerfile-scripts | Inline Dockerfile and Dockerfile.ci scripts | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| update-local-yml-file | Update mounts in the local yml file | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| update-migration-references | Update migration ref doc | * | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| update-providers-dependencies | Update cross-dependencies for providers packages | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| update-spelling-wordlist-to-be-sorted | Sort alphabetically and uniquify spelling_wordlist.txt | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| update-supported-versions | Updates supported versions in documentation | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| update-vendored-in-k8s-json-schema | Vendor k8s definitions into values.schema.json | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| update-version | Update version to the latest version in the documentation | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| yamllint | Check YAML files with yamllint | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ +| yesqa | Remove unnecessary noqa statements | | ++-----------------------------------------------------------+------------------------------------------------------------------+---------+ .. END AUTO-GENERATED STATIC CHECK LIST diff --git a/dev/breeze/src/airflow_breeze/pre_commit_ids.py b/dev/breeze/src/airflow_breeze/pre_commit_ids.py index b64d92b5c4cf7..4914cb09148cd 100644 --- a/dev/breeze/src/airflow_breeze/pre_commit_ids.py +++ b/dev/breeze/src/airflow_breeze/pre_commit_ids.py @@ -47,6 +47,7 @@ "check-incorrect-use-of-LoggingMixin", "check-init-decorator-arguments", "check-lazy-logging", + "check-links-to-example-dags-do-not-use-hardcoded-versions", "check-merge-conflict", "check-newsfragments-are-valid", "check-no-providers-in-core-examples", diff --git a/dev/example_dags/update_example_dags_paths.py b/dev/example_dags/update_example_dags_paths.py index 08251cd213d8d..36eea37158053 100755 --- a/dev/example_dags/update_example_dags_paths.py +++ b/dev/example_dags/update_example_dags_paths.py @@ -40,14 +40,14 @@ EXAMPLE_DAGS_URL_MATCHER = re.compile( r"^(.*)(https://github.com/apache/airflow/tree/(.*)/airflow/providers/(.*)/example_dags)(/?\".*)$" ) - SYSTEM_TESTS_URL_MATCHER = re.compile( r"^(.*)(https://github.com/apache/airflow/tree/(.*)/tests/system/providers/(.*))(/?\".*)$" ) def check_if_url_exists(url: str) -> bool: # type: ignore[return] - response = requests.head(url) + return True # uncomment to check URLs + response = requests.head(url, allow_redirects=True) if response.status_code == 200: return True if response.status_code == 404: @@ -57,12 +57,12 @@ def check_if_url_exists(url: str) -> bool: # type: ignore[return] def replace_match(file: str, line: str, provider: str, version: str) -> str | None: - for matcher in [EXAMPLE_DAGS_URL_MATCHER, SYSTEM_TESTS_URL_MATCHER]: + for index, matcher in enumerate([EXAMPLE_DAGS_URL_MATCHER, SYSTEM_TESTS_URL_MATCHER]): match = matcher.match(line) if match: url_path_to_dir = match.group(4) branch = match.group(3) - if branch.startswith("providers-"): + if branch.startswith("providers-") and branch.endswith(f"/{version}"): console.print(f"[green]Already corrected[/]: {provider}:{version}") continue system_tests_url = ( @@ -73,17 +73,17 @@ def replace_match(file: str, line: str, provider: str, version: str) -> str | No f"https://github.com/apache/airflow/tree/providers-{provider}/{version}" f"/airflow/providers/{url_path_to_dir}/example_dags" ) - if check_if_url_exists(system_tests_url): + if check_if_url_exists(system_tests_url) and index == 1: new_line = re.sub(matcher, r"\1" + system_tests_url + r"\5", line) - elif check_if_url_exists(example_dags_url): + elif check_if_url_exists(example_dags_url) and index == 0: new_line = re.sub(matcher, r"\1" + example_dags_url + r"\5", line) else: console.print( f"[yellow] Neither example dags nor system tests folder" - f" exists for {provider}:{version} -> removing:[/]" + f" exists for {provider}:{version} -> skipping:[/]" ) console.print(line) - return None + return line if line != new_line: console.print(f"[yellow] Replacing in {file}[/]\n{line.strip()}\n{new_line.strip()}") return new_line @@ -101,8 +101,8 @@ def find_matches(_file: Path, provider: str, version: str): if __name__ == "__main__": - curdir = Path(os.curdir).resolve() - dirs = list(filter(os.path.isdir, curdir.iterdir())) + curdir: Path = Path(os.curdir).resolve() + dirs: list[Path] = list(filter(os.path.isdir, curdir.iterdir())) with Progress(console=console) as progress: task = progress.add_task(f"Updating {len(dirs)}", total=len(dirs)) for directory in dirs: @@ -113,8 +113,8 @@ def find_matches(_file: Path, provider: str, version: str): for version_dir in version_dirs: version = version_dir.name console.print(version) - for file_name in ["index.html", "example-dags.html"]: - candidate_file = version_dir / file_name + for file in version_dir.rglob("*.html"): + candidate_file = file if candidate_file.exists(): find_matches(candidate_file, provider, version) progress.advance(task) diff --git a/docs/apache-airflow-providers-airbyte/index.rst b/docs/apache-airflow-providers-airbyte/index.rst index 2dce30a82e173..314ec9b9c2868 100644 --- a/docs/apache-airflow-providers-airbyte/index.rst +++ b/docs/apache-airflow-providers-airbyte/index.rst @@ -44,7 +44,7 @@ Content :maxdepth: 1 :caption: Resources - Example DAGs + Example DAGs PyPI Repository Installing from sources diff --git a/docs/apache-airflow-providers-alibaba/index.rst b/docs/apache-airflow-providers-alibaba/index.rst index 5439aadc2e4a1..de3eff524358c 100644 --- a/docs/apache-airflow-providers-alibaba/index.rst +++ b/docs/apache-airflow-providers-alibaba/index.rst @@ -45,7 +45,7 @@ Content :maxdepth: 1 :caption: Resources - Example DAGs + Example DAGs PyPI Repository Installing from sources diff --git a/docs/apache-airflow-providers-amazon/example-dags.rst b/docs/apache-airflow-providers-amazon/example-dags.rst index 253ae79041437..8881ac0174f02 100644 --- a/docs/apache-airflow-providers-amazon/example-dags.rst +++ b/docs/apache-airflow-providers-amazon/example-dags.rst @@ -20,5 +20,5 @@ Example DAGs You can learn how to use Amazon AWS integrations by analyzing the source code of the example DAGs: -* `Amazon AWS `__ -* `Amazon AWS (legacy) `__ +* `Amazon AWS `__ +* `Amazon AWS (legacy) `__ diff --git a/docs/apache-airflow-providers-amazon/operators/athena.rst b/docs/apache-airflow-providers-amazon/operators/athena.rst index 3b1fd3b6e83c2..4cb7fcb1faa99 100644 --- a/docs/apache-airflow-providers-amazon/operators/athena.rst +++ b/docs/apache-airflow-providers-amazon/operators/athena.rst @@ -43,7 +43,7 @@ to run a query in Amazon Athena. In the following example, we query an existing Athena table and send the results to an existing Amazon S3 bucket. For more examples of how to use this operator, please -see the `Sample DAG `__. +see the `Sample DAG `__. .. exampleinclude:: /../../tests/system/providers/amazon/aws/example_athena.py :language: python diff --git a/docs/apache-airflow-providers-apache-beam/index.rst b/docs/apache-airflow-providers-apache-beam/index.rst index 5c749f3b0dc76..55f40e3be5339 100644 --- a/docs/apache-airflow-providers-apache-beam/index.rst +++ b/docs/apache-airflow-providers-apache-beam/index.rst @@ -38,7 +38,7 @@ Content :caption: Resources PyPI Repository - Example DAGs + Example DAGs .. toctree:: :maxdepth: 1 diff --git a/docs/apache-airflow-providers-apache-cassandra/index.rst b/docs/apache-airflow-providers-apache-cassandra/index.rst index 6b3f0a5c34f8f..2efd4abaf05c6 100644 --- a/docs/apache-airflow-providers-apache-cassandra/index.rst +++ b/docs/apache-airflow-providers-apache-cassandra/index.rst @@ -44,7 +44,7 @@ Content :maxdepth: 1 :caption: Resources - Example DAGs + Example DAGs PyPI Repository Installing from sources diff --git a/docs/apache-airflow-providers-apache-drill/index.rst b/docs/apache-airflow-providers-apache-drill/index.rst index ba35dcaaa7986..a1a2a7358cc4d 100644 --- a/docs/apache-airflow-providers-apache-drill/index.rst +++ b/docs/apache-airflow-providers-apache-drill/index.rst @@ -44,7 +44,7 @@ Content :maxdepth: 1 :caption: Resources - Example DAGs + Example DAGs PyPI Repository Installing from sources diff --git a/docs/apache-airflow-providers-apache-druid/index.rst b/docs/apache-airflow-providers-apache-druid/index.rst index f5ab4acb65d68..17a85fe8638a1 100644 --- a/docs/apache-airflow-providers-apache-druid/index.rst +++ b/docs/apache-airflow-providers-apache-druid/index.rst @@ -46,7 +46,7 @@ Content PyPI Repository Installing from sources - Example DAGs + Example DAGs .. THE REMAINDER OF THE FILE IS AUTOMATICALLY GENERATED. IT WILL BE OVERWRITTEN AT RELEASE TIME! diff --git a/docs/apache-airflow-providers-apache-hive/index.rst b/docs/apache-airflow-providers-apache-hive/index.rst index a687327cecc58..07656a9bd6d1d 100644 --- a/docs/apache-airflow-providers-apache-hive/index.rst +++ b/docs/apache-airflow-providers-apache-hive/index.rst @@ -45,7 +45,7 @@ Content :maxdepth: 1 :caption: Resources - Example DAGs + Example DAGs PyPI Repository Installing from sources diff --git a/docs/apache-airflow-providers-apache-kylin/index.rst b/docs/apache-airflow-providers-apache-kylin/index.rst index 46c3675dd6c7b..ecdcb30832157 100644 --- a/docs/apache-airflow-providers-apache-kylin/index.rst +++ b/docs/apache-airflow-providers-apache-kylin/index.rst @@ -38,7 +38,7 @@ Content :maxdepth: 1 :caption: Resources - Example DAGs + Example DAGs PyPI Repository Installing from sources diff --git a/docs/apache-airflow-providers-apache-livy/index.rst b/docs/apache-airflow-providers-apache-livy/index.rst index c769bf91ba3fe..31699da682427 100644 --- a/docs/apache-airflow-providers-apache-livy/index.rst +++ b/docs/apache-airflow-providers-apache-livy/index.rst @@ -43,7 +43,7 @@ Content :maxdepth: 1 :caption: Resources - Example DAGs + Example DAGs PyPI Repository Installing from sources diff --git a/docs/apache-airflow-providers-apache-pig/index.rst b/docs/apache-airflow-providers-apache-pig/index.rst index 0f00fdd90d625..65a9978eef70a 100644 --- a/docs/apache-airflow-providers-apache-pig/index.rst +++ b/docs/apache-airflow-providers-apache-pig/index.rst @@ -43,7 +43,7 @@ Content :maxdepth: 1 :caption: Resources - Example DAGs + Example DAGs PyPI Repository Installing from sources diff --git a/docs/apache-airflow-providers-apache-spark/index.rst b/docs/apache-airflow-providers-apache-spark/index.rst index af7bc319a4edf..18fe4e16ef6e5 100644 --- a/docs/apache-airflow-providers-apache-spark/index.rst +++ b/docs/apache-airflow-providers-apache-spark/index.rst @@ -44,7 +44,7 @@ Content :maxdepth: 1 :caption: Resources - Example DAGs + Example DAGs PyPI Repository Installing from sources diff --git a/docs/apache-airflow-providers-arangodb/index.rst b/docs/apache-airflow-providers-arangodb/index.rst index 5b5d029546874..9d5c0833661c3 100644 --- a/docs/apache-airflow-providers-arangodb/index.rst +++ b/docs/apache-airflow-providers-arangodb/index.rst @@ -39,7 +39,7 @@ Content :maxdepth: 1 :caption: Resources - Example DAGs + Example DAGs .. toctree:: :maxdepth: 1 diff --git a/docs/apache-airflow-providers-asana/index.rst b/docs/apache-airflow-providers-asana/index.rst index f187013eb8d41..f1613ff51e52c 100644 --- a/docs/apache-airflow-providers-asana/index.rst +++ b/docs/apache-airflow-providers-asana/index.rst @@ -45,7 +45,7 @@ Content :maxdepth: 1 :caption: Resources - Example DAGs + Example DAGs PyPI Repository Installing from sources diff --git a/docs/apache-airflow-providers-cncf-kubernetes/index.rst b/docs/apache-airflow-providers-cncf-kubernetes/index.rst index 176b1608d9ad1..c87d995d078c3 100644 --- a/docs/apache-airflow-providers-cncf-kubernetes/index.rst +++ b/docs/apache-airflow-providers-cncf-kubernetes/index.rst @@ -45,7 +45,7 @@ Content :maxdepth: 1 :caption: Resources - Example DAGs + Example DAGs PyPI Repository Installing from sources diff --git a/docs/apache-airflow-providers-common-sql/index.rst b/docs/apache-airflow-providers-common-sql/index.rst index 1e6847b55b5f0..52c449e891a06 100644 --- a/docs/apache-airflow-providers-common-sql/index.rst +++ b/docs/apache-airflow-providers-common-sql/index.rst @@ -44,7 +44,7 @@ Content :maxdepth: 1 :caption: Resources - Example DAGs + Example DAGs PyPI Repository Installing from sources diff --git a/docs/apache-airflow-providers-databricks/index.rst b/docs/apache-airflow-providers-databricks/index.rst index d0df5494bd50f..98d99c8588662 100644 --- a/docs/apache-airflow-providers-databricks/index.rst +++ b/docs/apache-airflow-providers-databricks/index.rst @@ -45,7 +45,7 @@ Content :maxdepth: 1 :caption: Resources - Example DAGs + Example DAGs PyPI Repository Installing from sources diff --git a/docs/apache-airflow-providers-dbt-cloud/index.rst b/docs/apache-airflow-providers-dbt-cloud/index.rst index 0df50f9432656..28f05b42cb5f1 100644 --- a/docs/apache-airflow-providers-dbt-cloud/index.rst +++ b/docs/apache-airflow-providers-dbt-cloud/index.rst @@ -49,7 +49,7 @@ Content :maxdepth: 1 :caption: Resources - Example DAGs + Example DAGs PyPI Repository Installing from sources diff --git a/docs/apache-airflow-providers-dingding/index.rst b/docs/apache-airflow-providers-dingding/index.rst index c001f9d6d4e1d..0b4be2a481a1b 100644 --- a/docs/apache-airflow-providers-dingding/index.rst +++ b/docs/apache-airflow-providers-dingding/index.rst @@ -44,7 +44,7 @@ Content :maxdepth: 1 :caption: Resources - Example DAGs + Example DAGs PyPI Repository Installing from sources diff --git a/docs/apache-airflow-providers-docker/index.rst b/docs/apache-airflow-providers-docker/index.rst index 0629d753df480..c768e0a2a8e43 100644 --- a/docs/apache-airflow-providers-docker/index.rst +++ b/docs/apache-airflow-providers-docker/index.rst @@ -39,7 +39,7 @@ Content :maxdepth: 1 :caption: Resources - Example DAGs + Example DAGs PyPI Repository Installing from sources diff --git a/docs/apache-airflow-providers-elasticsearch/index.rst b/docs/apache-airflow-providers-elasticsearch/index.rst index 2862e00c3cf8d..0c7475a428709 100644 --- a/docs/apache-airflow-providers-elasticsearch/index.rst +++ b/docs/apache-airflow-providers-elasticsearch/index.rst @@ -46,7 +46,7 @@ Content :maxdepth: 1 :caption: Resources - Example DAGs + Example DAGs PyPI Repository Installing from sources diff --git a/docs/apache-airflow-providers-github/index.rst b/docs/apache-airflow-providers-github/index.rst index 61e2054c68ec2..44335d69aec35 100644 --- a/docs/apache-airflow-providers-github/index.rst +++ b/docs/apache-airflow-providers-github/index.rst @@ -45,7 +45,7 @@ Content :maxdepth: 1 :caption: Resources - Example DAGs + Example DAGs PyPI Repository Installing from sources diff --git a/docs/apache-airflow-providers-google/example-dags.rst b/docs/apache-airflow-providers-google/example-dags.rst index b912ef3ea4fe6..49f4fbf327f12 100644 --- a/docs/apache-airflow-providers-google/example-dags.rst +++ b/docs/apache-airflow-providers-google/example-dags.rst @@ -19,10 +19,10 @@ Example DAGs ============ You can learn how to use Google integrations by analyzing the source code of the example DAGs: -* `Google Ads `__ -* `Google Cloud (legacy) `__ -* `Google Cloud `__ -* `Google Firebase `__ -* `Google Marketing Platform `__ -* `Google Workplace `__ (formerly Google Suite) -* `Google LevelDB `__ +* `Google Ads `__ +* `Google Cloud (legacy) `__ +* `Google Cloud `__ +* `Google Firebase `__ +* `Google Marketing Platform `__ +* `Google Workplace `__ (formerly Google Suite) +* `Google LevelDB `__ diff --git a/docs/apache-airflow-providers-google/operators/cloud/index.rst b/docs/apache-airflow-providers-google/operators/cloud/index.rst index 22ded79012385..f974c0eb81a4f 100644 --- a/docs/apache-airflow-providers-google/operators/cloud/index.rst +++ b/docs/apache-airflow-providers-google/operators/cloud/index.rst @@ -29,4 +29,4 @@ Google Cloud Operators .. note:: You can learn how to use Google Cloud integrations by analyzing the - `source code `_ of the particular example DAGs. + `source code `_ of the particular example DAGs. diff --git a/docs/apache-airflow-providers-http/index.rst b/docs/apache-airflow-providers-http/index.rst index 60025a889c901..780a2e406161f 100644 --- a/docs/apache-airflow-providers-http/index.rst +++ b/docs/apache-airflow-providers-http/index.rst @@ -45,7 +45,7 @@ Content :maxdepth: 1 :caption: Resources - Example DAGs + Example DAGs PyPI Repository Installing from sources diff --git a/docs/apache-airflow-providers-influxdb/index.rst b/docs/apache-airflow-providers-influxdb/index.rst index 1a3300c06ae1d..2b1b9962b6faf 100644 --- a/docs/apache-airflow-providers-influxdb/index.rst +++ b/docs/apache-airflow-providers-influxdb/index.rst @@ -45,7 +45,7 @@ Content :maxdepth: 1 :caption: Resources - Example DAGs + Example DAGs PyPI Repository Installing from sources diff --git a/docs/apache-airflow-providers-jdbc/index.rst b/docs/apache-airflow-providers-jdbc/index.rst index 31fc9c4457fd4..653cfdd2795e4 100644 --- a/docs/apache-airflow-providers-jdbc/index.rst +++ b/docs/apache-airflow-providers-jdbc/index.rst @@ -45,7 +45,7 @@ Content :maxdepth: 1 :caption: Resources - Example DAGs + Example DAGs PyPI Repository Installing from sources diff --git a/docs/apache-airflow-providers-jenkins/index.rst b/docs/apache-airflow-providers-jenkins/index.rst index 5c1626ab657c1..b8afb8eb4f8eb 100644 --- a/docs/apache-airflow-providers-jenkins/index.rst +++ b/docs/apache-airflow-providers-jenkins/index.rst @@ -43,7 +43,7 @@ Content :maxdepth: 1 :caption: Resources - Example DAGs + Example DAGs PyPI Repository Installing from sources diff --git a/docs/apache-airflow-providers-microsoft-azure/index.rst b/docs/apache-airflow-providers-microsoft-azure/index.rst index dda9160a313b8..9405f476aa0f7 100644 --- a/docs/apache-airflow-providers-microsoft-azure/index.rst +++ b/docs/apache-airflow-providers-microsoft-azure/index.rst @@ -47,7 +47,7 @@ Content :maxdepth: 1 :caption: Resources - Example DAGs + Example DAGs PyPI Repository Installing from sources diff --git a/docs/apache-airflow-providers-microsoft-mssql/index.rst b/docs/apache-airflow-providers-microsoft-mssql/index.rst index 7ebfe909273db..14f586e27f659 100644 --- a/docs/apache-airflow-providers-microsoft-mssql/index.rst +++ b/docs/apache-airflow-providers-microsoft-mssql/index.rst @@ -45,7 +45,7 @@ Content :maxdepth: 1 :caption: Resources - Example DAGs + Example DAGs PyPI Repository Installing from sources diff --git a/docs/apache-airflow-providers-microsoft-winrm/index.rst b/docs/apache-airflow-providers-microsoft-winrm/index.rst index 042cd90b32781..ac244b5120a9c 100644 --- a/docs/apache-airflow-providers-microsoft-winrm/index.rst +++ b/docs/apache-airflow-providers-microsoft-winrm/index.rst @@ -44,7 +44,7 @@ Content :maxdepth: 1 :caption: Resources - Example DAGs + Example DAGs PyPI Repository Installing from sources diff --git a/docs/apache-airflow-providers-mysql/index.rst b/docs/apache-airflow-providers-mysql/index.rst index 0d58b78d54a60..5e1e7353e80a2 100644 --- a/docs/apache-airflow-providers-mysql/index.rst +++ b/docs/apache-airflow-providers-mysql/index.rst @@ -45,7 +45,7 @@ Content :maxdepth: 1 :caption: Resources - Example DAGs + Example DAGs PyPI Repository Installing from sources diff --git a/docs/apache-airflow-providers-neo4j/index.rst b/docs/apache-airflow-providers-neo4j/index.rst index c3d032575d608..f03cdae75c28d 100644 --- a/docs/apache-airflow-providers-neo4j/index.rst +++ b/docs/apache-airflow-providers-neo4j/index.rst @@ -45,7 +45,7 @@ Content :maxdepth: 1 :caption: Resources - Example DAGs + Example DAGs PyPI Repository Installing from sources diff --git a/docs/apache-airflow-providers-opsgenie/index.rst b/docs/apache-airflow-providers-opsgenie/index.rst index 9109513f16e6b..3dc179c0a1632 100644 --- a/docs/apache-airflow-providers-opsgenie/index.rst +++ b/docs/apache-airflow-providers-opsgenie/index.rst @@ -44,7 +44,7 @@ Content :maxdepth: 1 :caption: Resources - Example DAGs + Example DAGs PyPI Repository Installing from sources diff --git a/docs/apache-airflow-providers-papermill/index.rst b/docs/apache-airflow-providers-papermill/index.rst index 9b66d74602259..7a7f39daec878 100644 --- a/docs/apache-airflow-providers-papermill/index.rst +++ b/docs/apache-airflow-providers-papermill/index.rst @@ -44,7 +44,7 @@ Content :maxdepth: 1 :caption: Resources - Example DAGs + Example DAGs PyPI Repository Installing from sources diff --git a/docs/apache-airflow-providers-plexus/index.rst b/docs/apache-airflow-providers-plexus/index.rst index c738c121dceca..3726213c9c7b9 100644 --- a/docs/apache-airflow-providers-plexus/index.rst +++ b/docs/apache-airflow-providers-plexus/index.rst @@ -38,7 +38,7 @@ Content :maxdepth: 1 :caption: Resources - Example DAGs + Example DAGs PyPI Repository Installing from sources diff --git a/docs/apache-airflow-providers-postgres/index.rst b/docs/apache-airflow-providers-postgres/index.rst index 74cc26baf911e..2cb25ecf32a28 100644 --- a/docs/apache-airflow-providers-postgres/index.rst +++ b/docs/apache-airflow-providers-postgres/index.rst @@ -45,7 +45,7 @@ Content :maxdepth: 1 :caption: Resources - Example DAGs + Example DAGs PyPI Repository Installing from sources diff --git a/docs/apache-airflow-providers-presto/index.rst b/docs/apache-airflow-providers-presto/index.rst index 31d502077209a..b86681f23414a 100644 --- a/docs/apache-airflow-providers-presto/index.rst +++ b/docs/apache-airflow-providers-presto/index.rst @@ -44,7 +44,7 @@ Content :maxdepth: 1 :caption: Resources - Example DAGs + Example DAGs PyPI Repository Installing from sources diff --git a/docs/apache-airflow-providers-qubole/index.rst b/docs/apache-airflow-providers-qubole/index.rst index 83c39d506b9a1..8b8c838527536 100644 --- a/docs/apache-airflow-providers-qubole/index.rst +++ b/docs/apache-airflow-providers-qubole/index.rst @@ -44,7 +44,7 @@ Content :maxdepth: 1 :caption: Resources - Example DAGs + Example DAGs PyPI Repository Installing from sources diff --git a/docs/apache-airflow-providers-qubole/operators/index.rst b/docs/apache-airflow-providers-qubole/operators/index.rst index deefeb75392e4..1dbccd05442d5 100644 --- a/docs/apache-airflow-providers-qubole/operators/index.rst +++ b/docs/apache-airflow-providers-qubole/operators/index.rst @@ -29,4 +29,4 @@ Qubole Operators .. note:: You can learn how to use Google Cloud integrations by analyzing the - `source code `_ of the particular example DAGs. + `source code `_ of the particular example DAGs. diff --git a/docs/apache-airflow-providers-salesforce/index.rst b/docs/apache-airflow-providers-salesforce/index.rst index 0d1c99c4a56f8..ec317f3804535 100644 --- a/docs/apache-airflow-providers-salesforce/index.rst +++ b/docs/apache-airflow-providers-salesforce/index.rst @@ -45,7 +45,7 @@ Content :maxdepth: 1 :caption: Resources - Example DAGs + Example DAGs PyPI Repository Installing from sources diff --git a/docs/apache-airflow-providers-singularity/index.rst b/docs/apache-airflow-providers-singularity/index.rst index 76d61aabacad2..0595c5bccda96 100644 --- a/docs/apache-airflow-providers-singularity/index.rst +++ b/docs/apache-airflow-providers-singularity/index.rst @@ -38,7 +38,7 @@ Content :maxdepth: 1 :caption: Resources - Example DAGs + Example DAGs PyPI Repository Installing from sources diff --git a/docs/apache-airflow-providers-slack/index.rst b/docs/apache-airflow-providers-slack/index.rst index cf71f3b8be28b..8e76bc5dc5279 100644 --- a/docs/apache-airflow-providers-slack/index.rst +++ b/docs/apache-airflow-providers-slack/index.rst @@ -44,7 +44,7 @@ Content Connection Types Python API <_api/airflow/providers/slack/index> - Example DAGs + Example DAGs .. toctree:: :maxdepth: 1 diff --git a/docs/apache-airflow-providers-snowflake/index.rst b/docs/apache-airflow-providers-snowflake/index.rst index c0f40df55255a..2ed7610ab4148 100644 --- a/docs/apache-airflow-providers-snowflake/index.rst +++ b/docs/apache-airflow-providers-snowflake/index.rst @@ -45,7 +45,7 @@ Content :maxdepth: 1 :caption: Resources - Example DAGs + Example DAGs PyPI Repository Installing from sources diff --git a/docs/apache-airflow-providers-sqlite/index.rst b/docs/apache-airflow-providers-sqlite/index.rst index 8e4298e28b4ac..2dfe002cd10f0 100644 --- a/docs/apache-airflow-providers-sqlite/index.rst +++ b/docs/apache-airflow-providers-sqlite/index.rst @@ -45,7 +45,7 @@ Content :maxdepth: 1 :caption: Resources - Example DAGs + Example DAGs PyPI Repository Installing from sources diff --git a/docs/apache-airflow-providers-tableau/index.rst b/docs/apache-airflow-providers-tableau/index.rst index bacdbb1fa3497..ac99aba77348c 100644 --- a/docs/apache-airflow-providers-tableau/index.rst +++ b/docs/apache-airflow-providers-tableau/index.rst @@ -40,7 +40,7 @@ Content :maxdepth: 1 :caption: Resources - Example DAGs + Example DAGs PyPI Repository Installing from sources diff --git a/docs/apache-airflow-providers-tabular/index.rst b/docs/apache-airflow-providers-tabular/index.rst index be1193f3d6a30..72704e9c32d93 100644 --- a/docs/apache-airflow-providers-tabular/index.rst +++ b/docs/apache-airflow-providers-tabular/index.rst @@ -38,7 +38,7 @@ Content :maxdepth: 1 :caption: Resources - Example DAGs + Example DAGs PyPI Repository Installing from sources Python API <_api/airflow/providers/tabular/index> diff --git a/docs/apache-airflow-providers-telegram/index.rst b/docs/apache-airflow-providers-telegram/index.rst index aaf5637754b38..f137d6c027db5 100644 --- a/docs/apache-airflow-providers-telegram/index.rst +++ b/docs/apache-airflow-providers-telegram/index.rst @@ -44,7 +44,7 @@ Content :maxdepth: 1 :caption: Resources - Example DAGs + Example DAGs PyPI Repository Installing from sources diff --git a/docs/apache-airflow-providers-trino/index.rst b/docs/apache-airflow-providers-trino/index.rst index 48131e6e58f2a..3dbfcfeced3f7 100644 --- a/docs/apache-airflow-providers-trino/index.rst +++ b/docs/apache-airflow-providers-trino/index.rst @@ -46,7 +46,7 @@ Content :maxdepth: 1 :caption: Resources - Example DAGs + Example DAGs PyPI Repository Installing from sources diff --git a/docs/apache-airflow-providers-yandex/index.rst b/docs/apache-airflow-providers-yandex/index.rst index fbfe9ea1761f1..ce70186bda186 100644 --- a/docs/apache-airflow-providers-yandex/index.rst +++ b/docs/apache-airflow-providers-yandex/index.rst @@ -45,7 +45,7 @@ Content :maxdepth: 1 :caption: Resources - Example DAGs + Example DAGs PyPI Repository Installing from sources diff --git a/docs/apache-airflow-providers-yandex/operators.rst b/docs/apache-airflow-providers-yandex/operators.rst index 552fd4febd73b..4fab4468d67ab 100644 --- a/docs/apache-airflow-providers-yandex/operators.rst +++ b/docs/apache-airflow-providers-yandex/operators.rst @@ -37,4 +37,4 @@ Prerequisite Tasks Using the operators ^^^^^^^^^^^^^^^^^^^^^ -See the usage examples in `example DAGs `_ +See the usage examples in `example DAGs `_ diff --git a/docs/apache-airflow-providers-zendesk/index.rst b/docs/apache-airflow-providers-zendesk/index.rst index e82d3e233adcb..8497880c46281 100644 --- a/docs/apache-airflow-providers-zendesk/index.rst +++ b/docs/apache-airflow-providers-zendesk/index.rst @@ -38,7 +38,7 @@ Content :maxdepth: 1 :caption: Resources - Example DAGs + Example DAGs PyPI Repository Installing from sources diff --git a/docs/conf.py b/docs/conf.py index f75704d9dd666..208883177f237 100644 --- a/docs/conf.py +++ b/docs/conf.py @@ -172,6 +172,13 @@ elif PACKAGE_NAME == "docker-stack": # No extra extensions pass +elif PACKAGE_NAME.startswith("apache-airflow-providers-"): + extensions.extend( + [ + "extra_provider_files_with_substitutions", + "autoapi.extension", + ] + ) else: extensions.append("autoapi.extension") # List of patterns, relative to source directory, that match files and @@ -298,7 +305,8 @@ def _get_rst_filepath_from_path(filepath: pathlib.Path): "installation/installing-from-pypi.html", "installation/installing-from-sources.html", ] - +if PACKAGE_NAME.startswith("apache-airflow-providers"): + manual_substitutions_in_generated_html = ["example-dags.html", "operators.html", "index.html"] if PACKAGE_NAME == "docker-stack": # Replace "|version|" inside ```` quotes manual_substitutions_in_generated_html = ["build.html"] diff --git a/docs/exts/extra_provider_files_with_substitutions.py b/docs/exts/extra_provider_files_with_substitutions.py new file mode 100644 index 0000000000000..7e74c70d666db --- /dev/null +++ b/docs/exts/extra_provider_files_with_substitutions.py @@ -0,0 +1,46 @@ +# 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 __future__ import annotations + +from pathlib import Path + + +def fix_provider_references(app, exception): + """Sphinx "build-finished" event handler.""" + from sphinx.builders import html as builders + + if exception or not isinstance(app.builder, builders.StandaloneHTMLBuilder): + return + + # Replace `|version|` in the files that require manual substitution + for path in Path(app.outdir).rglob("*.html"): + if not path.exists(): + continue + with open(path) as input_file: + content = input_file.readlines() + with open(path, "w") as output_file: + for line in content: + output_file.write(line.replace("|version|", app.config.version)) + + +def setup(app): + """Setup plugin""" + app.connect("build-finished", fix_provider_references) + + return { + "parallel_write_safe": True, + } diff --git a/images/breeze/output-commands-hash.txt b/images/breeze/output-commands-hash.txt index 3e8c83e10fef7..ef8d5e0c86140 100644 --- a/images/breeze/output-commands-hash.txt +++ b/images/breeze/output-commands-hash.txt @@ -50,7 +50,7 @@ setup:version:123b462a421884dc2320ffc5e54b2478 setup:a3bd246c3a425f3e586d11bbdc8937cb shell:f810c333d2c179343bbbd99872131cf0 start-airflow:8ae982d2717006e6b73fe1f7928a19a9 -static-checks:61c8d96ffe3008bff89a9df39b423be2 +static-checks:7a39e28c87fbca0a9fae0ebfe1591b71 stop:8969537ccdd799f692ccb8600a7bbed6 testing:docker-compose-tests:b86c044b24138af0659a05ed6331576c testing:helm-tests:94a442e7f3f63b34c4831a84d165690a diff --git a/images/breeze/output-commands.svg b/images/breeze/output-commands.svg index 3a712e55da6ed..763f7d8c8051d 100644 --- a/images/breeze/output-commands.svg +++ b/images/breeze/output-commands.svg @@ -19,221 +19,221 @@ font-weight: 700; } - .terminal-2786916824-matrix { + .breeze-help-matrix { font-family: Fira Code, monospace; font-size: 20px; line-height: 24.4px; font-variant-east-asian: full-width; } - .terminal-2786916824-title { + .breeze-help-title { font-size: 18px; font-weight: bold; font-family: arial; } - .terminal-2786916824-r1 { fill: #c5c8c6;font-weight: bold } -.terminal-2786916824-r2 { fill: #c5c8c6 } -.terminal-2786916824-r3 { fill: #d0b344;font-weight: bold } -.terminal-2786916824-r4 { fill: #68a0b3;font-weight: bold } -.terminal-2786916824-r5 { fill: #868887 } -.terminal-2786916824-r6 { fill: #98a84b;font-weight: bold } -.terminal-2786916824-r7 { fill: #8d7b39 } + .breeze-help-r1 { fill: #c5c8c6;font-weight: bold } +.breeze-help-r2 { fill: #c5c8c6 } +.breeze-help-r3 { fill: #d0b344;font-weight: bold } +.breeze-help-r4 { fill: #68a0b3;font-weight: bold } +.breeze-help-r5 { fill: #868887 } +.breeze-help-r6 { fill: #98a84b;font-weight: bold } +.breeze-help-r7 { fill: #8d7b39 } - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - Breeze commands + Breeze commands - + - - -Usage: breeze [OPTIONSCOMMAND [ARGS]... - -╭─ Basic flags ────────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---python-pPython major/minor version used in Airflow image for images.(>3.7< | 3.8 | 3.9 | 3.10) -[default: 3.7]                                               ---backend-bDatabase backend to use.(>sqlite< | mysql | postgres | mssql)[default: sqlite] ---postgres-version-PVersion of Postgres used.(>11< | 12 | 13 | 14 | 15)[default: 11] ---mysql-version-MVersion of MySQL used.(>5.7< | 8)[default: 5.7] ---mssql-version-SVersion of MsSQL used.(>2017-latest< | 2019-latest)[default: 2017-latest] ---integrationIntegration(s) to enable when running (can be more than one).                             -(cassandra | kerberos | mongo | openldap | pinot | rabbitmq | redis | statsd | trino |    -all)                                                                                      ---forward-credentials-fForward local credentials to container when running. ---db-reset-dReset DB when entering the container. ---max-timeMaximum time that the command should take - if it takes longer, the command will fail. -(INTEGER RANGE)                                                                        ---github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---verbose-vPrint verbose information about performed steps. ---dry-run-DIf dry-run is set, commands are only printed, not executed. ---answer-aForce answer to questions.(y | n | q | yes | no | quit) ---help-hShow this message and exit. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Basic developer commands ───────────────────────────────────────────────────────────────────────────────────────────╮ -start-airflow     Enter breeze environment and starts all Airflow components in the tmux session. Compile assets   -if contents of www directory changed.                                                            -static-checks     Run static checks.                                                                               -build-docs        Build documentation in the container.                                                            -stop              Stop running breeze environment.                                                                 -shell             Enter breeze environment. this is the default command use when no other is selected.             -exec              Joins the interactive shell of running airflow container.                                        -compile-www-assetsCompiles www assets.                                                                             -cleanup           Cleans the cache of parameters, docker cache and optionally built CI/PROD images.                -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Advanced command groups ────────────────────────────────────────────────────────────────────────────────────────────╮ -testing                Tools that developers can use to run tests                                                  -ci-image               Tools that developers can use to manually manage CI images                                  -k8s                    Tools that developers use to run Kubernetes tests                                           -prod-image             Tools that developers can use to manually manage PROD images                                -setup                  Tools that developers can use to configure Breeze                                           -release-management     Tools that release managers can use to prepare and manage Airflow releases                  -ci                     Tools that CI workflows use to cleanup/manage CI environment                                -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + +Usage: breeze [OPTIONSCOMMAND [ARGS]... + +╭─ Basic flags ────────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--python-pPython major/minor version used in Airflow image for images.(>3.7< | 3.8 | 3.9 | 3.10) +[default: 3.7]                                               +--backend-bDatabase backend to use.(>sqlite< | mysql | postgres | mssql)[default: sqlite] +--postgres-version-PVersion of Postgres used.(>11< | 12 | 13 | 14 | 15)[default: 11] +--mysql-version-MVersion of MySQL used.(>5.7< | 8)[default: 5.7] +--mssql-version-SVersion of MsSQL used.(>2017-latest< | 2019-latest)[default: 2017-latest] +--integrationIntegration(s) to enable when running (can be more than one).                             +(cassandra | kerberos | mongo | openldap | pinot | rabbitmq | redis | statsd | trino |    +all)                                                                                      +--forward-credentials-fForward local credentials to container when running. +--db-reset-dReset DB when entering the container. +--max-timeMaximum time that the command should take - if it takes longer, the command will fail. +(INTEGER RANGE)                                                                        +--github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--answer-aForce answer to questions.(y | n | q | yes | no | quit) +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Basic developer commands ───────────────────────────────────────────────────────────────────────────────────────────╮ +start-airflow     Enter breeze environment and starts all Airflow components in the tmux session. Compile assets   +if contents of www directory changed.                                                            +static-checks     Run static checks.                                                                               +build-docs        Build documentation in the container.                                                            +stop              Stop running breeze environment.                                                                 +shell             Enter breeze environment. this is the default command use when no other is selected.             +exec              Joins the interactive shell of running airflow container.                                        +compile-www-assetsCompiles www assets.                                                                             +cleanup           Cleans the cache of parameters, docker cache and optionally built CI/PROD images.                +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Advanced command groups ────────────────────────────────────────────────────────────────────────────────────────────╮ +testing                Tools that developers can use to run tests                                                  +ci-image               Tools that developers can use to manually manage CI images                                  +k8s                    Tools that developers use to run Kubernetes tests                                           +prod-image             Tools that developers can use to manually manage PROD images                                +setup                  Tools that developers can use to configure Breeze                                           +release-management     Tools that release managers can use to prepare and manage Airflow releases                  +ci                     Tools that CI workflows use to cleanup/manage CI environment                                +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ diff --git a/images/breeze/output_static-checks.svg b/images/breeze/output_static-checks.svg index 0418a91abd205..3cd8b66bbc932 100644 --- a/images/breeze/output_static-checks.svg +++ b/images/breeze/output_static-checks.svg @@ -1,4 +1,4 @@ - + pretty-format-json | pydocstyle | python-no-log-warn | pyupgrade |                       replace-bad-characters | rst-backticks | run-flake8 | run-mypy | run-shellcheck |        static-check-autoflake | trailing-whitespace | ts-compile-and-lint-javascript |          -update-breeze-cmd-output | update-breeze-readme-config-hash |                            -update-common-sql-api-stubs | update-er-diagram | update-extras |                        -update-in-the-wild-to-be-sorted | update-inlined-dockerfile-scripts |                    -update-local-yml-file | update-migration-references | update-providers-dependencies |    -update-spelling-wordlist-to-be-sorted | update-supported-versions |                      -update-vendored-in-k8s-json-schema | update-version | yamllint | yesqa)                  ---file-fList of files to run the checks on.(PATH) ---all-files-aRun checks on all files. ---show-diff-on-failure-sShow diff for files modified by the checks. ---last-commit-cRun checks for all files in last commit. Mutually exclusive with --commit-ref. ---commit-ref-rRun checks for this commit reference only (can be any git commit-ish reference).         -Mutually exclusive with --last-commit.                                                   -(TEXT)                                                                                   ---github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---verbose-vPrint verbose information about performed steps. ---dry-run-DIf dry-run is set, commands are only printed, not executed. ---help-hShow this message and exit. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +update-breeze-cmd-output | update-breeze-readme-config-hash | update-er-diagram |        +update-extras | update-in-the-wild-to-be-sorted | update-inlined-dockerfile-scripts |    +update-local-yml-file | update-migration-references | update-providers-dependencies |    +update-spelling-wordlist-to-be-sorted | update-supported-versions |                      +update-vendored-in-k8s-json-schema | update-version | yamllint | yesqa)                  +--file-fList of files to run the checks on.(PATH) +--all-files-aRun checks on all files. +--show-diff-on-failure-sShow diff for files modified by the checks. +--last-commit-cRun checks for all files in last commit. Mutually exclusive with --commit-ref. +--commit-ref-rRun checks for this commit reference only (can be any git commit-ish reference).         +Mutually exclusive with --last-commit.                                                   +(TEXT)                                                                                   +--github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ From 369c635130a87c8c9375613e0cfce6ab5930acfd Mon Sep 17 00:00:00 2001 From: Jarek Potiuk Date: Mon, 2 Jan 2023 10:17:51 +0100 Subject: [PATCH 145/158] Make secret rendering test more resilient to implementation details (#28673) The newly released Pygments 2.14.0 changed slightly the way it wraps tags around rendeered variables and tour test made far too many assumptions about the rendered output which started to fail on main with the new Pygments. This PR fixes it by actually testing only what it was supposed to test: - lack of the secret in rendered output - presence of *** masking in the output (cherry picked from commit 81cd6c74788bc3182397dd28a4e7db3f21ff2d69) --- tests/www/views/test_views_rendered.py | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/tests/www/views/test_views_rendered.py b/tests/www/views/test_views_rendered.py index 970c56d4f081d..c54212dc9f9db 100644 --- a/tests/www/views/test_views_rendered.py +++ b/tests/www/views/test_views_rendered.py @@ -30,7 +30,7 @@ from airflow.utils.state import DagRunState, TaskInstanceState from airflow.utils.types import DagRunType from tests.test_utils.db import clear_db_dags, clear_db_runs, clear_rendered_ti_fields -from tests.test_utils.www import check_content_in_response +from tests.test_utils.www import check_content_in_response, check_content_not_in_response DEFAULT_DATE = timezone.datetime(2020, 3, 1) @@ -186,7 +186,7 @@ def test_user_defined_filter_and_macros_raise_error(admin_client, create_dag_run @pytest.mark.usefixtures("patch_app") def test_rendered_template_secret(admin_client, create_dag_run, task_secret): """Test that the Rendered View masks values retrieved from secret variables.""" - Variable.set("my_secret", "foo") + Variable.set("my_secret", "secret_unlikely_to_happen_accidentally") Variable.set("spam", "egg") assert task_secret.bash_command == "echo {{ var.value.my_secret }} && echo {{ var.value.spam }}" @@ -202,8 +202,7 @@ def test_rendered_template_secret(admin_client, create_dag_run, task_secret): url = f"rendered-templates?task_id=task_secret&dag_id=testdag&execution_date={date}" resp = admin_client.get(url, follow_redirects=True) - check_content_in_response( - 'echo *** && echo egg', resp - ) + check_content_in_response("***", resp) + check_content_not_in_response("secret_unlikely_to_happen_accidentally", resp) ti.refresh_from_task(task_secret) assert ti.state == TaskInstanceState.QUEUED From 15243922225ec5468c707c714dc2ff710689d530 Mon Sep 17 00:00:00 2001 From: Tzu-ping Chung Date: Thu, 12 Jan 2023 23:44:31 +0800 Subject: [PATCH 146/158] Mention mapped operator in cluster policy doc (#28885) (cherry picked from commit 49e178e4b0e7898eeb13527ea3211210ee6d94d6) --- .../administration-and-deployment/cluster-policies.rst | 2 +- .../authoring-and-scheduling/dynamic-task-mapping.rst | 2 ++ 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/docs/apache-airflow/administration-and-deployment/cluster-policies.rst b/docs/apache-airflow/administration-and-deployment/cluster-policies.rst index 242924a3f3249..76ede3bd67620 100644 --- a/docs/apache-airflow/administration-and-deployment/cluster-policies.rst +++ b/docs/apache-airflow/administration-and-deployment/cluster-policies.rst @@ -27,7 +27,7 @@ If you want to check or mutate DAGs or Tasks on a cluster-wide level, then a Clu There are three types of cluster policy: * ``dag_policy``: Takes a :class:`~airflow.models.dag.DAG` parameter called ``dag``. Runs at load time of the DAG from DagBag :class:`~airflow.models.dagbag.DagBag`. -* ``task_policy``: Takes a :class:`~airflow.models.baseoperator.BaseOperator` parameter called ``task``. The policy gets executed when the task is created during parsing of the task from DagBag at load time. This means that the whole task definition can be altered in the task policy. It does not relate to a specific task running in a DagRun. The ``task_policy`` defined is applied to all the task instances that will be executed in the future. +* ``task_policy``: Takes a parameter called ``task`` that is of type either :class:`~airflow.models.baseoperator.BaseOperator` or :class:`~airflow.models.mappedoperator.MappedOperator` (for `dynamically expanded tasks `_). The policy gets executed when the task is created during parsing of the task from DagBag at load time. This means that the whole task definition can be altered in the task policy. It does not relate to a specific task running in a DagRun. The ``task_policy`` defined is applied to all the task instances that will be executed in the future. * ``task_instance_mutation_hook``: Takes a :class:`~airflow.models.taskinstance.TaskInstance` parameter called ``task_instance``. The ``task_instance_mutation`` applies not to a task but to the instance of a task that relates to a particular DagRun. It is executed in a "worker", not in the dag file processor, just before the task instance is executed. The policy is only applied to the currently executed run (i.e. instance) of that task. The DAG and Task cluster policies can raise the :class:`~airflow.exceptions.AirflowClusterPolicyViolation` exception to indicate that the dag/task they were passed is not compliant and should not be loaded. diff --git a/docs/apache-airflow/authoring-and-scheduling/dynamic-task-mapping.rst b/docs/apache-airflow/authoring-and-scheduling/dynamic-task-mapping.rst index 8f5f2ee6e2eaa..4692edc199b77 100644 --- a/docs/apache-airflow/authoring-and-scheduling/dynamic-task-mapping.rst +++ b/docs/apache-airflow/authoring-and-scheduling/dynamic-task-mapping.rst @@ -15,6 +15,8 @@ specific language governing permissions and limitations under the License. +.. _dynamic-task-mapping: + ==================== Dynamic Task Mapping ==================== From fb274865ee3b4db635a1333238afae2d4e51f956 Mon Sep 17 00:00:00 2001 From: Jarek Potiuk Date: Thu, 12 Jan 2023 21:44:15 +0100 Subject: [PATCH 147/158] Skip pushing early image cache for non-main branches (#28895) The early cache is only needed when we push to main because it is only needed for PRs to the main brach. (cherry picked from commit 14783c63c5d36fcd70afaaebc29c2638073b7e34) --- .github/workflows/ci.yml | 30 +++++++++++++++++++----------- 1 file changed, 19 insertions(+), 11 deletions(-) diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index e8c18eb21eb3e..28038a9298ca8 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -256,18 +256,20 @@ jobs: steps: - name: Cleanup repo run: docker run -v "${GITHUB_WORKSPACE}:/workspace" -u 0:0 bash -c "rm -rf /workspace/*" - if: needs.build-info.outputs.canary-run == 'true' + if: needs.build-info.outputs.canary-run == 'true' && needs.build-info.outputs.default-branch == 'main' - name: "Checkout ${{ github.ref }} ( ${{ github.sha }} )" uses: actions/checkout@v3 with: persist-credentials: false - if: needs.build-info.outputs.canary-run == 'true' + if: needs.build-info.outputs.canary-run == 'true' && needs.build-info.outputs.default-branch == 'main' - name: "Install Breeze" uses: ./.github/actions/breeze - if: needs.build-info.outputs.canary-run == 'true' + if: needs.build-info.outputs.canary-run == 'true' && needs.build-info.outputs.default-branch == 'main' - name: "Start ARM instance" run: ./scripts/ci/images/ci_start_arm_instance_and_connect_to_docker.sh - if: matrix.platform == 'linux/arm64' && needs.build-info.outputs.canary-run == 'true' + if: > + matrix.platform == 'linux/arm64' && needs.build-info.outputs.canary-run == 'true' + && needs.build-info.outputs.default-branch == 'main' - name: "Push CI cache ${{ matrix.platform }}" run: > breeze ci-image build @@ -278,7 +280,7 @@ jobs: --platform ${{ matrix.platform }} env: DEBUG_RESOURCES: ${{needs.build-info.outputs.debug-resources}} - if: needs.build-info.outputs.canary-run == 'true' + if: needs.build-info.outputs.canary-run == 'true' && needs.build-info.outputs.default-branch == 'main' - name: "Push CI latest image ${{ matrix.platform }}" run: > breeze ci-image build @@ -287,18 +289,24 @@ jobs: DEBUG_RESOURCES: ${{needs.build-info.outputs.debug-resources}} # We only push "amd" image as it is really only needed for any kind of automated builds in CI # and currently there is not an easy way to make multi-platform image from two separate builds - if: matrix.platform == 'linux/amd64' && needs.build-info.outputs.canary-run == 'true' - + if: > + matrix.platform == 'linux/amd64' && needs.build-info.outputs.canary-run == 'true' + && needs.build-info.outputs.default-branch == 'main' - name: "Stop ARM instance" run: ./scripts/ci/images/ci_stop_arm_instance.sh - if: always() && matrix.platform == 'linux/arm64' && needs.build-info.outputs.canary-run == 'true' + if: > + always() && matrix.platform == 'linux/arm64' && needs.build-info.outputs.canary-run == 'true' + && needs.build-info.outputs.default-branch == 'main' - name: "Clean docker cache for ${{ matrix.platform }}" run: docker system prune --all --force - if: matrix.platform == 'linux/amd64' && needs.build-info.outputs.canary-run == 'true' + if: > + matrix.platform == 'linux/amd64' && needs.build-info.outputs.canary-run == 'true' + && needs.build-info.outputs.default-branch == 'main' - name: "Fix ownership" run: breeze ci fix-ownership - if: always() && needs.build-info.outputs.canary-run == 'true' - + if: > + always() && needs.build-info.outputs.canary-run == 'true' + && needs.build-info.outputs.default-branch == 'main' # Check that after earlier cache push, breeze command will build quickly check-that-image-builds-quickly: timeout-minutes: 5 From 6101280efb8dc912b84b9f6d91bd39d7f9fea4d1 Mon Sep 17 00:00:00 2001 From: Ash Berlin-Taylor Date: Tue, 10 Jan 2023 20:40:14 +0000 Subject: [PATCH 148/158] Google have re-worked their docs site ane appear not publish objects.inv (#28834) (cherry picked from commit 44d93bea3adbe9429b6fdee14de31f80ac51c3bb) --- docs/conf.py | 35 ------------------- .../docs_build/third_party_inventories.py | 24 ------------- docs/spelling_wordlist.txt | 7 ++++ 3 files changed, 7 insertions(+), 59 deletions(-) diff --git a/docs/conf.py b/docs/conf.py index 208883177f237..2e9340cfe8204 100644 --- a/docs/conf.py +++ b/docs/conf.py @@ -633,41 +633,6 @@ def _get_params(root_schema: dict, prefix: str = "", default_section: str = "") "sqlalchemy", ] } -if PACKAGE_NAME in ("apache-airflow-providers-google", "apache-airflow"): - intersphinx_mapping.update( - { - pkg_name: ( - f"{THIRD_PARTY_INDEXES[pkg_name]}/", - (f"{INVENTORY_CACHE_DIR}/{pkg_name}/objects.inv",), - ) - for pkg_name in [ - "google-api-core", - "google-cloud-automl", - "google-cloud-bigquery", - "google-cloud-bigquery-datatransfer", - "google-cloud-bigquery-storage", - "google-cloud-bigtable", - "google-cloud-container", - "google-cloud-core", - "google-cloud-datacatalog", - "google-cloud-datastore", - "google-cloud-dlp", - "google-cloud-kms", - "google-cloud-language", - "google-cloud-monitoring", - "google-cloud-pubsub", - "google-cloud-redis", - "google-cloud-spanner", - "google-cloud-speech", - "google-cloud-storage", - "google-cloud-tasks", - "google-cloud-texttospeech", - "google-cloud-translate", - "google-cloud-videointelligence", - "google-cloud-vision", - ] - } - ) # -- Options for sphinx.ext.viewcode ------------------------------------------- # See: https://www.sphinx-doc.org/es/master/usage/extensions/viewcode.html diff --git a/docs/exts/docs_build/third_party_inventories.py b/docs/exts/docs_build/third_party_inventories.py index 7ac33a5333801..720af261b55be 100644 --- a/docs/exts/docs_build/third_party_inventories.py +++ b/docs/exts/docs_build/third_party_inventories.py @@ -27,28 +27,4 @@ "python": "https://docs.python.org/3", "requests": "https://requests.readthedocs.io/en/stable", "sqlalchemy": "https://docs.sqlalchemy.org/en/latest", - "google-api-core": "https://googleapis.dev/python/google-api-core/latest", - "google-cloud-automl": "https://googleapis.dev/python/automl/latest", - "google-cloud-bigquery": "https://googleapis.dev/python/bigquery/latest", - "google-cloud-bigquery-datatransfer": "https://googleapis.dev/python/bigquerydatatransfer/latest", - "google-cloud-bigquery-storage": "https://googleapis.dev/python/bigquerystorage/latest", - "google-cloud-bigtable": "https://googleapis.dev/python/bigtable/latest", - "google-cloud-container": "https://googleapis.dev/python/container/latest", - "google-cloud-core": "https://googleapis.dev/python/google-cloud-core/latest", - "google-cloud-datacatalog": "https://googleapis.dev/python/datacatalog/latest", - "google-cloud-datastore": "https://googleapis.dev/python/datastore/latest", - "google-cloud-dlp": "https://googleapis.dev/python/dlp/latest", - "google-cloud-kms": "https://googleapis.dev/python/cloudkms/latest", - "google-cloud-language": "https://googleapis.dev/python/language/latest", - "google-cloud-monitoring": "https://googleapis.dev/python/monitoring/latest", - "google-cloud-pubsub": "https://googleapis.dev/python/pubsub/latest", - "google-cloud-redis": "https://googleapis.dev/python/redis/latest", - "google-cloud-spanner": "https://googleapis.dev/python/spanner/latest", - "google-cloud-speech": "https://googleapis.dev/python/speech/latest", - "google-cloud-storage": "https://googleapis.dev/python/storage/latest", - "google-cloud-tasks": "https://googleapis.dev/python/cloudtasks/latest", - "google-cloud-texttospeech": "https://googleapis.dev/python/texttospeech/latest", - "google-cloud-translate": "https://googleapis.dev/python/translation/latest", - "google-cloud-videointelligence": "https://googleapis.dev/python/videointelligence/latest", - "google-cloud-vision": "https://googleapis.dev/python/vision/latest", } diff --git a/docs/spelling_wordlist.txt b/docs/spelling_wordlist.txt index f019d2a8c0120..734723894a396 100644 --- a/docs/spelling_wordlist.txt +++ b/docs/spelling_wordlist.txt @@ -849,8 +849,11 @@ LineItem lineterminator linter linux +ListColumnSpecsPager +ListDatasetsPager ListGenerator ListInfoTypesResponse +ListTableSpecsPager Liveness liveness livy @@ -1349,6 +1352,7 @@ Stackdriver stackdriver stacktrace Standarization +StartManualTransferRunsResponse starttls StatefulSet StatefulSets @@ -1415,6 +1419,7 @@ symlink symlinking symlinks sync'ed +SynthesizeSpeechResponse sys syspath Systemd @@ -1486,6 +1491,8 @@ tooltip tooltips traceback tracebacks +TransferConfig +TransferRun travis triage triaging From e234bebdc040c86ddae85f7c71d64e43013a05e2 Mon Sep 17 00:00:00 2001 From: Andrey Anshin Date: Wed, 11 Jan 2023 02:31:26 +0400 Subject: [PATCH 149/158] Validate Sphinx Inventory file header (#28838) (cherry picked from commit 04306f18b0643dfed3ed97863bbcf24dc50a8973) --- docs/exts/docs_build/fetch_inventories.py | 38 +++++++++++++++++------ 1 file changed, 29 insertions(+), 9 deletions(-) diff --git a/docs/exts/docs_build/fetch_inventories.py b/docs/exts/docs_build/fetch_inventories.py index a104cadf77bc3..1cd3e94c1ce2f 100644 --- a/docs/exts/docs_build/fetch_inventories.py +++ b/docs/exts/docs_build/fetch_inventories.py @@ -24,11 +24,13 @@ import sys import traceback from itertools import repeat +from tempfile import NamedTemporaryFile from typing import Iterator import requests import urllib3.exceptions from requests.adapters import DEFAULT_POOLSIZE +from sphinx.util.inventory import InventoryFileReader from airflow.utils.helpers import partition from docs.exts.docs_build.docs_builder import get_available_providers_packages @@ -47,25 +49,41 @@ def _fetch_file(session: requests.Session, package_name: str, url: str, path: str) -> tuple[str, bool]: """ - Download a file and returns status information as a tuple with package + Download a file, validate Sphinx Inventory headers and returns status information as a tuple with package name and success status(bool value). """ try: response = session.get(url, allow_redirects=True, stream=True) except (requests.RequestException, urllib3.exceptions.HTTPError): - print(f"Failed to fetch inventory: {url}") + print(f"{package_name}: Failed to fetch inventory: {url}") traceback.print_exc(file=sys.stderr) return package_name, False if not response.ok: - print(f"Failed to fetch inventory: {url}") - print(f"Failed with status: {response.status_code}", file=sys.stderr) + print(f"{package_name}: Failed to fetch inventory: {url}") + print(f"{package_name}: Failed with status: {response.status_code}", file=sys.stderr) return package_name, False - os.makedirs(os.path.dirname(path), exist_ok=True) - with open(path, "wb") as f: - response.raw.decode_content = True - shutil.copyfileobj(response.raw, f) - print(f"Fetched inventory: {url}") + if response.url != url: + print(f"{package_name}: {url} redirected to {response.url}") + + with NamedTemporaryFile(suffix=package_name, mode="wb+") as tf: + for chunk in response.iter_content(chunk_size=4096): + tf.write(chunk) + + tf.flush() + tf.seek(0, 0) + + line = InventoryFileReader(tf).readline() + if not line.startswith("# Sphinx inventory version"): + print(f"{package_name}: Response contain unexpected Sphinx Inventory header: {line!r}.") + return package_name, False + + tf.seek(0, 0) + os.makedirs(os.path.dirname(path), exist_ok=True) + with open(path, "wb") as f: + shutil.copyfileobj(tf, f) + + print(f"{package_name}: Fetched inventory: {response.url}") return package_name, True @@ -136,5 +154,7 @@ def fetch_inventories(): print("Failed packages:") for pkg_no, (pkg_name, _) in enumerate(failed, start=1): print(f"{pkg_no}. {pkg_name}") + print("Terminate execution.") + raise SystemExit(1) return [pkg_name for pkg_name, status in failed] From 743c237ca99d5701f284f3b15defe7d0460118f8 Mon Sep 17 00:00:00 2001 From: Ephraim Anierobi Date: Thu, 12 Jan 2023 23:29:15 +0100 Subject: [PATCH 150/158] fixup! Restructure Docs (#27235) --- docs/apache-airflow/authoring-and-scheduling/index.rst | 1 - 1 file changed, 1 deletion(-) diff --git a/docs/apache-airflow/authoring-and-scheduling/index.rst b/docs/apache-airflow/authoring-and-scheduling/index.rst index db1ef3726cc67..ba83cbb156f26 100644 --- a/docs/apache-airflow/authoring-and-scheduling/index.rst +++ b/docs/apache-airflow/authoring-and-scheduling/index.rst @@ -29,7 +29,6 @@ It's recommended that you first review the pages in :doc:`core concepts Date: Mon, 5 Dec 2022 05:58:32 +0300 Subject: [PATCH 151/158] Rewrite kerberos security integration and unit tests (#28092) (cherry picked from commit e0bb6be71a3fe2ba75126b601ea11cd2be9465b5) --- tests/security/test_kerberos.py | 228 ++++++++++++++------------------ 1 file changed, 101 insertions(+), 127 deletions(-) diff --git a/tests/security/test_kerberos.py b/tests/security/test_kerberos.py index fa8be0fa21010..81f139c78206a 100644 --- a/tests/security/test_kerberos.py +++ b/tests/security/test_kerberos.py @@ -17,98 +17,68 @@ # under the License. from __future__ import annotations +import logging import os import shlex -import unittest -from argparse import Namespace +from contextlib import nullcontext from unittest import mock import pytest -from parameterized import parameterized from airflow.security import kerberos from airflow.security.kerberos import renew_from_kt from tests.test_utils.config import conf_vars -KRB5_KTNAME = os.environ.get("KRB5_KTNAME") +@pytest.mark.integration("kerberos") +class TestKerberosIntegration: + @classmethod + def setup_class(cls): + assert "KRB5_KTNAME" in os.environ, "Missing KRB5_KTNAME environment variable" + cls.keytab = os.environ["KRB5_KTNAME"] -@unittest.skipIf(KRB5_KTNAME is None, "Skipping Kerberos API tests due to missing KRB5_KTNAME") -class TestKerberos(unittest.TestCase): - def setUp(self): - self.args = Namespace( - keytab=KRB5_KTNAME, principal=None, pid=None, daemon=None, stdout=None, stderr=None, log_file=None - ) - - @conf_vars({("kerberos", "keytab"): KRB5_KTNAME}) - def test_renew_from_kt(self): - """ - We expect no result, but a successful run. No more TypeError - """ - assert renew_from_kt(principal=self.args.principal, keytab=self.args.keytab) is None - - @conf_vars({("kerberos", "keytab"): KRB5_KTNAME, ("kerberos", "include_ip"): ""}) - def test_renew_from_kt_include_ip_empty(self): - """ - We expect no result, but a successful run. - """ - assert renew_from_kt(principal=self.args.principal, keytab=self.args.keytab) is None - - @conf_vars({("kerberos", "keytab"): KRB5_KTNAME, ("kerberos", "include_ip"): "False"}) - def test_renew_from_kt_include_ip_false(self): - """ - We expect no result, but a successful run. - """ - assert renew_from_kt(principal=self.args.principal, keytab=self.args.keytab) is None - - @conf_vars({("kerberos", "keytab"): KRB5_KTNAME, ("kerberos", "include_ip"): "True"}) - def test_renew_from_kt_include_ip_true(self): - """ - We expect no result, but a successful run. - """ - assert renew_from_kt(principal=self.args.principal, keytab=self.args.keytab) is None - - # Validate forwardable kerberos option - @conf_vars({("kerberos", "keytab"): KRB5_KTNAME, ("kerberos", "forwardable"): ""}) - def test_renew_from_kt_forwardable_empty(self): - """ - We expect no result, but a successful run. - """ - assert renew_from_kt(principal=self.args.principal, keytab=self.args.keytab) is None - - @conf_vars({("kerberos", "keytab"): KRB5_KTNAME, ("kerberos", "forwardable"): "False"}) - def test_renew_from_kt_forwardable_false(self): - """ - We expect no result, but a successful run. - """ - assert renew_from_kt(principal=self.args.principal, keytab=self.args.keytab) is None - - @conf_vars({("kerberos", "keytab"): KRB5_KTNAME, ("kerberos", "forwardable"): "True"}) - def test_renew_from_kt_forwardable_true(self): - """ - We expect no result, but a successful run. - """ - assert renew_from_kt(principal=self.args.principal, keytab=self.args.keytab) is None - - @conf_vars({("kerberos", "keytab"): ""}) - def test_args_from_cli(self): - """ - We expect no result, but a run with sys.exit(1) because keytab not exist. - """ - with pytest.raises(SystemExit) as ctx: - renew_from_kt(principal=self.args.principal, keytab=self.args.keytab) - - with self.assertLogs(kerberos.log) as log: - assert ( - f"kinit: krb5_init_creds_set_keytab: Failed to find airflow@LUPUS.GRIDDYNAMICS.NET in " - f"keytab FILE:{self.args.keytab} (unknown enctype)" in log.output - ) - - assert ctx.value.code == 1 - + @pytest.mark.parametrize( + "kerberos_config", + [ + pytest.param({}, id="default-config"), + pytest.param({("kerberos", "include_ip"): "True"}, id="explicit-include-ip"), + pytest.param({("kerberos", "include_ip"): "False"}, id="explicit-not-include-ip"), + pytest.param({("kerberos", "forwardable"): "True"}, id="explicit-forwardable"), + pytest.param({("kerberos", "forwardable"): "False"}, id="explicit-not-forwardable"), + ], + ) + def test_renew_from_kt(self, kerberos_config): + """We expect return 0 (exit code) and successful run.""" + with conf_vars(kerberos_config): + assert renew_from_kt(principal=None, keytab=self.keytab) == 0 -class TestKerberosUnit(unittest.TestCase): - @parameterized.expand( + @pytest.mark.parametrize( + "exit_on_fail, expected_context", + [ + pytest.param(True, pytest.raises(SystemExit), id="exit-on-fail"), + pytest.param(False, nullcontext(), id="return-code-of-fail"), + ], + ) + def test_args_from_cli(self, exit_on_fail, expected_context, caplog): + """Test exit code if keytab not exist.""" + keytab = "/not/exists/keytab" + result = None + + with mock.patch.dict(os.environ, KRB5_KTNAME=keytab), conf_vars({("kerberos", "keytab"): keytab}): + with expected_context as ctx: + with caplog.at_level(logging.ERROR, logger=kerberos.log.name): + caplog.clear() + result = renew_from_kt(principal=None, keytab=keytab, exit_on_fail=exit_on_fail) + + # If `exit_on_fail` set to True than exit code in exception, otherwise in function return + exit_code = ctx.value.code if exit_on_fail else result + assert exit_code == 1 + assert caplog.record_tuples + + +class TestKerberos: + @pytest.mark.parametrize( + "kerberos_config, expected_cmd", [ ( {("kerberos", "reinit_frequency"): "42"}, @@ -158,31 +128,27 @@ class TestKerberosUnit(unittest.TestCase): "test-principal", ], ), - ] + ], ) - def test_renew_from_kt(self, kerberos_config, expected_cmd): - with self.assertLogs(kerberos.log) as log_ctx, conf_vars(kerberos_config), mock.patch( - "airflow.security.kerberos.subprocess" - ) as mock_subprocess, mock.patch( - "airflow.security.kerberos.NEED_KRB181_WORKAROUND", None - ), mock.patch( - "airflow.security.kerberos.open", mock.mock_open(read_data=b"X-CACHECONF:") - ), mock.patch( - "time.sleep", return_value=None - ): + @mock.patch("time.sleep", return_value=None) + @mock.patch("airflow.security.kerberos.open", mock.mock_open(read_data=b"X-CACHECONF:")) + @mock.patch("airflow.security.kerberos.NEED_KRB181_WORKAROUND", None) + @mock.patch("airflow.security.kerberos.subprocess") + def test_renew_from_kt(self, mock_subprocess, mock_sleep, kerberos_config, expected_cmd, caplog): + expected_cmd_text = " ".join(shlex.quote(f) for f in expected_cmd) + + with conf_vars(kerberos_config), caplog.at_level(logging.INFO, logger=kerberos.log.name): + caplog.clear() mock_subprocess.Popen.return_value.__enter__.return_value.returncode = 0 mock_subprocess.call.return_value = 0 renew_from_kt(principal="test-principal", keytab="keytab") - assert mock_subprocess.Popen.call_args[0][0] == expected_cmd - - expected_cmd_text = " ".join(shlex.quote(f) for f in expected_cmd) - assert log_ctx.output == [ - f"INFO:airflow.security.kerberos:Re-initialising kerberos from keytab: {expected_cmd_text}", - "INFO:airflow.security.kerberos:Renewing kerberos ticket to work around kerberos 1.8.1: " - "kinit -c /tmp/airflow_krb5_ccache -R", + assert caplog.messages == [ + f"Re-initialising kerberos from keytab: {expected_cmd_text}", + "Renewing kerberos ticket to work around kerberos 1.8.1: kinit -c /tmp/airflow_krb5_ccache -R", ] + assert mock_subprocess.Popen.call_args[0][0] == expected_cmd assert mock_subprocess.mock_calls == [ mock.call.Popen( expected_cmd, @@ -201,17 +167,17 @@ def test_renew_from_kt(self, kerberos_config, expected_cmd): @mock.patch("airflow.security.kerberos.subprocess") @mock.patch("airflow.security.kerberos.NEED_KRB181_WORKAROUND", None) @mock.patch("airflow.security.kerberos.open", mock.mock_open(read_data=b"")) - def test_renew_from_kt_without_workaround(self, mock_subprocess): + def test_renew_from_kt_without_workaround(self, mock_subprocess, caplog): mock_subprocess.Popen.return_value.__enter__.return_value.returncode = 0 mock_subprocess.call.return_value = 0 - with self.assertLogs(kerberos.log) as log_ctx: + with caplog.at_level(logging.INFO, logger=kerberos.log.name): + caplog.clear() renew_from_kt(principal="test-principal", keytab="keytab") - - assert log_ctx.output == [ - "INFO:airflow.security.kerberos:Re-initialising kerberos from keytab: " - "kinit -f -a -r 3600m -k -t keytab -c /tmp/airflow_krb5_ccache test-principal" - ] + assert caplog.messages == [ + "Re-initialising kerberos from keytab: " + "kinit -f -a -r 3600m -k -t keytab -c /tmp/airflow_krb5_ccache test-principal" + ] assert mock_subprocess.mock_calls == [ mock.call.Popen( @@ -241,21 +207,24 @@ def test_renew_from_kt_without_workaround(self, mock_subprocess): @mock.patch("airflow.security.kerberos.subprocess") @mock.patch("airflow.security.kerberos.NEED_KRB181_WORKAROUND", None) - def test_renew_from_kt_failed(self, mock_subprocess): + def test_renew_from_kt_failed(self, mock_subprocess, caplog): mock_subp = mock_subprocess.Popen.return_value.__enter__.return_value mock_subp.returncode = 1 mock_subp.stdout = mock.MagicMock(name="stdout", **{"readlines.return_value": ["STDOUT"]}) mock_subp.stderr = mock.MagicMock(name="stderr", **{"readlines.return_value": ["STDERR"]}) - with self.assertLogs(kerberos.log) as log_ctx, self.assertRaises(SystemExit): + with pytest.raises(SystemExit) as ctx: + caplog.clear() renew_from_kt(principal="test-principal", keytab="keytab") + assert ctx.value.code == 1 - assert log_ctx.output == [ - "INFO:airflow.security.kerberos:Re-initialising kerberos from keytab: " + log_records = [record for record in caplog.record_tuples if record[0] == kerberos.log.name] + assert len(log_records) == 2, log_records + assert [lr[1] for lr in log_records] == [logging.INFO, logging.ERROR] + assert [lr[2] for lr in log_records] == [ + "Re-initialising kerberos from keytab: " "kinit -f -a -r 3600m -k -t keytab -c /tmp/airflow_krb5_ccache test-principal", - "ERROR:airflow.security.kerberos:Couldn't reinit from keytab! `kinit' exited with 1.\n" - "STDOUT\n" - "STDERR", + "Couldn't reinit from keytab! `kinit' exited with 1.\nSTDOUT\nSTDERR", ] assert mock_subprocess.mock_calls == [ @@ -289,22 +258,25 @@ def test_renew_from_kt_failed(self, mock_subprocess): @mock.patch("airflow.security.kerberos.open", mock.mock_open(read_data=b"X-CACHECONF:")) @mock.patch("airflow.security.kerberos.get_hostname", return_value="HOST") @mock.patch("time.sleep", return_value=None) - def test_renew_from_kt_failed_workaround(self, mock_sleep, mock_getfqdn, mock_subprocess): + def test_renew_from_kt_failed_workaround(self, mock_sleep, mock_getfqdn, mock_subprocess, caplog): mock_subprocess.Popen.return_value.__enter__.return_value.returncode = 0 mock_subprocess.call.return_value = 1 - with self.assertLogs(kerberos.log) as log_ctx, self.assertRaises(SystemExit): + with pytest.raises(SystemExit) as ctx: + caplog.clear() renew_from_kt(principal="test-principal", keytab="keytab") + assert ctx.value.code == 1 - assert log_ctx.output == [ - "INFO:airflow.security.kerberos:Re-initialising kerberos from keytab: " + log_records = [record for record in caplog.record_tuples if record[0] == kerberos.log.name] + assert len(log_records) == 3, log_records + assert [lr[1] for lr in log_records] == [logging.INFO, logging.INFO, logging.ERROR] + assert [lr[2] for lr in log_records] == [ + "Re-initialising kerberos from keytab: " "kinit -f -a -r 3600m -k -t keytab -c /tmp/airflow_krb5_ccache test-principal", - "INFO:airflow.security.kerberos:Renewing kerberos ticket to work around kerberos 1.8.1: " - "kinit -c /tmp/airflow_krb5_ccache -R", - "ERROR:airflow.security.kerberos:Couldn't renew kerberos ticket in order to work around " + "Renewing kerberos ticket to work around kerberos 1.8.1: kinit -c /tmp/airflow_krb5_ccache -R", + "Couldn't renew kerberos ticket in order to work around " "Kerberos 1.8.1 issue. Please check that the ticket for 'test-principal/HOST' is still " - "renewable:\n" - " $ kinit -f -c /tmp/airflow_krb5_ccache\n" + "renewable:\n $ kinit -f -c /tmp/airflow_krb5_ccache\n" "If the 'renew until' date is the same as the 'valid starting' date, the ticket cannot be " "renewed. Please check your KDC configuration, and the ticket renewal policy (maxrenewlife) for " "the 'test-principal/HOST' and `krbtgt' principals.", @@ -337,19 +309,21 @@ def test_renew_from_kt_failed_workaround(self, mock_sleep, mock_getfqdn, mock_su mock.call.call(["kinit", "-c", "/tmp/airflow_krb5_ccache", "-R"], close_fds=True), ] - def test_run_without_keytab(self): - with self.assertLogs(kerberos.log) as log_ctx, self.assertRaises(SystemExit): - kerberos.run(principal="test-principal", keytab=None) - assert log_ctx.output == [ - "WARNING:airflow.security.kerberos:Keytab renewer not starting, no keytab configured" - ] + def test_run_without_keytab(self, caplog): + with pytest.raises(SystemExit) as ctx: + with caplog.at_level(logging.WARNING, logger=kerberos.log.name): + caplog.clear() + kerberos.run(principal="test-principal", keytab=None) + assert ctx.value.code == 0 + assert caplog.messages == ["Keytab renewer not starting, no keytab configured"] @mock.patch("airflow.security.kerberos.renew_from_kt") @mock.patch("time.sleep", return_value=None) def test_run(self, mock_sleep, mock_renew_from_kt): mock_renew_from_kt.side_effect = [1, 1, SystemExit(42)] - with self.assertRaises(SystemExit): + with pytest.raises(SystemExit) as ctx: kerberos.run(principal="test-principal", keytab="/tmp/keytab") + assert ctx.value.code == 42 assert mock_renew_from_kt.mock_calls == [ mock.call("test-principal", "/tmp/keytab"), mock.call("test-principal", "/tmp/keytab"), From 6c011179c8b2195d4968886310608b2c129ea290 Mon Sep 17 00:00:00 2001 From: Andrey Anshin Date: Sat, 26 Nov 2022 04:09:15 +0400 Subject: [PATCH 152/158] Replace `unittests` in providers tests by pure `pytest` [Wave-3] (#27696) (cherry picked from commit 518fd8479a1b9bfe303e184672231b6bc84687db) --- tests/providers/airbyte/hooks/test_airbyte.py | 35 ++++--- .../airbyte/operators/test_airbyte.py | 3 +- .../providers/airbyte/sensors/test_airbyte.py | 3 +- .../providers/alibaba/cloud/hooks/test_oss.py | 5 +- .../cloud/log/test_oss_task_handler.py | 5 +- .../alibaba/cloud/operators/test_oss.py | 13 ++- .../alibaba/cloud/sensors/test_oss_key.py | 5 +- .../providers/arangodb/hooks/test_arangodb.py | 6 +- .../arangodb/operators/test_arangodb.py | 3 +- .../arangodb/sensors/test_arangodb.py | 5 +- .../asana/operators/test_asana_tasks.py | 5 +- .../atlassian/jira/hooks/test_jira.py | 5 +- .../atlassian/jira/operators/test_jira.py | 5 +- .../atlassian/jira/sensors/test_jira.py | 5 +- .../celery/sensors/test_celery_queue.py | 5 +- .../providers/cloudant/hooks/test_cloudant.py | 5 +- .../operators/test_spark_kubernetes.py | 5 +- .../sensors/test_spark_kubernetes.py | 5 +- .../providers/common/sql/hooks/test_dbapi.py | 7 +- .../common/sql/operators/test_sql.py | 29 +++--- .../providers/common/sql/sensors/test_sql.py | 11 +-- .../databricks/hooks/test_databricks.py | 37 ++++---- .../databricks/hooks/test_databricks_sql.py | 5 +- .../databricks/operators/test_databricks.py | 25 +++-- .../operators/test_databricks_repos.py | 7 +- .../operators/test_databricks_sql.py | 5 +- .../providers/databricks/utils/databricks.py | 6 +- tests/providers/datadog/hooks/test_datadog.py | 29 +++--- .../providers/datadog/sensors/test_datadog.py | 5 +- .../providers/dingding/hooks/test_dingding.py | 5 +- .../dingding/operators/test_dingding.py | 5 +- .../discord/hooks/test_discord_webhook.py | 5 +- .../discord/operators/test_discord_webhook.py | 6 +- tests/providers/docker/hooks/test_docker.py | 11 +-- .../providers/docker/operators/test_docker.py | 35 ++++--- .../docker/operators/test_docker_swarm.py | 8 +- .../elasticsearch/hooks/test_elasticsearch.py | 20 ++-- .../elasticsearch/log/test_es_task_handler.py | 4 +- tests/providers/exasol/hooks/test_exasol.py | 13 +-- .../providers/exasol/operators/test_exasol.py | 3 +- tests/providers/ftp/hooks/test_ftp.py | 14 +-- tests/providers/ftp/sensors/test_ftp.py | 3 +- tests/providers/grpc/hooks/test_grpc.py | 2 +- tests/providers/grpc/operators/test_grpc.py | 3 +- .../_internal_client/test_vault_client.py | 3 +- tests/providers/hashicorp/hooks/test_vault.py | 25 ++--- .../providers/hashicorp/secrets/test_vault.py | 4 +- tests/providers/http/hooks/test_http.py | 93 ++++++++----------- tests/providers/http/operators/test_http.py | 19 ++-- tests/providers/http/sensors/test_http.py | 5 +- tests/providers/imap/hooks/test_imap.py | 5 +- .../imap/sensors/test_imap_attachment.py | 11 +-- .../providers/influxdb/hooks/test_influxdb.py | 6 +- .../influxdb/operators/test_influxdb.py | 3 +- tests/providers/jdbc/hooks/test_jdbc.py | 2 +- tests/providers/jdbc/operators/test_jdbc.py | 5 +- tests/providers/jenkins/hooks/test_jenkins.py | 9 +- .../operators/test_jenkins_job_trigger.py | 77 ++++----------- .../providers/jenkins/sensors/test_jenkins.py | 12 +-- tests/providers/mongo/hooks/test_mongo.py | 44 +++------ tests/providers/mongo/sensors/test_mongo.py | 8 +- tests/providers/mysql/hooks/test_mysql.py | 56 ++++------- tests/providers/mysql/operators/test_mysql.py | 29 ++---- .../mysql/transfers/test_presto_to_mysql.py | 19 ++-- .../mysql/transfers/test_s3_to_mysql.py | 7 +- .../mysql/transfers/test_trino_to_mysql.py | 19 ++-- .../mysql/transfers/test_vertica_to_mysql.py | 5 +- tests/providers/neo4j/hooks/test_neo4j.py | 28 +++--- tests/providers/neo4j/operators/test_neo4j.py | 3 +- .../providers/openfaas/hooks/test_openfaas.py | 62 ++++++------- .../providers/opsgenie/hooks/test_opsgenie.py | 5 +- .../opsgenie/operators/test_opsgenie.py | 13 ++- tests/providers/oracle/hooks/test_oracle.py | 21 +---- .../providers/oracle/operators/test_oracle.py | 22 +++-- .../oracle/transfers/test_oracle_to_oracle.py | 6 +- .../papermill/operators/test_papermill.py | 3 +- .../providers/postgres/hooks/test_postgres.py | 28 ++---- .../postgres/operators/test_postgres.py | 8 +- tests/providers/presto/hooks/test_presto.py | 15 ++- .../presto/transfers/test_gcs_presto.py | 3 +- tests/providers/qubole/hooks/test_qubole.py | 4 +- .../qubole/hooks/test_qubole_check.py | 4 +- .../qubole/operators/test_qubole_check.py | 7 +- tests/providers/qubole/sensors/test_qubole.py | 5 +- tests/providers/redis/hooks/test_redis.py | 3 +- .../redis/operators/test_redis_publish.py | 5 +- .../providers/redis/sensors/test_redis_key.py | 6 +- .../redis/sensors/test_redis_pub_sub.py | 5 +- .../salesforce/hooks/test_salesforce.py | 3 +- .../operators/test_salesforce_apex_rest.py | 3 +- .../providers/salesforce/sensors/__init__.py | 16 ---- tests/providers/samba/hooks/test_samba.py | 14 +-- tests/providers/segment/hooks/test_segment.py | 7 +- .../operators/test_segment_track_event.py | 8 +- .../providers/sendgrid/utils/test_emailer.py | 5 +- tests/providers/sftp/hooks/test_sftp.py | 46 ++++----- tests/providers/sftp/sensors/test_sftp.py | 3 +- .../singularity/operators/test_singularity.py | 25 ++--- tests/providers/slack/operators/test_slack.py | 6 +- .../snowflake/hooks/test_snowflake.py | 63 +++++-------- .../snowflake/operators/test_snowflake.py | 6 +- .../transfers/test_copy_into_snowflake.py | 3 +- tests/providers/sqlite/hooks/test_sqlite.py | 9 +- .../providers/sqlite/operators/test_sqlite.py | 8 +- tests/providers/ssh/hooks/test_ssh.py | 15 ++- tests/providers/tableau/hooks/test_tableau.py | 23 ++--- .../tableau/operators/test_tableau.py | 9 +- .../providers/tableau/sensors/test_tableau.py | 16 ++-- .../providers/telegram/hooks/test_telegram.py | 5 +- .../telegram/operators/test_telegram.py | 5 +- tests/providers/trino/hooks/test_trino.py | 20 ++-- tests/providers/trino/operators/test_trino.py | 18 ++-- .../trino/transfers/test_gcs_trino.py | 3 +- tests/providers/vertica/hooks/test_vertica.py | 13 +-- .../vertica/operators/test_vertica.py | 3 +- .../yandex/hooks/test_yandexcloud_dataproc.py | 9 +- .../operators/test_yandexcloud_dataproc.py | 5 +- tests/test_utils/asserts.py | 20 +++- 118 files changed, 615 insertions(+), 922 deletions(-) delete mode 100644 tests/providers/salesforce/sensors/__init__.py diff --git a/tests/providers/airbyte/hooks/test_airbyte.py b/tests/providers/airbyte/hooks/test_airbyte.py index af265e1ed136c..271b531822581 100644 --- a/tests/providers/airbyte/hooks/test_airbyte.py +++ b/tests/providers/airbyte/hooks/test_airbyte.py @@ -17,11 +17,9 @@ # under the License. from __future__ import annotations -import unittest from unittest import mock import pytest -import requests_mock from airflow.exceptions import AirflowException from airflow.models import Connection @@ -29,7 +27,7 @@ from airflow.utils import db -class TestAirbyteHook(unittest.TestCase): +class TestAirbyteHook: """ Test all functions from Airbyte Hook """ @@ -46,7 +44,7 @@ class TestAirbyteHook(unittest.TestCase): _mock_job_status_success_response_body = {"job": {"status": "succeeded"}} _mock_job_cancel_status = "cancelled" - def setUp(self): + def setup_method(self): db.merge_conn( Connection( conn_id="airbyte_conn_id_test", conn_type="airbyte", host="http://test-airbyte", port=8001 @@ -59,25 +57,26 @@ def return_value_get_job(self, status): response.json.return_value = {"job": {"status": status}} return response - @requests_mock.mock() - def test_submit_sync_connection(self, m): - m.post( + def test_submit_sync_connection(self, requests_mock): + requests_mock.post( self.sync_connection_endpoint, status_code=200, json=self._mock_sync_conn_success_response_body ) resp = self.hook.submit_sync_connection(connection_id=self.connection_id) assert resp.status_code == 200 assert resp.json() == self._mock_sync_conn_success_response_body - @requests_mock.mock() - def test_get_job_status(self, m): - m.post(self.get_job_endpoint, status_code=200, json=self._mock_job_status_success_response_body) + def test_get_job_status(self, requests_mock): + requests_mock.post( + self.get_job_endpoint, status_code=200, json=self._mock_job_status_success_response_body + ) resp = self.hook.get_job(job_id=self.job_id) assert resp.status_code == 200 assert resp.json() == self._mock_job_status_success_response_body - @requests_mock.mock() - def test_cancel_job(self, m): - m.post(self.cancel_job_endpoint, status_code=200, json=self._mock_job_status_success_response_body) + def test_cancel_job(self, requests_mock): + requests_mock.post( + self.cancel_job_endpoint, status_code=200, json=self._mock_job_status_success_response_body + ) resp = self.hook.cancel_job(job_id=self.job_id) assert resp.status_code == 200 @@ -147,9 +146,8 @@ def test_wait_for_job_cancelled(self, mock_get_job): calls = [mock.call(job_id=self.job_id), mock.call(job_id=self.job_id)] mock_get_job.assert_has_calls(calls) - @requests_mock.mock() - def test_connection_success(self, m): - m.get( + def test_connection_success(self, requests_mock): + requests_mock.get( self.health_endpoint, status_code=200, ) @@ -158,9 +156,8 @@ def test_connection_success(self, m): assert status is True assert msg == "Connection successfully tested" - @requests_mock.mock() - def test_connection_failure(self, m): - m.get(self.health_endpoint, status_code=500, json={"message": "internal server error"}) + def test_connection_failure(self, requests_mock): + requests_mock.get(self.health_endpoint, status_code=500, json={"message": "internal server error"}) status, msg = self.hook.test_connection() assert status is False diff --git a/tests/providers/airbyte/operators/test_airbyte.py b/tests/providers/airbyte/operators/test_airbyte.py index d0f7fbe6b7462..f8ecd15615c8d 100644 --- a/tests/providers/airbyte/operators/test_airbyte.py +++ b/tests/providers/airbyte/operators/test_airbyte.py @@ -17,13 +17,12 @@ # under the License. from __future__ import annotations -import unittest from unittest import mock from airflow.providers.airbyte.operators.airbyte import AirbyteTriggerSyncOperator -class TestAirbyteTriggerSyncOp(unittest.TestCase): +class TestAirbyteTriggerSyncOp: """ Test execute function from Airbyte Operator """ diff --git a/tests/providers/airbyte/sensors/test_airbyte.py b/tests/providers/airbyte/sensors/test_airbyte.py index 31e2f17de0c65..f6fd5ef972f53 100644 --- a/tests/providers/airbyte/sensors/test_airbyte.py +++ b/tests/providers/airbyte/sensors/test_airbyte.py @@ -16,7 +16,6 @@ # under the License. from __future__ import annotations -import unittest from unittest import mock import pytest @@ -25,7 +24,7 @@ from airflow.providers.airbyte.sensors.airbyte import AirbyteJobSensor -class TestAirbyteJobSensor(unittest.TestCase): +class TestAirbyteJobSensor: task_id = "task-id" airbyte_conn_id = "airbyte-conn-test" diff --git a/tests/providers/alibaba/cloud/hooks/test_oss.py b/tests/providers/alibaba/cloud/hooks/test_oss.py index e03c0241fe842..1c47aa10c9741 100644 --- a/tests/providers/alibaba/cloud/hooks/test_oss.py +++ b/tests/providers/alibaba/cloud/hooks/test_oss.py @@ -17,7 +17,6 @@ # under the License. from __future__ import annotations -import unittest from unittest import mock from airflow.providers.alibaba.cloud.hooks.oss import OSSHook @@ -32,8 +31,8 @@ MOCK_FILE_PATH = "mock_file_path" -class TestOSSHook(unittest.TestCase): - def setUp(self): +class TestOSSHook: + def setup_method(self): with mock.patch( OSS_STRING.format("OSSHook.__init__"), new=mock_oss_hook_default_project_id, diff --git a/tests/providers/alibaba/cloud/log/test_oss_task_handler.py b/tests/providers/alibaba/cloud/log/test_oss_task_handler.py index e1d4cbf1c2066..2cf999849143f 100644 --- a/tests/providers/alibaba/cloud/log/test_oss_task_handler.py +++ b/tests/providers/alibaba/cloud/log/test_oss_task_handler.py @@ -17,7 +17,6 @@ # under the License. from __future__ import annotations -import unittest from unittest import mock from unittest.mock import PropertyMock @@ -32,8 +31,8 @@ MOCK_FILE_PATH = "mock_file_path" -class TestOSSTaskHandler(unittest.TestCase): - def setUp(self): +class TestOSSTaskHandler: + def setup_method(self): self.base_log_folder = "local/airflow/logs/1.log" self.oss_log_folder = f"oss://{MOCK_BUCKET_NAME}/airflow/logs" self.oss_task_handler = OSSTaskHandler(self.base_log_folder, self.oss_log_folder) diff --git a/tests/providers/alibaba/cloud/operators/test_oss.py b/tests/providers/alibaba/cloud/operators/test_oss.py index 0b42db1013dec..e95d5cbe34009 100644 --- a/tests/providers/alibaba/cloud/operators/test_oss.py +++ b/tests/providers/alibaba/cloud/operators/test_oss.py @@ -17,7 +17,6 @@ # under the License. from __future__ import annotations -import unittest from unittest import mock from airflow.providers.alibaba.cloud.operators.oss import ( @@ -38,7 +37,7 @@ MOCK_CONTENT = "mock_content" -class TestOSSCreateBucketOperator(unittest.TestCase): +class TestOSSCreateBucketOperator: @mock.patch("airflow.providers.alibaba.cloud.operators.oss.OSSHook") def test_execute(self, mock_hook): operator = OSSCreateBucketOperator( @@ -49,7 +48,7 @@ def test_execute(self, mock_hook): mock_hook.return_value.create_bucket.assert_called_once_with(bucket_name=MOCK_BUCKET) -class TestOSSDeleteBucketOperator(unittest.TestCase): +class TestOSSDeleteBucketOperator: @mock.patch("airflow.providers.alibaba.cloud.operators.oss.OSSHook") def test_execute(self, mock_hook): operator = OSSDeleteBucketOperator( @@ -60,7 +59,7 @@ def test_execute(self, mock_hook): mock_hook.return_value.delete_bucket.assert_called_once_with(bucket_name=MOCK_BUCKET) -class TestOSSUploadObjectOperator(unittest.TestCase): +class TestOSSUploadObjectOperator: @mock.patch("airflow.providers.alibaba.cloud.operators.oss.OSSHook") def test_execute(self, mock_hook): operator = OSSUploadObjectOperator( @@ -78,7 +77,7 @@ def test_execute(self, mock_hook): ) -class TestOSSDownloadObjectOperator(unittest.TestCase): +class TestOSSDownloadObjectOperator: @mock.patch("airflow.providers.alibaba.cloud.operators.oss.OSSHook") def test_execute(self, mock_hook): operator = OSSDownloadObjectOperator( @@ -96,7 +95,7 @@ def test_execute(self, mock_hook): ) -class TestOSSDeleteBatchObjectOperator(unittest.TestCase): +class TestOSSDeleteBatchObjectOperator: @mock.patch("airflow.providers.alibaba.cloud.operators.oss.OSSHook") def test_execute(self, mock_hook): operator = OSSDeleteBatchObjectOperator( @@ -111,7 +110,7 @@ def test_execute(self, mock_hook): mock_hook.return_value.delete_objects.assert_called_once_with(bucket_name=MOCK_BUCKET, key=MOCK_KEYS) -class TestOSSDeleteObjectOperator(unittest.TestCase): +class TestOSSDeleteObjectOperator: @mock.patch("airflow.providers.alibaba.cloud.operators.oss.OSSHook") def test_execute(self, mock_hook): operator = OSSDeleteObjectOperator( diff --git a/tests/providers/alibaba/cloud/sensors/test_oss_key.py b/tests/providers/alibaba/cloud/sensors/test_oss_key.py index e191598d565d2..4304f37a52fdb 100644 --- a/tests/providers/alibaba/cloud/sensors/test_oss_key.py +++ b/tests/providers/alibaba/cloud/sensors/test_oss_key.py @@ -17,7 +17,6 @@ # under the License. from __future__ import annotations -import unittest from unittest import mock from unittest.mock import PropertyMock @@ -33,8 +32,8 @@ MOCK_CONTENT = "mock_content" -class TestOSSKeySensor(unittest.TestCase): - def setUp(self): +class TestOSSKeySensor: + def setup_method(self): self.sensor = OSSKeySensor( bucket_key=MOCK_KEY, oss_conn_id=MOCK_OSS_CONN_ID, diff --git a/tests/providers/arangodb/hooks/test_arangodb.py b/tests/providers/arangodb/hooks/test_arangodb.py index 748cc22a28bec..707e8a59d20d9 100644 --- a/tests/providers/arangodb/hooks/test_arangodb.py +++ b/tests/providers/arangodb/hooks/test_arangodb.py @@ -16,7 +16,6 @@ # under the License. from __future__ import annotations -import unittest from unittest.mock import Mock, patch from airflow.models import Connection @@ -26,9 +25,8 @@ arangodb_client_mock = Mock(name="arangodb_client_for_test") -class TestArangoDBHook(unittest.TestCase): - def setUp(self): - super().setUp() +class TestArangoDBHook: + def setup_method(self): db.merge_conn( Connection( conn_id="arangodb_default", diff --git a/tests/providers/arangodb/operators/test_arangodb.py b/tests/providers/arangodb/operators/test_arangodb.py index 60f0bd76c83dd..190f5d9f5a23e 100644 --- a/tests/providers/arangodb/operators/test_arangodb.py +++ b/tests/providers/arangodb/operators/test_arangodb.py @@ -16,13 +16,12 @@ # under the License. from __future__ import annotations -import unittest from unittest import mock from airflow.providers.arangodb.operators.arangodb import AQLOperator -class TestAQLOperator(unittest.TestCase): +class TestAQLOperator: @mock.patch("airflow.providers.arangodb.operators.arangodb.ArangoDBHook") def test_arangodb_operator_test(self, mock_hook): diff --git a/tests/providers/arangodb/sensors/test_arangodb.py b/tests/providers/arangodb/sensors/test_arangodb.py index 95c948adc6856..5b9273b77439b 100644 --- a/tests/providers/arangodb/sensors/test_arangodb.py +++ b/tests/providers/arangodb/sensors/test_arangodb.py @@ -17,7 +17,6 @@ # under the License. from __future__ import annotations -import unittest from unittest.mock import Mock, patch from airflow.models import Connection @@ -29,8 +28,8 @@ arangodb_hook_mock = Mock(name="arangodb_hook_for_test", **{"query.return_value.count.return_value": 1}) -class TestAQLSensor(unittest.TestCase): - def setUp(self): +class TestAQLSensor: + def setup_method(self): args = {"owner": "airflow", "start_date": DEFAULT_DATE} dag = DAG("test_dag_id", default_args=args) self.dag = dag diff --git a/tests/providers/asana/operators/test_asana_tasks.py b/tests/providers/asana/operators/test_asana_tasks.py index 157e8c8e700d0..6b7c373d69fcb 100644 --- a/tests/providers/asana/operators/test_asana_tasks.py +++ b/tests/providers/asana/operators/test_asana_tasks.py @@ -16,7 +16,6 @@ # under the License. from __future__ import annotations -import unittest from unittest.mock import Mock, patch from airflow.models import Connection @@ -34,12 +33,12 @@ asana_client_mock = Mock(name="asana_client_for_test") -class TestAsanaTaskOperators(unittest.TestCase): +class TestAsanaTaskOperators: """ Test that the AsanaTaskOperators are using the python-asana methods as expected. """ - def setUp(self): + def setup_method(self): args = {"owner": "airflow", "start_date": DEFAULT_DATE} dag = DAG(TEST_DAG_ID, default_args=args) self.dag = dag diff --git a/tests/providers/atlassian/jira/hooks/test_jira.py b/tests/providers/atlassian/jira/hooks/test_jira.py index b5229fba67b3a..a8069357b4555 100644 --- a/tests/providers/atlassian/jira/hooks/test_jira.py +++ b/tests/providers/atlassian/jira/hooks/test_jira.py @@ -17,7 +17,6 @@ # under the License. from __future__ import annotations -import unittest from unittest.mock import Mock, patch from airflow.models import Connection @@ -27,8 +26,8 @@ jira_client_mock = Mock(name="jira_client") -class TestJiraHook(unittest.TestCase): - def setUp(self): +class TestJiraHook: + def setup_method(self): db.merge_conn( Connection( conn_id="jira_default", diff --git a/tests/providers/atlassian/jira/operators/test_jira.py b/tests/providers/atlassian/jira/operators/test_jira.py index c8a4aaf43cdf9..76db8a7d692c2 100644 --- a/tests/providers/atlassian/jira/operators/test_jira.py +++ b/tests/providers/atlassian/jira/operators/test_jira.py @@ -17,7 +17,6 @@ # under the License. from __future__ import annotations -import unittest from unittest.mock import Mock, patch from airflow.models import Connection @@ -39,8 +38,8 @@ } -class TestJiraOperator(unittest.TestCase): - def setUp(self): +class TestJiraOperator: + def setup_method(self): args = {"owner": "airflow", "start_date": DEFAULT_DATE} dag = DAG("test_dag_id", default_args=args) self.dag = dag diff --git a/tests/providers/atlassian/jira/sensors/test_jira.py b/tests/providers/atlassian/jira/sensors/test_jira.py index bf814f8c2796a..ecd63ab3acb7b 100644 --- a/tests/providers/atlassian/jira/sensors/test_jira.py +++ b/tests/providers/atlassian/jira/sensors/test_jira.py @@ -17,7 +17,6 @@ # under the License. from __future__ import annotations -import unittest from unittest.mock import Mock, patch from airflow.models import Connection @@ -46,8 +45,8 @@ class _TicketFields: ) -class TestJiraSensor(unittest.TestCase): - def setUp(self): +class TestJiraSensor: + def setup_method(self): args = {"owner": "airflow", "start_date": DEFAULT_DATE} dag = DAG("test_dag_id", default_args=args) self.dag = dag diff --git a/tests/providers/celery/sensors/test_celery_queue.py b/tests/providers/celery/sensors/test_celery_queue.py index 6faa08561ebc6..8d09085352adf 100644 --- a/tests/providers/celery/sensors/test_celery_queue.py +++ b/tests/providers/celery/sensors/test_celery_queue.py @@ -17,14 +17,13 @@ # under the License. from __future__ import annotations -import unittest from unittest.mock import patch from airflow.providers.celery.sensors.celery_queue import CeleryQueueSensor -class TestCeleryQueueSensor(unittest.TestCase): - def setUp(self): +class TestCeleryQueueSensor: + def setup_method(self): class TestCeleryqueueSensor(CeleryQueueSensor): def _check_task_id(self, context): return True diff --git a/tests/providers/cloudant/hooks/test_cloudant.py b/tests/providers/cloudant/hooks/test_cloudant.py index 04c527689bb82..22911cf1e0d30 100644 --- a/tests/providers/cloudant/hooks/test_cloudant.py +++ b/tests/providers/cloudant/hooks/test_cloudant.py @@ -17,7 +17,6 @@ # under the License. from __future__ import annotations -import unittest from unittest.mock import patch import pytest @@ -27,8 +26,8 @@ from airflow.providers.cloudant.hooks.cloudant import CloudantHook -class TestCloudantHook(unittest.TestCase): - def setUp(self): +class TestCloudantHook: + def setup_method(self): self.cloudant_hook = CloudantHook() @patch( diff --git a/tests/providers/cncf/kubernetes/operators/test_spark_kubernetes.py b/tests/providers/cncf/kubernetes/operators/test_spark_kubernetes.py index 113562e21a4b9..6989337a0b276 100644 --- a/tests/providers/cncf/kubernetes/operators/test_spark_kubernetes.py +++ b/tests/providers/cncf/kubernetes/operators/test_spark_kubernetes.py @@ -17,7 +17,6 @@ from __future__ import annotations import json -import unittest from unittest.mock import patch from airflow import DAG @@ -239,8 +238,8 @@ @patch("kubernetes.client.api.custom_objects_api.CustomObjectsApi.delete_namespaced_custom_object") @patch("kubernetes.client.api.custom_objects_api.CustomObjectsApi.create_namespaced_custom_object") @patch("airflow.utils.context.Context") -class TestSparkKubernetesOperator(unittest.TestCase): - def setUp(self): +class TestSparkKubernetesOperator: + def setup_method(self): db.merge_conn( Connection( conn_id="kubernetes_default_kube_config", diff --git a/tests/providers/cncf/kubernetes/sensors/test_spark_kubernetes.py b/tests/providers/cncf/kubernetes/sensors/test_spark_kubernetes.py index 1aa7d2d5ac92e..a45a1b8a75cb0 100644 --- a/tests/providers/cncf/kubernetes/sensors/test_spark_kubernetes.py +++ b/tests/providers/cncf/kubernetes/sensors/test_spark_kubernetes.py @@ -18,7 +18,6 @@ from __future__ import annotations import json -import unittest from unittest.mock import patch import pytest @@ -550,8 +549,8 @@ @patch("airflow.providers.cncf.kubernetes.hooks.kubernetes.KubernetesHook.get_conn") -class TestSparkKubernetesSensor(unittest.TestCase): - def setUp(self): +class TestSparkKubernetesSensor: + def setup_method(self): db.merge_conn(Connection(conn_id="kubernetes_default", conn_type="kubernetes", extra=json.dumps({}))) db.merge_conn( Connection( diff --git a/tests/providers/common/sql/hooks/test_dbapi.py b/tests/providers/common/sql/hooks/test_dbapi.py index beca713949203..45b40d8e48fdb 100644 --- a/tests/providers/common/sql/hooks/test_dbapi.py +++ b/tests/providers/common/sql/hooks/test_dbapi.py @@ -18,7 +18,6 @@ from __future__ import annotations import json -import unittest from unittest import mock import pytest @@ -36,10 +35,8 @@ class NonDbApiHook(BaseHook): pass -class TestDbApiHook(unittest.TestCase): - def setUp(self): - super().setUp() - +class TestDbApiHook: + def setup_method(self): self.cur = mock.MagicMock( rowcount=0, spec=["description", "rowcount", "execute", "fetchall", "fetchone", "close"] ) diff --git a/tests/providers/common/sql/operators/test_sql.py b/tests/providers/common/sql/operators/test_sql.py index 3741a93ed36ac..1770ed8f5e659 100644 --- a/tests/providers/common/sql/operators/test_sql.py +++ b/tests/providers/common/sql/operators/test_sql.py @@ -18,7 +18,6 @@ from __future__ import annotations import datetime -import unittest from unittest import mock from unittest.mock import MagicMock @@ -43,7 +42,6 @@ from airflow.utils import timezone from airflow.utils.session import create_session from airflow.utils.state import State -from tests.providers.apache.hive import TestHiveEnvironment class MockHook: @@ -55,7 +53,7 @@ def _get_mock_db_hook(): return MockHook() -class TestSQLExecuteQueryOperator(unittest.TestCase): +class TestSQLExecuteQueryOperator: def _construct_operator(self, sql, **kwargs): dag = DAG("test_dag", start_date=datetime.datetime(2017, 1, 1)) return SQLExecuteQueryOperator( @@ -481,8 +479,8 @@ def test_sql_operator_hook_params_biguery(self, mock_get_conn): assert self._operator._hook.location == "us-east1" -class TestCheckOperator(unittest.TestCase): - def setUp(self): +class TestCheckOperator: + def setup_method(self): self._operator = SQLCheckOperator(task_id="test_task", sql="sql", parameters="parameters") @mock.patch.object(SQLCheckOperator, "get_db_hook") @@ -505,8 +503,8 @@ def test_sqlcheckoperator_parameters(self, mock_get_db_hook): mock_get_db_hook.return_value.get_first.assert_called_once_with("sql", "parameters") -class TestValueCheckOperator(unittest.TestCase): - def setUp(self): +class TestValueCheckOperator: + def setup_method(self): self.task_id = "test_task" self.conn_id = "default_conn" @@ -564,7 +562,7 @@ def test_execute_fail(self, mock_get_db_hook): operator.execute(context=MagicMock()) -class TestIntervalCheckOperator(unittest.TestCase): +class TestIntervalCheckOperator: def _construct_operator(self, table, metric_thresholds, ratio_formula, ignore_zero): return SQLIntervalCheckOperator( task_id="test_task", @@ -681,7 +679,7 @@ def returned_row(): operator.execute(context=MagicMock()) -class TestThresholdCheckOperator(unittest.TestCase): +class TestThresholdCheckOperator: def _construct_operator(self, sql, min_threshold, max_threshold): dag = DAG("test_dag", start_date=datetime.datetime(2017, 1, 1)) @@ -757,22 +755,19 @@ def test_fail_min_sql_max_value(self, mock_get_db_hook): operator.execute(context=MagicMock()) -class TestSqlBranch(TestHiveEnvironment, unittest.TestCase): +class TestSqlBranch: """ Test for SQL Branch Operator """ @classmethod - def setUpClass(cls): - super().setUpClass() - + def setup_class(cls): with create_session() as session: session.query(DagRun).delete() session.query(TI).delete() session.query(XCom).delete() - def setUp(self): - super().setUp() + def setup_method(self): self.dag = DAG( "sql_branch_operator_test", default_args={"owner": "airflow", "start_date": DEFAULT_DATE}, @@ -782,9 +777,7 @@ def setUp(self): self.branch_2 = EmptyOperator(task_id="branch_2", dag=self.dag) self.branch_3 = None - def tearDown(self): - super().tearDown() - + def teardown_method(self): with create_session() as session: session.query(DagRun).delete() session.query(TI).delete() diff --git a/tests/providers/common/sql/sensors/test_sql.py b/tests/providers/common/sql/sensors/test_sql.py index 77665f1c84a12..912478e16d462 100644 --- a/tests/providers/common/sql/sensors/test_sql.py +++ b/tests/providers/common/sql/sensors/test_sql.py @@ -18,7 +18,6 @@ from __future__ import annotations import os -import unittest from unittest import mock import pytest @@ -28,15 +27,13 @@ from airflow.providers.common.sql.hooks.sql import DbApiHook from airflow.providers.common.sql.sensors.sql import SqlSensor from airflow.utils.timezone import datetime -from tests.providers.apache.hive import TestHiveEnvironment DEFAULT_DATE = datetime(2015, 1, 1) TEST_DAG_ID = "unit_test_sql_dag" -class TestSqlSensor(TestHiveEnvironment): - def setUp(self): - super().setUp() +class TestSqlSensor: + def setup_method(self): args = {"owner": "airflow", "start_date": DEFAULT_DATE} self.dag = DAG(TEST_DAG_ID, default_args=args) @@ -245,8 +242,8 @@ def test_sql_sensor_postgres_poke_invalid_success(self, mock_hook): op.poke(None) assert "self.success is present, but not callable -> [1]" == str(ctx.value) - @unittest.skipIf( - "AIRFLOW_RUNALL_TESTS" not in os.environ, "Skipped because AIRFLOW_RUNALL_TESTS is not set" + @pytest.mark.skipif( + "AIRFLOW_RUNALL_TESTS" not in os.environ, reason="Skipped because AIRFLOW_RUNALL_TESTS is not set" ) def test_sql_sensor_presto(self): op = SqlSensor( diff --git a/tests/providers/databricks/hooks/test_databricks.py b/tests/providers/databricks/hooks/test_databricks.py index b77b8da7ba922..895be832b4cd0 100644 --- a/tests/providers/databricks/hooks/test_databricks.py +++ b/tests/providers/databricks/hooks/test_databricks.py @@ -21,7 +21,6 @@ import json import sys import time -import unittest import aiohttp import pytest @@ -228,13 +227,13 @@ def setup_mock_requests(mock_requests, exception, status_code=500, error_count=N ] -class TestDatabricksHook(unittest.TestCase): +class TestDatabricksHook: """ Tests for DatabricksHook. """ @provide_session - def setUp(self, session=None): + def setup_method(self, method, session=None): conn = session.query(Connection).filter(Connection.conn_id == DEFAULT_CONN_ID).first() conn.host = HOST conn.login = LOGIN @@ -616,11 +615,11 @@ def test_uninstall_libs_on_cluster(self, mock_requests): def test_is_aad_token_valid_returns_true(self): aad_token = {"token": "my_token", "expires_on": int(time.time()) + TOKEN_REFRESH_LEAD_TIME + 10} - self.assertTrue(self.hook._is_aad_token_valid(aad_token)) + assert self.hook._is_aad_token_valid(aad_token) def test_is_aad_token_valid_returns_false(self): aad_token = {"token": "my_token", "expires_on": int(time.time())} - self.assertFalse(self.hook._is_aad_token_valid(aad_token)) + assert not self.hook._is_aad_token_valid(aad_token) @mock.patch("airflow.providers.databricks.hooks.databricks_base.requests") def test_list_jobs_success_single_page(self, mock_requests): @@ -756,13 +755,13 @@ def test_connection_failure(self, mock_requests): ) -class TestDatabricksHookToken(unittest.TestCase): +class TestDatabricksHookToken: """ Tests for DatabricksHook when auth is done with token. """ @provide_session - def setUp(self, session=None): + def setup_method(self, method, session=None): conn = session.query(Connection).filter(Connection.conn_id == DEFAULT_CONN_ID).first() conn.extra = json.dumps({"token": TOKEN, "host": HOST}) @@ -785,13 +784,13 @@ def test_submit_run(self, mock_requests): assert kwargs["auth"].token == TOKEN -class TestDatabricksHookTokenInPassword(unittest.TestCase): +class TestDatabricksHookTokenInPassword: """ Tests for DatabricksHook. """ @provide_session - def setUp(self, session=None): + def setup_method(self, method, session=None): conn = session.query(Connection).filter(Connection.conn_id == DEFAULT_CONN_ID).first() conn.host = HOST conn.login = None @@ -818,7 +817,7 @@ def test_submit_run(self, mock_requests): class TestDatabricksHookTokenWhenNoHostIsProvidedInExtra(TestDatabricksHookToken): @provide_session - def setUp(self, session=None): + def setup_method(self, method, session=None): conn = session.query(Connection).filter(Connection.conn_id == DEFAULT_CONN_ID).first() conn.extra = json.dumps({"token": TOKEN}) @@ -827,7 +826,7 @@ def setUp(self, session=None): self.hook = DatabricksHook() -class TestRunState(unittest.TestCase): +class TestRunState: def test_is_terminal_true(self): terminal_states = ["TERMINATED", "SKIPPED", "INTERNAL_ERROR"] for state in terminal_states: @@ -874,13 +873,13 @@ def create_aad_token_for_resource(resource: str) -> dict: } -class TestDatabricksHookAadToken(unittest.TestCase): +class TestDatabricksHookAadToken: """ Tests for DatabricksHook when auth is done with AAD token for SP as user inside workspace. """ @provide_session - def setUp(self, session=None): + def setup_method(self, method, session=None): conn = session.query(Connection).filter(Connection.conn_id == DEFAULT_CONN_ID).first() conn.login = "9ff815a6-4404-4ab8-85cb-cd0e6f879c1d" conn.password = "secret" @@ -911,14 +910,14 @@ def test_submit_run(self, mock_requests): assert kwargs["auth"].token == TOKEN -class TestDatabricksHookAadTokenOtherClouds(unittest.TestCase): +class TestDatabricksHookAadTokenOtherClouds: """ Tests for DatabricksHook when auth is done with AAD token for SP as user inside workspace and using non-global Azure cloud (China, GovCloud, Germany) """ @provide_session - def setUp(self, session=None): + def setup_method(self, method, session=None): self.tenant_id = "3ff810a6-5504-4ab8-85cb-cd0e6f879c1d" self.ad_endpoint = "https://login.microsoftonline.de" self.client_id = "9ff815a6-4404-4ab8-85cb-cd0e6f879c1d" @@ -958,13 +957,13 @@ def test_submit_run(self, mock_requests): assert kwargs["auth"].token == TOKEN -class TestDatabricksHookAadTokenSpOutside(unittest.TestCase): +class TestDatabricksHookAadTokenSpOutside: """ Tests for DatabricksHook when auth is done with AAD token for SP outside of workspace. """ @provide_session - def setUp(self, session=None): + def setup_method(self, method, session=None): conn = session.query(Connection).filter(Connection.conn_id == DEFAULT_CONN_ID).first() self.tenant_id = "3ff810a6-5504-4ab8-85cb-cd0e6f879c1d" self.client_id = "9ff815a6-4404-4ab8-85cb-cd0e6f879c1d" @@ -1011,13 +1010,13 @@ def test_submit_run(self, mock_requests): assert kwargs["headers"]["X-Databricks-Azure-SP-Management-Token"] == TOKEN -class TestDatabricksHookAadTokenManagedIdentity(unittest.TestCase): +class TestDatabricksHookAadTokenManagedIdentity: """ Tests for DatabricksHook when auth is done with AAD leveraging Managed Identity authentication """ @provide_session - def setUp(self, session=None): + def setup_method(self, method, session=None): conn = session.query(Connection).filter(Connection.conn_id == DEFAULT_CONN_ID).first() conn.host = HOST conn.extra = json.dumps( diff --git a/tests/providers/databricks/hooks/test_databricks_sql.py b/tests/providers/databricks/hooks/test_databricks_sql.py index 7f1aaf493a4f0..673ac0bdddb4f 100644 --- a/tests/providers/databricks/hooks/test_databricks_sql.py +++ b/tests/providers/databricks/hooks/test_databricks_sql.py @@ -18,7 +18,6 @@ # from __future__ import annotations -import unittest from unittest import mock import pytest @@ -35,13 +34,13 @@ TOKEN = "token" -class TestDatabricksSqlHookQueryByName(unittest.TestCase): +class TestDatabricksSqlHookQueryByName: """ Tests for DatabricksHook. """ @provide_session - def setUp(self, session=None): + def setup_method(self, method, session=None): conn = session.query(Connection).filter(Connection.conn_id == DEFAULT_CONN_ID).first() conn.host = HOST conn.login = None diff --git a/tests/providers/databricks/operators/test_databricks.py b/tests/providers/databricks/operators/test_databricks.py index c3bca253a6e05..5236e67aa7aa5 100644 --- a/tests/providers/databricks/operators/test_databricks.py +++ b/tests/providers/databricks/operators/test_databricks.py @@ -17,7 +17,6 @@ # under the License. from __future__ import annotations -import unittest from datetime import datetime from unittest import mock from unittest.mock import MagicMock @@ -94,7 +93,7 @@ def make_run_with_state_mock( ) -class TestDatabricksSubmitRunOperator(unittest.TestCase): +class TestDatabricksSubmitRunOperator: def test_init_with_notebook_task_named_parameters(self): """ Test the initializer with the named parameters. @@ -437,7 +436,7 @@ def test_no_wait_for_termination(self, db_mock_class): db_mock.get_run.assert_not_called() -class TestDatabricksSubmitRunDeferrableOperator(unittest.TestCase): +class TestDatabricksSubmitRunDeferrableOperator: @mock.patch("airflow.providers.databricks.operators.databricks.DatabricksHook") def test_execute_task_deferred(self, db_mock_class): """ @@ -454,8 +453,8 @@ def test_execute_task_deferred(self, db_mock_class): with pytest.raises(TaskDeferred) as exc: op.execute(None) - self.assertTrue(isinstance(exc.value.trigger, DatabricksExecutionTrigger)) - self.assertEqual(exc.value.method_name, "execute_complete") + assert isinstance(exc.value.trigger, DatabricksExecutionTrigger) + assert exc.value.method_name == "execute_complete" expected = utils.normalise_json_content( {"new_cluster": NEW_CLUSTER, "notebook_task": NOTEBOOK_TASK, "run_name": TASK_ID} @@ -470,7 +469,7 @@ def test_execute_task_deferred(self, db_mock_class): db_mock.submit_run.assert_called_once_with(expected) db_mock.get_run_page_url.assert_called_once_with(RUN_ID) - self.assertEqual(RUN_ID, op.run_id) + assert op.run_id == RUN_ID def test_execute_complete_success(self): """ @@ -487,7 +486,7 @@ def test_execute_complete_success(self): } op = DatabricksSubmitRunDeferrableOperator(task_id=TASK_ID, json=run) - self.assertIsNone(op.execute_complete(context=None, event=event)) + assert op.execute_complete(context=None, event=event) is None @mock.patch("airflow.providers.databricks.operators.databricks.DatabricksHook") def test_execute_complete_failure(self, db_mock_class): @@ -523,7 +522,7 @@ def test_execute_complete_incorrect_event_validation_failure(self): op.execute_complete(context=None, event=event) -class TestDatabricksRunNowOperator(unittest.TestCase): +class TestDatabricksRunNowOperator: def test_init_with_named_parameters(self): """ Test the initializer with the named parameters. @@ -874,7 +873,7 @@ def test_exec_failure_if_job_id_not_found(self, db_mock_class): db_mock.find_job_id_by_name.assert_called_once_with(JOB_NAME) -class TestDatabricksRunNowDeferrableOperator(unittest.TestCase): +class TestDatabricksRunNowDeferrableOperator: @mock.patch("airflow.providers.databricks.operators.databricks.DatabricksHook") def test_execute_task_deferred(self, db_mock_class): """ @@ -888,8 +887,8 @@ def test_execute_task_deferred(self, db_mock_class): with pytest.raises(TaskDeferred) as exc: op.execute(None) - self.assertTrue(isinstance(exc.value.trigger, DatabricksExecutionTrigger)) - self.assertEqual(exc.value.method_name, "execute_complete") + assert isinstance(exc.value.trigger, DatabricksExecutionTrigger) + assert exc.value.method_name == "execute_complete" expected = utils.normalise_json_content( { @@ -910,7 +909,7 @@ def test_execute_task_deferred(self, db_mock_class): db_mock.run_now.assert_called_once_with(expected) db_mock.get_run_page_url.assert_called_once_with(RUN_ID) - self.assertEqual(RUN_ID, op.run_id) + assert op.run_id == RUN_ID def test_execute_complete_success(self): """ @@ -924,7 +923,7 @@ def test_execute_complete_success(self): } op = DatabricksRunNowDeferrableOperator(task_id=TASK_ID, job_id=JOB_ID, json=run) - self.assertIsNone(op.execute_complete(context=None, event=event)) + assert op.execute_complete(context=None, event=event) is None @mock.patch("airflow.providers.databricks.operators.databricks.DatabricksHook") def test_execute_complete_failure(self, db_mock_class): diff --git a/tests/providers/databricks/operators/test_databricks_repos.py b/tests/providers/databricks/operators/test_databricks_repos.py index 3c593b82fdd30..9e8770d3d9653 100644 --- a/tests/providers/databricks/operators/test_databricks_repos.py +++ b/tests/providers/databricks/operators/test_databricks_repos.py @@ -17,7 +17,6 @@ # under the License. from __future__ import annotations -import unittest from unittest import mock import pytest @@ -33,7 +32,7 @@ DEFAULT_CONN_ID = "databricks_default" -class TestDatabricksReposUpdateOperator(unittest.TestCase): +class TestDatabricksReposUpdateOperator: @mock.patch("airflow.providers.databricks.operators.databricks_repos.DatabricksHook") def test_update_with_id(self, db_mock_class): """ @@ -98,7 +97,7 @@ def test_init_exception(self): DatabricksReposUpdateOperator(task_id=TASK_ID, repo_id="123") -class TestDatabricksReposDeleteOperator(unittest.TestCase): +class TestDatabricksReposDeleteOperator: @mock.patch("airflow.providers.databricks.operators.databricks_repos.DatabricksHook") def test_delete_with_id(self, db_mock_class): """ @@ -153,7 +152,7 @@ def test_init_exception(self): DatabricksReposDeleteOperator(task_id=TASK_ID) -class TestDatabricksReposCreateOperator(unittest.TestCase): +class TestDatabricksReposCreateOperator: @mock.patch("airflow.providers.databricks.operators.databricks_repos.DatabricksHook") def test_create_plus_checkout(self, db_mock_class): """ diff --git a/tests/providers/databricks/operators/test_databricks_sql.py b/tests/providers/databricks/operators/test_databricks_sql.py index 9a989dfae368a..2663deeec20fb 100644 --- a/tests/providers/databricks/operators/test_databricks_sql.py +++ b/tests/providers/databricks/operators/test_databricks_sql.py @@ -19,7 +19,6 @@ import os import tempfile -import unittest from unittest import mock import pytest @@ -38,7 +37,7 @@ COPY_FILE_LOCATION = "s3://my-bucket/jsonData" -class TestDatabricksSqlOperator(unittest.TestCase): +class TestDatabricksSqlOperator: @mock.patch("airflow.providers.databricks.operators.databricks_sql.DatabricksSqlHook") def test_exec_success(self, db_mock_class): """ @@ -117,7 +116,7 @@ def test_exec_write_file(self, db_mock_class): ) -class TestDatabricksSqlCopyIntoOperator(unittest.TestCase): +class TestDatabricksSqlCopyIntoOperator: def test_copy_with_files(self): op = DatabricksCopyIntoOperator( file_location=COPY_FILE_LOCATION, diff --git a/tests/providers/databricks/utils/databricks.py b/tests/providers/databricks/utils/databricks.py index b918a2b0f2c63..7619bcb8ad07f 100644 --- a/tests/providers/databricks/utils/databricks.py +++ b/tests/providers/databricks/utils/databricks.py @@ -17,8 +17,6 @@ # under the License. from __future__ import annotations -import unittest - import pytest from airflow.exceptions import AirflowException @@ -29,7 +27,7 @@ RUN_PAGE_URL = "run-page-url" -class TestDatabricksOperatorSharedFunctions(unittest.TestCase): +class TestDatabricksOperatorSharedFunctions: def test_normalise_json_content(self): test_json = { "test_bool": True, @@ -56,7 +54,7 @@ def test_validate_trigger_event_success(self): "run_page_url": RUN_PAGE_URL, "run_state": RunState("TERMINATED", "SUCCESS", "").to_json(), } - self.assertIsNone(validate_trigger_event(event)) + assert validate_trigger_event(event) is None def test_validate_trigger_event_failure(self): event = {} diff --git a/tests/providers/datadog/hooks/test_datadog.py b/tests/providers/datadog/hooks/test_datadog.py index 5b026ff1c0bbd..b5a4d5aaf2290 100644 --- a/tests/providers/datadog/hooks/test_datadog.py +++ b/tests/providers/datadog/hooks/test_datadog.py @@ -18,7 +18,6 @@ from __future__ import annotations import json -import unittest from unittest import mock import pytest @@ -46,20 +45,20 @@ DEVICE_NAME = "device-name" -class TestDatadogHook(unittest.TestCase): - @mock.patch("airflow.providers.datadog.hooks.datadog.initialize") - @mock.patch("airflow.providers.datadog.hooks.datadog.DatadogHook.get_connection") - def setUp(self, mock_get_connection, mock_initialize): - mock_get_connection.return_value = Connection( - extra=json.dumps( - { - "app_key": APP_KEY, - "api_key": API_KEY, - "api_host": API_HOST, - } - ) - ) - self.hook = DatadogHook() +class TestDatadogHook: + def setup_method(self): + with mock.patch("airflow.providers.datadog.hooks.datadog.initialize"): + with mock.patch("airflow.providers.datadog.hooks.datadog.DatadogHook.get_connection") as m: + m.return_value = Connection( + extra=json.dumps( + { + "app_key": APP_KEY, + "api_key": API_KEY, + "api_host": API_HOST, + } + ) + ) + self.hook = DatadogHook() @mock.patch("airflow.providers.datadog.hooks.datadog.initialize") @mock.patch("airflow.providers.datadog.hooks.datadog.DatadogHook.get_connection") diff --git a/tests/providers/datadog/sensors/test_datadog.py b/tests/providers/datadog/sensors/test_datadog.py index ae33cc136d233..e8c146f1c955d 100644 --- a/tests/providers/datadog/sensors/test_datadog.py +++ b/tests/providers/datadog/sensors/test_datadog.py @@ -18,7 +18,6 @@ from __future__ import annotations import json -import unittest from unittest.mock import patch from airflow.models import Connection @@ -63,8 +62,8 @@ zero_events: list = [] -class TestDatadogSensor(unittest.TestCase): - def setUp(self): +class TestDatadogSensor: + def setup_method(self): db.merge_conn( Connection( conn_id="datadog_default", diff --git a/tests/providers/dingding/hooks/test_dingding.py b/tests/providers/dingding/hooks/test_dingding.py index 202c74a410a5f..745fc65bf5397 100644 --- a/tests/providers/dingding/hooks/test_dingding.py +++ b/tests/providers/dingding/hooks/test_dingding.py @@ -18,7 +18,6 @@ from __future__ import annotations import json -import unittest import pytest @@ -27,10 +26,10 @@ from airflow.utils import db -class TestDingdingHook(unittest.TestCase): +class TestDingdingHook: conn_id = "dingding_conn_id_test" - def setUp(self): + def setup_method(self): db.merge_conn( Connection( conn_id=self.conn_id, diff --git a/tests/providers/dingding/operators/test_dingding.py b/tests/providers/dingding/operators/test_dingding.py index 3858316ef9f66..d2b25c242e2ce 100644 --- a/tests/providers/dingding/operators/test_dingding.py +++ b/tests/providers/dingding/operators/test_dingding.py @@ -17,7 +17,6 @@ # under the License. from __future__ import annotations -import unittest from unittest import mock from airflow.models.dag import DAG @@ -27,7 +26,7 @@ DEFAULT_DATE = timezone.datetime(2017, 1, 1) -class TestDingdingOperator(unittest.TestCase): +class TestDingdingOperator: _config = { "dingding_conn_id": "dingding_default", "message_type": "text", @@ -36,7 +35,7 @@ class TestDingdingOperator(unittest.TestCase): "at_all": False, } - def setUp(self): + def setup_method(self): args = {"owner": "airflow", "start_date": DEFAULT_DATE} self.dag = DAG("test_dag_id", default_args=args) diff --git a/tests/providers/discord/hooks/test_discord_webhook.py b/tests/providers/discord/hooks/test_discord_webhook.py index 40dab6ec73a25..d1e0406ec87f4 100644 --- a/tests/providers/discord/hooks/test_discord_webhook.py +++ b/tests/providers/discord/hooks/test_discord_webhook.py @@ -18,7 +18,6 @@ from __future__ import annotations import json -import unittest import pytest @@ -28,7 +27,7 @@ from airflow.utils import db -class TestDiscordWebhookHook(unittest.TestCase): +class TestDiscordWebhookHook: _config = { "http_conn_id": "default-discord-webhook", @@ -49,7 +48,7 @@ class TestDiscordWebhookHook(unittest.TestCase): expected_payload = json.dumps(expected_payload_dict) - def setUp(self): + def setup_method(self): db.merge_conn( Connection( conn_id="default-discord-webhook", diff --git a/tests/providers/discord/operators/test_discord_webhook.py b/tests/providers/discord/operators/test_discord_webhook.py index 3e7b2352908d7..27cbe7d6d6532 100644 --- a/tests/providers/discord/operators/test_discord_webhook.py +++ b/tests/providers/discord/operators/test_discord_webhook.py @@ -17,8 +17,6 @@ # under the License. from __future__ import annotations -import unittest - from airflow.models.dag import DAG from airflow.providers.discord.operators.discord_webhook import DiscordWebhookOperator from airflow.utils import timezone @@ -26,7 +24,7 @@ DEFAULT_DATE = timezone.datetime(2018, 1, 1) -class TestDiscordWebhookOperator(unittest.TestCase): +class TestDiscordWebhookOperator: _config = { "http_conn_id": "discord-webhook-default", "webhook_endpoint": "webhooks/11111/some-discord-token_111", @@ -37,7 +35,7 @@ class TestDiscordWebhookOperator(unittest.TestCase): "proxy": "https://proxy.proxy.com:8888", } - def setUp(self): + def setup_method(self): args = {"owner": "airflow", "start_date": DEFAULT_DATE} self.dag = DAG("test_dag_id", default_args=args) diff --git a/tests/providers/docker/hooks/test_docker.py b/tests/providers/docker/hooks/test_docker.py index 18c4c27f1ccf1..b7d24aa908a99 100644 --- a/tests/providers/docker/hooks/test_docker.py +++ b/tests/providers/docker/hooks/test_docker.py @@ -17,24 +17,19 @@ # under the License. from __future__ import annotations -import unittest from unittest import mock import pytest from airflow.exceptions import AirflowException from airflow.models import Connection +from airflow.providers.docker.hooks.docker import DockerHook from airflow.utils import db -try: - from airflow.providers.docker.hooks.docker import DockerHook -except ImportError: - pass - @mock.patch("airflow.providers.docker.hooks.docker.APIClient", autospec=True) -class TestDockerHook(unittest.TestCase): - def setUp(self): +class TestDockerHook: + def setup_method(self): db.merge_conn( Connection( conn_id="docker_default", diff --git a/tests/providers/docker/operators/test_docker.py b/tests/providers/docker/operators/test_docker.py index c3506eed3b841..f17c6a01ebc0b 100644 --- a/tests/providers/docker/operators/test_docker.py +++ b/tests/providers/docker/operators/test_docker.py @@ -18,31 +18,24 @@ from __future__ import annotations import logging -import unittest from unittest import mock from unittest.mock import call import pytest +from docker import APIClient from docker.constants import DEFAULT_TIMEOUT_SECONDS from docker.errors import APIError +from docker.types import DeviceRequest, LogConfig, Mount from airflow.exceptions import AirflowException - -try: - from docker import APIClient - from docker.types import DeviceRequest, LogConfig, Mount - - from airflow.providers.docker.hooks.docker import DockerHook - from airflow.providers.docker.operators.docker import DockerOperator -except ImportError: - pass - +from airflow.providers.docker.hooks.docker import DockerHook +from airflow.providers.docker.operators.docker import DockerOperator TEMPDIR_MOCK_RETURN_VALUE = "/mkdtemp" -class TestDockerOperator(unittest.TestCase): - def setUp(self): +class TestDockerOperator: + def setup_method(self): self.tempdir_patcher = mock.patch("airflow.providers.docker.operators.docker.TemporaryDirectory") self.tempdir_mock = self.tempdir_patcher.start() self.tempdir_mock.return_value.__enter__.return_value = TEMPDIR_MOCK_RETURN_VALUE @@ -81,7 +74,7 @@ def dotenv_mock_return_value(**kwargs): self.dotenv_mock = self.dotenv_patcher.start() self.dotenv_mock.side_effect = dotenv_mock_return_value - def tearDown(self) -> None: + def teardown_method(self) -> None: self.tempdir_patcher.stop() self.client_class_patcher.stop() self.dotenv_patcher.stop() @@ -241,7 +234,7 @@ def test_execute_no_temp_dir(self): self.dotenv_mock.assert_called_once_with(stream="ENV=FILE\nVAR=VALUE") stringio_patcher.stop() - def test_execute_fallback_temp_dir(self): + def test_execute_fallback_temp_dir(self, caplog): self.client_mock.create_container.side_effect = [ APIError(message="wrong path: " + TEMPDIR_MOCK_RETURN_VALUE), {"Id": "some_id"}, @@ -270,12 +263,16 @@ def test_execute_fallback_temp_dir(self): container_name="test_container", tty=True, ) - with self.assertLogs(operator.log, level=logging.WARNING) as captured: + caplog.clear() + with caplog.at_level(logging.WARNING, logger=operator.log.name): operator.execute(None) - assert ( - "WARNING:airflow.task.operators:Using remote engine or docker-in-docker " - "and mounting temporary volume from host is not supported" in captured.output[0] + warning_message = ( + "Using remote engine or docker-in-docker and mounting temporary volume from host " + "is not supported. Falling back to `mount_tmp_dir=False` mode. " + "You can set `mount_tmp_dir` parameter to False to disable mounting and remove the warning" ) + assert warning_message in caplog.messages + self.client_class_mock.assert_called_once_with( base_url="unix://var/run/docker.sock", tls=None, version="1.19", timeout=DEFAULT_TIMEOUT_SECONDS ) diff --git a/tests/providers/docker/operators/test_docker_swarm.py b/tests/providers/docker/operators/test_docker_swarm.py index ed7217b4eeb0a..d12be721f3652 100644 --- a/tests/providers/docker/operators/test_docker_swarm.py +++ b/tests/providers/docker/operators/test_docker_swarm.py @@ -17,19 +17,17 @@ # under the License. from __future__ import annotations -import unittest from unittest import mock import pytest from docker import APIClient, types from docker.constants import DEFAULT_TIMEOUT_SECONDS -from parameterized import parameterized from airflow.exceptions import AirflowException from airflow.providers.docker.operators.docker_swarm import DockerSwarmOperator -class TestDockerSwarmOperator(unittest.TestCase): +class TestDockerSwarmOperator: @mock.patch("airflow.providers.docker.operators.docker.APIClient") @mock.patch("airflow.providers.docker.operators.docker_swarm.types") def test_execute(self, types_mock, client_class_mock): @@ -162,10 +160,10 @@ def test_no_auto_remove(self, types_mock, client_class_mock): client_mock.remove_service.call_count == 0 ), "Docker service being removed even when `auto_remove` set to `False`" - @parameterized.expand([("failed",), ("shutdown",), ("rejected",), ("orphaned",), ("remove",)]) + @pytest.mark.parametrize("status", ["failed", "shutdown", "rejected", "orphaned", "remove"]) @mock.patch("airflow.providers.docker.operators.docker.APIClient") @mock.patch("airflow.providers.docker.operators.docker_swarm.types") - def test_non_complete_service_raises_error(self, status, types_mock, client_class_mock): + def test_non_complete_service_raises_error(self, types_mock, client_class_mock, status): mock_obj = mock.Mock() diff --git a/tests/providers/elasticsearch/hooks/test_elasticsearch.py b/tests/providers/elasticsearch/hooks/test_elasticsearch.py index b853b18f42902..a3bf9bcde26a5 100644 --- a/tests/providers/elasticsearch/hooks/test_elasticsearch.py +++ b/tests/providers/elasticsearch/hooks/test_elasticsearch.py @@ -17,9 +17,9 @@ # under the License. from __future__ import annotations -import unittest from unittest import mock +import pytest from elasticsearch import Elasticsearch from airflow.models import Connection @@ -30,7 +30,7 @@ ) -class TestElasticsearchHook(unittest.TestCase): +class TestElasticsearchHook: def test_throws_warning(self): self.cur = mock.MagicMock(rowcount=0) self.conn = mock.MagicMock() @@ -38,7 +38,7 @@ def test_throws_warning(self): conn = self.conn self.connection = Connection(host="localhost", port=9200, schema="http") - with self.assertWarns(DeprecationWarning): + with pytest.warns(DeprecationWarning): class UnitTestElasticsearchHook(ElasticsearchHook): conn_name_attr = "test_conn_id" @@ -49,10 +49,8 @@ def get_conn(self): self.db_hook = UnitTestElasticsearchHook() -class TestElasticsearchSQLHookConn(unittest.TestCase): - def setUp(self): - super().setUp() - +class TestElasticsearchSQLHookConn: + def setup_method(self): self.connection = Connection(host="localhost", port=9200, schema="http") class UnitTestElasticsearchHook(ElasticsearchSQLHook): @@ -69,10 +67,8 @@ def test_get_conn(self, mock_connect): mock_connect.assert_called_with(host="localhost", port=9200, scheme="http", user=None, password=None) -class TestElasticsearcSQLhHook(unittest.TestCase): - def setUp(self): - super().setUp() - +class TestElasticsearchSQLHook: + def setup_method(self): self.cur = mock.MagicMock(rowcount=0) self.conn = mock.MagicMock() self.conn.cursor.return_value = self.cur @@ -131,7 +127,7 @@ def search(self, **kwargs): class TestElasticsearchPythonHook: - def setup(self): + def setup_method(self): self.elasticsearch_hook = ElasticsearchPythonHook(hosts=["http://localhost:9200"]) def test_client(self): diff --git a/tests/providers/elasticsearch/log/test_es_task_handler.py b/tests/providers/elasticsearch/log/test_es_task_handler.py index f8a54ab4c99de..bdf274732e555 100644 --- a/tests/providers/elasticsearch/log/test_es_task_handler.py +++ b/tests/providers/elasticsearch/log/test_es_task_handler.py @@ -75,7 +75,7 @@ def ti(self, create_task_instance, create_log_template): clear_db_dags() @elasticmock - def setup(self): + def setup_method(self, method): self.local_log_location = "local/log/location" self.end_of_log_mark = "end_of_log\n" self.write_stdout = False @@ -100,7 +100,7 @@ def setup(self): self.body = {"message": self.test_message, "log_id": self.LOG_ID, "offset": 1} self.es.index(index=self.index_name, doc_type=self.doc_type, body=self.body, id=1) - def teardown(self): + def teardown_method(self): shutil.rmtree(self.local_log_location.split(os.path.sep)[0], ignore_errors=True) def test_client(self): diff --git a/tests/providers/exasol/hooks/test_exasol.py b/tests/providers/exasol/hooks/test_exasol.py index 8345b36b25175..f208450826487 100644 --- a/tests/providers/exasol/hooks/test_exasol.py +++ b/tests/providers/exasol/hooks/test_exasol.py @@ -18,7 +18,6 @@ from __future__ import annotations import json -import unittest from unittest import mock import pytest @@ -27,10 +26,8 @@ from airflow.providers.exasol.hooks.exasol import ExasolHook -class TestExasolHookConn(unittest.TestCase): - def setUp(self): - super().setUp() - +class TestExasolHookConn: + def setup_method(self): self.connection = models.Connection( login="login", password="password", @@ -66,10 +63,8 @@ def test_get_conn_extra_args(self, mock_pyexasol): assert kwargs["encryption"] is True -class TestExasolHook(unittest.TestCase): - def setUp(self): - super().setUp() - +class TestExasolHook: + def setup_method(self): self.cur = mock.MagicMock(rowcount=0) self.conn = mock.MagicMock() self.conn.execute.return_value = self.cur diff --git a/tests/providers/exasol/operators/test_exasol.py b/tests/providers/exasol/operators/test_exasol.py index 6f1d00c999a96..ffc06f6eb601a 100644 --- a/tests/providers/exasol/operators/test_exasol.py +++ b/tests/providers/exasol/operators/test_exasol.py @@ -17,14 +17,13 @@ # under the License. from __future__ import annotations -import unittest from unittest import mock from airflow.providers.common.sql.hooks.sql import fetch_all_handler from airflow.providers.exasol.operators.exasol import ExasolOperator -class TestExasol(unittest.TestCase): +class TestExasol: @mock.patch("airflow.providers.common.sql.operators.sql.SQLExecuteQueryOperator.get_db_hook") def test_overwrite_autocommit(self, mock_get_db_hook): operator = ExasolOperator(task_id="TEST", sql="SELECT 1", autocommit=True) diff --git a/tests/providers/ftp/hooks/test_ftp.py b/tests/providers/ftp/hooks/test_ftp.py index e62344ee699d3..dcb2e4ac512c9 100644 --- a/tests/providers/ftp/hooks/test_ftp.py +++ b/tests/providers/ftp/hooks/test_ftp.py @@ -18,15 +18,13 @@ from __future__ import annotations import io -import unittest from unittest import mock from airflow.providers.ftp.hooks import ftp as fh -class TestFTPHook(unittest.TestCase): - def setUp(self): - super().setUp() +class TestFTPHook: + def setup_method(self): self.path = "/some/path" self.conn_mock = mock.MagicMock(name="conn") self.get_conn_orig = fh.FTPHook.get_conn @@ -37,9 +35,8 @@ def _get_conn_mock(hook): fh.FTPHook.get_conn = _get_conn_mock - def tearDown(self): + def teardown_method(self): fh.FTPHook.get_conn = self.get_conn_orig - super().tearDown() def test_close_conn(self): ftp_hook = fh.FTPHook() @@ -137,9 +134,8 @@ def test_connection_failure(self): assert msg == "Test" -class TestIntegrationFTPHook(unittest.TestCase): - def setUp(self): - super().setUp() +class TestIntegrationFTPHook: + def setup_method(self): from airflow.models import Connection from airflow.utils import db diff --git a/tests/providers/ftp/sensors/test_ftp.py b/tests/providers/ftp/sensors/test_ftp.py index 59a02ad543d3b..e79b46526aaee 100644 --- a/tests/providers/ftp/sensors/test_ftp.py +++ b/tests/providers/ftp/sensors/test_ftp.py @@ -17,7 +17,6 @@ # under the License. from __future__ import annotations -import unittest from ftplib import error_perm from unittest import mock @@ -27,7 +26,7 @@ from airflow.providers.ftp.sensors.ftp import FTPSensor -class TestFTPSensor(unittest.TestCase): +class TestFTPSensor: @mock.patch("airflow.providers.ftp.sensors.ftp.FTPHook", spec=FTPHook) def test_poke(self, mock_hook): op = FTPSensor(path="foobar.json", ftp_conn_id="bob_ftp", task_id="test_task") diff --git a/tests/providers/grpc/hooks/test_grpc.py b/tests/providers/grpc/hooks/test_grpc.py index 0813fd4167660..46ae9896e73b1 100644 --- a/tests/providers/grpc/hooks/test_grpc.py +++ b/tests/providers/grpc/hooks/test_grpc.py @@ -61,7 +61,7 @@ def stream_call(self, data): class TestGrpcHook: - def setup(self): + def setup_method(self): self.channel_mock = mock.patch("grpc.Channel").start() def custom_conn_func(self, _): diff --git a/tests/providers/grpc/operators/test_grpc.py b/tests/providers/grpc/operators/test_grpc.py index e69a23bc3cd6b..62540bbfeae22 100644 --- a/tests/providers/grpc/operators/test_grpc.py +++ b/tests/providers/grpc/operators/test_grpc.py @@ -16,7 +16,6 @@ # under the License. from __future__ import annotations -import unittest from unittest import mock from airflow.providers.grpc.operators.grpc import GrpcOperator @@ -30,7 +29,7 @@ def stream_call(self, data): pass -class TestGrpcOperator(unittest.TestCase): +class TestGrpcOperator: def custom_conn_func(self, connection): pass diff --git a/tests/providers/hashicorp/_internal_client/test_vault_client.py b/tests/providers/hashicorp/_internal_client/test_vault_client.py index 15aa1f19a8bc4..1bab652dc01d9 100644 --- a/tests/providers/hashicorp/_internal_client/test_vault_client.py +++ b/tests/providers/hashicorp/_internal_client/test_vault_client.py @@ -17,7 +17,6 @@ from __future__ import annotations from unittest import mock -from unittest.case import TestCase from unittest.mock import mock_open, patch import pytest @@ -26,7 +25,7 @@ from airflow.providers.hashicorp._internal_client.vault_client import _VaultClient -class TestVaultClient(TestCase): +class TestVaultClient: @mock.patch("airflow.providers.hashicorp._internal_client.vault_client.hvac") def test_version_wrong(self, mock_hvac): mock_client = mock.MagicMock() diff --git a/tests/providers/hashicorp/hooks/test_vault.py b/tests/providers/hashicorp/hooks/test_vault.py index c91b1a2d1f1dd..bad350fe65528 100644 --- a/tests/providers/hashicorp/hooks/test_vault.py +++ b/tests/providers/hashicorp/hooks/test_vault.py @@ -17,17 +17,15 @@ from __future__ import annotations from unittest import mock -from unittest.case import TestCase from unittest.mock import PropertyMock, mock_open, patch import pytest from hvac.exceptions import VaultError -from parameterized import parameterized from airflow.providers.hashicorp.hooks.vault import VaultHook -class TestVaultHook(TestCase): +class TestVaultHook: @staticmethod def get_mock_connection( conn_type="vault", schema="secret", host="localhost", port=8180, user="user", password="pass" @@ -58,15 +56,16 @@ def test_version_not_int(self, mock_hvac, mock_get_connection): with pytest.raises(VaultError, match="The version is not an int: text"): VaultHook(**kwargs) - @parameterized.expand( + @pytest.mark.parametrize( + "version, expected_version", [ ("2", 2), (1, 1), - ] + ], ) @mock.patch("airflow.providers.hashicorp.hooks.vault.VaultHook.get_connection") @mock.patch("airflow.providers.hashicorp._internal_client.vault_client.hvac") - def test_version(self, version, expected_version, mock_hvac, mock_get_connection): + def test_version(self, mock_hvac, mock_get_connection, version, expected_version): mock_client = mock.MagicMock() mock_hvac.Client.return_value = mock_client mock_connection = self.get_mock_connection() @@ -156,16 +155,17 @@ def test_version_one_dejson(self, mock_hvac, mock_get_connection): test_hook = VaultHook(**kwargs) assert 1 == test_hook.vault_client.kv_engine_version - @parameterized.expand( + @pytest.mark.parametrize( + "protocol, expected_url", [ ("vaults", "https://localhost:8180"), ("http", "http://localhost:8180"), ("https", "https://localhost:8180"), - ] + ], ) @mock.patch("airflow.providers.hashicorp.hooks.vault.VaultHook.get_connection") @mock.patch("airflow.providers.hashicorp._internal_client.vault_client.hvac") - def test_protocol(self, protocol, expected_url, mock_hvac, mock_get_connection): + def test_protocol(self, mock_hvac, mock_get_connection, protocol, expected_url): mock_client = mock.MagicMock() mock_hvac.Client.return_value = mock_client mock_connection = self.get_mock_connection(conn_type=protocol) @@ -1154,15 +1154,16 @@ def test_create_or_update_secret_v2_cas(self, mock_hvac, mock_get_connection): mount_point="secret", secret_path="path", secret={"key": "value"}, cas=10 ) - @parameterized.expand( + @pytest.mark.parametrize( + "method, expected_method", [ (None, None), ("post", "post"), - ] + ], ) @mock.patch("airflow.providers.hashicorp.hooks.vault.VaultHook.get_connection") @mock.patch("airflow.providers.hashicorp._internal_client.vault_client.hvac") - def test_create_or_update_secret_v1(self, method, expected_method, mock_hvac, mock_get_connection): + def test_create_or_update_secret_v1(self, mock_hvac, mock_get_connection, method, expected_method): mock_connection = self.get_mock_connection() mock_get_connection.return_value = mock_connection mock_client = mock.MagicMock() diff --git a/tests/providers/hashicorp/secrets/test_vault.py b/tests/providers/hashicorp/secrets/test_vault.py index 82c54cbeabd99..a29e6dc21e8c3 100644 --- a/tests/providers/hashicorp/secrets/test_vault.py +++ b/tests/providers/hashicorp/secrets/test_vault.py @@ -16,7 +16,7 @@ # under the License. from __future__ import annotations -from unittest import TestCase, mock +from unittest import mock import pytest from hvac.exceptions import InvalidPath, VaultError @@ -24,7 +24,7 @@ from airflow.providers.hashicorp.secrets.vault import VaultBackend -class TestVaultSecrets(TestCase): +class TestVaultSecrets: @mock.patch("airflow.providers.hashicorp._internal_client.vault_client.hvac") def test_get_conn_uri(self, mock_hvac): mock_client = mock.MagicMock() diff --git a/tests/providers/http/hooks/test_http.py b/tests/providers/http/hooks/test_http.py index 9f3efd340f55f..03bf09075c3c8 100644 --- a/tests/providers/http/hooks/test_http.py +++ b/tests/providers/http/hooks/test_http.py @@ -19,16 +19,13 @@ import json import os -import unittest from collections import OrderedDict from http import HTTPStatus from unittest import mock import pytest import requests -import requests_mock import tenacity -from parameterized import parameterized from requests.adapters import Response from airflow.exceptions import AirflowException @@ -44,10 +41,12 @@ def get_airflow_connection_with_port(unused_conn_id=None): return Connection(conn_id="http_default", conn_type="http", host="test.com", port=1234) -class TestHttpHook(unittest.TestCase): +class TestHttpHook: """Test get, post and raise_for_status""" - def setUp(self): + def setup_method(self): + import requests_mock + session = requests.Session() adapter = requests_mock.Adapter() session.mount("mock", adapter) @@ -55,18 +54,17 @@ def setUp(self): self.get_lowercase_hook = HttpHook(method="get") self.post_hook = HttpHook(method="POST") - @requests_mock.mock() - def test_raise_for_status_with_200(self, m): - - m.get("http://test:8080/v1/test", status_code=200, text='{"status":{"status": 200}}', reason="OK") + def test_raise_for_status_with_200(self, requests_mock): + requests_mock.get( + "http://test:8080/v1/test", status_code=200, text='{"status":{"status": 200}}', reason="OK" + ) with mock.patch("airflow.hooks.base.BaseHook.get_connection", side_effect=get_airflow_connection): resp = self.get_hook.run("v1/test") assert resp.text == '{"status":{"status": 200}}' - @requests_mock.mock() - @mock.patch("requests.Session") @mock.patch("requests.Request") - def test_get_request_with_port(self, mock_requests, request_mock, mock_session): + @mock.patch("requests.Session") + def test_get_request_with_port(self, mock_session, mock_request): from requests.exceptions import MissingSchema with mock.patch( @@ -80,16 +78,14 @@ def test_get_request_with_port(self, mock_requests, request_mock, mock_session): except MissingSchema: pass - request_mock.assert_called_once_with( + mock_request.assert_called_once_with( mock.ANY, expected_url, headers=mock.ANY, params=mock.ANY ) - request_mock.reset_mock() + mock_request.reset_mock() - @requests_mock.mock() - def test_get_request_do_not_raise_for_status_if_check_response_is_false(self, m): - - m.get( + def test_get_request_do_not_raise_for_status_if_check_response_is_false(self, requests_mock): + requests_mock.get( "http://test:8080/v1/test", status_code=404, text='{"status":{"status": 404}}', @@ -100,17 +96,15 @@ def test_get_request_do_not_raise_for_status_if_check_response_is_false(self, m) resp = self.get_hook.run("v1/test", extra_options={"check_response": False}) assert resp.text == '{"status":{"status": 404}}' - @requests_mock.mock() - def test_hook_contains_header_from_extra_field(self, mock_requests): + def test_hook_contains_header_from_extra_field(self): with mock.patch("airflow.hooks.base.BaseHook.get_connection", side_effect=get_airflow_connection): expected_conn = get_airflow_connection() conn = self.get_hook.get_conn() assert dict(conn.headers, **json.loads(expected_conn.extra)) == conn.headers assert conn.headers.get("bareer") == "test" - @requests_mock.mock() @mock.patch("requests.Request") - def test_hook_with_method_in_lowercase(self, mock_requests, request_mock): + def test_hook_with_method_in_lowercase(self, mock_requests): from requests.exceptions import InvalidURL, MissingSchema with mock.patch( @@ -121,27 +115,23 @@ def test_hook_with_method_in_lowercase(self, mock_requests, request_mock): self.get_lowercase_hook.run("v1/test", data=data) except (MissingSchema, InvalidURL): pass - request_mock.assert_called_once_with(mock.ANY, mock.ANY, headers=mock.ANY, params=data) + mock_requests.assert_called_once_with(mock.ANY, mock.ANY, headers=mock.ANY, params=data) - @requests_mock.mock() - def test_hook_uses_provided_header(self, mock_requests): + def test_hook_uses_provided_header(self): conn = self.get_hook.get_conn(headers={"bareer": "newT0k3n"}) assert conn.headers.get("bareer") == "newT0k3n" - @requests_mock.mock() - def test_hook_has_no_header_from_extra(self, mock_requests): + def test_hook_has_no_header_from_extra(self): conn = self.get_hook.get_conn() assert conn.headers.get("bareer") is None - @requests_mock.mock() - def test_hooks_header_from_extra_is_overridden(self, mock_requests): + def test_hooks_header_from_extra_is_overridden(self): with mock.patch("airflow.hooks.base.BaseHook.get_connection", side_effect=get_airflow_connection): conn = self.get_hook.get_conn(headers={"bareer": "newT0k3n"}) assert conn.headers.get("bareer") == "newT0k3n" - @requests_mock.mock() - def test_post_request(self, mock_requests): - mock_requests.post( + def test_post_request(self, requests_mock): + requests_mock.post( "http://test:8080/v1/test", status_code=200, text='{"status":{"status": 200}}', reason="OK" ) @@ -149,9 +139,8 @@ def test_post_request(self, mock_requests): resp = self.post_hook.run("v1/test") assert resp.status_code == 200 - @requests_mock.mock() - def test_post_request_with_error_code(self, mock_requests): - mock_requests.post( + def test_post_request_with_error_code(self, requests_mock): + requests_mock.post( "http://test:8080/v1/test", status_code=418, text='{"status":{"status": 418}}', @@ -162,9 +151,8 @@ def test_post_request_with_error_code(self, mock_requests): with pytest.raises(AirflowException): self.post_hook.run("v1/test") - @requests_mock.mock() - def test_post_request_do_not_raise_for_status_if_check_response_is_false(self, mock_requests): - mock_requests.post( + def test_post_request_do_not_raise_for_status_if_check_response_is_false(self, requests_mock): + requests_mock.post( "http://test:8080/v1/test", status_code=418, text='{"status":{"status": 418}}', @@ -193,10 +181,9 @@ def send_and_raise(unused_request, **kwargs): self.get_hook.run_with_advanced_retry(endpoint="v1/test", _retry_args=retry_args) assert self.get_hook._retry_obj.stop.max_attempt_number + 1 == mocked_session.call_count - @requests_mock.mock() - def test_run_with_advanced_retry(self, m): + def test_run_with_advanced_retry(self, requests_mock): - m.get("http://test:8080/v1/test", status_code=200, reason="OK") + requests_mock.get("http://test:8080/v1/test", status_code=200, reason="OK") retry_args = dict( wait=tenacity.wait_none(), @@ -266,20 +253,14 @@ def test_connection_without_host(self, mock_get_connection): hook.get_conn({}) assert hook.base_url == "http://" - @parameterized.expand( - [ - "GET", - "POST", - ] - ) - @requests_mock.mock() - def test_json_request(self, method, mock_requests): + @pytest.mark.parametrize("method", ["GET", "POST"]) + def test_json_request(self, method, requests_mock): obj1 = {"a": 1, "b": "abc", "c": [1, 2, {"d": 10}]} def match_obj1(request): return request.json() == obj1 - mock_requests.request(method=method, url="//test:8080/v1/test", additional_matcher=match_obj1) + requests_mock.request(method=method, url="//test:8080/v1/test", additional_matcher=match_obj1) with mock.patch("airflow.hooks.base.BaseHook.get_connection", side_effect=get_airflow_connection): # will raise NoMockAddress exception if obj1 != request.json() @@ -357,17 +338,17 @@ def test_verify_false_parameter_overwrites_set_requests_ca_bundle_env_var(self, verify=False, ) - @requests_mock.mock() - def test_connection_success(self, m): - m.get("http://test:8080", status_code=200, json={"status": {"status": 200}}, reason="OK") + def test_connection_success(self, requests_mock): + requests_mock.get("http://test:8080", status_code=200, json={"status": {"status": 200}}, reason="OK") with mock.patch("airflow.hooks.base.BaseHook.get_connection", side_effect=get_airflow_connection): status, msg = self.get_hook.test_connection() assert status is True assert msg == "Connection successfully tested" - @requests_mock.mock() - def test_connection_failure(self, m): - m.get("http://test:8080", status_code=500, json={"message": "internal server error"}, reason="NOT_OK") + def test_connection_failure(self, requests_mock): + requests_mock.get( + "http://test:8080", status_code=500, json={"message": "internal server error"}, reason="NOT_OK" + ) with mock.patch("airflow.hooks.base.BaseHook.get_connection", side_effect=get_airflow_connection): status, msg = self.get_hook.test_connection() assert status is False diff --git a/tests/providers/http/operators/test_http.py b/tests/providers/http/operators/test_http.py index 227ea03b273a6..fca910a7680eb 100644 --- a/tests/providers/http/operators/test_http.py +++ b/tests/providers/http/operators/test_http.py @@ -17,26 +17,23 @@ # under the License. from __future__ import annotations -import unittest from unittest import mock import pytest -import requests_mock from airflow.exceptions import AirflowException from airflow.providers.http.operators.http import SimpleHttpOperator @mock.patch.dict("os.environ", AIRFLOW_CONN_HTTP_EXAMPLE="http://www.example.com") -class TestSimpleHttpOp(unittest.TestCase): - @requests_mock.mock() - def test_response_in_logs(self, m): +class TestSimpleHttpOp: + def test_response_in_logs(self, requests_mock): """ Test that when using SimpleHttpOperator with 'GET', the log contains 'Example Domain' in it """ - m.get("http://www.example.com", text="Example.com fake response") + requests_mock.get("http://www.example.com", text="Example.com fake response") operator = SimpleHttpOperator( task_id="test_HTTP_op", method="GET", @@ -48,8 +45,7 @@ def test_response_in_logs(self, m): result = operator.execute("Example.com fake response") assert result == "Example.com fake response" - @requests_mock.mock() - def test_response_in_logs_after_failed_check(self, m): + def test_response_in_logs_after_failed_check(self, requests_mock): """ Test that when using SimpleHttpOperator with log_response=True, the response is logged even if request_check fails @@ -58,7 +54,7 @@ def test_response_in_logs_after_failed_check(self, m): def response_check(response): return response.text != "invalid response" - m.get("http://www.example.com", text="invalid response") + requests_mock.get("http://www.example.com", text="invalid response") operator = SimpleHttpOperator( task_id="test_HTTP_op", method="GET", @@ -74,9 +70,8 @@ def response_check(response): calls = [mock.call("Calling HTTP method"), mock.call("invalid response")] mock_info.assert_has_calls(calls, any_order=True) - @requests_mock.mock() - def test_filters_response(self, m): - m.get("http://www.example.com", json={"value": 5}) + def test_filters_response(self, requests_mock): + requests_mock.get("http://www.example.com", json={"value": 5}) operator = SimpleHttpOperator( task_id="test_HTTP_op", method="GET", diff --git a/tests/providers/http/sensors/test_http.py b/tests/providers/http/sensors/test_http.py index 7b801e28fc21e..0463a80a04fa6 100644 --- a/tests/providers/http/sensors/test_http.py +++ b/tests/providers/http/sensors/test_http.py @@ -17,7 +17,6 @@ # under the License. from __future__ import annotations -import unittest from unittest import mock from unittest.mock import patch @@ -206,8 +205,8 @@ def mount(self, prefix, adapter): pass -class TestHttpOpSensor(unittest.TestCase): - def setUp(self): +class TestHttpOpSensor: + def setup_method(self): args = {"owner": "airflow", "start_date": DEFAULT_DATE_ISO} dag = DAG(TEST_DAG_ID, default_args=args) self.dag = dag diff --git a/tests/providers/imap/hooks/test_imap.py b/tests/providers/imap/hooks/test_imap.py index 1912c37c5efaf..c2be420be6e59 100644 --- a/tests/providers/imap/hooks/test_imap.py +++ b/tests/providers/imap/hooks/test_imap.py @@ -19,7 +19,6 @@ import imaplib import json -import unittest from unittest.mock import Mock, mock_open, patch import pytest @@ -61,8 +60,8 @@ def _create_fake_imap(mock_imaplib, with_mail=False, attachment_name="test1.csv" return mock_conn -class TestImapHook(unittest.TestCase): - def setUp(self): +class TestImapHook: + def setup_method(self): db.merge_conn( Connection( conn_id="imap_default", diff --git a/tests/providers/imap/sensors/test_imap_attachment.py b/tests/providers/imap/sensors/test_imap_attachment.py index 6fd9db8b19837..41cda774fcf46 100644 --- a/tests/providers/imap/sensors/test_imap_attachment.py +++ b/tests/providers/imap/sensors/test_imap_attachment.py @@ -17,16 +17,15 @@ # under the License. from __future__ import annotations -import unittest from unittest.mock import Mock, patch -from parameterized import parameterized +import pytest from airflow.providers.imap.sensors.imap_attachment import ImapAttachmentSensor -class TestImapAttachmentSensor(unittest.TestCase): - def setUp(self): +class TestImapAttachmentSensor: + def setup_method(self): self.kwargs = dict( attachment_name="test_file", check_regex=False, @@ -36,9 +35,9 @@ def setUp(self): dag=None, ) - @parameterized.expand([(True,), (False,)]) + @pytest.mark.parametrize("has_attachment_return_value", [True, False]) @patch("airflow.providers.imap.sensors.imap_attachment.ImapHook") - def test_poke(self, has_attachment_return_value, mock_imap_hook): + def test_poke(self, mock_imap_hook, has_attachment_return_value): mock_imap_hook.return_value.__enter__ = Mock(return_value=mock_imap_hook) mock_imap_hook.has_mail_attachment.return_value = has_attachment_return_value diff --git a/tests/providers/influxdb/hooks/test_influxdb.py b/tests/providers/influxdb/hooks/test_influxdb.py index 7b0f2d7b78fab..630fe26263680 100644 --- a/tests/providers/influxdb/hooks/test_influxdb.py +++ b/tests/providers/influxdb/hooks/test_influxdb.py @@ -16,16 +16,14 @@ # under the License. from __future__ import annotations -import unittest from unittest import mock from airflow.models import Connection from airflow.providers.influxdb.hooks.influxdb import InfluxDBHook -class TestInfluxDbHook(unittest.TestCase): - def setUp(self): - super().setUp() +class TestInfluxDbHook: + def setup_method(self): self.influxdb_hook = InfluxDBHook() extra = {} extra["token"] = "123456789" diff --git a/tests/providers/influxdb/operators/test_influxdb.py b/tests/providers/influxdb/operators/test_influxdb.py index 2ae8807ac9e86..f39e99e12a403 100644 --- a/tests/providers/influxdb/operators/test_influxdb.py +++ b/tests/providers/influxdb/operators/test_influxdb.py @@ -16,13 +16,12 @@ # under the License. from __future__ import annotations -import unittest from unittest import mock from airflow.providers.influxdb.operators.influxdb import InfluxDBOperator -class TestInfluxDBOperator(unittest.TestCase): +class TestInfluxDBOperator: @mock.patch("airflow.providers.influxdb.operators.influxdb.InfluxDBHook") def test_influxdb_operator_test(self, mock_hook): diff --git a/tests/providers/jdbc/hooks/test_jdbc.py b/tests/providers/jdbc/hooks/test_jdbc.py index 6a4739e54b327..50913b023769b 100644 --- a/tests/providers/jdbc/hooks/test_jdbc.py +++ b/tests/providers/jdbc/hooks/test_jdbc.py @@ -32,7 +32,7 @@ class TestJdbcHook: - def setup(self): + def setup_method(self): db.merge_conn( Connection( conn_id="jdbc_default", diff --git a/tests/providers/jdbc/operators/test_jdbc.py b/tests/providers/jdbc/operators/test_jdbc.py index b9339ce584850..e027bdb96df01 100644 --- a/tests/providers/jdbc/operators/test_jdbc.py +++ b/tests/providers/jdbc/operators/test_jdbc.py @@ -17,15 +17,14 @@ # under the License. from __future__ import annotations -import unittest from unittest.mock import patch from airflow.providers.common.sql.hooks.sql import fetch_all_handler from airflow.providers.jdbc.operators.jdbc import JdbcOperator -class TestJdbcOperator(unittest.TestCase): - def setUp(self): +class TestJdbcOperator: + def setup_method(self): self.kwargs = dict(sql="sql", task_id="test_jdbc_operator", dag=None) @patch("airflow.providers.common.sql.operators.sql.SQLExecuteQueryOperator.get_db_hook") diff --git a/tests/providers/jenkins/hooks/test_jenkins.py b/tests/providers/jenkins/hooks/test_jenkins.py index 53895f8572856..0ad0ba7603304 100644 --- a/tests/providers/jenkins/hooks/test_jenkins.py +++ b/tests/providers/jenkins/hooks/test_jenkins.py @@ -17,15 +17,14 @@ # under the License. from __future__ import annotations -import unittest from unittest import mock -from parameterized import parameterized +import pytest from airflow.providers.jenkins.hooks.jenkins import JenkinsHook -class TestJenkinsHook(unittest.TestCase): +class TestJenkinsHook: @mock.patch("airflow.hooks.base.BaseHook.get_connection") def test_client_created_default_http(self, get_connection_mock): """tests `init` method to validate http client creation when all parameters are passed""" @@ -69,12 +68,12 @@ def test_client_created_default_https(self, get_connection_mock): assert hook.jenkins_server is not None assert hook.jenkins_server.server == complete_url - @parameterized.expand([(True,), (False,)]) + @pytest.mark.parametrize("param_building", [True, False]) @mock.patch("airflow.hooks.base.BaseHook.get_connection") @mock.patch("jenkins.Jenkins.get_job_info") @mock.patch("jenkins.Jenkins.get_build_info") def test_get_build_building_state( - self, param_building, mock_get_build_info, mock_get_job_info, get_connection_mock + self, mock_get_build_info, mock_get_job_info, get_connection_mock, param_building ): mock_get_build_info.return_value = {"building": param_building} diff --git a/tests/providers/jenkins/operators/test_jenkins_job_trigger.py b/tests/providers/jenkins/operators/test_jenkins_job_trigger.py index d4c328ea44e4d..53a32def164e0 100644 --- a/tests/providers/jenkins/operators/test_jenkins_job_trigger.py +++ b/tests/providers/jenkins/operators/test_jenkins_job_trigger.py @@ -17,36 +17,25 @@ # under the License. from __future__ import annotations -import unittest from unittest.mock import Mock, patch import jenkins import pytest -from parameterized import parameterized from airflow.exceptions import AirflowException from airflow.providers.jenkins.hooks.jenkins import JenkinsHook from airflow.providers.jenkins.operators.jenkins_job_trigger import JenkinsJobTriggerOperator +TEST_PARAMETERS = ( + pytest.param({"a_param": "blip", "another_param": "42"}, id="dict params"), + pytest.param('{"second_param": "beep", "third_param": "153"}', id="string params"), + pytest.param(["final_one", "bop", "real_final", "eggs"], id="list params"), +) -class TestJenkinsOperator(unittest.TestCase): - @parameterized.expand( - [ - ( - "dict params", - {"a_param": "blip", "another_param": "42"}, - ), - ( - "string params", - '{"second_param": "beep", "third_param": "153"}', - ), - ( - "list params", - ["final_one", "bop", "real_final", "eggs"], - ), - ] - ) - def test_execute(self, _, parameters): + +class TestJenkinsOperator: + @pytest.mark.parametrize("parameters", TEST_PARAMETERS) + def test_execute(self, parameters): jenkins_mock = Mock(spec=jenkins.Jenkins, auth="secret") jenkins_mock.get_build_info.return_value = { "result": "SUCCESS", @@ -80,23 +69,8 @@ def test_execute(self, _, parameters): assert jenkins_mock.get_build_info.call_count == 1 jenkins_mock.get_build_info.assert_called_once_with(name="a_job_on_jenkins", number="1") - @parameterized.expand( - [ - ( - "dict params", - {"a_param": "blip", "another_param": "42"}, - ), - ( - "string params", - '{"second_param": "beep", "third_param": "153"}', - ), - ( - "list params", - ["final_one", "bop", "real_final", "eggs"], - ), - ] - ) - def test_execute_job_polling_loop(self, _, parameters): + @pytest.mark.parametrize("parameters", TEST_PARAMETERS) + def test_execute_job_polling_loop(self, parameters): jenkins_mock = Mock(spec=jenkins.Jenkins, auth="secret") jenkins_mock.get_job_info.return_value = {"nextBuildNumber": "1"} jenkins_mock.get_build_info.side_effect = [ @@ -129,23 +103,8 @@ def test_execute_job_polling_loop(self, _, parameters): operator.execute(None) assert jenkins_mock.get_build_info.call_count == 2 - @parameterized.expand( - [ - ( - "dict params", - {"a_param": "blip", "another_param": "42"}, - ), - ( - "string params", - '{"second_param": "beep", "third_param": "153"}', - ), - ( - "list params", - ["final_one", "bop", "real_final", "eggs"], - ), - ] - ) - def test_execute_job_failure(self, _, parameters): + @pytest.mark.parametrize("parameters", TEST_PARAMETERS) + def test_execute_job_failure(self, parameters): jenkins_mock = Mock(spec=jenkins.Jenkins, auth="secret") jenkins_mock.get_job_info.return_value = {"nextBuildNumber": "1"} jenkins_mock.get_build_info.return_value = { @@ -178,7 +137,8 @@ def test_execute_job_failure(self, _, parameters): with pytest.raises(AirflowException): operator.execute(None) - @parameterized.expand( + @pytest.mark.parametrize( + "state, allowed_jenkins_states", [ ( "SUCCESS", @@ -196,7 +156,7 @@ def test_execute_job_failure(self, _, parameters): "SUCCESS", None, ), - ] + ], ) def test_allowed_jenkins_states(self, state, allowed_jenkins_states): jenkins_mock = Mock(spec=jenkins.Jenkins, auth="secret") @@ -233,7 +193,8 @@ def test_allowed_jenkins_states(self, state, allowed_jenkins_states): except AirflowException: pytest.fail(f"Job failed with state={state} while allowed states={allowed_jenkins_states}") - @parameterized.expand( + @pytest.mark.parametrize( + "state, allowed_jenkins_states", [ ( "FAILURE", @@ -255,7 +216,7 @@ def test_allowed_jenkins_states(self, state, allowed_jenkins_states): "UNSTABLE", None, ), - ] + ], ) def test_allowed_jenkins_states_failure(self, state, allowed_jenkins_states): jenkins_mock = Mock(spec=jenkins.Jenkins, auth="secret") diff --git a/tests/providers/jenkins/sensors/test_jenkins.py b/tests/providers/jenkins/sensors/test_jenkins.py index a3e9f0b5a4f58..8f69c41510d85 100644 --- a/tests/providers/jenkins/sensors/test_jenkins.py +++ b/tests/providers/jenkins/sensors/test_jenkins.py @@ -17,17 +17,17 @@ # under the License. from __future__ import annotations -import unittest from unittest.mock import MagicMock, patch -from parameterized import parameterized +import pytest from airflow.providers.jenkins.hooks.jenkins import JenkinsHook from airflow.providers.jenkins.sensors.jenkins import JenkinsBuildSensor -class TestJenkinsBuildSensor(unittest.TestCase): - @parameterized.expand( +class TestJenkinsBuildSensor: + @pytest.mark.parametrize( + "build_number, build_state", [ ( 1, @@ -41,10 +41,10 @@ class TestJenkinsBuildSensor(unittest.TestCase): 3, True, ), - ] + ], ) @patch("jenkins.Jenkins") - def test_poke(self, build_number, build_state, mock_jenkins): + def test_poke(self, mock_jenkins, build_number, build_state): target_build_number = build_number if build_number else 10 jenkins_mock = MagicMock() diff --git a/tests/providers/mongo/hooks/test_mongo.py b/tests/providers/mongo/hooks/test_mongo.py index e366a7d2c8f7d..0a106eaad3439 100644 --- a/tests/providers/mongo/hooks/test_mongo.py +++ b/tests/providers/mongo/hooks/test_mongo.py @@ -18,10 +18,10 @@ from __future__ import annotations import importlib -import unittest from types import ModuleType import pymongo +import pytest from airflow.models import Connection from airflow.providers.mongo.hooks.mongo import MongoHook @@ -48,8 +48,9 @@ def get_collection(self, mock_collection, mongo_db=None): return mock_collection -class TestMongoHook(unittest.TestCase): - def setUp(self): +@pytest.mark.skipif(mongomock is None, reason="mongomock package not present") +class TestMongoHook: + def setup_method(self): self.hook = MongoHookTest(conn_id="mongo_default", mongo_db="default") self.conn = self.hook.get_conn() db.merge_conn( @@ -57,22 +58,19 @@ def setUp(self): conn_id="mongo_default_with_srv", conn_type="mongo", host="mongo", - port="27017", + port=27017, extra='{"srv": true}', ) ) - @unittest.skipIf(mongomock is None, "mongomock package not present") def test_get_conn(self): assert self.hook.connection.port == 27017 assert isinstance(self.conn, pymongo.MongoClient) - @unittest.skipIf(mongomock is None, "mongomock package not present") def test_srv(self): hook = MongoHook(conn_id="mongo_default_with_srv") assert hook.uri.startswith("mongodb+srv://") - @unittest.skipIf(mongomock is None, "mongomock package not present") def test_insert_one(self): collection = mongomock.MongoClient().db.collection obj = {"test_insert_one": "test_value"} @@ -82,7 +80,6 @@ def test_insert_one(self): assert obj == result_obj - @unittest.skipIf(mongomock is None, "mongomock package not present") def test_insert_many(self): collection = mongomock.MongoClient().db.collection objs = [{"test_insert_many_1": "test_value"}, {"test_insert_many_2": "test_value"}] @@ -92,7 +89,6 @@ def test_insert_many(self): result_objs = list(collection.find()) assert len(result_objs) == 2 - @unittest.skipIf(mongomock is None, "mongomock package not present") def test_update_one(self): collection = mongomock.MongoClient().db.collection obj = {"_id": "1", "field": 0} @@ -106,7 +102,6 @@ def test_update_one(self): result_obj = collection.find_one(filter="1") assert 123 == result_obj["field"] - @unittest.skipIf(mongomock is None, "mongomock package not present") def test_update_one_with_upsert(self): collection = mongomock.MongoClient().db.collection @@ -118,7 +113,6 @@ def test_update_one_with_upsert(self): result_obj = collection.find_one(filter="1") assert 123 == result_obj["field"] - @unittest.skipIf(mongomock is None, "mongomock package not present") def test_update_many(self): collection = mongomock.MongoClient().db.collection obj1 = {"_id": "1", "field": 0} @@ -136,7 +130,6 @@ def test_update_many(self): result_obj = collection.find_one(filter="2") assert 123 == result_obj["field"] - @unittest.skipIf(mongomock is None, "mongomock package not present") def test_update_many_with_upsert(self): collection = mongomock.MongoClient().db.collection @@ -148,7 +141,6 @@ def test_update_many_with_upsert(self): result_obj = collection.find_one(filter="1") assert 123 == result_obj["field"] - @unittest.skipIf(mongomock is None, "mongomock package not present") def test_replace_one(self): collection = mongomock.MongoClient().db.collection obj1 = {"_id": "1", "field": "test_value_1"} @@ -165,7 +157,6 @@ def test_replace_one(self): result_obj = collection.find_one(filter="2") assert "test_value_2" == result_obj["field"] - @unittest.skipIf(mongomock is None, "mongomock package not present") def test_replace_one_with_filter(self): collection = mongomock.MongoClient().db.collection obj1 = {"_id": "1", "field": "test_value_1"} @@ -182,7 +173,6 @@ def test_replace_one_with_filter(self): result_obj = collection.find_one(filter="2") assert "test_value_2" == result_obj["field"] - @unittest.skipIf(mongomock is None, "mongomock package not present") def test_replace_one_with_upsert(self): collection = mongomock.MongoClient().db.collection @@ -192,7 +182,6 @@ def test_replace_one_with_upsert(self): result_obj = collection.find_one(filter="1") assert "test_value_1" == result_obj["field"] - @unittest.skipIf(mongomock is None, "mongomock package not present") def test_replace_many(self): collection = mongomock.MongoClient().db.collection obj1 = {"_id": "1", "field": "test_value_1"} @@ -209,7 +198,6 @@ def test_replace_many(self): result_obj = collection.find_one(filter="2") assert "test_value_2_updated" == result_obj["field"] - @unittest.skipIf(mongomock is None, "mongomock package not present") def test_replace_many_with_upsert(self): collection = mongomock.MongoClient().db.collection obj1 = {"_id": "1", "field": "test_value_1"} @@ -223,7 +211,6 @@ def test_replace_many_with_upsert(self): result_obj = collection.find_one(filter="2") assert "test_value_2" == result_obj["field"] - @unittest.skipIf(mongomock is None, "mongomock package not present") def test_delete_one(self): collection = mongomock.MongoClient().db.collection obj = {"_id": "1"} @@ -233,7 +220,6 @@ def test_delete_one(self): assert 0 == collection.count() - @unittest.skipIf(mongomock is None, "mongomock package not present") def test_delete_many(self): collection = mongomock.MongoClient().db.collection obj1 = {"_id": "1", "field": "value"} @@ -244,7 +230,6 @@ def test_delete_many(self): assert 0 == collection.count() - @unittest.skipIf(mongomock is None, "mongomock package not present") def test_find_one(self): collection = mongomock.MongoClient().db.collection obj = {"test_find_one": "test_value"} @@ -254,7 +239,6 @@ def test_find_one(self): result_obj = {result: result_obj[result] for result in result_obj} assert obj == result_obj - @unittest.skipIf(mongomock is None, "mongomock package not present") def test_find_many(self): collection = mongomock.MongoClient().db.collection objs = [{"_id": 1, "test_find_many_1": "test_value"}, {"_id": 2, "test_find_many_2": "test_value"}] @@ -264,7 +248,6 @@ def test_find_many(self): assert len(list(result_objs)) > 1 - @unittest.skipIf(mongomock is None, "mongomock package not present") def test_find_many_with_projection(self): collection = mongomock.MongoClient().db.collection objs = [ @@ -277,10 +260,8 @@ def test_find_many_with_projection(self): result_objs = self.hook.find( mongo_collection=collection, query={}, projection=projection, find_one=False ) + assert "_id" not in result_objs[0] - self.assertRaises(KeyError, lambda x: x[0]["_id"], result_objs) - - @unittest.skipIf(mongomock is None, "mongomock package not present") def test_aggregate(self): collection = mongomock.MongoClient().db.collection objs = [ @@ -296,11 +277,12 @@ def test_aggregate(self): results = self.hook.aggregate(collection, aggregate_query) assert len(list(results)) == 2 - def test_context_manager(self): - with MongoHook(conn_id="mongo_default", mongo_db="default") as ctx_hook: - ctx_hook.get_conn() - assert isinstance(ctx_hook, MongoHook) - assert ctx_hook.client is not None +def test_context_manager(): + with MongoHook(conn_id="mongo_default", mongo_db="default") as ctx_hook: + ctx_hook.get_conn() + + assert isinstance(ctx_hook, MongoHook) + assert ctx_hook.client is not None - assert ctx_hook.client is None + assert ctx_hook.client is None diff --git a/tests/providers/mongo/sensors/test_mongo.py b/tests/providers/mongo/sensors/test_mongo.py index dafc9942a1885..98eaec52738f7 100644 --- a/tests/providers/mongo/sensors/test_mongo.py +++ b/tests/providers/mongo/sensors/test_mongo.py @@ -17,8 +17,6 @@ # under the License. from __future__ import annotations -import unittest - import pytest from airflow.models import Connection @@ -31,10 +29,10 @@ @pytest.mark.integration("mongo") -class TestMongoSensor(unittest.TestCase): - def setUp(self): +class TestMongoSensor: + def setup_method(self): db.merge_conn( - Connection(conn_id="mongo_test", conn_type="mongo", host="mongo", port="27017", schema="test") + Connection(conn_id="mongo_test", conn_type="mongo", host="mongo", port=27017, schema="test") ) args = {"owner": "airflow", "start_date": DEFAULT_DATE} diff --git a/tests/providers/mysql/hooks/test_mysql.py b/tests/providers/mysql/hooks/test_mysql.py index f0c32b5091e62..4c4d9918641cf 100644 --- a/tests/providers/mysql/hooks/test_mysql.py +++ b/tests/providers/mysql/hooks/test_mysql.py @@ -19,27 +19,24 @@ import json import os -import unittest import uuid from contextlib import closing from unittest import mock import MySQLdb.cursors import pytest -from parameterized import parameterized from airflow.models import Connection from airflow.models.dag import DAG from airflow.providers.mysql.hooks.mysql import MySqlHook from airflow.utils import timezone +from tests.test_utils.asserts import assert_equal_ignore_multiple_spaces SSL_DICT = {"cert": "/tmp/client-cert.pem", "ca": "/tmp/server-ca.pem", "key": "/tmp/client-key.pem"} -class TestMySqlHookConn(unittest.TestCase): - def setUp(self): - super().setUp() - +class TestMySqlHookConn: + def setup_method(self): self.connection = Connection( conn_type="mysql", login="login", @@ -178,10 +175,8 @@ def test_get_conn_rds_iam(self, mock_client, mock_connect): ) -class TestMySqlHookConnMySqlConnectorPython(unittest.TestCase): - def setUp(self): - super().setUp() - +class TestMySqlHookConnMySqlConnectorPython: + def setup_method(self): self.connection = Connection( login="login", password="password", @@ -252,10 +247,8 @@ def autocommit(self, autocommit): self._autocommit = autocommit -class TestMySqlHook(unittest.TestCase): - def setUp(self): - super().setUp() - +class TestMySqlHook: + def setup_method(self): self.cur = mock.MagicMock(rowcount=0) self.conn = mock.MagicMock() self.conn.cursor.return_value = self.cur @@ -269,7 +262,7 @@ def get_conn(self): self.db_hook = SubMySqlHook() - @parameterized.expand([(True,), (False,)]) + @pytest.mark.parametrize("autocommit", [True, False]) def test_set_autocommit_mysql_connector(self, autocommit): conn = MockMySQLConnectorConnection() self.db_hook.set_autocommit(conn, autocommit) @@ -389,25 +382,20 @@ def __exit__(self, exc_type, exc_val, exc_tb): @pytest.mark.backend("mysql") -class TestMySql(unittest.TestCase): - def setUp(self): +class TestMySql: + def setup_method(self): args = {"owner": "airflow", "start_date": DEFAULT_DATE} dag = DAG(TEST_DAG_ID, default_args=args) self.dag = dag - def tearDown(self): + def teardown_method(self): drop_tables = {"test_mysql_to_mysql", "test_airflow"} with closing(MySqlHook().get_conn()) as conn: with closing(conn.cursor()) as cursor: for table in drop_tables: cursor.execute(f"DROP TABLE IF EXISTS {table}") - @parameterized.expand( - [ - ("mysqlclient",), - ("mysql-connector-python",), - ] - ) + @pytest.mark.parametrize("client", ["mysqlclient", "mysql-connector-python"]) @mock.patch.dict( "os.environ", { @@ -440,12 +428,7 @@ def test_mysql_hook_test_bulk_load(self, client): results = tuple(result[0] for result in cursor.fetchall()) assert sorted(results) == sorted(records) - @parameterized.expand( - [ - ("mysqlclient",), - ("mysql-connector-python",), - ] - ) + @pytest.mark.parametrize("client", ["mysqlclient", "mysql-connector-python"]) def test_mysql_hook_test_bulk_dump(self, client): with MySqlContext(client): hook = MySqlHook("airflow_db") @@ -462,14 +445,9 @@ def test_mysql_hook_test_bulk_dump(self, client): else: raise pytest.skip("Skip test_mysql_hook_test_bulk_load since file output is not permitted") - @parameterized.expand( - [ - ("mysqlclient",), - ("mysql-connector-python",), - ] - ) + @pytest.mark.parametrize("client", ["mysqlclient", "mysql-connector-python"]) @mock.patch("airflow.providers.mysql.hooks.mysql.MySqlHook.get_conn") - def test_mysql_hook_test_bulk_dump_mock(self, client, mock_get_conn): + def test_mysql_hook_test_bulk_dump_mock(self, mock_get_conn, client): with MySqlContext(client): mock_execute = mock.MagicMock() mock_get_conn.return_value.cursor.return_value.execute = mock_execute @@ -479,11 +457,9 @@ def test_mysql_hook_test_bulk_dump_mock(self, client, mock_get_conn): tmp_file = "/path/to/output/file" hook.bulk_dump(table, tmp_file) - from tests.test_utils.asserts import assert_equal_ignore_multiple_spaces - assert mock_execute.call_count == 1 query = f""" SELECT * INTO OUTFILE '{tmp_file}' FROM {table} """ - assert_equal_ignore_multiple_spaces(self, mock_execute.call_args[0][0], query) + assert_equal_ignore_multiple_spaces(None, mock_execute.call_args[0][0], query) diff --git a/tests/providers/mysql/operators/test_mysql.py b/tests/providers/mysql/operators/test_mysql.py index a272f1f3d489b..379459985aa6f 100644 --- a/tests/providers/mysql/operators/test_mysql.py +++ b/tests/providers/mysql/operators/test_mysql.py @@ -18,12 +18,10 @@ from __future__ import annotations import os -import unittest from contextlib import closing from tempfile import NamedTemporaryFile import pytest -from parameterized import parameterized from airflow.models.dag import DAG from airflow.providers.mysql.hooks.mysql import MySqlHook @@ -38,25 +36,20 @@ @pytest.mark.backend("mysql") -class TestMySql(unittest.TestCase): - def setUp(self): +class TestMySql: + def setup_method(self): args = {"owner": "airflow", "start_date": DEFAULT_DATE} dag = DAG(TEST_DAG_ID, default_args=args) self.dag = dag - def tearDown(self): + def teardown_method(self): drop_tables = {"test_mysql_to_mysql", "test_airflow"} with closing(MySqlHook().get_conn()) as conn: with closing(conn.cursor()) as cursor: for table in drop_tables: cursor.execute(f"DROP TABLE IF EXISTS {table}") - @parameterized.expand( - [ - ("mysqlclient",), - ("mysql-connector-python",), - ] - ) + @pytest.mark.parametrize("client", ["mysqlclient", "mysql-connector-python"]) def test_mysql_operator_test(self, client): with MySqlContext(client): sql = """ @@ -67,12 +60,7 @@ def test_mysql_operator_test(self, client): op = MySqlOperator(task_id="basic_mysql", sql=sql, dag=self.dag) op.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE, ignore_ti_state=True) - @parameterized.expand( - [ - ("mysqlclient",), - ("mysql-connector-python",), - ] - ) + @pytest.mark.parametrize("client", ["mysqlclient", "mysql-connector-python"]) def test_mysql_operator_test_multi(self, client): with MySqlContext(client): sql = [ @@ -87,12 +75,7 @@ def test_mysql_operator_test_multi(self, client): ) op.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE, ignore_ti_state=True) - @parameterized.expand( - [ - ("mysqlclient",), - ("mysql-connector-python",), - ] - ) + @pytest.mark.parametrize("client", ["mysqlclient", "mysql-connector-python"]) def test_overwrite_schema(self, client): """ Verifies option to overwrite connection schema diff --git a/tests/providers/mysql/transfers/test_presto_to_mysql.py b/tests/providers/mysql/transfers/test_presto_to_mysql.py index f5d572b7be792..7249cfe4ecad7 100644 --- a/tests/providers/mysql/transfers/test_presto_to_mysql.py +++ b/tests/providers/mysql/transfers/test_presto_to_mysql.py @@ -18,21 +18,26 @@ from __future__ import annotations import os -import unittest +from datetime import datetime from unittest.mock import patch +import pytest + +from airflow.models.dag import DAG from airflow.providers.mysql.transfers.presto_to_mysql import PrestoToMySqlOperator -from tests.providers.apache.hive import DEFAULT_DATE, TestHiveEnvironment + +DEFAULT_DATE = datetime(2022, 1, 1) -class TestPrestoToMySqlTransfer(TestHiveEnvironment): - def setUp(self): +class TestPrestoToMySqlTransfer: + def setup_method(self): self.kwargs = dict( sql="sql", mysql_table="mysql_table", task_id="test_presto_to_mysql_transfer", ) - super().setUp() + args = {"owner": "airflow", "start_date": DEFAULT_DATE} + self.dag = DAG("test_presto_to_mysql_transfer", default_args=args) @patch("airflow.providers.mysql.transfers.presto_to_mysql.MySqlHook") @patch("airflow.providers.mysql.transfers.presto_to_mysql.PrestoHook") @@ -57,8 +62,8 @@ def test_execute_with_mysql_preoperator(self, mock_presto_hook, mock_mysql_hook) table=self.kwargs["mysql_table"], rows=mock_presto_hook.return_value.get_records.return_value ) - @unittest.skipIf( - "AIRFLOW_RUNALL_TESTS" not in os.environ, "Skipped because AIRFLOW_RUNALL_TESTS is not set" + @pytest.mark.skipif( + "AIRFLOW_RUNALL_TESTS" not in os.environ, reason="Skipped because AIRFLOW_RUNALL_TESTS is not set" ) def test_presto_to_mysql(self): op = PrestoToMySqlOperator( diff --git a/tests/providers/mysql/transfers/test_s3_to_mysql.py b/tests/providers/mysql/transfers/test_s3_to_mysql.py index 743225d3c06f7..50e62bca9ce30 100644 --- a/tests/providers/mysql/transfers/test_s3_to_mysql.py +++ b/tests/providers/mysql/transfers/test_s3_to_mysql.py @@ -16,7 +16,6 @@ # under the License. from __future__ import annotations -import unittest from unittest.mock import patch import pytest @@ -28,8 +27,8 @@ from airflow.utils.session import create_session -class TestS3ToMySqlTransfer(unittest.TestCase): - def setUp(self): +class TestS3ToMySqlTransfer: + def setup_method(self): configuration.conf.load_test_config() db.merge_conn( @@ -99,7 +98,7 @@ def test_execute_exception(self, mock_remove, mock_bulk_load_custom, mock_downlo ) mock_remove.assert_called_once_with(mock_download_file.return_value) - def tearDown(self): + def teardown_method(self): with create_session() as session: ( session.query(models.Connection) diff --git a/tests/providers/mysql/transfers/test_trino_to_mysql.py b/tests/providers/mysql/transfers/test_trino_to_mysql.py index 6a7f01ad2bf16..390c84729b2b8 100644 --- a/tests/providers/mysql/transfers/test_trino_to_mysql.py +++ b/tests/providers/mysql/transfers/test_trino_to_mysql.py @@ -18,21 +18,26 @@ from __future__ import annotations import os -import unittest +from datetime import datetime from unittest.mock import patch +import pytest + +from airflow.models.dag import DAG from airflow.providers.mysql.transfers.trino_to_mysql import TrinoToMySqlOperator -from tests.providers.apache.hive import DEFAULT_DATE, TestHiveEnvironment + +DEFAULT_DATE = datetime(2022, 1, 1) -class TestTrinoToMySqlTransfer(TestHiveEnvironment): - def setUp(self): +class TestTrinoToMySqlTransfer: + def setup_method(self): self.kwargs = dict( sql="sql", mysql_table="mysql_table", task_id="test_trino_to_mysql_transfer", ) - super().setUp() + args = {"owner": "airflow", "start_date": DEFAULT_DATE} + self.dag = DAG("test_trino_to_mysql_transfer", default_args=args) @patch("airflow.providers.mysql.transfers.trino_to_mysql.MySqlHook") @patch("airflow.providers.mysql.transfers.trino_to_mysql.TrinoHook") @@ -57,8 +62,8 @@ def test_execute_with_mysql_preoperator(self, mock_trino_hook, mock_mysql_hook): table=self.kwargs["mysql_table"], rows=mock_trino_hook.return_value.get_records.return_value ) - @unittest.skipIf( - "AIRFLOW_RUNALL_TESTS" not in os.environ, "Skipped because AIRFLOW_RUNALL_TESTS is not set" + @pytest.mark.skipif( + "AIRFLOW_RUNALL_TESTS" not in os.environ, reason="Skipped because AIRFLOW_RUNALL_TESTS is not set" ) def test_trino_to_mysql(self): op = TrinoToMySqlOperator( diff --git a/tests/providers/mysql/transfers/test_vertica_to_mysql.py b/tests/providers/mysql/transfers/test_vertica_to_mysql.py index 9af6203866fdb..e13c5d05781c9 100644 --- a/tests/providers/mysql/transfers/test_vertica_to_mysql.py +++ b/tests/providers/mysql/transfers/test_vertica_to_mysql.py @@ -18,7 +18,6 @@ from __future__ import annotations import datetime -import unittest from unittest import mock from airflow.models.dag import DAG @@ -40,8 +39,8 @@ def mock_get_conn(): return conn_mock -class TestVerticaToMySqlTransfer(unittest.TestCase): - def setUp(self): +class TestVerticaToMySqlTransfer: + def setup_method(self): args = {"owner": "airflow", "start_date": datetime.datetime(2017, 1, 1)} self.dag = DAG("test_dag_id", default_args=args) diff --git a/tests/providers/neo4j/hooks/test_neo4j.py b/tests/providers/neo4j/hooks/test_neo4j.py index 1ac536335326f..2191b326b1ded 100644 --- a/tests/providers/neo4j/hooks/test_neo4j.py +++ b/tests/providers/neo4j/hooks/test_neo4j.py @@ -16,23 +16,23 @@ # under the License. from __future__ import annotations -import unittest from unittest import mock -from parameterized import parameterized +import pytest from airflow.models import Connection from airflow.providers.neo4j.hooks.neo4j import Neo4jHook -class TestNeo4jHookConn(unittest.TestCase): - @parameterized.expand( +class TestNeo4jHookConn: + @pytest.mark.parametrize( + "conn_extra, expected_uri", [ - [{}, "bolt://host:7687"], - [{"bolt_scheme": True}, "bolt://host:7687"], - [{"certs_self_signed": True, "bolt_scheme": True}, "bolt+ssc://host:7687"], - [{"certs_trusted_ca": True, "bolt_scheme": True}, "bolt+s://host:7687"], - ] + ({}, "bolt://host:7687"), + ({"bolt_scheme": True}, "bolt://host:7687"), + ({"certs_self_signed": True, "bolt_scheme": True}, "bolt+ssc://host:7687"), + ({"certs_trusted_ca": True, "bolt_scheme": True}, "bolt+s://host:7687"), + ], ) def test_get_uri_neo4j_scheme(self, conn_extra, expected_uri): connection = Connection( @@ -75,10 +75,7 @@ def test_run_with_schema(self, mock_graph_database): ] ) session = mock_graph_database.driver.return_value.session.return_value.__enter__.return_value - self.assertEqual( - session.run.return_value.data.return_value, - op_result, - ) + assert op_result == session.run.return_value.data.return_value @mock.patch("airflow.providers.neo4j.hooks.neo4j.GraphDatabase") def test_run_without_schema(self, mock_graph_database): @@ -103,7 +100,4 @@ def test_run_without_schema(self, mock_graph_database): ] ) session = mock_graph_database.driver.return_value.session.return_value.__enter__.return_value - self.assertEqual( - session.run.return_value.data.return_value, - op_result, - ) + assert op_result == session.run.return_value.data.return_value diff --git a/tests/providers/neo4j/operators/test_neo4j.py b/tests/providers/neo4j/operators/test_neo4j.py index a38960ebf27ea..2cf95ec3374f8 100644 --- a/tests/providers/neo4j/operators/test_neo4j.py +++ b/tests/providers/neo4j/operators/test_neo4j.py @@ -16,7 +16,6 @@ # under the License. from __future__ import annotations -import unittest from unittest import mock from airflow.providers.neo4j.operators.neo4j import Neo4jOperator @@ -28,7 +27,7 @@ TEST_DAG_ID = "unit_test_dag" -class TestNeo4jOperator(unittest.TestCase): +class TestNeo4jOperator: @mock.patch("airflow.providers.neo4j.operators.neo4j.Neo4jHook") def test_neo4j_operator_test(self, mock_hook): diff --git a/tests/providers/openfaas/hooks/test_openfaas.py b/tests/providers/openfaas/hooks/test_openfaas.py index a5a8a99e80ae3..17172cc383425 100644 --- a/tests/providers/openfaas/hooks/test_openfaas.py +++ b/tests/providers/openfaas/hooks/test_openfaas.py @@ -17,11 +17,9 @@ # under the License. from __future__ import annotations -import unittest from unittest import mock import pytest -import requests_mock from airflow.exceptions import AirflowException from airflow.hooks.base import BaseHook @@ -31,21 +29,20 @@ FUNCTION_NAME = "function_name" -class TestOpenFaasHook(unittest.TestCase): +class TestOpenFaasHook: GET_FUNCTION = "/system/function/" INVOKE_ASYNC_FUNCTION = "/async-function/" INVOKE_FUNCTION = "/function/" DEPLOY_FUNCTION = "/system/functions" UPDATE_FUNCTION = "/system/functions" - def setUp(self): + def setup_method(self): self.hook = OpenFaasHook(function_name=FUNCTION_NAME) self.mock_response = {"ans": "a"} @mock.patch.object(BaseHook, "get_connection") - @requests_mock.mock() - def test_is_function_exist_false(self, mock_get_connection, m): - m.get( + def test_is_function_exist_false(self, mock_get_connection, requests_mock): + requests_mock.get( "http://open-faas.io" + self.GET_FUNCTION + FUNCTION_NAME, json=self.mock_response, status_code=404, @@ -57,9 +54,8 @@ def test_is_function_exist_false(self, mock_get_connection, m): assert not does_function_exist @mock.patch.object(BaseHook, "get_connection") - @requests_mock.mock() - def test_is_function_exist_true(self, mock_get_connection, m): - m.get( + def test_is_function_exist_true(self, mock_get_connection, requests_mock): + requests_mock.get( "http://open-faas.io" + self.GET_FUNCTION + FUNCTION_NAME, json=self.mock_response, status_code=202, @@ -71,18 +67,20 @@ def test_is_function_exist_true(self, mock_get_connection, m): assert does_function_exist @mock.patch.object(BaseHook, "get_connection") - @requests_mock.mock() - def test_update_function_true(self, mock_get_connection, m): - m.put("http://open-faas.io" + self.UPDATE_FUNCTION, json=self.mock_response, status_code=202) + def test_update_function_true(self, mock_get_connection, requests_mock): + requests_mock.put( + "http://open-faas.io" + self.UPDATE_FUNCTION, json=self.mock_response, status_code=202 + ) mock_connection = Connection(host="http://open-faas.io") mock_get_connection.return_value = mock_connection self.hook.update_function({}) # returns None @mock.patch.object(BaseHook, "get_connection") - @requests_mock.mock() - def test_update_function_false(self, mock_get_connection, m): - m.put("http://open-faas.io" + self.UPDATE_FUNCTION, json=self.mock_response, status_code=400) + def test_update_function_false(self, mock_get_connection, requests_mock): + requests_mock.put( + "http://open-faas.io" + self.UPDATE_FUNCTION, json=self.mock_response, status_code=400 + ) mock_connection = Connection(host="http://open-faas.io") mock_get_connection.return_value = mock_connection @@ -91,9 +89,8 @@ def test_update_function_false(self, mock_get_connection, m): assert "failed to update " + FUNCTION_NAME in str(ctx.value) @mock.patch.object(BaseHook, "get_connection") - @requests_mock.mock() - def test_invoke_function_false(self, mock_get_connection, m): - m.post( + def test_invoke_function_false(self, mock_get_connection, requests_mock): + requests_mock.post( "http://open-faas.io" + self.INVOKE_FUNCTION + FUNCTION_NAME, json=self.mock_response, status_code=400, @@ -106,9 +103,8 @@ def test_invoke_function_false(self, mock_get_connection, m): assert "failed to invoke function" in str(ctx.value) @mock.patch.object(BaseHook, "get_connection") - @requests_mock.mock() - def test_invoke_function_true(self, mock_get_connection, m): - m.post( + def test_invoke_function_true(self, mock_get_connection, requests_mock): + requests_mock.post( "http://open-faas.io" + self.INVOKE_FUNCTION + FUNCTION_NAME, json=self.mock_response, status_code=200, @@ -118,9 +114,8 @@ def test_invoke_function_true(self, mock_get_connection, m): assert self.hook.invoke_function({}) is None @mock.patch.object(BaseHook, "get_connection") - @requests_mock.mock() - def test_invoke_async_function_false(self, mock_get_connection, m): - m.post( + def test_invoke_async_function_false(self, mock_get_connection, requests_mock): + requests_mock.post( "http://open-faas.io" + self.INVOKE_ASYNC_FUNCTION + FUNCTION_NAME, json=self.mock_response, status_code=400, @@ -133,9 +128,8 @@ def test_invoke_async_function_false(self, mock_get_connection, m): assert "failed to invoke function" in str(ctx.value) @mock.patch.object(BaseHook, "get_connection") - @requests_mock.mock() - def test_invoke_async_function_true(self, mock_get_connection, m): - m.post( + def test_invoke_async_function_true(self, mock_get_connection, requests_mock): + requests_mock.post( "http://open-faas.io" + self.INVOKE_ASYNC_FUNCTION + FUNCTION_NAME, json=self.mock_response, status_code=202, @@ -145,17 +139,17 @@ def test_invoke_async_function_true(self, mock_get_connection, m): assert self.hook.invoke_async_function({}) is None @mock.patch.object(BaseHook, "get_connection") - @requests_mock.mock() - def test_deploy_function_function_already_exist(self, mock_get_connection, m): - m.put("http://open-faas.io/" + self.UPDATE_FUNCTION, json=self.mock_response, status_code=202) + def test_deploy_function_function_already_exist(self, mock_get_connection, requests_mock): + requests_mock.put( + "http://open-faas.io/" + self.UPDATE_FUNCTION, json=self.mock_response, status_code=202 + ) mock_connection = Connection(host="http://open-faas.io/") mock_get_connection.return_value = mock_connection assert self.hook.deploy_function(True, {}) is None @mock.patch.object(BaseHook, "get_connection") - @requests_mock.mock() - def test_deploy_function_function_not_exist(self, mock_get_connection, m): - m.post("http://open-faas.io" + self.DEPLOY_FUNCTION, json={}, status_code=202) + def test_deploy_function_function_not_exist(self, mock_get_connection, requests_mock): + requests_mock.post("http://open-faas.io" + self.DEPLOY_FUNCTION, json={}, status_code=202) mock_connection = Connection(host="http://open-faas.io") mock_get_connection.return_value = mock_connection assert self.hook.deploy_function(False, {}) is None diff --git a/tests/providers/opsgenie/hooks/test_opsgenie.py b/tests/providers/opsgenie/hooks/test_opsgenie.py index f2d51476349c6..8b3a518356f12 100644 --- a/tests/providers/opsgenie/hooks/test_opsgenie.py +++ b/tests/providers/opsgenie/hooks/test_opsgenie.py @@ -17,7 +17,6 @@ # under the License. from __future__ import annotations -import unittest from unittest import mock import pytest @@ -29,7 +28,7 @@ from airflow.utils import db -class TestOpsgenieAlertHook(unittest.TestCase): +class TestOpsgenieAlertHook: conn_id = "opsgenie_conn_id_test" opsgenie_alert_endpoint = "https://api.opsgenie.com/v2/alerts" _create_alert_payload = { @@ -67,7 +66,7 @@ class TestOpsgenieAlertHook(unittest.TestCase): "request_id": "43a29c5c-3dbf-4fa4-9c26-f4f71023e120", } - def setUp(self): + def setup_method(self): db.merge_conn( Connection( conn_id=self.conn_id, diff --git a/tests/providers/opsgenie/operators/test_opsgenie.py b/tests/providers/opsgenie/operators/test_opsgenie.py index ee06dfc1ca1ff..0194660323f7f 100644 --- a/tests/providers/opsgenie/operators/test_opsgenie.py +++ b/tests/providers/opsgenie/operators/test_opsgenie.py @@ -17,7 +17,6 @@ # under the License. from __future__ import annotations -import unittest from unittest import mock from airflow.models.dag import DAG @@ -31,7 +30,7 @@ DEFAULT_DATE = timezone.datetime(2017, 1, 1) -class TestOpsgenieCreateAlertOperator(unittest.TestCase): +class TestOpsgenieCreateAlertOperator: _config = { "message": "An example alert message", "alias": "Life is too short for no alias", @@ -78,7 +77,7 @@ class TestOpsgenieCreateAlertOperator(unittest.TestCase): "note": _config["note"], } - def setUp(self): + def setup_method(self): args = {"owner": "airflow", "start_date": DEFAULT_DATE} self.dag = DAG("test_dag_id", default_args=args) @@ -111,7 +110,7 @@ def test_properties(self): assert self._config["note"] == operator.note -class TestOpsgenieCloseAlertOperator(unittest.TestCase): +class TestOpsgenieCloseAlertOperator: _config = {"user": "example_user", "note": "my_closing_note", "source": "some_source"} expected_payload_dict = { "user": _config["user"], @@ -119,7 +118,7 @@ class TestOpsgenieCloseAlertOperator(unittest.TestCase): "source": _config["source"], } - def setUp(self): + def setup_method(self): args = {"owner": "airflow", "start_date": DEFAULT_DATE} self.dag = DAG("test_dag_id", default_args=args) @@ -145,8 +144,8 @@ def test_properties(self): assert self._config["source"] == operator.source -class TestOpsgenieDeleteAlertOperator(unittest.TestCase): - def setUp(self): +class TestOpsgenieDeleteAlertOperator: + def setup_method(self): args = {"owner": "airflow", "start_date": DEFAULT_DATE} self.dag = DAG("test_dag_id", default_args=args) diff --git a/tests/providers/oracle/hooks/test_oracle.py b/tests/providers/oracle/hooks/test_oracle.py index 24c492d10c28c..9f49618b45435 100644 --- a/tests/providers/oracle/hooks/test_oracle.py +++ b/tests/providers/oracle/hooks/test_oracle.py @@ -18,27 +18,19 @@ from __future__ import annotations import json -import unittest from datetime import datetime from unittest import mock import numpy +import oracledb import pytest from airflow.models import Connection from airflow.providers.oracle.hooks.oracle import OracleHook -try: - import oracledb -except ImportError: - oracledb = None # type: ignore - - -@unittest.skipIf(oracledb is None, "oracledb package not present") -class TestOracleHookConn(unittest.TestCase): - def setUp(self): - super().setUp() +class TestOracleHookConn: + def setup_method(self): self.connection = Connection( login="login", password="password", host="host", schema="schema", port=1521 ) @@ -265,11 +257,8 @@ def test_type_checking_thick_mode_config_dir(self): self.db_hook.get_conn() -@unittest.skipIf(oracledb is None, "oracledb package not present") -class TestOracleHook(unittest.TestCase): - def setUp(self): - super().setUp() - +class TestOracleHook: + def setup_method(self): self.cur = mock.MagicMock(rowcount=0) self.conn = mock.MagicMock() self.conn.cursor.return_value = self.cur diff --git a/tests/providers/oracle/operators/test_oracle.py b/tests/providers/oracle/operators/test_oracle.py index ba5e82df3999b..2e5a8e10e63dc 100644 --- a/tests/providers/oracle/operators/test_oracle.py +++ b/tests/providers/oracle/operators/test_oracle.py @@ -16,15 +16,16 @@ # under the License. from __future__ import annotations -import unittest from unittest import mock +import pytest + from airflow.providers.common.sql.hooks.sql import fetch_all_handler from airflow.providers.oracle.hooks.oracle import OracleHook from airflow.providers.oracle.operators.oracle import OracleOperator, OracleStoredProcedureOperator -class TestOracleOperator(unittest.TestCase): +class TestOracleOperator: @mock.patch("airflow.providers.common.sql.operators.sql.SQLExecuteQueryOperator.get_db_hook") def test_execute(self, mock_get_db_hook): sql = "SELECT * FROM test_table" @@ -34,13 +35,14 @@ def test_execute(self, mock_get_db_hook): context = "test_context" task_id = "test_task_id" - operator = OracleOperator( - sql=sql, - oracle_conn_id=oracle_conn_id, - parameters=parameters, - autocommit=autocommit, - task_id=task_id, - ) + with pytest.warns(DeprecationWarning, match="This class is deprecated.*"): + operator = OracleOperator( + sql=sql, + oracle_conn_id=oracle_conn_id, + parameters=parameters, + autocommit=autocommit, + task_id=task_id, + ) operator.execute(context=context) mock_get_db_hook.return_value.run.assert_called_once_with( sql=sql, @@ -52,7 +54,7 @@ def test_execute(self, mock_get_db_hook): ) -class TestOracleStoredProcedureOperator(unittest.TestCase): +class TestOracleStoredProcedureOperator: @mock.patch.object(OracleHook, "run", autospec=OracleHook.run) def test_execute(self, mock_run): procedure = "test" diff --git a/tests/providers/oracle/transfers/test_oracle_to_oracle.py b/tests/providers/oracle/transfers/test_oracle_to_oracle.py index 587238620aebc..e2e66706da852 100644 --- a/tests/providers/oracle/transfers/test_oracle_to_oracle.py +++ b/tests/providers/oracle/transfers/test_oracle_to_oracle.py @@ -17,16 +17,14 @@ # under the License. from __future__ import annotations -import unittest from unittest import mock from unittest.mock import MagicMock from airflow.providers.oracle.transfers.oracle_to_oracle import OracleToOracleOperator -class TestOracleToOracleTransfer(unittest.TestCase): - @staticmethod - def test_execute(): +class TestOracleToOracleTransfer: + def test_execute(self): oracle_destination_conn_id = "oracle_destination_conn_id" destination_table = "destination_table" oracle_source_conn_id = "oracle_source_conn_id" diff --git a/tests/providers/papermill/operators/test_papermill.py b/tests/providers/papermill/operators/test_papermill.py index dc072b9e3f2d8..2ab23280a73be 100644 --- a/tests/providers/papermill/operators/test_papermill.py +++ b/tests/providers/papermill/operators/test_papermill.py @@ -17,7 +17,6 @@ # under the License. from __future__ import annotations -import unittest from unittest.mock import patch from airflow.models import DAG, DagRun, TaskInstance @@ -27,7 +26,7 @@ DEFAULT_DATE = timezone.datetime(2021, 1, 1) -class TestPapermillOperator(unittest.TestCase): +class TestPapermillOperator: @patch("airflow.providers.papermill.operators.papermill.pm") def test_execute(self, mock_papermill): in_nb = "/tmp/does_not_exist" diff --git a/tests/providers/postgres/hooks/test_postgres.py b/tests/providers/postgres/hooks/test_postgres.py index bc0c1974381d8..70ca7823835ac 100644 --- a/tests/providers/postgres/hooks/test_postgres.py +++ b/tests/providers/postgres/hooks/test_postgres.py @@ -18,7 +18,6 @@ from __future__ import annotations import json -import unittest from tempfile import NamedTemporaryFile from unittest import mock @@ -31,8 +30,7 @@ class TestPostgresHookConn: - @pytest.fixture(autouse=True) - def setup(self): + def setup_method(self): self.connection = Connection(login="login", password="password", host="host", schema="database") class UnitTestPostgresHook(PostgresHook): @@ -258,14 +256,11 @@ def test_schema_kwarg_database_kwarg_compatibility(self): assert hook.database == database -class TestPostgresHook(unittest.TestCase): - def __init__(self, *args, **kwargs): - super().__init__(*args, **kwargs) - self.table = "test_postgres_hook_table" - - def setUp(self): - super().setUp() +@pytest.mark.backend("postgres") +class TestPostgresHook: + table = "test_postgres_hook_table" + def setup_method(self): self.cur = mock.MagicMock(rowcount=0) self.conn = conn = mock.MagicMock() self.conn.cursor.return_value = self.cur @@ -278,14 +273,11 @@ def get_conn(self): self.db_hook = UnitTestPostgresHook() - def tearDown(self): - super().tearDown() - + def teardown_method(self): with PostgresHook().get_conn() as conn: with conn.cursor() as cur: cur.execute(f"DROP TABLE IF EXISTS {self.table}") - @pytest.mark.backend("postgres") def test_copy_expert(self): open_mock = mock.mock_open(read_data='{"some": "json"}') with mock.patch("airflow.providers.postgres.hooks.postgres.open", open_mock): @@ -302,7 +294,6 @@ def test_copy_expert(self): self.cur.copy_expert.assert_called_once_with(statement, open_mock.return_value) assert open_mock.call_args[0] == (filename, "r+") - @pytest.mark.backend("postgres") def test_bulk_load(self): hook = PostgresHook() input_data = ["foo", "bar", "baz"] @@ -322,7 +313,6 @@ def test_bulk_load(self): assert sorted(input_data) == sorted(results) - @pytest.mark.backend("postgres") def test_bulk_dump(self): hook = PostgresHook() input_data = ["foo", "bar", "baz"] @@ -341,7 +331,6 @@ def test_bulk_dump(self): assert sorted(input_data) == sorted(results) - @pytest.mark.backend("postgres") def test_insert_rows(self): table = "table" rows = [("hello",), ("world",)] @@ -358,7 +347,6 @@ def test_insert_rows(self): for row in rows: self.cur.execute.assert_any_call(sql, row) - @pytest.mark.backend("postgres") def test_insert_rows_replace(self): table = "table" rows = [ @@ -388,7 +376,6 @@ def test_insert_rows_replace(self): for row in rows: self.cur.execute.assert_any_call(sql, row) - @pytest.mark.backend("postgres") def test_insert_rows_replace_missing_target_field_arg(self): table = "table" rows = [ @@ -407,7 +394,6 @@ def test_insert_rows_replace_missing_target_field_arg(self): assert str(ctx.value) == "PostgreSQL ON CONFLICT upsert syntax requires column names" - @pytest.mark.backend("postgres") def test_insert_rows_replace_missing_replace_index_arg(self): table = "table" rows = [ @@ -426,7 +412,6 @@ def test_insert_rows_replace_missing_replace_index_arg(self): assert str(ctx.value) == "PostgreSQL ON CONFLICT upsert syntax requires an unique index" - @pytest.mark.backend("postgres") def test_insert_rows_replace_all_index(self): table = "table" rows = [ @@ -456,7 +441,6 @@ def test_insert_rows_replace_all_index(self): for row in rows: self.cur.execute.assert_any_call(sql, row) - @pytest.mark.backend("postgres") def test_rowcount(self): hook = PostgresHook() input_data = ["foo", "bar", "baz"] diff --git a/tests/providers/postgres/operators/test_postgres.py b/tests/providers/postgres/operators/test_postgres.py index 394cfc2618fd6..4b615917ea28e 100644 --- a/tests/providers/postgres/operators/test_postgres.py +++ b/tests/providers/postgres/operators/test_postgres.py @@ -17,8 +17,6 @@ # under the License. from __future__ import annotations -import unittest - import pytest from airflow.models.dag import DAG @@ -32,13 +30,13 @@ @pytest.mark.backend("postgres") -class TestPostgres(unittest.TestCase): - def setUp(self): +class TestPostgres: + def setup_method(self): args = {"owner": "airflow", "start_date": DEFAULT_DATE} dag = DAG(TEST_DAG_ID, default_args=args) self.dag = dag - def tearDown(self): + def teardown_method(self): tables_to_drop = ["test_postgres_to_postgres", "test_airflow"] from airflow.providers.postgres.hooks.postgres import PostgresHook diff --git a/tests/providers/presto/hooks/test_presto.py b/tests/providers/presto/hooks/test_presto.py index 7831e441c9e8c..423d4d20751fc 100644 --- a/tests/providers/presto/hooks/test_presto.py +++ b/tests/providers/presto/hooks/test_presto.py @@ -19,12 +19,10 @@ import json import re -import unittest from unittest import mock from unittest.mock import patch import pytest -from parameterized import parameterized from prestodb.transaction import IsolationLevel from airflow import AirflowException @@ -56,7 +54,7 @@ def test_generate_airflow_presto_client_info_header(): assert generate_presto_client_info() == expected -class TestPrestoHookConn(unittest.TestCase): +class TestPrestoHookConn: @patch("airflow.providers.presto.hooks.presto.prestodb.auth.BasicAuthentication") @patch("airflow.providers.presto.hooks.presto.prestodb.dbapi.connect") @patch("airflow.providers.presto.hooks.presto.PrestoHook.get_connection") @@ -190,14 +188,15 @@ def test_http_headers( mock_basic_auth.assert_called_once_with("login", "password") assert mock_connect.return_value == conn - @parameterized.expand( + @pytest.mark.parametrize( + "current_verify, expected_verify", [ ("False", False), ("false", False), ("true", True), ("true", True), ("/tmp/cert.crt", "/tmp/cert.crt"), - ] + ], ) def test_get_conn_verify(self, current_verify, expected_verify): patcher_connect = patch("airflow.providers.presto.hooks.presto.prestodb.dbapi.connect") @@ -215,10 +214,8 @@ def test_get_conn_verify(self, current_verify, expected_verify): assert mock_connect.return_value == conn -class TestPrestoHook(unittest.TestCase): - def setUp(self): - super().setUp() - +class TestPrestoHook: + def setup_method(self): self.cur = mock.MagicMock(rowcount=0) self.conn = mock.MagicMock() self.conn.cursor.return_value = self.cur diff --git a/tests/providers/presto/transfers/test_gcs_presto.py b/tests/providers/presto/transfers/test_gcs_presto.py index 247bb414bc149..fe0689099bbae 100644 --- a/tests/providers/presto/transfers/test_gcs_presto.py +++ b/tests/providers/presto/transfers/test_gcs_presto.py @@ -17,7 +17,6 @@ # under the License. from __future__ import annotations -import unittest from unittest import mock from airflow.providers.presto.transfers.gcs_to_presto import GCSToPrestoOperator @@ -33,7 +32,7 @@ SCHEMA_JSON = "path/to/file.json" -class TestGCSToPrestoOperator(unittest.TestCase): +class TestGCSToPrestoOperator: @mock.patch("airflow.providers.presto.transfers.gcs_to_presto.PrestoHook") @mock.patch("airflow.providers.presto.transfers.gcs_to_presto.GCSHook") @mock.patch("airflow.providers.presto.transfers.gcs_to_presto.NamedTemporaryFile") diff --git a/tests/providers/qubole/hooks/test_qubole.py b/tests/providers/qubole/hooks/test_qubole.py index aee6eb70690da..a037b8110a87e 100644 --- a/tests/providers/qubole/hooks/test_qubole.py +++ b/tests/providers/qubole/hooks/test_qubole.py @@ -17,7 +17,7 @@ # under the License. from __future__ import annotations -from unittest import TestCase, mock +from unittest import mock from qds_sdk.commands import PrestoCommand @@ -38,7 +38,7 @@ def get_result_mock(fp, inline, delim, fetch, arguments): fp.write(bytearray(RESULTS_WITH_NO_HEADER, "utf-8")) -class TestQuboleHook(TestCase): +class TestQuboleHook: def test_add_string_to_tags(self): tags = {"dag_id", "task_id"} add_tags(tags, "string") diff --git a/tests/providers/qubole/hooks/test_qubole_check.py b/tests/providers/qubole/hooks/test_qubole_check.py index a7bedc52cc66d..8c5b27da520ab 100644 --- a/tests/providers/qubole/hooks/test_qubole_check.py +++ b/tests/providers/qubole/hooks/test_qubole_check.py @@ -17,12 +17,10 @@ # under the License. from __future__ import annotations -import unittest - from airflow.providers.qubole.hooks.qubole_check import parse_first_row -class TestQuboleCheckHook(unittest.TestCase): +class TestQuboleCheckHook: def test_single_row_bool(self): query_result = ["true\ttrue"] record_list = parse_first_row(query_result) diff --git a/tests/providers/qubole/operators/test_qubole_check.py b/tests/providers/qubole/operators/test_qubole_check.py index 3e8ccfc968dff..74af6a29d1267 100644 --- a/tests/providers/qubole/operators/test_qubole_check.py +++ b/tests/providers/qubole/operators/test_qubole_check.py @@ -17,7 +17,6 @@ # under the License. from __future__ import annotations -import unittest from datetime import datetime from unittest import mock from unittest.mock import MagicMock @@ -49,7 +48,7 @@ ], ) class TestQuboleCheckMixin: - def setup(self): + def setup_method(self): self.task_id = "test_task" def __construct_operator(self, operator_class, **kwargs): @@ -95,8 +94,8 @@ def test_execute_fail(self, mock_handle_airflow_exception, operator_class, kwarg mock_handle_airflow_exception.assert_called_once() -class TestQuboleValueCheckOperator(unittest.TestCase): - def setUp(self): +class TestQuboleValueCheckOperator: + def setup_method(self): self.task_id = "test_task" self.conn_id = "default_conn" diff --git a/tests/providers/qubole/sensors/test_qubole.py b/tests/providers/qubole/sensors/test_qubole.py index b0212974613a0..02fd29ba7fc65 100644 --- a/tests/providers/qubole/sensors/test_qubole.py +++ b/tests/providers/qubole/sensors/test_qubole.py @@ -17,7 +17,6 @@ # under the License. from __future__ import annotations -import unittest from datetime import datetime from unittest.mock import patch @@ -35,8 +34,8 @@ DEFAULT_DATE = datetime(2017, 1, 1) -class TestQuboleSensor(unittest.TestCase): - def setUp(self): +class TestQuboleSensor: + def setup_method(self): db.merge_conn(Connection(conn_id=DEFAULT_CONN, conn_type="HTTP")) @patch("airflow.providers.qubole.sensors.qubole.QuboleFileSensor.poke") diff --git a/tests/providers/redis/hooks/test_redis.py b/tests/providers/redis/hooks/test_redis.py index efc7521d7fd88..0eb4edd76fc0a 100644 --- a/tests/providers/redis/hooks/test_redis.py +++ b/tests/providers/redis/hooks/test_redis.py @@ -17,7 +17,6 @@ # under the License. from __future__ import annotations -import unittest from unittest import mock import pytest @@ -26,7 +25,7 @@ from airflow.providers.redis.hooks.redis import RedisHook -class TestRedisHook(unittest.TestCase): +class TestRedisHook: def test_get_conn(self): hook = RedisHook(redis_conn_id="redis_default") assert hook.redis is None diff --git a/tests/providers/redis/operators/test_redis_publish.py b/tests/providers/redis/operators/test_redis_publish.py index 4fca70b3a6cc6..c5ea8a65bd626 100644 --- a/tests/providers/redis/operators/test_redis_publish.py +++ b/tests/providers/redis/operators/test_redis_publish.py @@ -17,7 +17,6 @@ # under the License. from __future__ import annotations -import unittest from unittest.mock import MagicMock import pytest @@ -31,8 +30,8 @@ @pytest.mark.integration("redis") -class TestRedisPublishOperator(unittest.TestCase): - def setUp(self): +class TestRedisPublishOperator: + def setup_method(self): args = {"owner": "airflow", "start_date": DEFAULT_DATE} self.dag = DAG("test_redis_dag_id", default_args=args) diff --git a/tests/providers/redis/sensors/test_redis_key.py b/tests/providers/redis/sensors/test_redis_key.py index 6a338d429179f..f54e816c29a50 100644 --- a/tests/providers/redis/sensors/test_redis_key.py +++ b/tests/providers/redis/sensors/test_redis_key.py @@ -17,8 +17,6 @@ # under the License. from __future__ import annotations -import unittest - import pytest from airflow.models.dag import DAG @@ -30,8 +28,8 @@ @pytest.mark.integration("redis") -class TestRedisSensor(unittest.TestCase): - def setUp(self): +class TestRedisSensor: + def setup_method(self): args = {"owner": "airflow", "start_date": DEFAULT_DATE} self.dag = DAG("test_dag_id", default_args=args) diff --git a/tests/providers/redis/sensors/test_redis_pub_sub.py b/tests/providers/redis/sensors/test_redis_pub_sub.py index 7ef56da33efb8..5ed0c40db71bd 100644 --- a/tests/providers/redis/sensors/test_redis_pub_sub.py +++ b/tests/providers/redis/sensors/test_redis_pub_sub.py @@ -17,7 +17,6 @@ # under the License. from __future__ import annotations -import unittest from time import sleep from unittest.mock import MagicMock, call, patch @@ -31,8 +30,8 @@ DEFAULT_DATE = timezone.datetime(2017, 1, 1) -class TestRedisPubSubSensor(unittest.TestCase): - def setUp(self): +class TestRedisPubSubSensor: + def setup_method(self): args = {"owner": "airflow", "start_date": DEFAULT_DATE} self.dag = DAG("test_dag_id", default_args=args) diff --git a/tests/providers/salesforce/hooks/test_salesforce.py b/tests/providers/salesforce/hooks/test_salesforce.py index 4b936c32fe1e1..9906d1c4d1428 100644 --- a/tests/providers/salesforce/hooks/test_salesforce.py +++ b/tests/providers/salesforce/hooks/test_salesforce.py @@ -33,9 +33,10 @@ class TestSalesforceHook: - def setup(self): + def setup_method(self): self.salesforce_hook = SalesforceHook(salesforce_conn_id="conn_id") + @staticmethod def _insert_conn_db_entry(conn_id, conn_object): with create_session() as session: session.query(Connection).filter(Connection.conn_id == conn_id).delete() diff --git a/tests/providers/salesforce/operators/test_salesforce_apex_rest.py b/tests/providers/salesforce/operators/test_salesforce_apex_rest.py index 6bb84d3ae09ea..822d3667be924 100644 --- a/tests/providers/salesforce/operators/test_salesforce_apex_rest.py +++ b/tests/providers/salesforce/operators/test_salesforce_apex_rest.py @@ -16,13 +16,12 @@ # under the License. from __future__ import annotations -import unittest from unittest.mock import Mock, patch from airflow.providers.salesforce.operators.salesforce_apex_rest import SalesforceApexRestOperator -class TestSalesforceApexRestOperator(unittest.TestCase): +class TestSalesforceApexRestOperator: """ Test class for SalesforceApexRestOperator """ diff --git a/tests/providers/salesforce/sensors/__init__.py b/tests/providers/salesforce/sensors/__init__.py deleted file mode 100644 index 13a83393a9124..0000000000000 --- a/tests/providers/salesforce/sensors/__init__.py +++ /dev/null @@ -1,16 +0,0 @@ -# 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. diff --git a/tests/providers/samba/hooks/test_samba.py b/tests/providers/samba/hooks/test_samba.py index 944122eb2148c..06ec32cb48193 100644 --- a/tests/providers/samba/hooks/test_samba.py +++ b/tests/providers/samba/hooks/test_samba.py @@ -17,12 +17,10 @@ # under the License. from __future__ import annotations -import unittest from inspect import getfullargspec from unittest import mock import pytest -from parameterized import parameterized from airflow.exceptions import AirflowException from airflow.models import Connection @@ -38,7 +36,7 @@ ) -class TestSambaHook(unittest.TestCase): +class TestSambaHook: def test_get_conn_should_fail_if_conn_id_does_not_exist(self): with pytest.raises(AirflowException): SambaHook("conn") @@ -65,7 +63,8 @@ def test_context_manager(self, get_conn_mock, register_session): # Test that the connection was disconnected upon exit. assert len(mock_connection.disconnect.mock_calls) == 1 - @parameterized.expand( + @pytest.mark.parametrize( + "name", [ "getxattr", "link", @@ -94,7 +93,7 @@ def test_context_manager(self, get_conn_mock, register_session): ], ) @mock.patch("airflow.hooks.base.BaseHook.get_connection") - def test_method(self, name, get_conn_mock): + def test_method(self, get_conn_mock, name): get_conn_mock.return_value = CONNECTION hook = SambaHook("samba_default") connection_settings = { @@ -132,14 +131,15 @@ def test_method(self, name, get_conn_mock): # We expect keyword arguments to include the connection settings. assert dict(kwargs, **connection_settings) == p_kwargs - @parameterized.expand( + @pytest.mark.parametrize( + "path, full_path", [ ("/start/path/with/slash", "//ip/share/start/path/with/slash"), ("start/path/without/slash", "//ip/share/start/path/without/slash"), ], ) @mock.patch("airflow.hooks.base.BaseHook.get_connection") - def test__join_path(self, path, full_path, get_conn_mock): + def test__join_path(self, get_conn_mock, path, full_path): get_conn_mock.return_value = CONNECTION hook = SambaHook("samba_default") assert hook._join_path(path) == full_path diff --git a/tests/providers/segment/hooks/test_segment.py b/tests/providers/segment/hooks/test_segment.py index b766caf639adc..47cc5884f5820 100644 --- a/tests/providers/segment/hooks/test_segment.py +++ b/tests/providers/segment/hooks/test_segment.py @@ -17,7 +17,6 @@ # under the License. from __future__ import annotations -import unittest from unittest import mock import pytest @@ -29,10 +28,8 @@ WRITE_KEY = "foo" -class TestSegmentHook(unittest.TestCase): - def setUp(self): - super().setUp() - +class TestSegmentHook: + def setup_method(self): self.conn = conn = mock.MagicMock() conn.write_key = WRITE_KEY self.expected_write_key = WRITE_KEY diff --git a/tests/providers/segment/operators/test_segment_track_event.py b/tests/providers/segment/operators/test_segment_track_event.py index b5ae582026831..bf7e4bd0d0d26 100644 --- a/tests/providers/segment/operators/test_segment_track_event.py +++ b/tests/providers/segment/operators/test_segment_track_event.py @@ -17,7 +17,6 @@ # under the License. from __future__ import annotations -import unittest from unittest import mock import pytest @@ -30,9 +29,8 @@ WRITE_KEY = "foo" -class TestSegmentHook(unittest.TestCase): - def setUp(self): - super().setUp() +class TestSegmentHook: + def setup_method(self): self.conn = conn = mock.MagicMock() conn.write_key = WRITE_KEY @@ -59,7 +57,7 @@ def test_on_error(self): self.test_hook.on_error("error", ["items"]) -class TestSegmentTrackEventOperator(unittest.TestCase): +class TestSegmentTrackEventOperator: @mock.patch("airflow.providers.segment.operators.segment_track_event.SegmentHook") def test_execute(self, mock_hook): # Given diff --git a/tests/providers/sendgrid/utils/test_emailer.py b/tests/providers/sendgrid/utils/test_emailer.py index eace39557e47e..e3428e349efa7 100644 --- a/tests/providers/sendgrid/utils/test_emailer.py +++ b/tests/providers/sendgrid/utils/test_emailer.py @@ -20,15 +20,14 @@ import copy import os import tempfile -import unittest from unittest import mock from airflow.providers.sendgrid.utils.emailer import send_email -class TestSendEmailSendGrid(unittest.TestCase): +class TestSendEmailSendGrid: # Unit test for sendgrid.send_email() - def setUp(self): + def setup_method(self): self.recipients = ["foo@foo.com", "bar@bar.com"] self.subject = "sendgrid-send-email unit test" self.html_content = "Foo bar" diff --git a/tests/providers/sftp/hooks/test_sftp.py b/tests/providers/sftp/hooks/test_sftp.py index b471dc88b053f..4d7f7bb5623d0 100644 --- a/tests/providers/sftp/hooks/test_sftp.py +++ b/tests/providers/sftp/hooks/test_sftp.py @@ -20,13 +20,11 @@ import json import os import shutil -import unittest from io import StringIO from unittest import mock import paramiko import pytest -from parameterized import parameterized from airflow.exceptions import AirflowException from airflow.models import Connection @@ -58,7 +56,7 @@ def generate_host_key(pkey: paramiko.PKey): TEST_KEY_FILE = "~/.ssh/id_rsa" -class TestSFTPHook(unittest.TestCase): +class TestSFTPHook: @provide_session def update_connection(self, login, session=None): connection = session.query(Connection).filter(Connection.conn_id == "sftp_default").first() @@ -72,7 +70,7 @@ def _create_additional_test_file(self, file_name): with open(os.path.join(TMP_PATH, file_name), "a") as file: file.write("Test file") - def setUp(self): + def setup_method(self): self.old_login = self.update_connection(SFTP_CONNECTION_USER) self.hook = SFTPHook() os.makedirs(os.path.join(TMP_PATH, TMP_DIR_FOR_TESTS, SUB_DIR)) @@ -180,11 +178,11 @@ def test_no_host_key_check_default(self, get_connection): @mock.patch("airflow.providers.sftp.hooks.sftp.SFTPHook.get_connection") def test_no_host_key_check_enabled(self, get_connection): - connection = Connection(login="login", host="host", extra='{"no_host_key_check": false}') + connection = Connection(login="login", host="host", extra='{"no_host_key_check": true}') get_connection.return_value = connection hook = SFTPHook() - assert hook.no_host_key_check is False + assert hook.no_host_key_check is True @mock.patch("airflow.providers.sftp.hooks.sftp.SFTPHook.get_connection") def test_no_host_key_check_disabled(self, get_connection): @@ -290,19 +288,21 @@ def test_key_file(self, get_connection): hook = SFTPHook() assert hook.key_file == TEST_KEY_FILE - @parameterized.expand( + @pytest.mark.parametrize( + "path, exists", [ (os.path.join(TMP_PATH, TMP_DIR_FOR_TESTS), True), (os.path.join(TMP_PATH, TMP_FILE_FOR_TESTS), True), (os.path.join(TMP_PATH, TMP_DIR_FOR_TESTS + "abc"), False), (os.path.join(TMP_PATH, TMP_DIR_FOR_TESTS, "abc"), False), - ] + ], ) def test_path_exists(self, path, exists): result = self.hook.path_exists(path) assert result == exists - @parameterized.expand( + @pytest.mark.parametrize( + "path, prefix, delimiter, match", [ ("test/path/file.bin", None, None, True), ("test/path/file.bin", "test", None, True), @@ -315,7 +315,7 @@ def test_path_exists(self, path, exists): ("test/path/file.bin", "test//", None, False), ("test/path/file.bin", None, ".txt", False), ("test/path/file.bin", "diff", ".txt", False), - ] + ], ) def test_path_match(self, path, prefix, delimiter, match): result = self.hook._is_path_match(path=path, prefix=prefix, delimiter=delimiter) @@ -366,11 +366,11 @@ def test_deprecation_ftp_conn_id(self, mock_get_connection): connection = Connection(conn_id="ftp_default", login="login", host="host") mock_get_connection.return_value = connection # If `ftp_conn_id` is provided, it will be used but would show a deprecation warning. - with self.assertWarnsRegex(DeprecationWarning, "Parameter `ftp_conn_id` is deprecated"): + with pytest.warns(DeprecationWarning, match=r"Parameter `ftp_conn_id` is deprecated"): assert SFTPHook(ftp_conn_id="ftp_default").ssh_conn_id == "ftp_default" # If both are provided, ftp_conn_id will be used but would show a deprecation warning. - with self.assertWarnsRegex(DeprecationWarning, "Parameter `ftp_conn_id` is deprecated"): + with pytest.warns(DeprecationWarning, match=r"Parameter `ftp_conn_id` is deprecated"): assert ( SFTPHook(ftp_conn_id="ftp_default", ssh_conn_id="sftp_default").ssh_conn_id == "ftp_default" ) @@ -385,41 +385,45 @@ def test_invalid_ssh_hook(self, mock_get_connection): with pytest.raises(AirflowException, match="ssh_hook must be an instance of SSHHook"): connection = Connection(conn_id="sftp_default", login="root", host="localhost") mock_get_connection.return_value = connection - SFTPHook(ssh_hook="invalid_hook") # type: ignore + with pytest.warns(DeprecationWarning, match=r"Parameter `ssh_hook` is deprecated.*"): + SFTPHook(ssh_hook="invalid_hook") @mock.patch("airflow.providers.ssh.hooks.ssh.SSHHook.get_connection") def test_valid_ssh_hook(self, mock_get_connection): connection = Connection(conn_id="sftp_test", login="root", host="localhost") mock_get_connection.return_value = connection - hook = SFTPHook(ssh_hook=SSHHook(ssh_conn_id="sftp_test")) + with pytest.warns(DeprecationWarning, match=r"Parameter `ssh_hook` is deprecated.*"): + hook = SFTPHook(ssh_hook=SSHHook(ssh_conn_id="sftp_test")) assert hook.ssh_conn_id == "sftp_test" assert isinstance(hook.get_conn(), paramiko.SFTPClient) def test_get_suffix_pattern_match(self): output = self.hook.get_file_by_pattern(TMP_PATH, "*.txt") - self.assertTrue(output, TMP_FILE_FOR_TESTS) + # In CI files might have different name, so we check that file found rather than actual name + assert output, TMP_FILE_FOR_TESTS def test_get_prefix_pattern_match(self): output = self.hook.get_file_by_pattern(TMP_PATH, "test*") - self.assertTrue(output, TMP_FILE_FOR_TESTS) + # In CI files might have different name, so we check that file found rather than actual name + assert output, TMP_FILE_FOR_TESTS def test_get_pattern_not_match(self): output = self.hook.get_file_by_pattern(TMP_PATH, "*.text") - self.assertFalse(output) + assert output == "" def test_get_several_pattern_match(self): output = self.hook.get_file_by_pattern(TMP_PATH, "*.log") - self.assertEqual(LOG_FILE_FOR_TESTS, output) + assert output == LOG_FILE_FOR_TESTS def test_get_first_pattern_match(self): output = self.hook.get_file_by_pattern(TMP_PATH, "test_*.txt") - self.assertEqual(TMP_FILE_FOR_TESTS, output) + assert output == TMP_FILE_FOR_TESTS def test_get_middle_pattern_match(self): output = self.hook.get_file_by_pattern(TMP_PATH, "*_file_*.txt") - self.assertEqual(ANOTHER_FILE_FOR_TESTS, output) + assert output == ANOTHER_FILE_FOR_TESTS - def tearDown(self): + def teardown_method(self): shutil.rmtree(os.path.join(TMP_PATH, TMP_DIR_FOR_TESTS)) for file_name in [TMP_FILE_FOR_TESTS, ANOTHER_FILE_FOR_TESTS, LOG_FILE_FOR_TESTS]: os.remove(os.path.join(TMP_PATH, file_name)) diff --git a/tests/providers/sftp/sensors/test_sftp.py b/tests/providers/sftp/sensors/test_sftp.py index 575f739477d3a..585b10cf892e3 100644 --- a/tests/providers/sftp/sensors/test_sftp.py +++ b/tests/providers/sftp/sensors/test_sftp.py @@ -17,7 +17,6 @@ # under the License. from __future__ import annotations -import unittest from datetime import datetime from unittest.mock import patch @@ -28,7 +27,7 @@ from airflow.providers.sftp.sensors.sftp import SFTPSensor -class TestSFTPSensor(unittest.TestCase): +class TestSFTPSensor: @patch("airflow.providers.sftp.sensors.sftp.SFTPHook") def test_file_present(self, sftp_hook_mock): sftp_hook_mock.return_value.get_mod_time.return_value = "19700101000000" diff --git a/tests/providers/singularity/operators/test_singularity.py b/tests/providers/singularity/operators/test_singularity.py index 7ac389e0bee8a..34508e7b01ba1 100644 --- a/tests/providers/singularity/operators/test_singularity.py +++ b/tests/providers/singularity/operators/test_singularity.py @@ -17,18 +17,16 @@ # under the License. from __future__ import annotations -import unittest from unittest import mock import pytest -from parameterized import parameterized from spython.instance import Instance from airflow.exceptions import AirflowException from airflow.providers.singularity.operators.singularity import SingularityOperator -class SingularityOperatorTestCase(unittest.TestCase): +class TestSingularityOperator: @mock.patch("airflow.providers.singularity.operators.singularity.Client") def test_execute(self, client_mock): instance = mock.Mock( @@ -55,12 +53,7 @@ def test_execute(self, client_mock): instance.start.assert_called_once_with() instance.stop.assert_called_once_with() - @parameterized.expand( - [ - ("",), - (None,), - ] - ) + @pytest.mark.parametrize("command", [pytest.param("", id="empty"), pytest.param(None, id="none")]) def test_command_is_required(self, command): task = SingularityOperator(task_id="task-id", image="docker://busybox", command=command) with pytest.raises(AirflowException, match="You must define a command."): @@ -95,7 +88,8 @@ def test_image_should_be_pulled_when_not_exists(self, client_mock): client_mock.pull.assert_called_once_with("docker://busybox", stream=True, pull_folder="/tmp") client_mock.execute.assert_called_once_with(mock.ANY, "echo hello", return_result=True) - @parameterized.expand( + @pytest.mark.parametrize( + "volumes, expected_options", [ ( None, @@ -117,10 +111,10 @@ def test_image_should_be_pulled_when_not_exists(self, client_mock): ["AAA", "BBB", "CCC"], ["--bind", "AAA", "--bind", "BBB", "--bind", "CCC"], ), - ] + ], ) @mock.patch("airflow.providers.singularity.operators.singularity.Client") - def test_bind_options(self, volumes, expected_options, client_mock): + def test_bind_options(self, client_mock, volumes, expected_options): instance = mock.Mock( autospec=Instance, **{ @@ -145,7 +139,8 @@ def test_bind_options(self, volumes, expected_options, client_mock): "docker://busybox", options=expected_options, args=None, start=False ) - @parameterized.expand( + @pytest.mark.parametrize( + "working_dir, expected_working_dir", [ ( None, @@ -159,10 +154,10 @@ def test_bind_options(self, volumes, expected_options, client_mock): "/work-dir/", ["--workdir", "/work-dir/"], ), - ] + ], ) @mock.patch("airflow.providers.singularity.operators.singularity.Client") - def test_working_dir(self, working_dir, expected_working_dir, client_mock): + def test_working_dir(self, client_mock, working_dir, expected_working_dir): instance = mock.Mock( autospec=Instance, **{ diff --git a/tests/providers/slack/operators/test_slack.py b/tests/providers/slack/operators/test_slack.py index 08ad3312bb6cb..ef40ac4f6c92d 100644 --- a/tests/providers/slack/operators/test_slack.py +++ b/tests/providers/slack/operators/test_slack.py @@ -74,8 +74,7 @@ def test_hook(self, mock_slack_hook_cls, token, conn_id): class TestSlackAPIPostOperator: - @pytest.fixture(autouse=True) - def setup(self): + def setup_method(self): self.test_username = "test_username" self.test_channel = "#test_slack_channel" self.test_text = "test_text" @@ -184,8 +183,7 @@ def test_api_call_params_with_default_args(self, mock_hook): class TestSlackAPIFileOperator: - @pytest.fixture(autouse=True) - def setup(self): + def setup_method(self): self.test_username = "test_username" self.test_channel = "#test_slack_channel" self.test_initial_comment = "test text file test_filename.txt" diff --git a/tests/providers/snowflake/hooks/test_snowflake.py b/tests/providers/snowflake/hooks/test_snowflake.py index 87c661043a643..f175ec5f087ac 100644 --- a/tests/providers/snowflake/hooks/test_snowflake.py +++ b/tests/providers/snowflake/hooks/test_snowflake.py @@ -18,7 +18,6 @@ from __future__ import annotations import json -import unittest from copy import deepcopy from pathlib import Path from typing import Any @@ -268,9 +267,7 @@ class TestPytestSnowflakeHook: def test_hook_should_support_prepare_basic_conn_params_and_uri( self, connection_kwargs, expected_uri, expected_conn_params ): - with unittest.mock.patch.dict( - "os.environ", AIRFLOW_CONN_TEST_CONN=Connection(**connection_kwargs).get_uri() - ): + with mock.patch.dict("os.environ", AIRFLOW_CONN_TEST_CONN=Connection(**connection_kwargs).get_uri()): assert SnowflakeHook(snowflake_conn_id="test_conn").get_uri() == expected_uri assert SnowflakeHook(snowflake_conn_id="test_conn")._get_conn_params() == expected_conn_params @@ -289,9 +286,7 @@ def test_get_conn_params_should_support_private_auth_in_connection( "private_key_content": str(encrypted_temporary_private_key.read_text()), }, } - with unittest.mock.patch.dict( - "os.environ", AIRFLOW_CONN_TEST_CONN=Connection(**connection_kwargs).get_uri() - ): + with mock.patch.dict("os.environ", AIRFLOW_CONN_TEST_CONN=Connection(**connection_kwargs).get_uri()): assert "private_key" in SnowflakeHook(snowflake_conn_id="test_conn")._get_conn_params() @pytest.mark.parametrize("include_params", [True, False]) @@ -308,7 +303,7 @@ def test_hook_param_beats_extra(self, include_params): session_parameters="session_parameters", ) extras = {k: f"{v}_extra" for k, v in hook_params.items()} - with unittest.mock.patch.dict( + with mock.patch.dict( "os.environ", AIRFLOW_CONN_TEST_CONN=Connection(conn_type="any", extra=json.dumps(extras)).get_uri(), ): @@ -336,7 +331,7 @@ def test_extra_short_beats_long(self, include_unprefixed): role="role", ) extras_prefixed = {f"extra__snowflake__{k}": f"{v}_prefixed" for k, v in extras.items()} - with unittest.mock.patch.dict( + with mock.patch.dict( "os.environ", AIRFLOW_CONN_TEST_CONN=Connection( conn_type="any", @@ -369,9 +364,7 @@ def test_get_conn_params_should_support_private_auth_with_encrypted_key( "private_key_file": str(encrypted_temporary_private_key), }, } - with unittest.mock.patch.dict( - "os.environ", AIRFLOW_CONN_TEST_CONN=Connection(**connection_kwargs).get_uri() - ): + with mock.patch.dict("os.environ", AIRFLOW_CONN_TEST_CONN=Connection(**connection_kwargs).get_uri()): assert "private_key" in SnowflakeHook(snowflake_conn_id="test_conn")._get_conn_params() def test_get_conn_params_should_support_private_auth_with_unencrypted_key( @@ -389,23 +382,19 @@ def test_get_conn_params_should_support_private_auth_with_unencrypted_key( "private_key_file": str(non_encrypted_temporary_private_key), }, } - with unittest.mock.patch.dict( - "os.environ", AIRFLOW_CONN_TEST_CONN=Connection(**connection_kwargs).get_uri() - ): + with mock.patch.dict("os.environ", AIRFLOW_CONN_TEST_CONN=Connection(**connection_kwargs).get_uri()): assert "private_key" in SnowflakeHook(snowflake_conn_id="test_conn")._get_conn_params() connection_kwargs["password"] = "" - with unittest.mock.patch.dict( - "os.environ", AIRFLOW_CONN_TEST_CONN=Connection(**connection_kwargs).get_uri() - ): + with mock.patch.dict("os.environ", AIRFLOW_CONN_TEST_CONN=Connection(**connection_kwargs).get_uri()): assert "private_key" in SnowflakeHook(snowflake_conn_id="test_conn")._get_conn_params() connection_kwargs["password"] = _PASSWORD - with unittest.mock.patch.dict( + with mock.patch.dict( "os.environ", AIRFLOW_CONN_TEST_CONN=Connection(**connection_kwargs).get_uri() ), pytest.raises(TypeError, match="Password was given but private key is not encrypted."): SnowflakeHook(snowflake_conn_id="test_conn")._get_conn_params() def test_should_add_partner_info(self): - with unittest.mock.patch.dict( + with mock.patch.dict( "os.environ", AIRFLOW_CONN_TEST_CONN=Connection(**BASE_CONNECTION_KWARGS).get_uri(), AIRFLOW_SNOWFLAKE_PARTNER="PARTNER_NAME", @@ -416,20 +405,18 @@ def test_should_add_partner_info(self): ) def test_get_conn_should_call_connect(self): - with unittest.mock.patch.dict( + with mock.patch.dict( "os.environ", AIRFLOW_CONN_TEST_CONN=Connection(**BASE_CONNECTION_KWARGS).get_uri() - ), unittest.mock.patch("airflow.providers.snowflake.hooks.snowflake.connector") as mock_connector: + ), mock.patch("airflow.providers.snowflake.hooks.snowflake.connector") as mock_connector: hook = SnowflakeHook(snowflake_conn_id="test_conn") conn = hook.get_conn() mock_connector.connect.assert_called_once_with(**hook._get_conn_params()) assert mock_connector.connect.return_value == conn def test_get_sqlalchemy_engine_should_support_pass_auth(self): - with unittest.mock.patch.dict( + with mock.patch.dict( "os.environ", AIRFLOW_CONN_TEST_CONN=Connection(**BASE_CONNECTION_KWARGS).get_uri() - ), unittest.mock.patch( - "airflow.providers.snowflake.hooks.snowflake.create_engine" - ) as mock_create_engine: + ), mock.patch("airflow.providers.snowflake.hooks.snowflake.create_engine") as mock_create_engine: hook = SnowflakeHook(snowflake_conn_id="test_conn") conn = hook.get_sqlalchemy_engine() mock_create_engine.assert_called_once_with( @@ -442,11 +429,9 @@ def test_get_sqlalchemy_engine_should_support_insecure_mode(self): connection_kwargs = deepcopy(BASE_CONNECTION_KWARGS) connection_kwargs["extra"]["extra__snowflake__insecure_mode"] = "True" - with unittest.mock.patch.dict( + with mock.patch.dict( "os.environ", AIRFLOW_CONN_TEST_CONN=Connection(**connection_kwargs).get_uri() - ), unittest.mock.patch( - "airflow.providers.snowflake.hooks.snowflake.create_engine" - ) as mock_create_engine: + ), mock.patch("airflow.providers.snowflake.hooks.snowflake.create_engine") as mock_create_engine: hook = SnowflakeHook(snowflake_conn_id="test_conn") conn = hook.get_sqlalchemy_engine() mock_create_engine.assert_called_once_with( @@ -460,11 +445,9 @@ def test_get_sqlalchemy_engine_should_support_session_parameters(self): connection_kwargs = deepcopy(BASE_CONNECTION_KWARGS) connection_kwargs["extra"]["session_parameters"] = {"TEST_PARAM": "AA", "TEST_PARAM_B": 123} - with unittest.mock.patch.dict( + with mock.patch.dict( "os.environ", AIRFLOW_CONN_TEST_CONN=Connection(**connection_kwargs).get_uri() - ), unittest.mock.patch( - "airflow.providers.snowflake.hooks.snowflake.create_engine" - ) as mock_create_engine: + ), mock.patch("airflow.providers.snowflake.hooks.snowflake.create_engine") as mock_create_engine: hook = SnowflakeHook(snowflake_conn_id="test_conn") conn = hook.get_sqlalchemy_engine() mock_create_engine.assert_called_once_with( @@ -479,18 +462,16 @@ def test_get_sqlalchemy_engine_should_support_private_key_auth(self, non_encrypt connection_kwargs["password"] = "" connection_kwargs["extra"]["private_key_file"] = str(non_encrypted_temporary_private_key) - with unittest.mock.patch.dict( + with mock.patch.dict( "os.environ", AIRFLOW_CONN_TEST_CONN=Connection(**connection_kwargs).get_uri() - ), unittest.mock.patch( - "airflow.providers.snowflake.hooks.snowflake.create_engine" - ) as mock_create_engine: + ), mock.patch("airflow.providers.snowflake.hooks.snowflake.create_engine") as mock_create_engine: hook = SnowflakeHook(snowflake_conn_id="test_conn") conn = hook.get_sqlalchemy_engine() assert "private_key" in mock_create_engine.call_args[1]["connect_args"] assert mock_create_engine.return_value == conn def test_hook_parameters_should_take_precedence(self): - with unittest.mock.patch.dict( + with mock.patch.dict( "os.environ", AIRFLOW_CONN_TEST_CONN=Connection(**BASE_CONNECTION_KWARGS).get_uri() ): hook = SnowflakeHook( @@ -555,7 +536,7 @@ def test_run_storing_query_ids_extra(self, mock_conn, sql, expected_sql, expecte @mock.patch("airflow.providers.common.sql.hooks.sql.DbApiHook.get_first") def test_connection_success(self, mock_get_first): - with unittest.mock.patch.dict( + with mock.patch.dict( "os.environ", AIRFLOW_CONN_SNOWFLAKE_DEFAULT=Connection(**BASE_CONNECTION_KWARGS).get_uri() ): hook = SnowflakeHook() @@ -570,7 +551,7 @@ def test_connection_success(self, mock_get_first): side_effect=Exception("Connection Errors"), ) def test_connection_failure(self, mock_get_first): - with unittest.mock.patch.dict( + with mock.patch.dict( "os.environ", AIRFLOW_CONN_SNOWFLAKE_DEFAULT=Connection(**BASE_CONNECTION_KWARGS).get_uri() ): hook = SnowflakeHook() diff --git a/tests/providers/snowflake/operators/test_snowflake.py b/tests/providers/snowflake/operators/test_snowflake.py index 7cd2d040cfbc7..dde2f9addeb60 100644 --- a/tests/providers/snowflake/operators/test_snowflake.py +++ b/tests/providers/snowflake/operators/test_snowflake.py @@ -17,7 +17,6 @@ # under the License. from __future__ import annotations -import unittest from unittest import mock import pytest @@ -37,9 +36,8 @@ TEST_DAG_ID = "unit_test_dag" -class TestSnowflakeOperator(unittest.TestCase): - def setUp(self): - super().setUp() +class TestSnowflakeOperator: + def setup_method(self): args = {"owner": "airflow", "start_date": DEFAULT_DATE} dag = DAG(TEST_DAG_ID, default_args=args) self.dag = dag diff --git a/tests/providers/snowflake/transfers/test_copy_into_snowflake.py b/tests/providers/snowflake/transfers/test_copy_into_snowflake.py index 821c883913b92..f0e7a61fd5311 100644 --- a/tests/providers/snowflake/transfers/test_copy_into_snowflake.py +++ b/tests/providers/snowflake/transfers/test_copy_into_snowflake.py @@ -16,13 +16,12 @@ # under the License. from __future__ import annotations -import unittest from unittest import mock from airflow.providers.snowflake.transfers.copy_into_snowflake import CopyFromExternalStageToSnowflakeOperator -class TestCopyFromExternalStageToSnowflake(unittest.TestCase): +class TestCopyFromExternalStageToSnowflake: @mock.patch("airflow.providers.snowflake.transfers.copy_into_snowflake.SnowflakeHook") def test_execute(self, mock_hook): CopyFromExternalStageToSnowflakeOperator( diff --git a/tests/providers/sqlite/hooks/test_sqlite.py b/tests/providers/sqlite/hooks/test_sqlite.py index 0a1dde81e2fd1..af61442ece4fe 100644 --- a/tests/providers/sqlite/hooks/test_sqlite.py +++ b/tests/providers/sqlite/hooks/test_sqlite.py @@ -17,7 +17,6 @@ # under the License. from __future__ import annotations -import unittest from unittest import mock from unittest.mock import patch @@ -27,8 +26,8 @@ from airflow.providers.sqlite.hooks.sqlite import SqliteHook -class TestSqliteHookConn(unittest.TestCase): - def setUp(self): +class TestSqliteHookConn: + def setup_method(self): self.connection = Connection(host="host") @@ -52,8 +51,8 @@ def test_get_conn_non_default_id(self, mock_connect): self.db_hook.get_connection.assert_called_once_with("non_default") -class TestSqliteHook(unittest.TestCase): - def setUp(self): +class TestSqliteHook: + def setup_method(self): self.cur = mock.MagicMock(rowcount=0) self.conn = mock.MagicMock() diff --git a/tests/providers/sqlite/operators/test_sqlite.py b/tests/providers/sqlite/operators/test_sqlite.py index d0de4ca32ff88..1584bd909d746 100644 --- a/tests/providers/sqlite/operators/test_sqlite.py +++ b/tests/providers/sqlite/operators/test_sqlite.py @@ -17,8 +17,6 @@ # under the License. from __future__ import annotations -import unittest - import pytest from airflow.models.dag import DAG @@ -32,13 +30,13 @@ @pytest.mark.backend("sqlite") -class TestSqliteOperator(unittest.TestCase): - def setUp(self): +class TestSqliteOperator: + def setup_method(self): args = {"owner": "airflow", "start_date": DEFAULT_DATE} dag = DAG(TEST_DAG_ID, default_args=args) self.dag = dag - def tearDown(self): + def teardown_method(self): tables_to_drop = ["test_airflow", "test_airflow2"] from airflow.providers.sqlite.hooks.sqlite import SqliteHook diff --git a/tests/providers/ssh/hooks/test_ssh.py b/tests/providers/ssh/hooks/test_ssh.py index 324db86b744d3..6448d88efe497 100644 --- a/tests/providers/ssh/hooks/test_ssh.py +++ b/tests/providers/ssh/hooks/test_ssh.py @@ -21,13 +21,11 @@ import random import string import textwrap -import unittest from io import StringIO from unittest import mock import paramiko import pytest -from parameterized import parameterized from airflow import settings from airflow.exceptions import AirflowException @@ -83,7 +81,7 @@ def generate_host_key(pkey: paramiko.PKey): TEST_CIPHERS = ["aes128-ctr", "aes192-ctr", "aes256-ctr"] -class TestSSHHook(unittest.TestCase): +class TestSSHHook: CONN_SSH_WITH_NO_EXTRA = "ssh_with_no_extra" CONN_SSH_WITH_PRIVATE_KEY_EXTRA = "ssh_with_private_key_extra" CONN_SSH_WITH_PRIVATE_KEY_ECDSA_EXTRA = "ssh_with_private_key_ecdsa_extra" @@ -112,7 +110,7 @@ class TestSSHHook(unittest.TestCase): ) @classmethod - def tearDownClass(cls) -> None: + def teardown_class(cls) -> None: with create_session() as session: conns_to_reset = [ cls.CONN_SSH_WITH_NO_EXTRA, @@ -139,7 +137,7 @@ def tearDownClass(cls) -> None: session.commit() @classmethod - def setUpClass(cls) -> None: + def setup_class(cls) -> None: db.merge_conn( Connection( conn_id=cls.CONN_SSH_WITH_NO_EXTRA, @@ -741,7 +739,8 @@ def test_ssh_connection_with_timeout_extra_and_conn_timeout_extra(self, ssh_mock look_for_keys=True, ) - @parameterized.expand( + @pytest.mark.parametrize( + "timeout, conn_timeout, timeoutextra, conn_timeoutextra, expected_value", [ (TEST_TIMEOUT, TEST_CONN_TIMEOUT, True, True, TEST_CONN_TIMEOUT), (TEST_TIMEOUT, TEST_CONN_TIMEOUT, True, False, TEST_CONN_TIMEOUT), @@ -759,11 +758,11 @@ def test_ssh_connection_with_timeout_extra_and_conn_timeout_extra(self, ssh_mock (None, None, True, False, TEST_TIMEOUT), (None, None, False, True, TEST_CONN_TIMEOUT), (None, None, False, False, 10), - ] + ], ) @mock.patch("airflow.providers.ssh.hooks.ssh.paramiko.SSHClient") def test_ssh_connection_with_all_timeout_param_and_extra_combinations( - self, timeout, conn_timeout, timeoutextra, conn_timeoutextra, expected_value, ssh_mock + self, ssh_mock, timeout, conn_timeout, timeoutextra, conn_timeoutextra, expected_value ): if timeoutextra and conn_timeoutextra: diff --git a/tests/providers/tableau/hooks/test_tableau.py b/tests/providers/tableau/hooks/test_tableau.py index fd69cb4c6beda..b463fce31017c 100644 --- a/tests/providers/tableau/hooks/test_tableau.py +++ b/tests/providers/tableau/hooks/test_tableau.py @@ -16,25 +16,21 @@ # under the License. from __future__ import annotations -import unittest from unittest.mock import MagicMock, patch -from parameterized import parameterized +import pytest from airflow import configuration, models from airflow.providers.tableau.hooks.tableau import TableauHook, TableauJobFinishCode from airflow.utils import db -class TestTableauHook(unittest.TestCase): +class TestTableauHook: """ Test class for TableauHook """ - def setUp(self): - """ - setup - """ + def setup_method(self): configuration.conf.load_test_config() db.merge_conn( @@ -222,15 +218,16 @@ def test_get_all(self, mock_pager, mock_server, mock_tableau_auth): mock_pager.assert_called_once_with(mock_server.return_value.jobs.get) - @parameterized.expand( + @pytest.mark.parametrize( + "finish_code, expected_status", [ - (0, TableauJobFinishCode.SUCCESS), - (1, TableauJobFinishCode.ERROR), - (2, TableauJobFinishCode.CANCELED), - ] + pytest.param(0, TableauJobFinishCode.SUCCESS, id="SUCCESS"), + pytest.param(1, TableauJobFinishCode.ERROR, id="ERROR"), + pytest.param(2, TableauJobFinishCode.CANCELED, id="CANCELED"), + ], ) @patch("airflow.providers.tableau.hooks.tableau.Server") - def test_get_job_status(self, finish_code, expected_status, mock_tableau_server): + def test_get_job_status(self, mock_tableau_server, finish_code, expected_status): """ Test get job status """ diff --git a/tests/providers/tableau/operators/test_tableau.py b/tests/providers/tableau/operators/test_tableau.py index a97abb81b1951..994aafaaf023d 100644 --- a/tests/providers/tableau/operators/test_tableau.py +++ b/tests/providers/tableau/operators/test_tableau.py @@ -16,7 +16,6 @@ # under the License. from __future__ import annotations -import unittest from unittest.mock import Mock, patch import pytest @@ -26,16 +25,12 @@ from airflow.providers.tableau.operators.tableau import TableauOperator -class TestTableauOperator(unittest.TestCase): +class TestTableauOperator: """ Test class for TableauOperator """ - def setUp(self): - """ - setup - """ - + def setup_method(self): self.mocked_workbooks = [] self.mock_datasources = [] diff --git a/tests/providers/tableau/sensors/test_tableau.py b/tests/providers/tableau/sensors/test_tableau.py index 22159991a2f3d..0c873e1f33eeb 100644 --- a/tests/providers/tableau/sensors/test_tableau.py +++ b/tests/providers/tableau/sensors/test_tableau.py @@ -16,11 +16,9 @@ # under the License. from __future__ import annotations -import unittest from unittest.mock import Mock, patch import pytest -from parameterized import parameterized from airflow.providers.tableau.sensors.tableau import ( TableauJobFailedException, @@ -29,12 +27,12 @@ ) -class TestTableauJobStatusSensor(unittest.TestCase): +class TestTableauJobStatusSensor: """ Test Class for JobStatusSensor """ - def setUp(self): + def setup_method(self): self.kwargs = {"job_id": "job_2", "site_id": "test_site", "task_id": "task", "dag": None} @patch("airflow.providers.tableau.sensors.tableau.TableauHook") @@ -51,9 +49,15 @@ def test_poke(self, mock_tableau_hook): assert job_finished mock_tableau_hook.get_job_status.assert_called_once_with(job_id=sensor.job_id) - @parameterized.expand([(TableauJobFinishCode.ERROR,), (TableauJobFinishCode.CANCELED,)]) + @pytest.mark.parametrize( + "finish_code", + [ + pytest.param(TableauJobFinishCode.ERROR, id="ERROR"), + pytest.param(TableauJobFinishCode.CANCELED, id="CANCELED"), + ], + ) @patch("airflow.providers.tableau.sensors.tableau.TableauHook") - def test_poke_failed(self, finish_code, mock_tableau_hook): + def test_poke_failed(self, mock_tableau_hook, finish_code): """ Test poke failed """ diff --git a/tests/providers/telegram/hooks/test_telegram.py b/tests/providers/telegram/hooks/test_telegram.py index a915f6d3285ee..de4722c78f43e 100644 --- a/tests/providers/telegram/hooks/test_telegram.py +++ b/tests/providers/telegram/hooks/test_telegram.py @@ -17,7 +17,6 @@ # under the License. from __future__ import annotations -import unittest from unittest import mock import pytest @@ -31,8 +30,8 @@ TELEGRAM_TOKEN = "dummy token" -class TestTelegramHook(unittest.TestCase): - def setUp(self): +class TestTelegramHook: + def setup_method(self): db.merge_conn( Connection( conn_id="telegram-webhook-without-token", diff --git a/tests/providers/telegram/operators/test_telegram.py b/tests/providers/telegram/operators/test_telegram.py index 6eef156602fc7..f375ca3ad4557 100644 --- a/tests/providers/telegram/operators/test_telegram.py +++ b/tests/providers/telegram/operators/test_telegram.py @@ -17,7 +17,6 @@ # under the License. from __future__ import annotations -import unittest from unittest import mock import pytest @@ -31,8 +30,8 @@ TELEGRAM_TOKEN = "xxx:xxx" -class TestTelegramOperator(unittest.TestCase): - def setUp(self): +class TestTelegramOperator: + def setup_method(self): db.merge_conn( Connection( conn_id="telegram_default", diff --git a/tests/providers/trino/hooks/test_trino.py b/tests/providers/trino/hooks/test_trino.py index 85bfb0e194533..d4560a50ace12 100644 --- a/tests/providers/trino/hooks/test_trino.py +++ b/tests/providers/trino/hooks/test_trino.py @@ -19,12 +19,10 @@ import json import re -import unittest from unittest import mock from unittest.mock import patch import pytest -from parameterized import parameterized from trino.transaction import IsolationLevel from airflow import AirflowException @@ -179,18 +177,19 @@ def test_get_conn_client_tags(self, mock_connect, mock_get_connection): self.assert_connection_called_with(mock_connect, client_tags=extras["client_tags"]) - @parameterized.expand( + @pytest.mark.parametrize( + "current_verify, expected_verify", [ ("False", False), ("false", False), ("true", True), ("true", True), ("/tmp/cert.crt", "/tmp/cert.crt"), - ] + ], ) @patch(HOOK_GET_CONNECTION) @patch(TRINO_DBAPI_CONNECT) - def test_get_conn_verify(self, current_verify, expected_verify, mock_connect, mock_get_connection): + def test_get_conn_verify(self, mock_connect, mock_get_connection, current_verify, expected_verify): extras = {"verify": current_verify} self.set_get_connection_return_value(mock_get_connection, extra=json.dumps(extras)) TrinoHook().get_conn() @@ -224,10 +223,8 @@ def assert_connection_called_with( ) -class TestTrinoHook(unittest.TestCase): - def setUp(self): - super().setUp() - +class TestTrinoHook: + def setup_method(self): self.cur = mock.MagicMock(rowcount=0) self.conn = mock.MagicMock() self.conn.cursor.return_value = self.cur @@ -316,8 +313,8 @@ def test_serialize_cell(self): assert 1 == self.db_hook._serialize_cell(1, None) -class TestTrinoHookIntegration(unittest.TestCase): - @pytest.mark.integration("trino") +@pytest.mark.integration("trino") +class TestTrinoHookIntegration: @mock.patch.dict("os.environ", AIRFLOW_CONN_TRINO_DEFAULT="trino://airflow@trino:8080/") def test_should_record_records(self): hook = TrinoHook() @@ -325,7 +322,6 @@ def test_should_record_records(self): records = hook.get_records(sql) assert [["Customer#000000001"], ["Customer#000000002"], ["Customer#000000003"]] == records - @pytest.mark.integration("trino") @pytest.mark.integration("kerberos") def test_should_record_records_with_kerberos_auth(self): conn_url = ( diff --git a/tests/providers/trino/operators/test_trino.py b/tests/providers/trino/operators/test_trino.py index 3caa2dad8ee6c..8fef756ecf8f2 100644 --- a/tests/providers/trino/operators/test_trino.py +++ b/tests/providers/trino/operators/test_trino.py @@ -17,26 +17,28 @@ # under the License. from __future__ import annotations -import unittest from unittest import mock +import pytest + from airflow.providers.trino.operators.trino import TrinoOperator TRINO_CONN_ID = "test_trino" TASK_ID = "test_trino_task" -class TestTrinoOperator(unittest.TestCase): +class TestTrinoOperator: @mock.patch("airflow.providers.common.sql.operators.sql.SQLExecuteQueryOperator.get_db_hook") def test_execute(self, mock_get_db_hook): """Asserts that the run method is called when a TrinoOperator task is executed""" - op = TrinoOperator( - task_id=TASK_ID, - sql="SELECT 1;", - trino_conn_id=TRINO_CONN_ID, - handler=list, - ) + with pytest.warns(DeprecationWarning, match="This class is deprecated.*"): + op = TrinoOperator( + task_id=TASK_ID, + sql="SELECT 1;", + trino_conn_id=TRINO_CONN_ID, + handler=list, + ) op.execute(None) mock_get_db_hook.return_value.run.assert_called_once_with( diff --git a/tests/providers/trino/transfers/test_gcs_trino.py b/tests/providers/trino/transfers/test_gcs_trino.py index 624d7167b281f..ce5d2f8d10aea 100644 --- a/tests/providers/trino/transfers/test_gcs_trino.py +++ b/tests/providers/trino/transfers/test_gcs_trino.py @@ -17,7 +17,6 @@ # under the License. from __future__ import annotations -import unittest from unittest import mock from airflow.providers.trino.transfers.gcs_to_trino import GCSToTrinoOperator @@ -33,7 +32,7 @@ SCHEMA_JSON = "path/to/file.json" -class TestGCSToTrinoOperator(unittest.TestCase): +class TestGCSToTrinoOperator: @mock.patch("airflow.providers.trino.transfers.gcs_to_trino.TrinoHook") @mock.patch("airflow.providers.trino.transfers.gcs_to_trino.GCSHook") @mock.patch("airflow.providers.trino.transfers.gcs_to_trino.NamedTemporaryFile") diff --git a/tests/providers/vertica/hooks/test_vertica.py b/tests/providers/vertica/hooks/test_vertica.py index 93d20cd2384b9..e78c2a0c5c813 100644 --- a/tests/providers/vertica/hooks/test_vertica.py +++ b/tests/providers/vertica/hooks/test_vertica.py @@ -17,7 +17,6 @@ # under the License. from __future__ import annotations -import unittest from unittest import mock from unittest.mock import patch @@ -25,10 +24,8 @@ from airflow.providers.vertica.hooks.vertica import VerticaHook -class TestVerticaHookConn(unittest.TestCase): - def setUp(self): - super().setUp() - +class TestVerticaHookConn: + def setup_method(self): self.connection = Connection( login="login", password="password", @@ -51,10 +48,8 @@ def test_get_conn(self, mock_connect): ) -class TestVerticaHook(unittest.TestCase): - def setUp(self): - super().setUp() - +class TestVerticaHook: + def setup_method(self): self.cur = mock.MagicMock(rowcount=0) self.conn = mock.MagicMock() self.conn.cursor.return_value = self.cur diff --git a/tests/providers/vertica/operators/test_vertica.py b/tests/providers/vertica/operators/test_vertica.py index 836f324cb8289..3fd6aa3e52d84 100644 --- a/tests/providers/vertica/operators/test_vertica.py +++ b/tests/providers/vertica/operators/test_vertica.py @@ -17,14 +17,13 @@ # under the License. from __future__ import annotations -import unittest from unittest import mock from airflow.providers.common.sql.hooks.sql import fetch_all_handler from airflow.providers.vertica.operators.vertica import VerticaOperator -class TestVerticaOperator(unittest.TestCase): +class TestVerticaOperator: @mock.patch("airflow.providers.common.sql.operators.sql.SQLExecuteQueryOperator.get_db_hook") def test_execute(self, mock_get_db_hook): sql = "select a, b, c" diff --git a/tests/providers/yandex/hooks/test_yandexcloud_dataproc.py b/tests/providers/yandex/hooks/test_yandexcloud_dataproc.py index b93a499a13fa6..cf436a9b8c02d 100644 --- a/tests/providers/yandex/hooks/test_yandexcloud_dataproc.py +++ b/tests/providers/yandex/hooks/test_yandexcloud_dataproc.py @@ -17,9 +17,10 @@ from __future__ import annotations import json -import unittest from unittest.mock import patch +import pytest + from airflow.models import Connection try: @@ -64,14 +65,14 @@ HAS_CREDENTIALS = OAUTH_TOKEN != "my_oauth_token" -@unittest.skipIf(yandexcloud is None, "Skipping Yandex.Cloud hook test: no yandexcloud module") -class TestYandexCloudDataprocHook(unittest.TestCase): +@pytest.mark.skipif(yandexcloud is None, reason="Skipping Yandex.Cloud hook test: no yandexcloud module") +class TestYandexCloudDataprocHook: def _init_hook(self): with patch("airflow.hooks.base.BaseHook.get_connection") as get_connection_mock: get_connection_mock.return_value = self.connection self.hook = DataprocHook() - def setUp(self): + def setup_method(self): self.connection = Connection(extra=json.dumps({"oauth": OAUTH_TOKEN})) self._init_hook() diff --git a/tests/providers/yandex/operators/test_yandexcloud_dataproc.py b/tests/providers/yandex/operators/test_yandexcloud_dataproc.py index 3b3aa69a5d059..8ffe2050a037c 100644 --- a/tests/providers/yandex/operators/test_yandexcloud_dataproc.py +++ b/tests/providers/yandex/operators/test_yandexcloud_dataproc.py @@ -17,7 +17,6 @@ from __future__ import annotations import datetime -from unittest import TestCase from unittest.mock import MagicMock, call, patch from airflow.models.dag import DAG @@ -64,8 +63,8 @@ LOG_GROUP_ID = "my_log_group_id" -class DataprocClusterCreateOperatorTest(TestCase): - def setUp(self): +class TestDataprocClusterCreateOperator: + def setup_method(self): dag_id = "test_dag" self.dag = DAG( dag_id, diff --git a/tests/test_utils/asserts.py b/tests/test_utils/asserts.py index e16123453e21a..9f3eab630b680 100644 --- a/tests/test_utils/asserts.py +++ b/tests/test_utils/asserts.py @@ -19,22 +19,38 @@ import logging import re import traceback +import warnings from collections import Counter from contextlib import contextmanager +from typing import TYPE_CHECKING from sqlalchemy import event # Long import to not create a copy of the reference, but to refer to one place. import airflow.settings +if TYPE_CHECKING: + from unittest import TestCase + log = logging.getLogger(__name__) -def assert_equal_ignore_multiple_spaces(case, first, second, msg=None): +def assert_equal_ignore_multiple_spaces(case: TestCase | None, first, second, msg=None): def _trim(s): return re.sub(r"\s+", " ", s.strip()) - return case.assertEqual(_trim(first), _trim(second), msg) + if case: + warnings.warn( + "Passing `case` has no effect and will be remove in the future. " + "Please set to `None` for avoid this warning.", + FutureWarning, + stacklevel=3, + ) + + if not msg: + assert _trim(first) == _trim(second) + else: + assert _trim(first) == _trim(second), msg class CountQueries: From dd14501d862ed314870ef2eb7d2a61d07a02ffd0 Mon Sep 17 00:00:00 2001 From: Andrey Anshin Date: Mon, 28 Nov 2022 03:05:40 +0400 Subject: [PATCH 153/158] Replace `unittests` in providers-apache tests by pure `pytest` (#27948) (cherry picked from commit 3ea4739e2172e994e2215d61dd3b4f37f2c64759) --- .../providers/apache/beam/hooks/test_beam.py | 60 +- .../apache/beam/operators/test_beam.py | 49 +- .../apache/cassandra/hooks/test_cassandra.py | 9 +- .../apache/cassandra/sensors/test_record.py | 3 +- .../apache/cassandra/sensors/test_table.py | 3 +- .../apache/drill/hooks/test_drill.py | 5 +- .../apache/drill/operators/test_drill.py | 8 +- .../apache/druid/hooks/test_druid.py | 51 +- .../druid/transfers/test_hive_to_druid.py | 9 +- .../providers/apache/hdfs/hooks/test_hdfs.py | 8 +- .../apache/hdfs/hooks/test_webhdfs.py | 85 +-- .../apache/hdfs/sensors/test_hdfs.py | 56 +- .../apache/hdfs/sensors/test_web_hdfs.py | 3 +- tests/providers/apache/hive/__init__.py | 5 +- .../providers/apache/hive/hooks/test_hive.py | 56 +- .../apache/hive/operators/test_hive.py | 7 +- .../apache/hive/operators/test_hive_stats.py | 9 +- .../apache/hive/sensors/test_hdfs.py | 7 +- .../hive/sensors/test_hive_partition.py | 7 +- .../hive/sensors/test_metastore_partition.py | 7 +- .../hive/sensors/test_named_hive_partition.py | 9 +- .../hive/transfers/test_hive_to_mysql.py | 11 +- .../hive/transfers/test_hive_to_samba.py | 11 +- .../hive/transfers/test_mssql_to_hive.py | 5 +- .../apache/hive/transfers/test_s3_to_hive.py | 29 +- .../hive/transfers/test_vertica_to_hive.py | 5 +- .../apache/kylin/hooks/test_kylin.py | 5 +- .../apache/kylin/operators/test_kylin_cube.py | 5 +- .../providers/apache/livy/hooks/test_livy.py | 574 ++++++++---------- .../apache/livy/operators/test_livy.py | 18 +- .../apache/livy/sensors/test_livy.py | 29 +- tests/providers/apache/pig/hooks/test_pig.py | 7 +- .../apache/pig/operators/test_pig.py | 3 +- .../apache/pinot/hooks/test_pinot.py | 15 +- .../apache/spark/hooks/test_spark_jdbc.py | 6 +- .../apache/spark/hooks/test_spark_sql.py | 7 +- .../apache/spark/hooks/test_spark_submit.py | 13 +- .../apache/spark/operators/test_spark_jdbc.py | 6 +- .../apache/spark/operators/test_spark_sql.py | 5 +- .../spark/operators/test_spark_submit.py | 5 +- .../apache/sqoop/hooks/test_sqoop.py | 5 +- .../apache/sqoop/operators/test_sqoop.py | 5 +- 42 files changed, 540 insertions(+), 685 deletions(-) diff --git a/tests/providers/apache/beam/hooks/test_beam.py b/tests/providers/apache/beam/hooks/test_beam.py index bc9e8187f008b..f509c18b0a618 100644 --- a/tests/providers/apache/beam/hooks/test_beam.py +++ b/tests/providers/apache/beam/hooks/test_beam.py @@ -20,12 +20,10 @@ import os import re import subprocess -import unittest from unittest import mock from unittest.mock import MagicMock import pytest -from parameterized import parameterized from airflow.exceptions import AirflowException from airflow.providers.apache.beam.hooks.beam import BeamCommandRunner, BeamHook, beam_options_to_args @@ -58,7 +56,7 @@ """ -class TestBeamHook(unittest.TestCase): +class TestBeamHook: @mock.patch(BEAM_STRING.format("BeamCommandRunner")) @mock.patch("airflow.providers.apache.beam.hooks.beam.subprocess.check_output", return_value=b"2.39.0") def test_start_python_pipeline(self, mock_check_output, mock_runner): @@ -106,18 +104,19 @@ def test_start_python_pipeline_unsupported_option(self, mock_check_output): process_line_callback=MagicMock(), ) - @parameterized.expand( + @pytest.mark.parametrize( + "py_interpreter", [ - ("default_to_python3", "python3"), - ("major_version_2", "python2"), - ("major_version_3", "python3"), - ("minor_version", "python3.6"), - ] + pytest.param("python", id="default python"), + pytest.param("python2", id="major python version 2.x"), + pytest.param("python3", id="major python version 3.x"), + pytest.param("python3.6", id="major.minor python version"), + ], ) @mock.patch(BEAM_STRING.format("BeamCommandRunner")) @mock.patch("airflow.providers.apache.beam.hooks.beam.subprocess.check_output", return_value=b"2.39.0") def test_start_python_pipeline_with_custom_interpreter( - self, _, py_interpreter, mock_check_output, mock_runner + self, mock_check_output, mock_runner, py_interpreter ): hook = BeamHook(runner=DEFAULT_RUNNER) wait_for_done = mock_runner.return_value.wait_for_done @@ -144,23 +143,24 @@ def test_start_python_pipeline_with_custom_interpreter( ) wait_for_done.assert_called_once_with() - @parameterized.expand( + @pytest.mark.parametrize( + "current_py_requirements, current_py_system_site_packages", [ - (["foo-bar"], False), - (["foo-bar"], True), - ([], True), - ] + pytest.param("foo-bar", False, id="requirements without system site-packages"), + pytest.param("foo-bar", True, id="requirements with system site-packages"), + pytest.param([], True, id="only system site-packages"), + ], ) @mock.patch(BEAM_STRING.format("prepare_virtualenv")) @mock.patch(BEAM_STRING.format("BeamCommandRunner")) @mock.patch("airflow.providers.apache.beam.hooks.beam.subprocess.check_output", return_value=b"2.39.0") def test_start_python_pipeline_with_non_empty_py_requirements_and_without_system_packages( self, - current_py_requirements, - current_py_system_site_packages, mock_check_output, mock_runner, mock_virtualenv, + current_py_requirements, + current_py_system_site_packages, ): hook = BeamHook(runner=DEFAULT_RUNNER) wait_for_done = mock_runner.return_value.wait_for_done @@ -204,7 +204,7 @@ def test_start_python_pipeline_with_empty_py_requirements_and_without_system_pac wait_for_done = mock_runner.return_value.wait_for_done process_line_callback = MagicMock() - with self.assertRaisesRegex(AirflowException, "Invalid method invocation."): + with pytest.raises(AirflowException, match=r"Invalid method invocation\."): hook.start_python_pipeline( variables=copy.deepcopy(BEAM_VARIABLES_PY), py_file=PY_FILE, @@ -302,20 +302,18 @@ def test_start_go_pipeline_without_go_installed_raises(self, mock_which): mock_which.return_value = None hook = BeamHook(runner=DEFAULT_RUNNER) - with self.assertRaises(AirflowException) as ex_ctx: + error_message = ( + r"You need to have Go installed to run beam go pipeline\. See .* " + "installation guide. If you are running airflow in Docker see more info at '.*'" + ) + with pytest.raises(AirflowException, match=error_message): hook.start_go_pipeline( go_file=GO_FILE, variables=copy.deepcopy(BEAM_VARIABLES_GO), ) - assert ( - "You need to have Go installed to run beam go pipeline. See https://go.dev/doc/install " - "installation guide. If you are running airflow in Docker see more info at " - "'https://airflow.apache.org/docs/docker-stack/recipes.html'." == str(ex_ctx.exception) - ) - -class TestBeamRunner(unittest.TestCase): +class TestBeamRunner: @mock.patch("airflow.providers.apache.beam.hooks.beam.BeamCommandRunner.log") @mock.patch("subprocess.Popen") @mock.patch("select.select") @@ -343,18 +341,20 @@ def poll_resp_error(): mock_popen.assert_called_once_with( cmd, shell=False, stdout=subprocess.PIPE, stderr=subprocess.PIPE, close_fds=True, cwd=None ) - self.assertRaises(Exception, beam.wait_for_done) + with pytest.raises(Exception): + beam.wait_for_done() -class TestBeamOptionsToArgs(unittest.TestCase): - @parameterized.expand( +class TestBeamOptionsToArgs: + @pytest.mark.parametrize( + "options, expected_args", [ ({"key": "val"}, ["--key=val"]), ({"key": None}, ["--key"]), ({"key": True}, ["--key"]), ({"key": False}, ["--key=False"]), ({"key": ["a", "b", "c"]}, ["--key=a", "--key=b", "--key=c"]), - ] + ], ) def test_beam_options_to_args(self, options, expected_args): args = beam_options_to_args(options) diff --git a/tests/providers/apache/beam/operators/test_beam.py b/tests/providers/apache/beam/operators/test_beam.py index b1885cf58b279..d5e0bfd58d46d 100644 --- a/tests/providers/apache/beam/operators/test_beam.py +++ b/tests/providers/apache/beam/operators/test_beam.py @@ -16,7 +16,6 @@ # under the License. from __future__ import annotations -import unittest from unittest import mock from unittest.mock import MagicMock @@ -51,8 +50,8 @@ TEST_IMPERSONATION_ACCOUNT = "test@impersonation.com" -class TestBeamRunPythonPipelineOperator(unittest.TestCase): - def setUp(self): +class TestBeamRunPythonPipelineOperator: + def setup_method(self): self.operator = BeamRunPythonPipelineOperator( task_id=TASK_ID, py_file=PY_FILE, @@ -63,13 +62,13 @@ def setUp(self): def test_init(self): """Test BeamRunPythonPipelineOperator instance is properly initialized.""" - self.assertEqual(self.operator.task_id, TASK_ID) - self.assertEqual(self.operator.py_file, PY_FILE) - self.assertEqual(self.operator.runner, DEFAULT_RUNNER) - self.assertEqual(self.operator.py_options, PY_OPTIONS) - self.assertEqual(self.operator.py_interpreter, PY_INTERPRETER) - self.assertEqual(self.operator.default_pipeline_options, DEFAULT_OPTIONS_PYTHON) - self.assertEqual(self.operator.pipeline_options, EXPECTED_ADDITIONAL_OPTIONS) + assert self.operator.task_id == TASK_ID + assert self.operator.py_file == PY_FILE + assert self.operator.runner == DEFAULT_RUNNER + assert self.operator.py_options == PY_OPTIONS + assert self.operator.py_interpreter == PY_INTERPRETER + assert self.operator.default_pipeline_options == DEFAULT_OPTIONS_PYTHON + assert self.operator.pipeline_options == EXPECTED_ADDITIONAL_OPTIONS @mock.patch("airflow.providers.apache.beam.operators.beam.BeamHook") @mock.patch("airflow.providers.apache.beam.operators.beam.GCSHook") @@ -180,8 +179,8 @@ def test_on_kill_direct_runner(self, _, dataflow_mock, __): dataflow_cancel_job.assert_not_called() -class TestBeamRunJavaPipelineOperator(unittest.TestCase): - def setUp(self): +class TestBeamRunJavaPipelineOperator: + def setup_method(self): self.operator = BeamRunJavaPipelineOperator( task_id=TASK_ID, jar=JAR_FILE, @@ -192,12 +191,12 @@ def setUp(self): def test_init(self): """Test BeamRunJavaPipelineOperator instance is properly initialized.""" - self.assertEqual(self.operator.task_id, TASK_ID) - self.assertEqual(self.operator.runner, DEFAULT_RUNNER) - self.assertEqual(self.operator.default_pipeline_options, DEFAULT_OPTIONS_JAVA) - self.assertEqual(self.operator.job_class, JOB_CLASS) - self.assertEqual(self.operator.jar, JAR_FILE) - self.assertEqual(self.operator.pipeline_options, ADDITIONAL_OPTIONS) + assert self.operator.task_id == TASK_ID + assert self.operator.runner == DEFAULT_RUNNER + assert self.operator.default_pipeline_options == DEFAULT_OPTIONS_JAVA + assert self.operator.job_class == JOB_CLASS + assert self.operator.jar == JAR_FILE + assert self.operator.pipeline_options == ADDITIONAL_OPTIONS @mock.patch("airflow.providers.apache.beam.operators.beam.BeamHook") @mock.patch("airflow.providers.apache.beam.operators.beam.GCSHook") @@ -299,8 +298,8 @@ def test_on_kill_direct_runner(self, _, dataflow_mock, __): dataflow_cancel_job.assert_not_called() -class TestBeamRunGoPipelineOperator(unittest.TestCase): - def setUp(self): +class TestBeamRunGoPipelineOperator: + def setup_method(self): self.operator = BeamRunGoPipelineOperator( task_id=TASK_ID, go_file=GO_FILE, @@ -310,11 +309,11 @@ def setUp(self): def test_init(self): """Test BeamRunGoPipelineOperator instance is properly initialized.""" - self.assertEqual(self.operator.task_id, TASK_ID) - self.assertEqual(self.operator.go_file, GO_FILE) - self.assertEqual(self.operator.runner, DEFAULT_RUNNER) - self.assertEqual(self.operator.default_pipeline_options, DEFAULT_OPTIONS_PYTHON) - self.assertEqual(self.operator.pipeline_options, EXPECTED_ADDITIONAL_OPTIONS) + assert self.operator.task_id == TASK_ID + assert self.operator.go_file == GO_FILE + assert self.operator.runner == DEFAULT_RUNNER + assert self.operator.default_pipeline_options == DEFAULT_OPTIONS_PYTHON + assert self.operator.pipeline_options == EXPECTED_ADDITIONAL_OPTIONS @mock.patch( "tempfile.TemporaryDirectory", diff --git a/tests/providers/apache/cassandra/hooks/test_cassandra.py b/tests/providers/apache/cassandra/hooks/test_cassandra.py index b265e8aed4e41..70bf5008e13c0 100644 --- a/tests/providers/apache/cassandra/hooks/test_cassandra.py +++ b/tests/providers/apache/cassandra/hooks/test_cassandra.py @@ -17,7 +17,6 @@ # under the License. from __future__ import annotations -import unittest from unittest import mock import pytest @@ -35,14 +34,14 @@ @pytest.mark.integration("cassandra") -class TestCassandraHook(unittest.TestCase): - def setUp(self): +class TestCassandraHook: + def setup_method(self): db.merge_conn( Connection( conn_id="cassandra_test", conn_type="cassandra", host="host-1,host-2", - port="9042", + port=9042, schema="test_keyspace", extra='{"load_balancing_policy":"TokenAwarePolicy","protocol_version":4}', ) @@ -52,7 +51,7 @@ def setUp(self): conn_id="cassandra_default_with_schema", conn_type="cassandra", host="cassandra", - port="9042", + port=9042, schema="s", ) ) diff --git a/tests/providers/apache/cassandra/sensors/test_record.py b/tests/providers/apache/cassandra/sensors/test_record.py index 35705d353a85c..309c168af2a05 100644 --- a/tests/providers/apache/cassandra/sensors/test_record.py +++ b/tests/providers/apache/cassandra/sensors/test_record.py @@ -17,7 +17,6 @@ # under the License. from __future__ import annotations -import unittest from unittest.mock import patch from airflow.providers.apache.cassandra.sensors.record import CassandraRecordSensor @@ -27,7 +26,7 @@ TEST_CASSANDRA_KEY = {"foo": "bar"} -class TestCassandraRecordSensor(unittest.TestCase): +class TestCassandraRecordSensor: @patch("airflow.providers.apache.cassandra.sensors.record.CassandraHook") def test_poke(self, mock_hook): sensor = CassandraRecordSensor( diff --git a/tests/providers/apache/cassandra/sensors/test_table.py b/tests/providers/apache/cassandra/sensors/test_table.py index a81b82fa07d1d..9f02f0413bb85 100644 --- a/tests/providers/apache/cassandra/sensors/test_table.py +++ b/tests/providers/apache/cassandra/sensors/test_table.py @@ -17,7 +17,6 @@ # under the License. from __future__ import annotations -import unittest from unittest.mock import patch from airflow.providers.apache.cassandra.sensors.table import CassandraTableSensor @@ -27,7 +26,7 @@ TEST_CASSANDRA_TABLE_WITH_KEYSPACE = "keyspacename.tablename" -class TestCassandraTableSensor(unittest.TestCase): +class TestCassandraTableSensor: @patch("airflow.providers.apache.cassandra.sensors.table.CassandraHook") def test_poke(self, mock_hook): sensor = CassandraTableSensor( diff --git a/tests/providers/apache/drill/hooks/test_drill.py b/tests/providers/apache/drill/hooks/test_drill.py index 996d8739eab52..c873b1ceefe32 100644 --- a/tests/providers/apache/drill/hooks/test_drill.py +++ b/tests/providers/apache/drill/hooks/test_drill.py @@ -17,14 +17,13 @@ # under the License. from __future__ import annotations -import unittest from unittest.mock import MagicMock from airflow.providers.apache.drill.hooks.drill import DrillHook -class TestDrillHook(unittest.TestCase): - def setUp(self): +class TestDrillHook: + def setup_method(self): self.cur = MagicMock(rowcount=0) self.conn = conn = MagicMock() self.conn.login = "drill_user" diff --git a/tests/providers/apache/drill/operators/test_drill.py b/tests/providers/apache/drill/operators/test_drill.py index e80fdddd80931..25113cf4c8729 100644 --- a/tests/providers/apache/drill/operators/test_drill.py +++ b/tests/providers/apache/drill/operators/test_drill.py @@ -17,8 +17,6 @@ # under the License. from __future__ import annotations -import unittest - import pytest from airflow.models.dag import DAG @@ -32,13 +30,13 @@ @pytest.mark.backend("drill") -class TestDrillOperator(unittest.TestCase): - def setUp(self): +class TestDrillOperator: + def setup_method(self): args = {"owner": "airflow", "start_date": DEFAULT_DATE} dag = DAG(TEST_DAG_ID, default_args=args) self.dag = dag - def tearDown(self): + def teardown_method(self): tables_to_drop = ["dfs.tmp.test_airflow"] from airflow.providers.apache.drill.hooks.drill import DrillHook diff --git a/tests/providers/apache/druid/hooks/test_druid.py b/tests/providers/apache/druid/hooks/test_druid.py index ddc3defa6e3a7..7d97f857b44d0 100644 --- a/tests/providers/apache/druid/hooks/test_druid.py +++ b/tests/providers/apache/druid/hooks/test_druid.py @@ -17,20 +17,19 @@ # under the License. from __future__ import annotations -import unittest from unittest.mock import MagicMock, patch import pytest import requests -import requests_mock from airflow.exceptions import AirflowException from airflow.providers.apache.druid.hooks.druid import DruidDbApiHook, DruidHook -class TestDruidHook(unittest.TestCase): - def setUp(self): - super().setUp() +class TestDruidHook: + def setup_method(self): + import requests_mock + session = requests.Session() adapter = requests_mock.Adapter() session.mount("mock", adapter) @@ -41,13 +40,12 @@ def get_conn_url(self): self.db_hook = TestDRuidhook() - @requests_mock.mock() - def test_submit_gone_wrong(self, m): - task_post = m.post( + def test_submit_gone_wrong(self, requests_mock): + task_post = requests_mock.post( "http://druid-overlord:8081/druid/indexer/v1/task", text='{"task":"9f8a7359-77d4-4612-b0cd-cc2f6a3c28de"}', ) - status_check = m.get( + status_check = requests_mock.get( "http://druid-overlord:8081/druid/indexer/v1/task/9f8a7359-77d4-4612-b0cd-cc2f6a3c28de/status", text='{"status":{"status": "FAILED"}}', ) @@ -59,13 +57,12 @@ def test_submit_gone_wrong(self, m): assert task_post.called_once assert status_check.called_once - @requests_mock.mock() - def test_submit_ok(self, m): - task_post = m.post( + def test_submit_ok(self, requests_mock): + task_post = requests_mock.post( "http://druid-overlord:8081/druid/indexer/v1/task", text='{"task":"9f8a7359-77d4-4612-b0cd-cc2f6a3c28de"}', ) - status_check = m.get( + status_check = requests_mock.get( "http://druid-overlord:8081/druid/indexer/v1/task/9f8a7359-77d4-4612-b0cd-cc2f6a3c28de/status", text='{"status":{"status": "SUCCESS"}}', ) @@ -76,13 +73,12 @@ def test_submit_ok(self, m): assert task_post.called_once assert status_check.called_once - @requests_mock.mock() - def test_submit_correct_json_body(self, m): - task_post = m.post( + def test_submit_correct_json_body(self, requests_mock): + task_post = requests_mock.post( "http://druid-overlord:8081/druid/indexer/v1/task", text='{"task":"9f8a7359-77d4-4612-b0cd-cc2f6a3c28de"}', ) - status_check = m.get( + status_check = requests_mock.get( "http://druid-overlord:8081/druid/indexer/v1/task/9f8a7359-77d4-4612-b0cd-cc2f6a3c28de/status", text='{"status":{"status": "SUCCESS"}}', ) @@ -100,13 +96,12 @@ def test_submit_correct_json_body(self, m): req_body = task_post.request_history[0].json() assert req_body["task"] == "9f8a7359-77d4-4612-b0cd-cc2f6a3c28de" - @requests_mock.mock() - def test_submit_unknown_response(self, m): - task_post = m.post( + def test_submit_unknown_response(self, requests_mock): + task_post = requests_mock.post( "http://druid-overlord:8081/druid/indexer/v1/task", text='{"task":"9f8a7359-77d4-4612-b0cd-cc2f6a3c28de"}', ) - status_check = m.get( + status_check = requests_mock.get( "http://druid-overlord:8081/druid/indexer/v1/task/9f8a7359-77d4-4612-b0cd-cc2f6a3c28de/status", text='{"status":{"status": "UNKNOWN"}}', ) @@ -118,19 +113,18 @@ def test_submit_unknown_response(self, m): assert task_post.called_once assert status_check.called_once - @requests_mock.mock() - def test_submit_timeout(self, m): + def test_submit_timeout(self, requests_mock): self.db_hook.timeout = 1 self.db_hook.max_ingestion_time = 5 - task_post = m.post( + task_post = requests_mock.post( "http://druid-overlord:8081/druid/indexer/v1/task", text='{"task":"9f8a7359-77d4-4612-b0cd-cc2f6a3c28de"}', ) - status_check = m.get( + status_check = requests_mock.get( "http://druid-overlord:8081/druid/indexer/v1/task/9f8a7359-77d4-4612-b0cd-cc2f6a3c28de/status", text='{"status":{"status": "RUNNING"}}', ) - shutdown_post = m.post( + shutdown_post = requests_mock.post( "http://druid-overlord:8081/druid/indexer/v1/task/" "9f8a7359-77d4-4612-b0cd-cc2f6a3c28de/shutdown", text='{"task":"9f8a7359-77d4-4612-b0cd-cc2f6a3c28de"}', @@ -189,9 +183,8 @@ def test_get_auth_with_no_user_and_password(self, mock_get_connection): assert self.db_hook.get_auth() is None -class TestDruidDbApiHook(unittest.TestCase): - def setUp(self): - super().setUp() +class TestDruidDbApiHook: + def setup_method(self): self.cur = MagicMock(rowcount=0) self.conn = conn = MagicMock() self.conn.host = "host" diff --git a/tests/providers/apache/druid/transfers/test_hive_to_druid.py b/tests/providers/apache/druid/transfers/test_hive_to_druid.py index f9d9999411c4c..cf990f400e5d1 100644 --- a/tests/providers/apache/druid/transfers/test_hive_to_druid.py +++ b/tests/providers/apache/druid/transfers/test_hive_to_druid.py @@ -17,16 +17,13 @@ # under the License. from __future__ import annotations -import unittest - import requests -import requests_mock from airflow.models.dag import DAG from airflow.providers.apache.druid.transfers.hive_to_druid import HiveToDruidOperator -class TestDruidHook(unittest.TestCase): +class TestDruidHook: # To debug the large json diff maxDiff = None @@ -57,8 +54,8 @@ class TestDruidHook(unittest.TestCase): index_spec_config = {"static_path": "/apps/db/warehouse/hive/", "columns": ["country", "segment"]} - def setUp(self): - super().setUp() + def setup_method(self): + import requests_mock args = {"owner": "airflow", "start_date": "2017-01-01"} self.dag = DAG("hive_to_druid", default_args=args) diff --git a/tests/providers/apache/hdfs/hooks/test_hdfs.py b/tests/providers/apache/hdfs/hooks/test_hdfs.py index 5e642fd6f81f9..3b4f7e6d11225 100644 --- a/tests/providers/apache/hdfs/hooks/test_hdfs.py +++ b/tests/providers/apache/hdfs/hooks/test_hdfs.py @@ -18,7 +18,6 @@ from __future__ import annotations import json -import unittest from unittest import mock import pytest @@ -26,13 +25,10 @@ from airflow.models import Connection from airflow.providers.apache.hdfs.hooks.hdfs import HDFSHook -try: - import snakebite -except ImportError: - pytestmark = pytest.mark.skip("Skipping test because HDFSHook is not installed") +snakebite = pytest.importorskip("snakebite") -class TestHDFSHook(unittest.TestCase): +class TestHDFSHook: @mock.patch.dict( "os.environ", { diff --git a/tests/providers/apache/hdfs/hooks/test_webhdfs.py b/tests/providers/apache/hdfs/hooks/test_webhdfs.py index 71e1231dac3d9..ca584b3ea9515 100644 --- a/tests/providers/apache/hdfs/hooks/test_webhdfs.py +++ b/tests/providers/apache/hdfs/hooks/test_webhdfs.py @@ -17,7 +17,6 @@ # under the License. from __future__ import annotations -import unittest from unittest.mock import call, patch import pytest @@ -27,8 +26,8 @@ from airflow.providers.apache.hdfs.hooks.webhdfs import AirflowWebHDFSHookException, WebHDFSHook -class TestWebHDFSHook(unittest.TestCase): - def setUp(self): +class TestWebHDFSHook: + def setup_method(self): self.webhdfs_hook = WebHDFSHook() @patch("airflow.providers.apache.hdfs.hooks.webhdfs.requests.Session", return_value="session") @@ -85,61 +84,39 @@ def test_get_conn_with_schema(self, socket_mock, mock_get_connection, mock_insec mock_insecure_client.return_value.status.assert_called_once_with("/") assert conn == mock_insecure_client.return_value - @patch("airflow.providers.apache.hdfs.hooks.webhdfs.requests.Session", return_value="session") - @patch("airflow.providers.apache.hdfs.hooks.webhdfs.InsecureClient") - @patch( - "airflow.providers.apache.hdfs.hooks.webhdfs.WebHDFSHook.get_connection", - return_value=Connection(host="host_1.com,host_2.com", login="user"), + @pytest.mark.parametrize( + "test_connection", + [ + pytest.param(Connection(host="host_1.com,host_2.com", login="user"), id="without-password"), + pytest.param( + Connection(host="host_1.com,host_2.com", login="user", password="password"), + id="with-password", + ), + ], ) - @patch("airflow.providers.apache.hdfs.hooks.webhdfs.socket") - def test_get_conn_without_port_schema( - self, socket_mock, mock_get_connection, mock_insecure_client, mock_session - ): - mock_insecure_client.side_effect = [HdfsError("Error"), mock_insecure_client.return_value] - socket_mock.socket.return_value.connect_ex.return_value = 0 - conn = self.webhdfs_hook.get_conn() - connection = mock_get_connection.return_value - hosts = connection.host.split(",") - mock_insecure_client.assert_has_calls( - [ - call( - f"http://{host}", - user=connection.login, - session=mock_session.return_value, - ) - for host in hosts - ] - ) - mock_insecure_client.return_value.status.assert_called_once_with("/") - assert conn == mock_insecure_client.return_value - - @patch("airflow.providers.apache.hdfs.hooks.webhdfs.requests.Session", create=True) + @patch("airflow.providers.apache.hdfs.hooks.webhdfs.requests.Session") @patch("airflow.providers.apache.hdfs.hooks.webhdfs.InsecureClient") - @patch( - "airflow.providers.apache.hdfs.hooks.webhdfs.WebHDFSHook.get_connection", - return_value=Connection(host="host_1.com,host_2.com", login="user", password="password"), - ) @patch("airflow.providers.apache.hdfs.hooks.webhdfs.socket") - def test_get_conn_with_password_without_port_schema( - self, socket_mock, mock_get_connection, mock_insecure_client, mock_session + def test_get_conn_without_port_schema( + self, socket_mock, mock_insecure_client, mock_session, test_connection ): - mock_insecure_client.side_effect = [HdfsError("Error"), mock_insecure_client.return_value] - socket_mock.socket.return_value.connect_ex.return_value = 0 - conn = self.webhdfs_hook.get_conn() - connection = mock_get_connection.return_value - hosts = connection.host.split(",") - mock_insecure_client.assert_has_calls( - [ - call( - f"http://{host}", - user=connection.login, - session=mock_session.return_value, - ) - for host in hosts - ] - ) - mock_insecure_client.return_value.status.assert_called_once_with("/") - assert conn == mock_insecure_client.return_value + with patch.object(WebHDFSHook, "get_connection", return_value=test_connection): + mock_insecure_client.side_effect = [HdfsError("Error"), mock_insecure_client.return_value] + socket_mock.socket.return_value.connect_ex.return_value = 0 + conn = self.webhdfs_hook.get_conn() + hosts = test_connection.host.split(",") + mock_insecure_client.assert_has_calls( + [ + call( + f"http://{host}", + user=test_connection.login, + session=mock_session.return_value, + ) + for host in hosts + ] + ) + mock_insecure_client.return_value.status.assert_called_once_with("/") + assert conn == mock_insecure_client.return_value @patch("airflow.providers.apache.hdfs.hooks.webhdfs.InsecureClient", side_effect=HdfsError("Error")) @patch( diff --git a/tests/providers/apache/hdfs/sensors/test_hdfs.py b/tests/providers/apache/hdfs/sensors/test_hdfs.py index 71a4560eb2954..c5deccb65f4af 100644 --- a/tests/providers/apache/hdfs/sensors/test_hdfs.py +++ b/tests/providers/apache/hdfs/sensors/test_hdfs.py @@ -19,7 +19,6 @@ import logging import re -import unittest from datetime import timedelta import pytest @@ -33,8 +32,8 @@ TEST_DAG_ID = "unit_test_dag" -class TestHdfsSensor(unittest.TestCase): - def setUp(self): +class TestHdfsSensor: + def setup_method(self): self.hook = FakeHDFSHook def test_legacy_file_exist(self): @@ -97,11 +96,15 @@ def test_legacy_file_does_not_exists(self): task.execute(None) -class TestHdfsSensorFolder(unittest.TestCase): - def setUp(self): +class TestHdfsSensorFolder: + def setup_method(self, method): self.hook = FakeHDFSHook - self.log = logging.getLogger() - self.log.setLevel(logging.DEBUG) + + logger = logging.getLogger(__name__) + logger.setLevel(logging.DEBUG) + logger.debug("#" * 10) + logger.debug("Running test case: %s.%s", self.__class__.__name__, method.__name__) + logger.debug("#" * 10) def test_should_be_empty_directory(self): """ @@ -109,9 +112,6 @@ def test_should_be_empty_directory(self): :return: """ # Given - self.log.debug("#" * 10) - self.log.debug("Running %s", self._testMethodName) - self.log.debug("#" * 10) task = HdfsFolderSensor( task_id="Should_be_empty_directory", filepath="/datadirectory/empty_directory", @@ -134,9 +134,6 @@ def test_should_be_empty_directory_fail(self): :return: """ # Given - self.log.debug("#" * 10) - self.log.debug("Running %s", self._testMethodName) - self.log.debug("#" * 10) task = HdfsFolderSensor( task_id="Should_be_empty_directory_fail", filepath="/datadirectory/not_empty_directory", @@ -158,9 +155,6 @@ def test_should_be_a_non_empty_directory(self): :return: """ # Given - self.log.debug("#" * 10) - self.log.debug("Running %s", self._testMethodName) - self.log.debug("#" * 10) task = HdfsFolderSensor( task_id="Should_be_non_empty_directory", filepath="/datadirectory/not_empty_directory", @@ -182,9 +176,6 @@ def test_should_be_non_empty_directory_fail(self): :return: """ # Given - self.log.debug("#" * 10) - self.log.debug("Running %s", self._testMethodName) - self.log.debug("#" * 10) task = HdfsFolderSensor( task_id="Should_be_empty_directory_fail", filepath="/datadirectory/empty_directory", @@ -200,11 +191,15 @@ def test_should_be_non_empty_directory_fail(self): task.execute(None) -class TestHdfsSensorRegex(unittest.TestCase): - def setUp(self): +class TestHdfsSensorRegex: + def setup_method(self, method): self.hook = FakeHDFSHook - self.log = logging.getLogger() - self.log.setLevel(logging.DEBUG) + + logger = logging.getLogger(__name__) + logger.setLevel(logging.DEBUG) + logger.debug("#" * 10) + logger.debug("Running test case: %s.%s", self.__class__.__name__, method.__name__) + logger.debug("#" * 10) def test_should_match_regex(self): """ @@ -212,9 +207,6 @@ def test_should_match_regex(self): :return: """ # Given - self.log.debug("#" * 10) - self.log.debug("Running %s", self._testMethodName) - self.log.debug("#" * 10) compiled_regex = re.compile("test[1-2]file") task = HdfsRegexSensor( task_id="Should_match_the_regex", @@ -238,9 +230,6 @@ def test_should_not_match_regex(self): :return: """ # Given - self.log.debug("#" * 10) - self.log.debug("Running %s", self._testMethodName) - self.log.debug("#" * 10) compiled_regex = re.compile("^IDoNotExist") task = HdfsRegexSensor( task_id="Should_not_match_the_regex", @@ -263,9 +252,6 @@ def test_should_match_regex_and_filesize(self): :return: """ # Given - self.log.debug("#" * 10) - self.log.debug("Running %s", self._testMethodName) - self.log.debug("#" * 10) compiled_regex = re.compile("test[1-2]file") task = HdfsRegexSensor( task_id="Should_match_the_regex_and_filesize", @@ -292,9 +278,6 @@ def test_should_match_regex_but_filesize(self): :return: """ # Given - self.log.debug("#" * 10) - self.log.debug("Running %s", self._testMethodName) - self.log.debug("#" * 10) compiled_regex = re.compile("test[1-2]file") task = HdfsRegexSensor( task_id="Should_match_the_regex_but_filesize", @@ -318,9 +301,6 @@ def test_should_match_regex_but_copyingext(self): :return: """ # Given - self.log.debug("#" * 10) - self.log.debug("Running %s", self._testMethodName) - self.log.debug("#" * 10) compiled_regex = re.compile(r"copying_file_\d+.txt") task = HdfsRegexSensor( task_id="Should_match_the_regex_but_filesize", diff --git a/tests/providers/apache/hdfs/sensors/test_web_hdfs.py b/tests/providers/apache/hdfs/sensors/test_web_hdfs.py index 40b9c389438b5..ad1276814b7ab 100644 --- a/tests/providers/apache/hdfs/sensors/test_web_hdfs.py +++ b/tests/providers/apache/hdfs/sensors/test_web_hdfs.py @@ -20,13 +20,12 @@ from unittest import mock from airflow.providers.apache.hdfs.sensors.web_hdfs import WebHdfsSensor -from tests.providers.apache.hive import TestHiveEnvironment TEST_HDFS_CONN = "webhdfs_default" TEST_HDFS_PATH = "hdfs://user/hive/warehouse/airflow.db/static_babynames" -class TestWebHdfsSensor(TestHiveEnvironment): +class TestWebHdfsSensor: @mock.patch("airflow.providers.apache.hdfs.hooks.webhdfs.WebHDFSHook") def test_poke(self, mock_hook): sensor = WebHdfsSensor( diff --git a/tests/providers/apache/hive/__init__.py b/tests/providers/apache/hive/__init__.py index 51ff81911e474..3db74244b8395 100644 --- a/tests/providers/apache/hive/__init__.py +++ b/tests/providers/apache/hive/__init__.py @@ -18,7 +18,6 @@ from __future__ import annotations from datetime import datetime -from unittest import TestCase from unittest.mock import MagicMock from airflow.models.dag import DAG @@ -30,8 +29,8 @@ DEFAULT_DATE_DS = DEFAULT_DATE_ISO[:10] -class TestHiveEnvironment(TestCase): - def setUp(self): +class TestHiveEnvironment: + def setup_method(self, method): args = {"owner": "airflow", "start_date": DEFAULT_DATE} dag = DAG("test_dag_id", default_args=args) self.dag = dag diff --git a/tests/providers/apache/hive/hooks/test_hive.py b/tests/providers/apache/hive/hooks/test_hive.py index 130554283bb94..852fc48061a77 100644 --- a/tests/providers/apache/hive/hooks/test_hive.py +++ b/tests/providers/apache/hive/hooks/test_hive.py @@ -19,8 +19,6 @@ import datetime import itertools -import os -import unittest from collections import OrderedDict, namedtuple from unittest import mock @@ -55,23 +53,7 @@ def __init__(self): self.iterable = [] -class TestHiveEnvironment(unittest.TestCase): - def setUp(self): - self.next_day = (DEFAULT_DATE + datetime.timedelta(days=1)).isoformat()[:10] - self.database = "airflow" - self.partition_by = "ds" - self.table = "static_babynames_partitioned" - with mock.patch( - "airflow.providers.apache.hive.hooks.hive.HiveMetastoreHook.get_metastore_client" - ) as get_metastore_mock, mock.patch( - "airflow.providers.apache.hive.hooks.hive.HiveMetastoreHook.get_connection" - ): - get_metastore_mock.return_value = mock.MagicMock() - - self.hook = HiveMetastoreHook() - - -class TestHiveCliHook(unittest.TestCase): +class TestHiveCliHook: @mock.patch("tempfile.tempdir", "/tmp/") @mock.patch("tempfile._RandomNameSequence.__next__") @mock.patch("subprocess.Popen") @@ -340,10 +322,24 @@ def test_load_df_with_data_types(self, mock_run_cli): STORED AS textfile ; """ - assert_equal_ignore_multiple_spaces(self, mock_run_cli.call_args_list[0][0][0], query) + assert_equal_ignore_multiple_spaces(None, mock_run_cli.call_args_list[0][0][0], query) -class TestHiveMetastoreHook(TestHiveEnvironment): +class TestHiveMetastoreHook: + def setup_method(self): + self.next_day = (DEFAULT_DATE + datetime.timedelta(days=1)).isoformat()[:10] + self.database = "airflow" + self.partition_by = "ds" + self.table = "static_babynames_partitioned" + with mock.patch( + "airflow.providers.apache.hive.hooks.hive.HiveMetastoreHook.get_metastore_client" + ) as get_metastore_mock, mock.patch( + "airflow.providers.apache.hive.hooks.hive.HiveMetastoreHook.get_connection" + ): + get_metastore_mock.return_value = mock.MagicMock() + + self.hook = HiveMetastoreHook() + VALID_FILTER_MAP = {"key2": "value2"} def test_get_max_partition_from_empty_part_specs(self): @@ -352,7 +348,6 @@ def test_get_max_partition_from_empty_part_specs(self): ) assert max_partition is None - # @mock.patch('airflow.providers.apache.hive.hooks.hive.HiveMetastoreHook', 'get_metastore_client') def test_get_max_partition_from_valid_part_specs_and_invalid_filter_map(self): with pytest.raises(AirflowException): HiveMetastoreHook._get_max_partition_from_part_specs( @@ -586,13 +581,13 @@ def test_drop_partition(self, get_metastore_client_mock, table_exist_mock): assert metastore_mock.drop_partition(self.table, db=self.database, part_vals=[DEFAULT_DATE_DS]), ret -class TestHiveServer2Hook(unittest.TestCase): +class TestHiveServer2Hook: def _upload_dataframe(self): df = pd.DataFrame({"a": [1, 2], "b": [1, 2]}) self.local_path = "/tmp/TestHiveServer2Hook.csv" df.to_csv(self.local_path, header=False, index=False) - def setUp(self): + def setup_method(self): self._upload_dataframe() args = {"owner": "airflow", "start_date": DEFAULT_DATE} self.dag = DAG("test_dag_id", default_args=args) @@ -772,13 +767,9 @@ def test_multi_statements(self): "AIRFLOW_CTX_DAG_EMAIL": "test@airflow.com", }, ): - # df = hook.get_pandas_df(query, schema=self.database) results = hook.get_records(sqls, schema=self.database) assert results == [(1, 1), (2, 2)] - # self.assertEqual(len(df), 2) - # self.assertListEqual(df["hive_server_hook.a"].values.tolist(), [1, 2]) - hook.get_conn.assert_called_with(self.database) hook.mock_cursor.execute.assert_any_call("CREATE TABLE IF NOT EXISTS test_multi_statements (i INT)") hook.mock_cursor.execute.assert_any_call(f"SELECT * FROM {self.table}") @@ -841,13 +832,10 @@ def test_get_results_with_hive_conf(self): assert "test_dag_run_id" in output -class TestHiveCli(unittest.TestCase): - def setUp(self): +@mock.patch.dict("os.environ", AIRFLOW__CORE__SECURITY="kerberos") +class TestHiveCli: + def setup_method(self): self.nondefault_schema = "nondefault" - os.environ["AIRFLOW__CORE__SECURITY"] = "kerberos" - - def tearDown(self): - del os.environ["AIRFLOW__CORE__SECURITY"] def test_get_proxy_user_value(self): hook = MockHiveCliHook() diff --git a/tests/providers/apache/hive/operators/test_hive.py b/tests/providers/apache/hive/operators/test_hive.py index a74c42925eccf..d64a2f7bc3422 100644 --- a/tests/providers/apache/hive/operators/test_hive.py +++ b/tests/providers/apache/hive/operators/test_hive.py @@ -18,9 +18,10 @@ from __future__ import annotations import os -import unittest from unittest import mock +import pytest + from airflow.configuration import conf from airflow.models import DagRun, TaskInstance from airflow.providers.apache.hive.operators.hive import HiveOperator @@ -95,7 +96,9 @@ def test_mapred_job_name(self, mock_get_hook): ) -@unittest.skipIf("AIRFLOW_RUNALL_TESTS" not in os.environ, "Skipped because AIRFLOW_RUNALL_TESTS is not set") +@pytest.mark.skipif( + "AIRFLOW_RUNALL_TESTS" not in os.environ, reason="Skipped because AIRFLOW_RUNALL_TESTS is not set" +) class TestHivePresto(TestHiveEnvironment): @mock.patch("tempfile.tempdir", "/tmp/") @mock.patch("tempfile._RandomNameSequence.__next__") diff --git a/tests/providers/apache/hive/operators/test_hive_stats.py b/tests/providers/apache/hive/operators/test_hive_stats.py index aba56f5f2f520..27a7772236a4e 100644 --- a/tests/providers/apache/hive/operators/test_hive_stats.py +++ b/tests/providers/apache/hive/operators/test_hive_stats.py @@ -19,7 +19,6 @@ import os import re -import unittest from collections import OrderedDict from unittest.mock import MagicMock, patch @@ -62,7 +61,7 @@ def get_connection(self, *args): class TestHiveStatsCollectionOperator(TestHiveEnvironment): - def setUp(self): + def setup_method(self, method): self.kwargs = dict( table="table", partition=dict(col="col", value="value"), @@ -71,7 +70,7 @@ def setUp(self): mysql_conn_id="mysql_conn_id", task_id="test_hive_stats_collection_operator", ) - super().setUp() + super().setup_method(method) def test_get_default_exprs(self): col = "col" @@ -305,8 +304,8 @@ def test_execute_delete_previous_runs_rows( """ mock_mysql_hook.return_value.run.assert_called_once_with(sql) - @unittest.skipIf( - "AIRFLOW_RUNALL_TESTS" not in os.environ, "Skipped because AIRFLOW_RUNALL_TESTS is not set" + @pytest.mark.skipif( + "AIRFLOW_RUNALL_TESTS" not in os.environ, reason="Skipped because AIRFLOW_RUNALL_TESTS is not set" ) @patch( "airflow.providers.apache.hive.operators.hive_stats.HiveMetastoreHook", diff --git a/tests/providers/apache/hive/sensors/test_hdfs.py b/tests/providers/apache/hive/sensors/test_hdfs.py index 6046e38d8fac6..a5cb9c6e91713 100644 --- a/tests/providers/apache/hive/sensors/test_hdfs.py +++ b/tests/providers/apache/hive/sensors/test_hdfs.py @@ -18,13 +18,16 @@ from __future__ import annotations import os -import unittest + +import pytest from airflow.providers.apache.hdfs.sensors.hdfs import HdfsSensor from tests.providers.apache.hive import DEFAULT_DATE, TestHiveEnvironment -@unittest.skipIf("AIRFLOW_RUNALL_TESTS" not in os.environ, "Skipped because AIRFLOW_RUNALL_TESTS is not set") +@pytest.mark.skipif( + "AIRFLOW_RUNALL_TESTS" not in os.environ, reason="Skipped because AIRFLOW_RUNALL_TESTS is not set" +) class TestHdfsSensor(TestHiveEnvironment): def test_hdfs_sensor(self): op = HdfsSensor( diff --git a/tests/providers/apache/hive/sensors/test_hive_partition.py b/tests/providers/apache/hive/sensors/test_hive_partition.py index 5ff5277a208d9..45e10783d81ea 100644 --- a/tests/providers/apache/hive/sensors/test_hive_partition.py +++ b/tests/providers/apache/hive/sensors/test_hive_partition.py @@ -18,14 +18,17 @@ from __future__ import annotations import os -import unittest from unittest.mock import patch +import pytest + from airflow.providers.apache.hive.sensors.hive_partition import HivePartitionSensor from tests.providers.apache.hive import DEFAULT_DATE, MockHiveMetastoreHook, TestHiveEnvironment -@unittest.skipIf("AIRFLOW_RUNALL_TESTS" not in os.environ, "Skipped because AIRFLOW_RUNALL_TESTS is not set") +@pytest.mark.skipif( + "AIRFLOW_RUNALL_TESTS" not in os.environ, reason="Skipped because AIRFLOW_RUNALL_TESTS is not set" +) @patch( "airflow.providers.apache.hive.sensors.hive_partition.HiveMetastoreHook", side_effect=MockHiveMetastoreHook, diff --git a/tests/providers/apache/hive/sensors/test_metastore_partition.py b/tests/providers/apache/hive/sensors/test_metastore_partition.py index f49fe0c2f2bf8..04a251339dc30 100644 --- a/tests/providers/apache/hive/sensors/test_metastore_partition.py +++ b/tests/providers/apache/hive/sensors/test_metastore_partition.py @@ -18,14 +18,17 @@ from __future__ import annotations import os -import unittest from unittest import mock +import pytest + from airflow.providers.apache.hive.sensors.metastore_partition import MetastorePartitionSensor from tests.providers.apache.hive import DEFAULT_DATE, DEFAULT_DATE_DS, MockDBConnection, TestHiveEnvironment -@unittest.skipIf("AIRFLOW_RUNALL_TESTS" not in os.environ, "Skipped because AIRFLOW_RUNALL_TESTS is not set") +@pytest.mark.skipif( + "AIRFLOW_RUNALL_TESTS" not in os.environ, reason="Skipped because AIRFLOW_RUNALL_TESTS is not set" +) class TestHivePartitionSensor(TestHiveEnvironment): def test_hive_metastore_sql_sensor(self): op = MetastorePartitionSensor( diff --git a/tests/providers/apache/hive/sensors/test_named_hive_partition.py b/tests/providers/apache/hive/sensors/test_named_hive_partition.py index bf4db9fcf2819..ab63156e5e9df 100644 --- a/tests/providers/apache/hive/sensors/test_named_hive_partition.py +++ b/tests/providers/apache/hive/sensors/test_named_hive_partition.py @@ -18,7 +18,6 @@ from __future__ import annotations import os -import unittest from datetime import timedelta from unittest import mock @@ -35,8 +34,8 @@ DEFAULT_DATE_DS = DEFAULT_DATE_ISO[:10] -class TestNamedHivePartitionSensor(unittest.TestCase): - def setUp(self): +class TestNamedHivePartitionSensor: + def setup_method(self): args = {"owner": "airflow", "start_date": DEFAULT_DATE} self.dag = DAG("test_dag_id", default_args=args) self.next_day = (DEFAULT_DATE + timedelta(days=1)).isoformat()[:10] @@ -114,7 +113,9 @@ def test_poke_non_existing(self): ) -@unittest.skipIf("AIRFLOW_RUNALL_TESTS" not in os.environ, "Skipped because AIRFLOW_RUNALL_TESTS is not set") +@pytest.mark.skipif( + "AIRFLOW_RUNALL_TESTS" not in os.environ, reason="Skipped because AIRFLOW_RUNALL_TESTS is not set" +) class TestPartitions(TestHiveEnvironment): def test_succeeds_on_one_partition(self): mock_hive_metastore_hook = MockHiveMetastoreHook() diff --git a/tests/providers/apache/hive/transfers/test_hive_to_mysql.py b/tests/providers/apache/hive/transfers/test_hive_to_mysql.py index afb84788991e7..d8f39812b2f5a 100644 --- a/tests/providers/apache/hive/transfers/test_hive_to_mysql.py +++ b/tests/providers/apache/hive/transfers/test_hive_to_mysql.py @@ -19,9 +19,10 @@ import os import re -import unittest from unittest.mock import MagicMock, patch +import pytest + from airflow.providers.apache.hive.transfers.hive_to_mysql import HiveToMySqlOperator from airflow.utils import timezone from airflow.utils.operator_helpers import context_to_airflow_vars @@ -31,7 +32,7 @@ class TestHiveToMySqlTransfer(TestHiveEnvironment): - def setUp(self): + def setup_method(self, method): self.kwargs = dict( sql="sql", mysql_table="table", @@ -39,7 +40,7 @@ def setUp(self): mysql_conn_id="mysql_default", task_id="test_hive_to_mysql", ) - super().setUp() + super().setup_method(method) @patch("airflow.providers.apache.hive.transfers.hive_to_mysql.MySqlHook") @patch("airflow.providers.apache.hive.transfers.hive_to_mysql.HiveServer2Hook") @@ -116,8 +117,8 @@ def test_execute_with_hive_conf(self, mock_mysql_hook): mock_hive_hook.get_records.assert_called_once_with(self.kwargs["sql"], parameters=hive_conf) - @unittest.skipIf( - "AIRFLOW_RUNALL_TESTS" not in os.environ, "Skipped because AIRFLOW_RUNALL_TESTS is not set" + @pytest.mark.skipif( + "AIRFLOW_RUNALL_TESTS" not in os.environ, reason="Skipped because AIRFLOW_RUNALL_TESTS is not set" ) def test_hive_to_mysql(self): test_hive_results = "test_hive_results" diff --git a/tests/providers/apache/hive/transfers/test_hive_to_samba.py b/tests/providers/apache/hive/transfers/test_hive_to_samba.py index ad5248700579c..7eb50e44b337e 100644 --- a/tests/providers/apache/hive/transfers/test_hive_to_samba.py +++ b/tests/providers/apache/hive/transfers/test_hive_to_samba.py @@ -18,9 +18,10 @@ from __future__ import annotations import os -import unittest from unittest.mock import MagicMock, Mock, PropertyMock, patch +import pytest + from airflow.providers.apache.hive.transfers.hive_to_samba import HiveToSambaOperator from airflow.providers.samba.hooks.samba import SambaHook from airflow.utils.operator_helpers import context_to_airflow_vars @@ -44,7 +45,7 @@ def get_connection(self, *args): class TestHive2SambaOperator(TestHiveEnvironment): - def setUp(self): + def setup_method(self, method): self.kwargs = dict( hql="hql", destination_filepath="destination_filepath", @@ -52,7 +53,7 @@ def setUp(self): hiveserver2_conn_id="hiveserver2_default", task_id="test_hive_to_samba_operator", ) - super().setUp() + super().setup_method(method) @patch("airflow.providers.apache.hive.transfers.hive_to_samba.SambaHook") @patch("airflow.providers.apache.hive.transfers.hive_to_samba.HiveServer2Hook") @@ -75,8 +76,8 @@ def test_execute(self, mock_tmp_file, mock_hive_hook, mock_samba_hook): self.kwargs["destination_filepath"], mock_tmp_file.name ) - @unittest.skipIf( - "AIRFLOW_RUNALL_TESTS" not in os.environ, "Skipped because AIRFLOW_RUNALL_TESTS is not set" + @pytest.mark.skipif( + "AIRFLOW_RUNALL_TESTS" not in os.environ, reason="Skipped because AIRFLOW_RUNALL_TESTS is not set" ) @patch("tempfile.tempdir", "/tmp/") @patch("tempfile._RandomNameSequence.__next__") diff --git a/tests/providers/apache/hive/transfers/test_mssql_to_hive.py b/tests/providers/apache/hive/transfers/test_mssql_to_hive.py index dc4d0332d93eb..53e49207e5399 100644 --- a/tests/providers/apache/hive/transfers/test_mssql_to_hive.py +++ b/tests/providers/apache/hive/transfers/test_mssql_to_hive.py @@ -17,7 +17,6 @@ # under the License. from __future__ import annotations -import unittest from collections import OrderedDict from unittest.mock import Mock, PropertyMock, patch @@ -26,8 +25,8 @@ from airflow.providers.apache.hive.transfers.mssql_to_hive import MsSqlToHiveOperator -class TestMsSqlToHiveTransfer(unittest.TestCase): - def setUp(self): +class TestMsSqlToHiveTransfer: + def setup_method(self): self.kwargs = dict(sql="sql", hive_table="table", task_id="test_mssql_to_hive", dag=None) def test_type_map_binary(self): diff --git a/tests/providers/apache/hive/transfers/test_s3_to_hive.py b/tests/providers/apache/hive/transfers/test_s3_to_hive.py index 926b4f20a7429..a3369256a7217 100644 --- a/tests/providers/apache/hive/transfers/test_s3_to_hive.py +++ b/tests/providers/apache/hive/transfers/test_s3_to_hive.py @@ -33,11 +33,8 @@ from airflow.exceptions import AirflowException from airflow.providers.apache.hive.transfers.s3_to_hive import S3ToHiveOperator -try: - import boto3 - from moto import mock_s3 -except ImportError: - mock_s3 = None +boto3 = pytest.importorskip("boto3") +moto = pytest.importorskip("moto") class TestS3ToHiveTransfer: @@ -196,13 +193,16 @@ def test__delete_top_row_and_compress(self): fn_bz2 = self._get_fn(".bz2", False) assert self._check_file_equality(bz2_txt_nh, fn_bz2, ".bz2"), "bz2 Compressed file not as expected" - @pytest.mark.skipif(mock is None, reason="mock package not present") - @pytest.mark.skipif(mock_s3 is None, reason="moto package not present") @mock.patch("airflow.providers.apache.hive.transfers.s3_to_hive.HiveCliHook") - @mock_s3 + @moto.mock_s3 def test_execute(self, mock_hiveclihook): conn = boto3.client("s3") - conn.create_bucket(Bucket="bucket") + if conn.meta.region_name == "us-east-1": + conn.create_bucket(Bucket="bucket") + else: + conn.create_bucket( + Bucket="bucket", CreateBucketConfiguration={"LocationConstraint": conn.meta.region_name} + ) # Testing txt, zip, bz2 files with and without header row for (ext, has_header) in product([".txt", ".gz", ".bz2", ".GZ"], [True, False]): @@ -226,13 +226,16 @@ def test_execute(self, mock_hiveclihook): s32hive = S3ToHiveOperator(**self.kwargs) s32hive.execute(None) - @pytest.mark.skipif(mock is None, reason="mock package not present") - @pytest.mark.skipif(mock_s3 is None, reason="moto package not present") @mock.patch("airflow.providers.apache.hive.transfers.s3_to_hive.HiveCliHook") - @mock_s3 + @moto.mock_s3 def test_execute_with_select_expression(self, mock_hiveclihook): conn = boto3.client("s3") - conn.create_bucket(Bucket="bucket") + if conn.meta.region_name == "us-east-1": + conn.create_bucket(Bucket="bucket") + else: + conn.create_bucket( + Bucket="bucket", CreateBucketConfiguration={"LocationConstraint": conn.meta.region_name} + ) select_expression = "SELECT * FROM S3Object s" bucket = "bucket" diff --git a/tests/providers/apache/hive/transfers/test_vertica_to_hive.py b/tests/providers/apache/hive/transfers/test_vertica_to_hive.py index 1d2e3640f77eb..142323c41ee81 100644 --- a/tests/providers/apache/hive/transfers/test_vertica_to_hive.py +++ b/tests/providers/apache/hive/transfers/test_vertica_to_hive.py @@ -18,7 +18,6 @@ from __future__ import annotations import datetime -import unittest from unittest import mock from airflow.models.dag import DAG @@ -40,8 +39,8 @@ def mock_get_conn(): return conn_mock -class TestVerticaToHiveTransfer(unittest.TestCase): - def setUp(self): +class TestVerticaToHiveTransfer: + def setup_method(self): args = {"owner": "airflow", "start_date": datetime.datetime(2017, 1, 1)} self.dag = DAG("test_dag_id", default_args=args) diff --git a/tests/providers/apache/kylin/hooks/test_kylin.py b/tests/providers/apache/kylin/hooks/test_kylin.py index 704e721c8a9b0..812cc8f72cb3e 100644 --- a/tests/providers/apache/kylin/hooks/test_kylin.py +++ b/tests/providers/apache/kylin/hooks/test_kylin.py @@ -17,7 +17,6 @@ # under the License. from __future__ import annotations -import unittest from unittest.mock import MagicMock, patch import pytest @@ -27,8 +26,8 @@ from airflow.providers.apache.kylin.hooks.kylin import KylinHook -class TestKylinHook(unittest.TestCase): - def setUp(self) -> None: +class TestKylinHook: + def setup_method(self) -> None: self.hook = KylinHook(kylin_conn_id="kylin_default", project="learn_kylin") @patch("kylinpy.Kylin.get_job") diff --git a/tests/providers/apache/kylin/operators/test_kylin_cube.py b/tests/providers/apache/kylin/operators/test_kylin_cube.py index 3f9a2c11d31a5..baa61ad80fc4a 100644 --- a/tests/providers/apache/kylin/operators/test_kylin_cube.py +++ b/tests/providers/apache/kylin/operators/test_kylin_cube.py @@ -17,7 +17,6 @@ # under the License. from __future__ import annotations -import unittest from datetime import datetime from unittest.mock import MagicMock, patch @@ -32,7 +31,7 @@ DEFAULT_DATE = timezone.datetime(2020, 1, 1) -class TestKylinCubeOperator(unittest.TestCase): +class TestKylinCubeOperator: _config = { "kylin_conn_id": "kylin_default", "project": "learn_kylin", @@ -59,7 +58,7 @@ class TestKylinCubeOperator(unittest.TestCase): build_response = {"uuid": "c143e0e4-ac5f-434d-acf3-46b0d15e3dc6"} - def setUp(self): + def setup_method(self): args = {"owner": "airflow", "start_date": DEFAULT_DATE} self.dag = DAG("test_dag_id", default_args=args) diff --git a/tests/providers/apache/livy/hooks/test_livy.py b/tests/providers/apache/livy/hooks/test_livy.py index 893a553908e05..1b70a72d251da 100644 --- a/tests/providers/apache/livy/hooks/test_livy.py +++ b/tests/providers/apache/livy/hooks/test_livy.py @@ -17,34 +17,55 @@ from __future__ import annotations import json -import unittest from unittest.mock import MagicMock, patch import pytest -import requests_mock from requests.exceptions import RequestException from airflow.exceptions import AirflowException from airflow.models import Connection from airflow.providers.apache.livy.hooks.livy import BatchState, LivyHook from airflow.utils import db +from tests.test_utils.db import clear_db_connections + +DEFAULT_CONN_ID = LivyHook.default_conn_name +DEFAULT_HOST = "livy" +DEFAULT_SCHEMA = "http" +DEFAULT_PORT = 8998 +MATCH_URL = f"//{DEFAULT_HOST}:{DEFAULT_PORT}" BATCH_ID = 100 SAMPLE_GET_RESPONSE = {"id": BATCH_ID, "state": BatchState.SUCCESS.value} - - -class TestLivyHook(unittest.TestCase): +VALID_SESSION_ID_TEST_CASES = [ + pytest.param(BATCH_ID, id="integer"), + pytest.param(str(BATCH_ID), id="integer as string"), +] +INVALID_SESSION_ID_TEST_CASES = [ + pytest.param(None, id="none"), + pytest.param("forty two", id="invalid string"), + pytest.param({"a": "b"}, id="dictionary"), +] + + +class TestLivyHook: @classmethod - def setUpClass(cls): + def setup_class(cls): + clear_db_connections(add_default_connections_back=False) db.merge_conn( - Connection(conn_id="livy_default", conn_type="http", host="host", schema="http", port=8998) + Connection( + conn_id=DEFAULT_CONN_ID, + conn_type="http", + host=DEFAULT_HOST, + schema=DEFAULT_SCHEMA, + port=DEFAULT_PORT, + ) ) db.merge_conn(Connection(conn_id="default_port", conn_type="http", host="http://host")) db.merge_conn(Connection(conn_id="default_protocol", conn_type="http", host="host")) db.merge_conn(Connection(conn_id="port_set", host="host", conn_type="http", port=1234)) - db.merge_conn(Connection(conn_id="schema_set", host="host", conn_type="http", schema="zzz")) + db.merge_conn(Connection(conn_id="schema_set", host="host", conn_type="http", schema="https")) db.merge_conn( - Connection(conn_id="dont_override_schema", conn_type="http", host="http://host", schema="zzz") + Connection(conn_id="dont_override_schema", conn_type="http", host="http://host", schema="https") ) db.merge_conn(Connection(conn_id="missing_host", conn_type="http", port=1234)) db.merge_conn(Connection(conn_id="invalid_uri", uri="http://invalid_uri:4321")) @@ -54,198 +75,153 @@ def setUpClass(cls): ) ) - def test_build_get_hook(self): - - connection_url_mapping = { - # id, expected - "default_port": "http://host", - "default_protocol": "http://host", - "port_set": "http://host:1234", - "schema_set": "zzz://host", - "dont_override_schema": "http://host", - } - - for conn_id, expected in connection_url_mapping.items(): - with self.subTest(conn_id): - hook = LivyHook(livy_conn_id=conn_id) - - hook.get_conn() - assert hook.base_url == expected + @classmethod + def teardown_class(cls): + clear_db_connections(add_default_connections_back=True) + + @pytest.mark.parametrize( + "conn_id, expected", + [ + pytest.param("default_port", "http://host", id="default-port"), + pytest.param("default_protocol", "http://host", id="default-protocol"), + pytest.param("port_set", "http://host:1234", id="with-defined-port"), + pytest.param("schema_set", "https://host", id="with-defined-schema"), + pytest.param("dont_override_schema", "http://host", id="ignore-defined-schema"), + ], + ) + def test_build_get_hook(self, conn_id, expected): + hook = LivyHook(livy_conn_id=conn_id) + hook.get_conn() + assert hook.base_url == expected - @unittest.skip("inherited HttpHook does not handle missing hostname") + @pytest.mark.skip("Inherited HttpHook does not handle missing hostname") def test_missing_host(self): with pytest.raises(AirflowException): LivyHook(livy_conn_id="missing_host").get_conn() - def test_build_body(self): - with self.subTest("minimal request"): - body = LivyHook.build_post_batch_body(file="appname") - - assert body == {"file": "appname"} - - with self.subTest("complex request"): - body = LivyHook.build_post_batch_body( - file="appname", - class_name="org.example.livy", - proxy_user="proxyUser", - args=["a", "1"], - jars=["jar1", "jar2"], - files=["file1", "file2"], - py_files=["py1", "py2"], - archives=["arch1", "arch2"], - queue="queue", - name="name", - conf={"a": "b"}, - driver_cores=2, - driver_memory="1M", - executor_memory="1m", - executor_cores="1", - num_executors="10", - ) + def test_build_body_minimal_request(self): + assert LivyHook.build_post_batch_body(file="appname") == {"file": "appname"} + + def test_build_body_complex_request(self): + body = LivyHook.build_post_batch_body( + file="appname", + class_name="org.example.livy", + proxy_user="proxyUser", + args=["a", "1"], + jars=["jar1", "jar2"], + files=["file1", "file2"], + py_files=["py1", "py2"], + archives=["arch1", "arch2"], + queue="queue", + name="name", + conf={"a": "b"}, + driver_cores=2, + driver_memory="1M", + executor_memory="1m", + executor_cores="1", + num_executors="10", + ) - assert body == { - "file": "appname", - "className": "org.example.livy", - "proxyUser": "proxyUser", - "args": ["a", "1"], - "jars": ["jar1", "jar2"], - "files": ["file1", "file2"], - "pyFiles": ["py1", "py2"], - "archives": ["arch1", "arch2"], - "queue": "queue", - "name": "name", - "conf": {"a": "b"}, - "driverCores": 2, - "driverMemory": "1M", - "executorMemory": "1m", - "executorCores": "1", - "numExecutors": "10", - } + assert body == { + "file": "appname", + "className": "org.example.livy", + "proxyUser": "proxyUser", + "args": ["a", "1"], + "jars": ["jar1", "jar2"], + "files": ["file1", "file2"], + "pyFiles": ["py1", "py2"], + "archives": ["arch1", "arch2"], + "queue": "queue", + "name": "name", + "conf": {"a": "b"}, + "driverCores": 2, + "driverMemory": "1M", + "executorMemory": "1m", + "executorCores": "1", + "numExecutors": "10", + } def test_parameters_validation(self): - with self.subTest("not a size"): - with pytest.raises(ValueError): - LivyHook.build_post_batch_body(file="appname", executor_memory="xxx") - - with self.subTest("list of stringables"): - assert LivyHook.build_post_batch_body(file="appname", args=["a", 1, 0.1])["args"] == [ - "a", - "1", - "0.1", - ] - - def test_validate_size_format(self): - with self.subTest("lower 1"): - assert LivyHook._validate_size_format("1m") - - with self.subTest("lower 2"): - assert LivyHook._validate_size_format("1mb") - - with self.subTest("upper 1"): - assert LivyHook._validate_size_format("1G") - - with self.subTest("upper 2"): - assert LivyHook._validate_size_format("1GB") - - with self.subTest("snake 1"): - assert LivyHook._validate_size_format("1Gb") - - with self.subTest("fullmatch"): - with pytest.raises(ValueError): - assert LivyHook._validate_size_format("1Gb foo") - - with self.subTest("missing size"): - with pytest.raises(ValueError): - assert LivyHook._validate_size_format("10") - - with self.subTest("numeric"): - with pytest.raises(ValueError): - LivyHook._validate_size_format(1) - - with self.subTest("None"): - assert LivyHook._validate_size_format(None) - - def test_validate_list_of_stringables(self): - with self.subTest("valid list"): - try: - LivyHook._validate_list_of_stringables([1, "string"]) - except ValueError: - self.fail("Exception raised") - - with self.subTest("valid tuple"): - try: - LivyHook._validate_list_of_stringables((1, "string")) - except ValueError: - self.fail("Exception raised") - - with self.subTest("empty list"): - try: - LivyHook._validate_list_of_stringables([]) - except ValueError: - self.fail("Exception raised") - - with self.subTest("dict"): - with pytest.raises(ValueError): - LivyHook._validate_list_of_stringables({"a": "a"}) - - with self.subTest("invalid element"): - with pytest.raises(ValueError): - LivyHook._validate_list_of_stringables([1, {}]) - - with self.subTest("dict"): - with pytest.raises(ValueError): - LivyHook._validate_list_of_stringables([1, None]) - - with self.subTest("None"): - with pytest.raises(ValueError): - LivyHook._validate_list_of_stringables(None) - - with self.subTest("int"): - with pytest.raises(ValueError): - LivyHook._validate_list_of_stringables(1) - - with self.subTest("string"): - with pytest.raises(ValueError): - LivyHook._validate_list_of_stringables("string") - - def test_validate_extra_conf(self): - with self.subTest("valid"): - try: - LivyHook._validate_extra_conf({"k1": "v1", "k2": 0}) - except ValueError: - self.fail("Exception raised") - - with self.subTest("empty dict"): - try: - LivyHook._validate_extra_conf({}) - except ValueError: - self.fail("Exception raised") - - with self.subTest("none"): - try: - LivyHook._validate_extra_conf(None) - except ValueError: - self.fail("Exception raised") - - with self.subTest("not a dict 1"): - with pytest.raises(ValueError): - LivyHook._validate_extra_conf("k1=v1") - - with self.subTest("not a dict 2"): - with pytest.raises(ValueError): - LivyHook._validate_extra_conf([("k1", "v1"), ("k2", 0)]) - - with self.subTest("nested dict"): - with pytest.raises(ValueError): - LivyHook._validate_extra_conf({"outer": {"inner": "val"}}) - - with self.subTest("empty items"): - with pytest.raises(ValueError): - LivyHook._validate_extra_conf({"has_val": "val", "no_val": None}) - - with self.subTest("empty string"): - with pytest.raises(ValueError): - LivyHook._validate_extra_conf({"has_val": "val", "no_val": ""}) + with pytest.raises(ValueError): + LivyHook.build_post_batch_body(file="appname", executor_memory="xxx") + + assert LivyHook.build_post_batch_body(file="appname", args=["a", 1, 0.1])["args"] == ["a", "1", "0.1"] + + @pytest.mark.parametrize( + "size", + [ + pytest.param("1m", id="lowercase-short"), + pytest.param("1mb", id="lowercase-long"), + pytest.param("1mb", id="uppercase-short"), + pytest.param("1GB", id="uppercase-long"), + pytest.param("1Gb", id="mix-case"), + pytest.param(None, id="none"), + ], + ) + def test_validate_size_format(self, size): + assert LivyHook._validate_size_format(size) + + @pytest.mark.parametrize( + "size", + [ + pytest.param("1Gb foo", id="fullmatch"), + pytest.param("10", id="missing size"), + pytest.param(1, id="integer"), + ], + ) + def test_validate_size_format_failed(self, size): + with pytest.raises(ValueError, match=rf"Invalid java size format for string'{size}'"): + assert LivyHook._validate_size_format(size) + + @pytest.mark.parametrize( + "value", + [ + pytest.param([1, "string"], id="list"), + pytest.param((1, "string"), id="tuple"), + pytest.param([], id="empty list"), + ], + ) + def test_validate_list_of_stringables(self, value): + assert LivyHook._validate_list_of_stringables(value) + + @pytest.mark.parametrize( + "value", + [ + pytest.param({"a": "a"}, id="dict"), + pytest.param([1, {}], id="invalid element"), + pytest.param(None, id="none"), + pytest.param(42, id="integer"), + pytest.param("foo-bar", id="string"), + ], + ) + def test_validate_list_of_stringables_failed(self, value): + with pytest.raises(ValueError, match="List of strings expected"): + assert LivyHook._validate_list_of_stringables(value) + + @pytest.mark.parametrize( + "config", + [ + pytest.param({"k1": "v1", "k2": 0}, id="valid dictionary config"), + pytest.param({}, id="empty dictionary"), + pytest.param(None, id="none"), + ], + ) + def test_validate_extra_conf(self, config): + LivyHook._validate_extra_conf(config) + + @pytest.mark.parametrize( + "config", + [ + pytest.param("k1=v1", id="string"), + pytest.param([("k1", "v1"), ("k2", 0)], id="list of tuples"), + pytest.param({"outer": {"inner": "val"}}, id="nested dictionary"), + pytest.param({"has_val": "val", "no_val": None}, id="none values in dictionary"), + pytest.param({"has_val": "val", "no_val": ""}, id="empty values in dictionary"), + ], + ) + def test_validate_extra_conf_failed(self, config): + with pytest.raises(ValueError): + LivyHook._validate_extra_conf(config) @patch("airflow.providers.apache.livy.hooks.livy.LivyHook.run_method") def test_post_batch_arguments(self, mock_request): @@ -257,8 +233,7 @@ def test_post_batch_arguments(self, mock_request): "log": [], } - hook = LivyHook() - resp = hook.post_batch(file="sparkapp") + resp = LivyHook().post_batch(file="sparkapp") mock_request.assert_called_once_with( method="POST", endpoint="/batches", data=json.dumps({"file": "sparkapp"}), headers={} @@ -271,65 +246,51 @@ def test_post_batch_arguments(self, mock_request): assert isinstance(resp, int) assert resp == BATCH_ID - @requests_mock.mock() - def test_post_batch_success(self, mock): - mock.register_uri( + def test_post_batch_success(self, requests_mock): + requests_mock.register_uri( "POST", "//livy:8998/batches", json={"id": BATCH_ID, "state": BatchState.STARTING.value, "log": []}, status_code=201, ) - resp = LivyHook().post_batch(file="sparkapp") - assert isinstance(resp, int) assert resp == BATCH_ID - @requests_mock.mock() - def test_post_batch_fail(self, mock): - mock.register_uri("POST", "//livy:8998/batches", json={}, status_code=400, reason="ERROR") - - hook = LivyHook() + def test_post_batch_fail(self, requests_mock): + requests_mock.register_uri("POST", f"{MATCH_URL}/batches", json={}, status_code=400, reason="ERROR") with pytest.raises(AirflowException): - hook.post_batch(file="sparkapp") - - @requests_mock.mock() - def test_get_batch_success(self, mock): - mock.register_uri("GET", f"//livy:8998/batches/{BATCH_ID}", json={"id": BATCH_ID}, status_code=200) - - hook = LivyHook() - resp = hook.get_batch(BATCH_ID) + LivyHook().post_batch(file="sparkapp") + def test_get_batch_success(self, requests_mock): + requests_mock.register_uri( + "GET", f"{MATCH_URL}/batches/{BATCH_ID}", json={"id": BATCH_ID}, status_code=200 + ) + resp = LivyHook().get_batch(BATCH_ID) assert isinstance(resp, dict) assert "id" in resp - @requests_mock.mock() - def test_get_batch_fail(self, mock): - mock.register_uri( + def test_get_batch_fail(self, requests_mock): + requests_mock.register_uri( "GET", - f"//livy:8998/batches/{BATCH_ID}", + f"{MATCH_URL}/batches/{BATCH_ID}", json={"msg": "Unable to find batch"}, status_code=404, reason="ERROR", ) - - hook = LivyHook() with pytest.raises(AirflowException): - hook.get_batch(BATCH_ID) + LivyHook().get_batch(BATCH_ID) def test_invalid_uri(self): - hook = LivyHook(livy_conn_id="invalid_uri") with pytest.raises(RequestException): - hook.post_batch(file="sparkapp") - - @requests_mock.mock() - def test_get_batch_state_success(self, mock): + LivyHook(livy_conn_id="invalid_uri").post_batch(file="sparkapp") + def test_get_batch_state_success(self, requests_mock): running = BatchState.RUNNING - mock.register_uri( + requests_mock.register_uri( "GET", - f"//livy:8998/batches/{BATCH_ID}/state", + f"{MATCH_URL}/batches/{BATCH_ID}/state", json={"id": BATCH_ID, "state": running.value}, status_code=200, ) @@ -339,123 +300,86 @@ def test_get_batch_state_success(self, mock): assert isinstance(state, BatchState) assert state == running - @requests_mock.mock() - def test_get_batch_state_fail(self, mock): - mock.register_uri( - "GET", f"//livy:8998/batches/{BATCH_ID}/state", json={}, status_code=400, reason="ERROR" + def test_get_batch_state_fail(self, requests_mock): + requests_mock.register_uri( + "GET", f"{MATCH_URL}/batches/{BATCH_ID}/state", json={}, status_code=400, reason="ERROR" ) - - hook = LivyHook() with pytest.raises(AirflowException): - hook.get_batch_state(BATCH_ID) - - @requests_mock.mock() - def test_get_batch_state_missing(self, mock): - mock.register_uri("GET", f"//livy:8998/batches/{BATCH_ID}/state", json={}, status_code=200) + LivyHook().get_batch_state(BATCH_ID) - hook = LivyHook() + def test_get_batch_state_missing(self, requests_mock): + requests_mock.register_uri("GET", f"{MATCH_URL}/batches/{BATCH_ID}/state", json={}, status_code=200) with pytest.raises(AirflowException): - hook.get_batch_state(BATCH_ID) + LivyHook().get_batch_state(BATCH_ID) def test_parse_post_response(self): res_id = LivyHook._parse_post_response({"id": BATCH_ID, "log": []}) - assert BATCH_ID == res_id - @requests_mock.mock() - def test_delete_batch_success(self, mock): - mock.register_uri( - "DELETE", f"//livy:8998/batches/{BATCH_ID}", json={"msg": "deleted"}, status_code=200 + def test_delete_batch_success(self, requests_mock): + requests_mock.register_uri( + "DELETE", f"{MATCH_URL}/batches/{BATCH_ID}", json={"msg": "deleted"}, status_code=200 ) + assert LivyHook().delete_batch(BATCH_ID) == {"msg": "deleted"} - resp = LivyHook().delete_batch(BATCH_ID) + def test_delete_batch_fail(self, requests_mock): + requests_mock.register_uri( + "DELETE", f"{MATCH_URL}/batches/{BATCH_ID}", json={}, status_code=400, reason="ERROR" + ) + with pytest.raises(AirflowException): + LivyHook().delete_batch(BATCH_ID) - assert resp == {"msg": "deleted"} + def test_missing_batch_id(self, requests_mock): + requests_mock.register_uri("POST", f"{MATCH_URL}/batches", json={}, status_code=201) + with pytest.raises(AirflowException): + LivyHook().post_batch(file="sparkapp") - @requests_mock.mock() - def test_delete_batch_fail(self, mock): - mock.register_uri( - "DELETE", f"//livy:8998/batches/{BATCH_ID}", json={}, status_code=400, reason="ERROR" + @pytest.mark.parametrize("session_id", VALID_SESSION_ID_TEST_CASES) + def test_get_batch_validation(self, session_id, requests_mock): + requests_mock.register_uri( + "GET", f"{MATCH_URL}/batches/{session_id}", json=SAMPLE_GET_RESPONSE, status_code=200 ) + assert LivyHook().get_batch(session_id) == SAMPLE_GET_RESPONSE - hook = LivyHook() - with pytest.raises(AirflowException): - hook.delete_batch(BATCH_ID) + @pytest.mark.parametrize("session_id", INVALID_SESSION_ID_TEST_CASES) + def test_get_batch_validation_failed(self, session_id): + with pytest.raises(TypeError, match=r"\'session_id\' must be an integer"): + LivyHook().get_batch(session_id) - @requests_mock.mock() - def test_missing_batch_id(self, mock): - mock.register_uri("POST", "//livy:8998/batches", json={}, status_code=201) + @pytest.mark.parametrize("session_id", VALID_SESSION_ID_TEST_CASES) + def test_get_batch_state_validation(self, session_id, requests_mock): + requests_mock.register_uri( + "GET", f"{MATCH_URL}/batches/{session_id}/state", json=SAMPLE_GET_RESPONSE, status_code=200 + ) + assert LivyHook().get_batch_state(session_id) == BatchState.SUCCESS - hook = LivyHook() - with pytest.raises(AirflowException): - hook.post_batch(file="sparkapp") - - @requests_mock.mock() - def test_get_batch_validation(self, mock): - mock.register_uri("GET", f"//livy:8998/batches/{BATCH_ID}", json=SAMPLE_GET_RESPONSE, status_code=200) - - hook = LivyHook() - with self.subTest("get_batch"): - hook.get_batch(BATCH_ID) - - # make sure blocked by validation - for val in [None, "one", {"a": "b"}]: - with self.subTest(f"get_batch {val}"): - with pytest.raises(TypeError): - hook.get_batch(val) - - @requests_mock.mock() - def test_get_batch_state_validation(self, mock): - mock.register_uri( - "GET", f"//livy:8998/batches/{BATCH_ID}/state", json=SAMPLE_GET_RESPONSE, status_code=200 + @pytest.mark.parametrize("session_id", INVALID_SESSION_ID_TEST_CASES) + def test_get_batch_state_validation_failed(self, session_id): + with pytest.raises(TypeError, match=r"\'session_id\' must be an integer"): + LivyHook().get_batch_state(session_id) + + def test_delete_batch_validation(self, requests_mock): + requests_mock.register_uri( + "DELETE", f"{MATCH_URL}/batches/{BATCH_ID}", json={"id": BATCH_ID}, status_code=200 ) + assert LivyHook().delete_batch(BATCH_ID) == {"id": BATCH_ID} + + @pytest.mark.parametrize("session_id", INVALID_SESSION_ID_TEST_CASES) + def test_delete_batch_validation_failed(self, session_id): + with pytest.raises(TypeError, match=r"\'session_id\' must be an integer"): + LivyHook().delete_batch(session_id) + + @pytest.mark.parametrize("session_id", VALID_SESSION_ID_TEST_CASES) + def test_check_session_id(self, session_id): + LivyHook._validate_session_id(session_id) # Should not raise any error + + @pytest.mark.parametrize("session_id", INVALID_SESSION_ID_TEST_CASES) + def test_check_session_id_failed(self, session_id): + with pytest.raises(TypeError, match=r"\'session_id\' must be an integer"): + LivyHook._validate_session_id("asd") - hook = LivyHook() - with self.subTest("get_batch"): - hook.get_batch_state(BATCH_ID) - - for val in [None, "one", {"a": "b"}]: - with self.subTest(f"get_batch {val}"): - with pytest.raises(TypeError): - hook.get_batch_state(val) - - @requests_mock.mock() - def test_delete_batch_validation(self, mock): - mock.register_uri("DELETE", f"//livy:8998/batches/{BATCH_ID}", json={"id": BATCH_ID}, status_code=200) - - hook = LivyHook() - with self.subTest("get_batch"): - hook.delete_batch(BATCH_ID) - - for val in [None, "one", {"a": "b"}]: - with self.subTest(f"get_batch {val}"): - with pytest.raises(TypeError): - hook.delete_batch(val) - - def test_check_session_id(self): - with self.subTest("valid 00"): - try: - LivyHook._validate_session_id(100) - except TypeError: - self.fail("") - - with self.subTest("valid 01"): - try: - LivyHook._validate_session_id(0) - except TypeError: - self.fail("") - - with self.subTest("None"): - with pytest.raises(TypeError): - LivyHook._validate_session_id(None) - - with self.subTest("random string"): - with pytest.raises(TypeError): - LivyHook._validate_session_id("asd") - - @requests_mock.mock() - def test_extra_headers(self, mock): - mock.register_uri( + def test_extra_headers(self, requests_mock): + requests_mock.register_uri( "POST", "//livy:8998/batches", json={"id": BATCH_ID, "state": BatchState.STARTING.value, "log": []}, diff --git a/tests/providers/apache/livy/operators/test_livy.py b/tests/providers/apache/livy/operators/test_livy.py index d49cba1c9cf41..eb305e1c2d7de 100644 --- a/tests/providers/apache/livy/operators/test_livy.py +++ b/tests/providers/apache/livy/operators/test_livy.py @@ -17,7 +17,6 @@ from __future__ import annotations import logging -import unittest from unittest.mock import MagicMock, patch import pytest @@ -36,8 +35,8 @@ LOG_RESPONSE = {"total": 3, "log": ["first_line", "second_line", "third_line"]} -class TestLivyOperator(unittest.TestCase): - def setUp(self): +class TestLivyOperator: + def setup_method(self): args = {"owner": "airflow", "start_date": DEFAULT_DATE} self.dag = DAG("test_dag_id", default_args=args) db.merge_conn( @@ -163,7 +162,7 @@ def test_injected_hook(self): @patch("airflow.providers.apache.livy.operators.livy.LivyHook.get_batch_logs", return_value=LOG_RESPONSE) @patch("airflow.providers.apache.livy.operators.livy.LivyHook.post_batch", return_value=BATCH_ID) @patch("airflow.providers.apache.livy.operators.livy.LivyHook.get_batch", return_value=GET_BATCH) - def test_log_dump(self, mock_get_batch, mock_post, mock_get_logs, mock_get): + def test_log_dump(self, mock_get_batch, mock_post, mock_get_logs, mock_get, caplog): task = LivyOperator( livy_conn_id="livyunittest", file="sparkapp", @@ -171,10 +170,13 @@ def test_log_dump(self, mock_get_batch, mock_post, mock_get_logs, mock_get): task_id="livy_example", polling_interval=1, ) - with self.assertLogs(task.get_hook().log, level=logging.INFO) as cm: + caplog.clear() + with caplog.at_level(level=logging.INFO, logger=task.get_hook().log.name): task.execute(context=self.mock_context) - assert "INFO:airflow.providers.apache.livy.hooks.livy.LivyHook:first_line" in cm.output - assert "INFO:airflow.providers.apache.livy.hooks.livy.LivyHook:second_line" in cm.output - assert "INFO:airflow.providers.apache.livy.hooks.livy.LivyHook:third_line" in cm.output + + assert "first_line" in caplog.messages + assert "second_line" in caplog.messages + assert "third_line" in caplog.messages + mock_get.assert_called_once_with(BATCH_ID, retry_args=None) mock_get_logs.assert_called_once_with(BATCH_ID, 0, 100) diff --git a/tests/providers/apache/livy/sensors/test_livy.py b/tests/providers/apache/livy/sensors/test_livy.py index c51a39776384d..d25f72ccd5c43 100644 --- a/tests/providers/apache/livy/sensors/test_livy.py +++ b/tests/providers/apache/livy/sensors/test_livy.py @@ -16,9 +16,10 @@ # under the License. from __future__ import annotations -import unittest from unittest.mock import patch +import pytest + from airflow.models import Connection from airflow.models.dag import DAG from airflow.providers.apache.livy.hooks.livy import BatchState, LivyHook @@ -28,19 +29,29 @@ DEFAULT_DATE = timezone.datetime(2017, 1, 1) -class TestLivySensor(unittest.TestCase): - def setUp(self): +class TestLivySensor: + def setup_method(self): args = {"owner": "airflow", "start_date": DEFAULT_DATE} self.dag = DAG("test_dag_id", default_args=args) db.merge_conn(Connection(conn_id="livyunittest", conn_type="livy", host="http://localhost:8998")) - @patch("airflow.providers.apache.livy.hooks.livy.LivyHook.get_batch_state") - def test_poke(self, mock_state): + @pytest.mark.parametrize( + "batch_state", [pytest.param(bs, id=bs.name) for bs in BatchState if bs in LivyHook.TERMINAL_STATES] + ) + def test_poke_on_terminal_state(self, batch_state): sensor = LivySensor( livy_conn_id="livyunittest", task_id="livy_sensor_test", dag=self.dag, batch_id=100 ) + with patch.object(LivyHook, "get_batch_state", return_value=batch_state): + assert sensor.poke({}) - for state in BatchState: - with self.subTest(state.value): - mock_state.return_value = state - assert sensor.poke({}) == (state in LivyHook.TERMINAL_STATES) + @pytest.mark.parametrize( + "batch_state", + [pytest.param(bs, id=bs.name) for bs in BatchState if bs not in LivyHook.TERMINAL_STATES], + ) + def test_poke_on_non_terminal_state(self, batch_state): + sensor = LivySensor( + livy_conn_id="livyunittest", task_id="livy_sensor_test", dag=self.dag, batch_id=100 + ) + with patch.object(LivyHook, "get_batch_state", return_value=batch_state): + assert not sensor.poke({}) diff --git a/tests/providers/apache/pig/hooks/test_pig.py b/tests/providers/apache/pig/hooks/test_pig.py index 7599014e12fae..1399c14a1ee83 100644 --- a/tests/providers/apache/pig/hooks/test_pig.py +++ b/tests/providers/apache/pig/hooks/test_pig.py @@ -17,7 +17,6 @@ # under the License. from __future__ import annotations -import unittest from unittest import mock import pytest @@ -25,10 +24,8 @@ from airflow.providers.apache.pig.hooks.pig import PigCliHook -class TestPigCliHook(unittest.TestCase): - def setUp(self): - super().setUp() - +class TestPigCliHook: + def setup_method(self): self.extra_dejson = mock.MagicMock() self.extra_dejson.get.return_value = None self.conn = mock.MagicMock() diff --git a/tests/providers/apache/pig/operators/test_pig.py b/tests/providers/apache/pig/operators/test_pig.py index f9d248766e87f..1b7e5d8c9f91b 100644 --- a/tests/providers/apache/pig/operators/test_pig.py +++ b/tests/providers/apache/pig/operators/test_pig.py @@ -16,7 +16,6 @@ # under the License. from __future__ import annotations -import unittest from unittest import mock from airflow.providers.apache.pig.hooks.pig import PigCliHook @@ -27,7 +26,7 @@ PIG = "ls /;" -class TestPigOperator(unittest.TestCase): +class TestPigOperator: def test_prepare_template(self): pig = "sh echo $DATE;" task_id = TEST_TASK_ID diff --git a/tests/providers/apache/pinot/hooks/test_pinot.py b/tests/providers/apache/pinot/hooks/test_pinot.py index a9b25a0480f17..0ca2dfb1c581a 100644 --- a/tests/providers/apache/pinot/hooks/test_pinot.py +++ b/tests/providers/apache/pinot/hooks/test_pinot.py @@ -20,7 +20,6 @@ import io import os import subprocess -import unittest from unittest import mock import pytest @@ -29,9 +28,8 @@ from airflow.providers.apache.pinot.hooks.pinot import PinotAdminHook, PinotDbApiHook -class TestPinotAdminHook(unittest.TestCase): - def setUp(self): - super().setUp() +class TestPinotAdminHook: + def setup_method(self): self.conn = conn = mock.MagicMock() self.conn.host = "host" self.conn.port = "1000" @@ -213,9 +211,8 @@ def test_exception_when_keeping_cmd_path(self): PinotAdminHook(cmd_path="pinot-admin.sh") -class TestPinotDbApiHook(unittest.TestCase): - def setUp(self): - super().setUp() +class TestPinotDbApiHook: + def setup_method(self): self.conn = conn = mock.MagicMock() self.conn.host = "host" self.conn.port = "1000" @@ -276,8 +273,8 @@ def test_get_pandas_df(self): assert item[0] == df.values.tolist()[i][0] -class TestPinotDbApiHookIntegration(unittest.TestCase): - @pytest.mark.integration("pinot") +@pytest.mark.integration("pinot") +class TestPinotDbApiHookIntegration: @mock.patch.dict("os.environ", AIRFLOW_CONN_PINOT_BROKER_DEFAULT="pinot://pinot:8000/") def test_should_return_records(self): hook = PinotDbApiHook() diff --git a/tests/providers/apache/spark/hooks/test_spark_jdbc.py b/tests/providers/apache/spark/hooks/test_spark_jdbc.py index 8daa4ecec656e..c9b8245b954ff 100644 --- a/tests/providers/apache/spark/hooks/test_spark_jdbc.py +++ b/tests/providers/apache/spark/hooks/test_spark_jdbc.py @@ -17,14 +17,12 @@ # under the License. from __future__ import annotations -import unittest - from airflow.models import Connection from airflow.providers.apache.spark.hooks.spark_jdbc import SparkJDBCHook from airflow.utils import db -class TestSparkJDBCHook(unittest.TestCase): +class TestSparkJDBCHook: _config = { "cmd_type": "spark_to_jdbc", @@ -63,7 +61,7 @@ class TestSparkJDBCHook(unittest.TestCase): "comments VARCHAR(1024)", } - def setUp(self): + def setup_method(self): db.merge_conn( Connection( conn_id="spark-default", diff --git a/tests/providers/apache/spark/hooks/test_spark_sql.py b/tests/providers/apache/spark/hooks/test_spark_sql.py index e3059d82917fe..1666c51946e6b 100644 --- a/tests/providers/apache/spark/hooks/test_spark_sql.py +++ b/tests/providers/apache/spark/hooks/test_spark_sql.py @@ -18,7 +18,6 @@ from __future__ import annotations import io -import unittest from itertools import dropwhile from unittest.mock import call, patch @@ -38,7 +37,7 @@ def get_after(sentinel, iterable): return next(truncated) -class TestSparkSqlHook(unittest.TestCase): +class TestSparkSqlHook: _config = { "conn_id": "spark_default", "executor_cores": 4, @@ -52,12 +51,12 @@ class TestSparkSqlHook(unittest.TestCase): } @classmethod - def setUpClass(cls) -> None: + def setup_class(cls) -> None: clear_db_connections(add_default_connections_back=False) db.merge_conn(Connection(conn_id="spark_default", conn_type="spark", host="yarn://yarn-master")) @classmethod - def tearDownClass(cls) -> None: + def teardown_class(cls) -> None: clear_db_connections(add_default_connections_back=True) def test_build_command(self): diff --git a/tests/providers/apache/spark/hooks/test_spark_submit.py b/tests/providers/apache/spark/hooks/test_spark_submit.py index f0736c84907e2..34c441edf1b0e 100644 --- a/tests/providers/apache/spark/hooks/test_spark_submit.py +++ b/tests/providers/apache/spark/hooks/test_spark_submit.py @@ -19,11 +19,9 @@ import io import os -import unittest from unittest.mock import call, patch import pytest -from parameterized import parameterized from airflow.exceptions import AirflowException from airflow.models import Connection @@ -31,7 +29,7 @@ from airflow.utils import db -class TestSparkSubmitHook(unittest.TestCase): +class TestSparkSubmitHook: _spark_job_file = "test_application.py" _config = { @@ -75,7 +73,7 @@ def cmd_args_to_dict(list_cmd): return_dict[arg] = list_cmd[pos + 1] return return_dict - def setUp(self): + def setup_method(self): db.merge_conn( Connection( conn_id="spark_yarn_cluster", @@ -795,8 +793,9 @@ def test_k8s_process_on_kill(self, mock_popen, mock_client_method): "spark-pi-edf2ace37be7353a958b38733a12f8e6-driver", "mynamespace", **kwargs ) - @parameterized.expand( - ( + @pytest.mark.parametrize( + "command, expected", + [ ( ("spark-submit", "foo", "--bar", "baz", "--password='secret'", "--foo", "bar"), "spark-submit foo --bar baz --password='******' --foo bar", @@ -829,7 +828,7 @@ def test_k8s_process_on_kill(self, mock_popen, mock_client_method): ("spark-submit",), "spark-submit", ), - ) + ], ) def test_masks_passwords(self, command: str, expected: str) -> None: # Given diff --git a/tests/providers/apache/spark/operators/test_spark_jdbc.py b/tests/providers/apache/spark/operators/test_spark_jdbc.py index f417d7c259cb9..9060fd63b72bb 100644 --- a/tests/providers/apache/spark/operators/test_spark_jdbc.py +++ b/tests/providers/apache/spark/operators/test_spark_jdbc.py @@ -17,8 +17,6 @@ # under the License. from __future__ import annotations -import unittest - from airflow.models.dag import DAG from airflow.providers.apache.spark.operators.spark_jdbc import SparkJDBCOperator from airflow.utils import timezone @@ -26,7 +24,7 @@ DEFAULT_DATE = timezone.datetime(2017, 1, 1) -class TestSparkJDBCOperator(unittest.TestCase): +class TestSparkJDBCOperator: _config = { "spark_app_name": "{{ task_instance.task_id }}", "spark_conf": {"parquet.compression": "SNAPPY"}, @@ -57,7 +55,7 @@ class TestSparkJDBCOperator(unittest.TestCase): "comments VARCHAR(1024)", } - def setUp(self): + def setup_method(self): args = {"owner": "airflow", "start_date": DEFAULT_DATE} self.dag = DAG("test_dag_id", default_args=args) diff --git a/tests/providers/apache/spark/operators/test_spark_sql.py b/tests/providers/apache/spark/operators/test_spark_sql.py index 396adcd24ce8c..8bfa4b1ed6ba7 100644 --- a/tests/providers/apache/spark/operators/test_spark_sql.py +++ b/tests/providers/apache/spark/operators/test_spark_sql.py @@ -18,7 +18,6 @@ from __future__ import annotations import datetime -import unittest from airflow.models.dag import DAG from airflow.providers.apache.spark.operators.spark_sql import SparkSqlOperator @@ -26,7 +25,7 @@ DEFAULT_DATE = datetime.datetime(2017, 1, 1) -class TestSparkSqlOperator(unittest.TestCase): +class TestSparkSqlOperator: _config = { "sql": "SELECT 22", "conn_id": "spark_special_conn_id", @@ -42,7 +41,7 @@ class TestSparkSqlOperator(unittest.TestCase): "yarn_queue": "special-queue", } - def setUp(self): + def setup_method(self): args = {"owner": "airflow", "start_date": DEFAULT_DATE} self.dag = DAG("test_dag_id", default_args=args) diff --git a/tests/providers/apache/spark/operators/test_spark_submit.py b/tests/providers/apache/spark/operators/test_spark_submit.py index c25f29c2cf9ee..4158b2fa71e9f 100644 --- a/tests/providers/apache/spark/operators/test_spark_submit.py +++ b/tests/providers/apache/spark/operators/test_spark_submit.py @@ -17,7 +17,6 @@ # under the License. from __future__ import annotations -import unittest from datetime import timedelta from airflow.models import DagRun, TaskInstance @@ -28,7 +27,7 @@ DEFAULT_DATE = timezone.datetime(2017, 1, 1) -class TestSparkSubmitOperator(unittest.TestCase): +class TestSparkSubmitOperator: _config = { "conf": {"parquet.compression": "SNAPPY"}, @@ -67,7 +66,7 @@ class TestSparkSubmitOperator(unittest.TestCase): ], } - def setUp(self): + def setup_method(self): args = {"owner": "airflow", "start_date": DEFAULT_DATE} self.dag = DAG("test_dag_id", default_args=args) diff --git a/tests/providers/apache/sqoop/hooks/test_sqoop.py b/tests/providers/apache/sqoop/hooks/test_sqoop.py index 89a34b12bde62..bf58533af52e6 100644 --- a/tests/providers/apache/sqoop/hooks/test_sqoop.py +++ b/tests/providers/apache/sqoop/hooks/test_sqoop.py @@ -19,7 +19,6 @@ import collections import json -import unittest from io import StringIO from unittest import mock from unittest.mock import call, patch @@ -32,7 +31,7 @@ from airflow.utils import db -class TestSqoopHook(unittest.TestCase): +class TestSqoopHook: _config = { "conn_id": "sqoop_test", "num_mappers": 22, @@ -82,7 +81,7 @@ class TestSqoopHook(unittest.TestCase): "archives": "/path/to/archives", } - def setUp(self): + def setup_method(self): db.merge_conn( Connection( conn_id="sqoop_test", diff --git a/tests/providers/apache/sqoop/operators/test_sqoop.py b/tests/providers/apache/sqoop/operators/test_sqoop.py index 90e6ae1a876ea..883422bf20b2a 100644 --- a/tests/providers/apache/sqoop/operators/test_sqoop.py +++ b/tests/providers/apache/sqoop/operators/test_sqoop.py @@ -18,7 +18,6 @@ from __future__ import annotations import datetime -import unittest import pytest @@ -27,7 +26,7 @@ from airflow.providers.apache.sqoop.operators.sqoop import SqoopOperator -class TestSqoopOperator(unittest.TestCase): +class TestSqoopOperator: _config = { "conn_id": "sqoop_default", "cmd_type": "export", @@ -62,7 +61,7 @@ class TestSqoopOperator(unittest.TestCase): "schema": "myschema", } - def setUp(self): + def setup_method(self): args = {"owner": "airflow", "start_date": datetime.datetime(2017, 1, 1)} self.dag = DAG("test_dag_id", default_args=args) From d88aa5d3505a0c5c6d21faac8a552e7b76b7081a Mon Sep 17 00:00:00 2001 From: Jarek Potiuk Date: Tue, 6 Dec 2022 22:46:11 +0100 Subject: [PATCH 154/158] Move Integration tests to separate package under tests (#28170) This is the first stage of improving the way how integration tests are run in our CI - first we want to separate them in a separate packages, and then we want to run them separately - one integration each in the CI. (cherry picked from commit df608fe63f868992afaec5cb6e527405a12dbfe2) --- .pre-commit-config.yaml | 1 + Dockerfile.ci | 7 +- .../commands/testing_commands.py | 2 +- scripts/docker/entrypoint_ci.sh | 7 +- tests/cli/commands/test_celery_command.py | 31 -- tests/executors/test_celery_executor.py | 259 +------------ .../hooks => integration}/__init__.py | 0 tests/integration/api/__init__.py | 16 + tests/integration/api/auth/__init__.py | 16 + .../integration/api/auth/backend/__init__.py | 16 + .../api/auth/backend/test_kerberos_auth.py | 0 tests/integration/cli/__init__.py | 16 + tests/integration/cli/commands/__init__.py | 16 + .../cli/commands/test_celery_command.py | 57 +++ tests/integration/executors/__init__.py | 16 + .../executors/test_celery_executor.py | 339 ++++++++++++++++++ tests/integration/providers/__init__.py | 16 + .../integration/providers/apache/__init__.py | 16 + .../providers/apache/cassandra/__init__.py | 16 + .../apache/cassandra/hooks/__init__.py | 16 + .../apache/cassandra/hooks/test_cassandra.py | 0 .../providers/apache/pinot/__init__.py | 16 + .../providers/apache/pinot/hooks/__init__.py | 16 + .../apache/pinot/hooks/test_pinot.py | 34 ++ .../integration/providers/google/__init__.py | 16 + .../providers/google/cloud/__init__.py | 16 + .../google/cloud/transfers/__init__.py | 16 + .../cloud/transfers/test_presto_to_gcs.py | 0 .../transfers/test_presto_to_gcs_system.py | 0 .../cloud/transfers/test_trino_to_gcs.py | 0 tests/integration/providers/mongo/__init__.py | 16 + .../providers/mongo/sensors/__init__.py | 16 + .../providers/mongo/sensors/test_mongo.py | 0 tests/integration/providers/redis/__init__.py | 16 + .../providers/redis/hooks/__init__.py | 16 + .../providers/redis/hooks/test_redis.py | 39 ++ .../providers/redis/operators/__init__.py | 16 + .../redis/operators/test_redis_publish.py | 0 .../providers/redis/sensors/__init__.py | 16 + .../providers/redis/sensors/test_redis_key.py | 0 .../redis/sensors/test_redis_pub_sub.py | 80 +++++ tests/integration/providers/trino/__init__.py | 16 + .../providers/trino/hooks/__init__.py | 16 + .../providers/trino/hooks/test_trino.py | 48 +++ tests/integration/security/__init__.py | 16 + tests/integration/security/test_kerberos.py | 75 ++++ .../apache/pinot/hooks/test_pinot.py | 10 - tests/providers/redis/hooks/test_redis.py | 18 - .../redis/sensors/test_redis_pub_sub.py | 46 --- tests/providers/trino/hooks/test_trino.py | 24 -- tests/security/test_kerberos.py | 48 --- 51 files changed, 1069 insertions(+), 440 deletions(-) rename tests/{providers/apache/cassandra/hooks => integration}/__init__.py (100%) create mode 100644 tests/integration/api/__init__.py create mode 100644 tests/integration/api/auth/__init__.py create mode 100644 tests/integration/api/auth/backend/__init__.py rename tests/{ => integration}/api/auth/backend/test_kerberos_auth.py (100%) create mode 100644 tests/integration/cli/__init__.py create mode 100644 tests/integration/cli/commands/__init__.py create mode 100644 tests/integration/cli/commands/test_celery_command.py create mode 100644 tests/integration/executors/__init__.py create mode 100644 tests/integration/executors/test_celery_executor.py create mode 100644 tests/integration/providers/__init__.py create mode 100644 tests/integration/providers/apache/__init__.py create mode 100644 tests/integration/providers/apache/cassandra/__init__.py create mode 100644 tests/integration/providers/apache/cassandra/hooks/__init__.py rename tests/{ => integration}/providers/apache/cassandra/hooks/test_cassandra.py (100%) create mode 100644 tests/integration/providers/apache/pinot/__init__.py create mode 100644 tests/integration/providers/apache/pinot/hooks/__init__.py create mode 100644 tests/integration/providers/apache/pinot/hooks/test_pinot.py create mode 100644 tests/integration/providers/google/__init__.py create mode 100644 tests/integration/providers/google/cloud/__init__.py create mode 100644 tests/integration/providers/google/cloud/transfers/__init__.py rename tests/{ => integration}/providers/google/cloud/transfers/test_presto_to_gcs.py (100%) rename tests/{ => integration}/providers/google/cloud/transfers/test_presto_to_gcs_system.py (100%) rename tests/{ => integration}/providers/google/cloud/transfers/test_trino_to_gcs.py (100%) create mode 100644 tests/integration/providers/mongo/__init__.py create mode 100644 tests/integration/providers/mongo/sensors/__init__.py rename tests/{ => integration}/providers/mongo/sensors/test_mongo.py (100%) create mode 100644 tests/integration/providers/redis/__init__.py create mode 100644 tests/integration/providers/redis/hooks/__init__.py create mode 100644 tests/integration/providers/redis/hooks/test_redis.py create mode 100644 tests/integration/providers/redis/operators/__init__.py rename tests/{ => integration}/providers/redis/operators/test_redis_publish.py (100%) create mode 100644 tests/integration/providers/redis/sensors/__init__.py rename tests/{ => integration}/providers/redis/sensors/test_redis_key.py (100%) create mode 100644 tests/integration/providers/redis/sensors/test_redis_pub_sub.py create mode 100644 tests/integration/providers/trino/__init__.py create mode 100644 tests/integration/providers/trino/hooks/__init__.py create mode 100644 tests/integration/providers/trino/hooks/test_trino.py create mode 100644 tests/integration/security/__init__.py create mode 100644 tests/integration/security/test_kerberos.py diff --git a/.pre-commit-config.yaml b/.pre-commit-config.yaml index ed082b1944e37..248029bfe582c 100644 --- a/.pre-commit-config.yaml +++ b/.pre-commit-config.yaml @@ -459,6 +459,7 @@ repos: ^airflow/www/static/| ^airflow/providers/| ^tests/providers/apache/cassandra/hooks/test_cassandra.py$| + ^tests/integration/providers/apache/cassandra/hooks/test_cassandra.py$| ^tests/system/providers/apache/spark/example_spark_dag.py$| ^docs/apache-airflow-providers-apache-cassandra/connections/cassandra.rst$| ^docs/apache-airflow-providers-apache-hive/commits.rst$| diff --git a/Dockerfile.ci b/Dockerfile.ci index 96172445fe181..e645b332426ca 100644 --- a/Dockerfile.ci +++ b/Dockerfile.ci @@ -913,11 +913,13 @@ else ) WWW_TESTS=("tests/www") HELM_CHART_TESTS=("tests/charts") + INTEGRATION_TESTS=("tests/integration") ALL_TESTS=("tests") ALL_PRESELECTED_TESTS=( "${CLI_TESTS[@]}" "${API_TESTS[@]}" "${HELM_CHART_TESTS[@]}" + "${INTEGRATION_TESTS[@]}" "${PROVIDERS_TESTS[@]}" "${CORE_TESTS[@]}" "${ALWAYS_TESTS[@]}" @@ -938,14 +940,15 @@ else SELECTED_TESTS=("${WWW_TESTS[@]}") elif [[ ${TEST_TYPE:=""} == "Helm" ]]; then SELECTED_TESTS=("${HELM_CHART_TESTS[@]}") + elif [[ ${TEST_TYPE:=""} == "Integration" ]]; then + SELECTED_TESTS=("${INTEGRATION_TESTS[@]}") elif [[ ${TEST_TYPE:=""} == "Other" ]]; then find_all_other_tests SELECTED_TESTS=("${ALL_OTHER_TESTS[@]}") elif [[ ${TEST_TYPE:=""} == "All" || ${TEST_TYPE} == "Quarantined" || \ ${TEST_TYPE} == "Always" || \ ${TEST_TYPE} == "Postgres" || ${TEST_TYPE} == "MySQL" || \ - ${TEST_TYPE} == "Long" || \ - ${TEST_TYPE} == "Integration" ]]; then + ${TEST_TYPE} == "Long" ]]; then SELECTED_TESTS=("${ALL_TESTS[@]}") elif [[ ${TEST_TYPE} =~ Providers\[(.*)\] ]]; then SELECTED_TESTS=() diff --git a/dev/breeze/src/airflow_breeze/commands/testing_commands.py b/dev/breeze/src/airflow_breeze/commands/testing_commands.py index 58d0b509a917f..d3f80fd21b0dc 100644 --- a/dev/breeze/src/airflow_breeze/commands/testing_commands.py +++ b/dev/breeze/src/airflow_breeze/commands/testing_commands.py @@ -226,7 +226,7 @@ def _run_tests_in_pool( progress_matcher=GenericRegexpProgressMatcher( regexp=TEST_PROGRESS_REGEXP, regexp_for_joined_line=PERCENT_TEST_PROGRESS_REGEXP, - lines_to_search=40, + lines_to_search=200, ), ) as (pool, outputs): results = [ diff --git a/scripts/docker/entrypoint_ci.sh b/scripts/docker/entrypoint_ci.sh index 0301cf4538416..298420f2c9a43 100755 --- a/scripts/docker/entrypoint_ci.sh +++ b/scripts/docker/entrypoint_ci.sh @@ -365,11 +365,13 @@ else ) WWW_TESTS=("tests/www") HELM_CHART_TESTS=("tests/charts") + INTEGRATION_TESTS=("tests/integration") ALL_TESTS=("tests") ALL_PRESELECTED_TESTS=( "${CLI_TESTS[@]}" "${API_TESTS[@]}" "${HELM_CHART_TESTS[@]}" + "${INTEGRATION_TESTS[@]}" "${PROVIDERS_TESTS[@]}" "${CORE_TESTS[@]}" "${ALWAYS_TESTS[@]}" @@ -390,14 +392,15 @@ else SELECTED_TESTS=("${WWW_TESTS[@]}") elif [[ ${TEST_TYPE:=""} == "Helm" ]]; then SELECTED_TESTS=("${HELM_CHART_TESTS[@]}") + elif [[ ${TEST_TYPE:=""} == "Integration" ]]; then + SELECTED_TESTS=("${INTEGRATION_TESTS[@]}") elif [[ ${TEST_TYPE:=""} == "Other" ]]; then find_all_other_tests SELECTED_TESTS=("${ALL_OTHER_TESTS[@]}") elif [[ ${TEST_TYPE:=""} == "All" || ${TEST_TYPE} == "Quarantined" || \ ${TEST_TYPE} == "Always" || \ ${TEST_TYPE} == "Postgres" || ${TEST_TYPE} == "MySQL" || \ - ${TEST_TYPE} == "Long" || \ - ${TEST_TYPE} == "Integration" ]]; then + ${TEST_TYPE} == "Long" ]]; then SELECTED_TESTS=("${ALL_TESTS[@]}") elif [[ ${TEST_TYPE} =~ Providers\[(.*)\] ]]; then SELECTED_TESTS=() diff --git a/tests/cli/commands/test_celery_command.py b/tests/cli/commands/test_celery_command.py index 9271e1b5812e4..9acea81c60193 100644 --- a/tests/cli/commands/test_celery_command.py +++ b/tests/cli/commands/test_celery_command.py @@ -61,37 +61,6 @@ def test_validate_session_dbapi_exception(self, mock_session): assert airflow.settings.validate_session() is False -@pytest.mark.integration("redis") -@pytest.mark.integration("rabbitmq") -@pytest.mark.backend("mysql", "postgres") -class TestWorkerServeLogs: - @classmethod - def setup_class(cls): - cls.parser = cli_parser.get_parser() - - @mock.patch("airflow.cli.commands.celery_command.celery_app") - @conf_vars({("core", "executor"): "CeleryExecutor"}) - def test_serve_logs_on_worker_start(self, mock_celery_app): - with mock.patch("airflow.cli.commands.celery_command.Process") as mock_process: - args = self.parser.parse_args(["celery", "worker", "--concurrency", "1"]) - - with mock.patch("celery.platforms.check_privileges") as mock_privil: - mock_privil.return_value = 0 - celery_command.worker(args) - mock_process.assert_called() - - @mock.patch("airflow.cli.commands.celery_command.celery_app") - @conf_vars({("core", "executor"): "CeleryExecutor"}) - def test_skip_serve_logs_on_worker_start(self, mock_celery_app): - with mock.patch("airflow.cli.commands.celery_command.Process") as mock_popen: - args = self.parser.parse_args(["celery", "worker", "--concurrency", "1", "--skip-serve-logs"]) - - with mock.patch("celery.platforms.check_privileges") as mock_privil: - mock_privil.return_value = 0 - celery_command.worker(args) - mock_popen.assert_not_called() - - @pytest.mark.backend("mysql", "postgres") class TestCeleryStopCommand: @classmethod diff --git a/tests/executors/test_celery_executor.py b/tests/executors/test_celery_executor.py index 6ab99d0706bf9..cbbe64c5649c6 100644 --- a/tests/executors/test_celery_executor.py +++ b/tests/executors/test_celery_executor.py @@ -18,12 +18,9 @@ from __future__ import annotations import contextlib -import json -import logging import os import signal import sys -import unittest from datetime import datetime, timedelta from unittest import mock @@ -31,22 +28,16 @@ import celery.contrib.testing.tasks # noqa: F401 import pytest from celery import Celery -from celery.backends.base import BaseBackend, BaseKeyValueStoreBackend -from celery.backends.database import DatabaseBackend -from celery.contrib.testing.worker import start_worker from celery.result import AsyncResult from freezegun import freeze_time from kombu.asynchronous import set_event_loop from parameterized import parameterized from airflow.configuration import conf -from airflow.exceptions import AirflowException, AirflowTaskTimeout from airflow.executors import celery_executor -from airflow.executors.celery_executor import BulkStateFetcher from airflow.models.baseoperator import BaseOperator from airflow.models.dag import DAG -from airflow.models.taskinstance import SimpleTaskInstance, TaskInstance, TaskInstanceKey -from airflow.operators.bash import BashOperator +from airflow.models.taskinstance import TaskInstance, TaskInstanceKey from airflow.utils import timezone from airflow.utils.state import State from tests.test_utils import db @@ -107,149 +98,6 @@ def teardown_method(self) -> None: db.clear_db_runs() db.clear_db_jobs() - @parameterized.expand(_prepare_test_bodies()) - @pytest.mark.integration("redis") - @pytest.mark.integration("rabbitmq") - @pytest.mark.backend("mysql", "postgres") - def test_celery_integration(self, broker_url): - success_command = ["airflow", "tasks", "run", "true", "some_parameter"] - fail_command = ["airflow", "version"] - - def fake_execute_command(command): - if command != success_command: - raise AirflowException("fail") - - with _prepare_app(broker_url, execute=fake_execute_command) as app: - executor = celery_executor.CeleryExecutor() - assert executor.tasks == {} - executor.start() - - with start_worker(app=app, logfile=sys.stdout, loglevel="info"): - execute_date = datetime.now() - - task_tuples_to_send = [ - ( - ("success", "fake_simple_ti", execute_date, 0), - success_command, - celery_executor.celery_configuration["task_default_queue"], - celery_executor.execute_command, - ), - ( - ("fail", "fake_simple_ti", execute_date, 0), - fail_command, - celery_executor.celery_configuration["task_default_queue"], - celery_executor.execute_command, - ), - ] - - # "Enqueue" them. We don't have a real SimpleTaskInstance, so directly edit the dict - for (key, command, queue, task) in task_tuples_to_send: - executor.queued_tasks[key] = (command, 1, queue, None) - executor.task_publish_retries[key] = 1 - - executor._process_tasks(task_tuples_to_send) - - assert list(executor.tasks.keys()) == [ - ("success", "fake_simple_ti", execute_date, 0), - ("fail", "fake_simple_ti", execute_date, 0), - ] - assert ( - executor.event_buffer[("success", "fake_simple_ti", execute_date, 0)][0] == State.QUEUED - ) - assert executor.event_buffer[("fail", "fake_simple_ti", execute_date, 0)][0] == State.QUEUED - - executor.end(synchronous=True) - - assert executor.event_buffer[("success", "fake_simple_ti", execute_date, 0)][0] == State.SUCCESS - assert executor.event_buffer[("fail", "fake_simple_ti", execute_date, 0)][0] == State.FAILED - - assert "success" not in executor.tasks - assert "fail" not in executor.tasks - - assert executor.queued_tasks == {} - assert timedelta(0, 600) == executor.task_adoption_timeout - - @pytest.mark.integration("redis") - @pytest.mark.integration("rabbitmq") - @pytest.mark.backend("mysql", "postgres") - def test_error_sending_task(self): - def fake_execute_command(): - pass - - with _prepare_app(execute=fake_execute_command): - # fake_execute_command takes no arguments while execute_command takes 1, - # which will cause TypeError when calling task.apply_async() - executor = celery_executor.CeleryExecutor() - task = BashOperator( - task_id="test", bash_command="true", dag=DAG(dag_id="id"), start_date=datetime.now() - ) - when = datetime.now() - value_tuple = ( - "command", - 1, - None, - SimpleTaskInstance.from_ti(ti=TaskInstance(task=task, run_id=None)), - ) - key = ("fail", "fake_simple_ti", when, 0) - executor.queued_tasks[key] = value_tuple - executor.task_publish_retries[key] = 1 - executor.heartbeat() - assert 0 == len(executor.queued_tasks), "Task should no longer be queued" - assert executor.event_buffer[("fail", "fake_simple_ti", when, 0)][0] == State.FAILED - - @pytest.mark.integration("redis") - @pytest.mark.integration("rabbitmq") - @pytest.mark.backend("mysql", "postgres") - def test_retry_on_error_sending_task(self, caplog): - """Test that Airflow retries publishing tasks to Celery Broker at least 3 times""" - - with _prepare_app(), caplog.at_level(logging.INFO), mock.patch.object( - # Mock `with timeout()` to _instantly_ fail. - celery_executor.timeout, - "__enter__", - side_effect=AirflowTaskTimeout, - ): - executor = celery_executor.CeleryExecutor() - assert executor.task_publish_retries == {} - assert executor.task_publish_max_retries == 3, "Assert Default Max Retries is 3" - - task = BashOperator( - task_id="test", bash_command="true", dag=DAG(dag_id="id"), start_date=datetime.now() - ) - when = datetime.now() - value_tuple = ( - "command", - 1, - None, - SimpleTaskInstance.from_ti(ti=TaskInstance(task=task, run_id=None)), - ) - key = ("fail", "fake_simple_ti", when, 0) - executor.queued_tasks[key] = value_tuple - - # Test that when heartbeat is called again, task is published again to Celery Queue - executor.heartbeat() - assert dict(executor.task_publish_retries) == {key: 1} - assert 1 == len(executor.queued_tasks), "Task should remain in queue" - assert executor.event_buffer == {} - assert f"[Try 1 of 3] Task Timeout Error for Task: ({key})." in caplog.text - - executor.heartbeat() - assert dict(executor.task_publish_retries) == {key: 2} - assert 1 == len(executor.queued_tasks), "Task should remain in queue" - assert executor.event_buffer == {} - assert f"[Try 2 of 3] Task Timeout Error for Task: ({key})." in caplog.text - - executor.heartbeat() - assert dict(executor.task_publish_retries) == {key: 3} - assert 1 == len(executor.queued_tasks), "Task should remain in queue" - assert executor.event_buffer == {} - assert f"[Try 3 of 3] Task Timeout Error for Task: ({key})." in caplog.text - - executor.heartbeat() - assert dict(executor.task_publish_retries) == {} - assert 0 == len(executor.queued_tasks), "Task should no longer be in queue" - assert executor.event_buffer[("fail", "fake_simple_ti", when, 0)][0] == State.FAILED - @pytest.mark.quarantined @pytest.mark.backend("mysql", "postgres") def test_exception_propagation(self): @@ -481,111 +329,6 @@ def test_operation_timeout_config(): assert celery_executor.OPERATION_TIMEOUT == 1 -class ClassWithCustomAttributes: - """Class for testing purpose: allows to create objects with custom attributes in one single statement.""" - - def __init__(self, **kwargs): - for key, value in kwargs.items(): - setattr(self, key, value) - - def __str__(self): - return f"{ClassWithCustomAttributes.__name__}({str(self.__dict__)})" - - def __repr__(self): - return self.__str__() - - def __eq__(self, other): - return self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not self.__eq__(other) - - -class TestBulkStateFetcher(unittest.TestCase): - @mock.patch( - "celery.backends.base.BaseKeyValueStoreBackend.mget", - return_value=[json.dumps({"status": "SUCCESS", "task_id": "123"})], - ) - @pytest.mark.integration("redis") - @pytest.mark.integration("rabbitmq") - @pytest.mark.backend("mysql", "postgres") - def test_should_support_kv_backend(self, mock_mget): - with _prepare_app(): - mock_backend = BaseKeyValueStoreBackend(app=celery_executor.app) - with mock.patch( - "airflow.executors.celery_executor.Celery.backend", mock_backend - ), self.assertLogs("airflow.executors.celery_executor.BulkStateFetcher", level="DEBUG") as cm: - fetcher = BulkStateFetcher() - result = fetcher.get_many( - [ - mock.MagicMock(task_id="123"), - mock.MagicMock(task_id="456"), - ] - ) - - # Assert called - ignore order - mget_args, _ = mock_mget.call_args - assert set(mget_args[0]) == {b"celery-task-meta-456", b"celery-task-meta-123"} - mock_mget.assert_called_once_with(mock.ANY) - - assert result == {"123": ("SUCCESS", None), "456": ("PENDING", None)} - assert [ - "DEBUG:airflow.executors.celery_executor.BulkStateFetcher:Fetched 2 state(s) for 2 task(s)" - ] == cm.output - - @mock.patch("celery.backends.database.DatabaseBackend.ResultSession") - @pytest.mark.integration("redis") - @pytest.mark.integration("rabbitmq") - @pytest.mark.backend("mysql", "postgres") - def test_should_support_db_backend(self, mock_session): - with _prepare_app(): - mock_backend = DatabaseBackend(app=celery_executor.app, url="sqlite3://") - - with mock.patch( - "airflow.executors.celery_executor.Celery.backend", mock_backend - ), self.assertLogs("airflow.executors.celery_executor.BulkStateFetcher", level="DEBUG") as cm: - mock_session = mock_backend.ResultSession.return_value - mock_session.query.return_value.filter.return_value.all.return_value = [ - mock.MagicMock(**{"to_dict.return_value": {"status": "SUCCESS", "task_id": "123"}}) - ] - - fetcher = BulkStateFetcher() - result = fetcher.get_many( - [ - mock.MagicMock(task_id="123"), - mock.MagicMock(task_id="456"), - ] - ) - - assert result == {"123": ("SUCCESS", None), "456": ("PENDING", None)} - assert [ - "DEBUG:airflow.executors.celery_executor.BulkStateFetcher:Fetched 2 state(s) for 2 task(s)" - ] == cm.output - - @pytest.mark.integration("redis") - @pytest.mark.integration("rabbitmq") - @pytest.mark.backend("mysql", "postgres") - def test_should_support_base_backend(self): - with _prepare_app(): - mock_backend = mock.MagicMock(autospec=BaseBackend) - - with mock.patch( - "airflow.executors.celery_executor.Celery.backend", mock_backend - ), self.assertLogs("airflow.executors.celery_executor.BulkStateFetcher", level="DEBUG") as cm: - fetcher = BulkStateFetcher(1) - result = fetcher.get_many( - [ - ClassWithCustomAttributes(task_id="123", state="SUCCESS"), - ClassWithCustomAttributes(task_id="456", state="PENDING"), - ] - ) - - assert result == {"123": ("SUCCESS", None), "456": ("PENDING", None)} - assert [ - "DEBUG:airflow.executors.celery_executor.BulkStateFetcher:Fetched 2 state(s) for 2 task(s)" - ] == cm.output - - class MockTask: """ A picklable object used to mock tasks sent to Celery. Can't use the mock library diff --git a/tests/providers/apache/cassandra/hooks/__init__.py b/tests/integration/__init__.py similarity index 100% rename from tests/providers/apache/cassandra/hooks/__init__.py rename to tests/integration/__init__.py diff --git a/tests/integration/api/__init__.py b/tests/integration/api/__init__.py new file mode 100644 index 0000000000000..13a83393a9124 --- /dev/null +++ b/tests/integration/api/__init__.py @@ -0,0 +1,16 @@ +# 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. diff --git a/tests/integration/api/auth/__init__.py b/tests/integration/api/auth/__init__.py new file mode 100644 index 0000000000000..13a83393a9124 --- /dev/null +++ b/tests/integration/api/auth/__init__.py @@ -0,0 +1,16 @@ +# 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. diff --git a/tests/integration/api/auth/backend/__init__.py b/tests/integration/api/auth/backend/__init__.py new file mode 100644 index 0000000000000..13a83393a9124 --- /dev/null +++ b/tests/integration/api/auth/backend/__init__.py @@ -0,0 +1,16 @@ +# 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. diff --git a/tests/api/auth/backend/test_kerberos_auth.py b/tests/integration/api/auth/backend/test_kerberos_auth.py similarity index 100% rename from tests/api/auth/backend/test_kerberos_auth.py rename to tests/integration/api/auth/backend/test_kerberos_auth.py diff --git a/tests/integration/cli/__init__.py b/tests/integration/cli/__init__.py new file mode 100644 index 0000000000000..13a83393a9124 --- /dev/null +++ b/tests/integration/cli/__init__.py @@ -0,0 +1,16 @@ +# 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. diff --git a/tests/integration/cli/commands/__init__.py b/tests/integration/cli/commands/__init__.py new file mode 100644 index 0000000000000..13a83393a9124 --- /dev/null +++ b/tests/integration/cli/commands/__init__.py @@ -0,0 +1,16 @@ +# 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. diff --git a/tests/integration/cli/commands/test_celery_command.py b/tests/integration/cli/commands/test_celery_command.py new file mode 100644 index 0000000000000..3cc9da0a13473 --- /dev/null +++ b/tests/integration/cli/commands/test_celery_command.py @@ -0,0 +1,57 @@ +# 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 __future__ import annotations + +from unittest import mock + +import pytest + +from airflow.cli import cli_parser +from airflow.cli.commands import celery_command +from tests.test_utils.config import conf_vars + + +@pytest.mark.integration("redis") +@pytest.mark.integration("rabbitmq") +@pytest.mark.backend("mysql", "postgres") +class TestWorkerServeLogs: + @classmethod + def setup_class(cls): + cls.parser = cli_parser.get_parser() + + @mock.patch("airflow.cli.commands.celery_command.celery_app") + @conf_vars({("core", "executor"): "CeleryExecutor"}) + def test_serve_logs_on_worker_start(self, mock_celery_app): + with mock.patch("airflow.cli.commands.celery_command.Process") as mock_process: + args = self.parser.parse_args(["celery", "worker", "--concurrency", "1"]) + + with mock.patch("celery.platforms.check_privileges") as mock_privil: + mock_privil.return_value = 0 + celery_command.worker(args) + mock_process.assert_called() + + @mock.patch("airflow.cli.commands.celery_command.celery_app") + @conf_vars({("core", "executor"): "CeleryExecutor"}) + def test_skip_serve_logs_on_worker_start(self, mock_celery_app): + with mock.patch("airflow.cli.commands.celery_command.Process") as mock_popen: + args = self.parser.parse_args(["celery", "worker", "--concurrency", "1", "--skip-serve-logs"]) + + with mock.patch("celery.platforms.check_privileges") as mock_privil: + mock_privil.return_value = 0 + celery_command.worker(args) + mock_popen.assert_not_called() diff --git a/tests/integration/executors/__init__.py b/tests/integration/executors/__init__.py new file mode 100644 index 0000000000000..13a83393a9124 --- /dev/null +++ b/tests/integration/executors/__init__.py @@ -0,0 +1,16 @@ +# 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. diff --git a/tests/integration/executors/test_celery_executor.py b/tests/integration/executors/test_celery_executor.py new file mode 100644 index 0000000000000..f7f69fcfce789 --- /dev/null +++ b/tests/integration/executors/test_celery_executor.py @@ -0,0 +1,339 @@ +# +# 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 __future__ import annotations + +import contextlib +import json +import logging +import os +import sys +import unittest +from datetime import datetime, timedelta +from unittest import mock + +# leave this it is used by the test worker +import celery.contrib.testing.tasks # noqa: F401 +import pytest +from celery import Celery +from celery.backends.base import BaseBackend, BaseKeyValueStoreBackend +from celery.backends.database import DatabaseBackend +from celery.contrib.testing.worker import start_worker +from kombu.asynchronous import set_event_loop +from parameterized import parameterized + +from airflow.configuration import conf +from airflow.exceptions import AirflowException, AirflowTaskTimeout +from airflow.executors import celery_executor +from airflow.executors.celery_executor import BulkStateFetcher +from airflow.models.dag import DAG +from airflow.models.taskinstance import SimpleTaskInstance, TaskInstance +from airflow.operators.bash import BashOperator +from airflow.utils.state import State +from tests.test_utils import db + + +def _prepare_test_bodies(): + if "CELERY_BROKER_URLS" in os.environ: + return [(url,) for url in os.environ["CELERY_BROKER_URLS"].split(",")] + return [(conf.get("celery", "BROKER_URL"))] + + +class FakeCeleryResult: + @property + def state(self): + raise Exception() + + def task_id(self): + return "task_id" + + +@contextlib.contextmanager +def _prepare_app(broker_url=None, execute=None): + broker_url = broker_url or conf.get("celery", "BROKER_URL") + execute = execute or celery_executor.execute_command.__wrapped__ + + test_config = dict(celery_executor.celery_configuration) + test_config.update({"broker_url": broker_url}) + test_app = Celery(broker_url, config_source=test_config) + test_execute = test_app.task(execute) + patch_app = mock.patch("airflow.executors.celery_executor.app", test_app) + patch_execute = mock.patch("airflow.executors.celery_executor.execute_command", test_execute) + + backend = test_app.backend + + if hasattr(backend, "ResultSession"): + # Pre-create the database tables now, otherwise SQLA vis Celery has a + # race condition where it one of the subprocesses can die with "Table + # already exists" error, because SQLA checks for which tables exist, + # then issues a CREATE TABLE, rather than doing CREATE TABLE IF NOT + # EXISTS + session = backend.ResultSession() + session.close() + + with patch_app, patch_execute: + try: + yield test_app + finally: + # Clear event loop to tear down each celery instance + set_event_loop(None) + + +@pytest.mark.integration("redis") +@pytest.mark.integration("rabbitmq") +@pytest.mark.backend("mysql", "postgres") +class TestCeleryExecutor: + def setup_method(self) -> None: + db.clear_db_runs() + db.clear_db_jobs() + + def teardown_method(self) -> None: + db.clear_db_runs() + db.clear_db_jobs() + + @parameterized.expand(_prepare_test_bodies()) + def test_celery_integration(self, broker_url): + success_command = ["airflow", "tasks", "run", "true", "some_parameter"] + fail_command = ["airflow", "version"] + + def fake_execute_command(command): + if command != success_command: + raise AirflowException("fail") + + with _prepare_app(broker_url, execute=fake_execute_command) as app: + executor = celery_executor.CeleryExecutor() + assert executor.tasks == {} + executor.start() + + with start_worker(app=app, logfile=sys.stdout, loglevel="info"): + execute_date = datetime.now() + + task_tuples_to_send = [ + ( + ("success", "fake_simple_ti", execute_date, 0), + success_command, + celery_executor.celery_configuration["task_default_queue"], + celery_executor.execute_command, + ), + ( + ("fail", "fake_simple_ti", execute_date, 0), + fail_command, + celery_executor.celery_configuration["task_default_queue"], + celery_executor.execute_command, + ), + ] + + # "Enqueue" them. We don't have a real SimpleTaskInstance, so directly edit the dict + for (key, command, queue, task) in task_tuples_to_send: + executor.queued_tasks[key] = (command, 1, queue, None) + executor.task_publish_retries[key] = 1 + + executor._process_tasks(task_tuples_to_send) + + assert list(executor.tasks.keys()) == [ + ("success", "fake_simple_ti", execute_date, 0), + ("fail", "fake_simple_ti", execute_date, 0), + ] + assert ( + executor.event_buffer[("success", "fake_simple_ti", execute_date, 0)][0] == State.QUEUED + ) + assert executor.event_buffer[("fail", "fake_simple_ti", execute_date, 0)][0] == State.QUEUED + + executor.end(synchronous=True) + + assert executor.event_buffer[("success", "fake_simple_ti", execute_date, 0)][0] == State.SUCCESS + assert executor.event_buffer[("fail", "fake_simple_ti", execute_date, 0)][0] == State.FAILED + + assert "success" not in executor.tasks + assert "fail" not in executor.tasks + + assert executor.queued_tasks == {} + assert timedelta(0, 600) == executor.task_adoption_timeout + + def test_error_sending_task(self): + def fake_execute_command(): + pass + + with _prepare_app(execute=fake_execute_command): + # fake_execute_command takes no arguments while execute_command takes 1, + # which will cause TypeError when calling task.apply_async() + executor = celery_executor.CeleryExecutor() + task = BashOperator( + task_id="test", bash_command="true", dag=DAG(dag_id="id"), start_date=datetime.now() + ) + when = datetime.now() + value_tuple = ( + "command", + 1, + None, + SimpleTaskInstance.from_ti(ti=TaskInstance(task=task, run_id=None)), + ) + key = ("fail", "fake_simple_ti", when, 0) + executor.queued_tasks[key] = value_tuple + executor.task_publish_retries[key] = 1 + executor.heartbeat() + assert 0 == len(executor.queued_tasks), "Task should no longer be queued" + assert executor.event_buffer[("fail", "fake_simple_ti", when, 0)][0] == State.FAILED + + def test_retry_on_error_sending_task(self, caplog): + """Test that Airflow retries publishing tasks to Celery Broker at least 3 times""" + + with _prepare_app(), caplog.at_level(logging.INFO), mock.patch.object( + # Mock `with timeout()` to _instantly_ fail. + celery_executor.timeout, + "__enter__", + side_effect=AirflowTaskTimeout, + ): + executor = celery_executor.CeleryExecutor() + assert executor.task_publish_retries == {} + assert executor.task_publish_max_retries == 3, "Assert Default Max Retries is 3" + + task = BashOperator( + task_id="test", bash_command="true", dag=DAG(dag_id="id"), start_date=datetime.now() + ) + when = datetime.now() + value_tuple = ( + "command", + 1, + None, + SimpleTaskInstance.from_ti(ti=TaskInstance(task=task, run_id=None)), + ) + key = ("fail", "fake_simple_ti", when, 0) + executor.queued_tasks[key] = value_tuple + + # Test that when heartbeat is called again, task is published again to Celery Queue + executor.heartbeat() + assert dict(executor.task_publish_retries) == {key: 1} + assert 1 == len(executor.queued_tasks), "Task should remain in queue" + assert executor.event_buffer == {} + assert f"[Try 1 of 3] Task Timeout Error for Task: ({key})." in caplog.text + + executor.heartbeat() + assert dict(executor.task_publish_retries) == {key: 2} + assert 1 == len(executor.queued_tasks), "Task should remain in queue" + assert executor.event_buffer == {} + assert f"[Try 2 of 3] Task Timeout Error for Task: ({key})." in caplog.text + + executor.heartbeat() + assert dict(executor.task_publish_retries) == {key: 3} + assert 1 == len(executor.queued_tasks), "Task should remain in queue" + assert executor.event_buffer == {} + assert f"[Try 3 of 3] Task Timeout Error for Task: ({key})." in caplog.text + + executor.heartbeat() + assert dict(executor.task_publish_retries) == {} + assert 0 == len(executor.queued_tasks), "Task should no longer be in queue" + assert executor.event_buffer[("fail", "fake_simple_ti", when, 0)][0] == State.FAILED + + +class ClassWithCustomAttributes: + """Class for testing purpose: allows to create objects with custom attributes in one single statement.""" + + def __init__(self, **kwargs): + for key, value in kwargs.items(): + setattr(self, key, value) + + def __str__(self): + return f"{ClassWithCustomAttributes.__name__}({str(self.__dict__)})" + + def __repr__(self): + return self.__str__() + + def __eq__(self, other): + return self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not self.__eq__(other) + + +@pytest.mark.integration("redis") +@pytest.mark.integration("rabbitmq") +@pytest.mark.backend("mysql", "postgres") +class TestBulkStateFetcher(unittest.TestCase): + @mock.patch( + "celery.backends.base.BaseKeyValueStoreBackend.mget", + return_value=[json.dumps({"status": "SUCCESS", "task_id": "123"})], + ) + def test_should_support_kv_backend(self, mock_mget): + with _prepare_app(): + mock_backend = BaseKeyValueStoreBackend(app=celery_executor.app) + with mock.patch( + "airflow.executors.celery_executor.Celery.backend", mock_backend + ), self.assertLogs("airflow.executors.celery_executor.BulkStateFetcher", level="DEBUG") as cm: + fetcher = BulkStateFetcher() + result = fetcher.get_many( + [ + mock.MagicMock(task_id="123"), + mock.MagicMock(task_id="456"), + ] + ) + + # Assert called - ignore order + mget_args, _ = mock_mget.call_args + assert set(mget_args[0]) == {b"celery-task-meta-456", b"celery-task-meta-123"} + mock_mget.assert_called_once_with(mock.ANY) + + assert result == {"123": ("SUCCESS", None), "456": ("PENDING", None)} + assert [ + "DEBUG:airflow.executors.celery_executor.BulkStateFetcher:Fetched 2 state(s) for 2 task(s)" + ] == cm.output + + @mock.patch("celery.backends.database.DatabaseBackend.ResultSession") + def test_should_support_db_backend(self, mock_session): + with _prepare_app(): + mock_backend = DatabaseBackend(app=celery_executor.app, url="sqlite3://") + + with mock.patch( + "airflow.executors.celery_executor.Celery.backend", mock_backend + ), self.assertLogs("airflow.executors.celery_executor.BulkStateFetcher", level="DEBUG") as cm: + mock_session = mock_backend.ResultSession.return_value + mock_session.query.return_value.filter.return_value.all.return_value = [ + mock.MagicMock(**{"to_dict.return_value": {"status": "SUCCESS", "task_id": "123"}}) + ] + + fetcher = BulkStateFetcher() + result = fetcher.get_many( + [ + mock.MagicMock(task_id="123"), + mock.MagicMock(task_id="456"), + ] + ) + + assert result == {"123": ("SUCCESS", None), "456": ("PENDING", None)} + assert [ + "DEBUG:airflow.executors.celery_executor.BulkStateFetcher:Fetched 2 state(s) for 2 task(s)" + ] == cm.output + + def test_should_support_base_backend(self): + with _prepare_app(): + mock_backend = mock.MagicMock(autospec=BaseBackend) + + with mock.patch( + "airflow.executors.celery_executor.Celery.backend", mock_backend + ), self.assertLogs("airflow.executors.celery_executor.BulkStateFetcher", level="DEBUG") as cm: + fetcher = BulkStateFetcher(1) + result = fetcher.get_many( + [ + ClassWithCustomAttributes(task_id="123", state="SUCCESS"), + ClassWithCustomAttributes(task_id="456", state="PENDING"), + ] + ) + + assert result == {"123": ("SUCCESS", None), "456": ("PENDING", None)} + assert [ + "DEBUG:airflow.executors.celery_executor.BulkStateFetcher:Fetched 2 state(s) for 2 task(s)" + ] == cm.output diff --git a/tests/integration/providers/__init__.py b/tests/integration/providers/__init__.py new file mode 100644 index 0000000000000..13a83393a9124 --- /dev/null +++ b/tests/integration/providers/__init__.py @@ -0,0 +1,16 @@ +# 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. diff --git a/tests/integration/providers/apache/__init__.py b/tests/integration/providers/apache/__init__.py new file mode 100644 index 0000000000000..13a83393a9124 --- /dev/null +++ b/tests/integration/providers/apache/__init__.py @@ -0,0 +1,16 @@ +# 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. diff --git a/tests/integration/providers/apache/cassandra/__init__.py b/tests/integration/providers/apache/cassandra/__init__.py new file mode 100644 index 0000000000000..13a83393a9124 --- /dev/null +++ b/tests/integration/providers/apache/cassandra/__init__.py @@ -0,0 +1,16 @@ +# 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. diff --git a/tests/integration/providers/apache/cassandra/hooks/__init__.py b/tests/integration/providers/apache/cassandra/hooks/__init__.py new file mode 100644 index 0000000000000..13a83393a9124 --- /dev/null +++ b/tests/integration/providers/apache/cassandra/hooks/__init__.py @@ -0,0 +1,16 @@ +# 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. diff --git a/tests/providers/apache/cassandra/hooks/test_cassandra.py b/tests/integration/providers/apache/cassandra/hooks/test_cassandra.py similarity index 100% rename from tests/providers/apache/cassandra/hooks/test_cassandra.py rename to tests/integration/providers/apache/cassandra/hooks/test_cassandra.py diff --git a/tests/integration/providers/apache/pinot/__init__.py b/tests/integration/providers/apache/pinot/__init__.py new file mode 100644 index 0000000000000..13a83393a9124 --- /dev/null +++ b/tests/integration/providers/apache/pinot/__init__.py @@ -0,0 +1,16 @@ +# 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. diff --git a/tests/integration/providers/apache/pinot/hooks/__init__.py b/tests/integration/providers/apache/pinot/hooks/__init__.py new file mode 100644 index 0000000000000..13a83393a9124 --- /dev/null +++ b/tests/integration/providers/apache/pinot/hooks/__init__.py @@ -0,0 +1,16 @@ +# 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. diff --git a/tests/integration/providers/apache/pinot/hooks/test_pinot.py b/tests/integration/providers/apache/pinot/hooks/test_pinot.py new file mode 100644 index 0000000000000..d99e8efdf4808 --- /dev/null +++ b/tests/integration/providers/apache/pinot/hooks/test_pinot.py @@ -0,0 +1,34 @@ +# 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 __future__ import annotations + +from unittest import mock + +import pytest + +from airflow.providers.apache.pinot.hooks.pinot import PinotDbApiHook + + +@pytest.mark.integration("pinot") +class TestPinotDbApiHookIntegration: + @mock.patch.dict("os.environ", AIRFLOW_CONN_PINOT_BROKER_DEFAULT="pinot://pinot:8000/") + def test_should_return_records(self): + hook = PinotDbApiHook() + sql = "select playerName from baseballStats ORDER BY playerName limit 5" + records = hook.get_records(sql) + assert [["A. Harry"], ["A. Harry"], ["Aaron"], ["Aaron Albert"], ["Aaron Albert"]] == records diff --git a/tests/integration/providers/google/__init__.py b/tests/integration/providers/google/__init__.py new file mode 100644 index 0000000000000..13a83393a9124 --- /dev/null +++ b/tests/integration/providers/google/__init__.py @@ -0,0 +1,16 @@ +# 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. diff --git a/tests/integration/providers/google/cloud/__init__.py b/tests/integration/providers/google/cloud/__init__.py new file mode 100644 index 0000000000000..13a83393a9124 --- /dev/null +++ b/tests/integration/providers/google/cloud/__init__.py @@ -0,0 +1,16 @@ +# 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. diff --git a/tests/integration/providers/google/cloud/transfers/__init__.py b/tests/integration/providers/google/cloud/transfers/__init__.py new file mode 100644 index 0000000000000..13a83393a9124 --- /dev/null +++ b/tests/integration/providers/google/cloud/transfers/__init__.py @@ -0,0 +1,16 @@ +# 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. diff --git a/tests/providers/google/cloud/transfers/test_presto_to_gcs.py b/tests/integration/providers/google/cloud/transfers/test_presto_to_gcs.py similarity index 100% rename from tests/providers/google/cloud/transfers/test_presto_to_gcs.py rename to tests/integration/providers/google/cloud/transfers/test_presto_to_gcs.py diff --git a/tests/providers/google/cloud/transfers/test_presto_to_gcs_system.py b/tests/integration/providers/google/cloud/transfers/test_presto_to_gcs_system.py similarity index 100% rename from tests/providers/google/cloud/transfers/test_presto_to_gcs_system.py rename to tests/integration/providers/google/cloud/transfers/test_presto_to_gcs_system.py diff --git a/tests/providers/google/cloud/transfers/test_trino_to_gcs.py b/tests/integration/providers/google/cloud/transfers/test_trino_to_gcs.py similarity index 100% rename from tests/providers/google/cloud/transfers/test_trino_to_gcs.py rename to tests/integration/providers/google/cloud/transfers/test_trino_to_gcs.py diff --git a/tests/integration/providers/mongo/__init__.py b/tests/integration/providers/mongo/__init__.py new file mode 100644 index 0000000000000..13a83393a9124 --- /dev/null +++ b/tests/integration/providers/mongo/__init__.py @@ -0,0 +1,16 @@ +# 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. diff --git a/tests/integration/providers/mongo/sensors/__init__.py b/tests/integration/providers/mongo/sensors/__init__.py new file mode 100644 index 0000000000000..13a83393a9124 --- /dev/null +++ b/tests/integration/providers/mongo/sensors/__init__.py @@ -0,0 +1,16 @@ +# 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. diff --git a/tests/providers/mongo/sensors/test_mongo.py b/tests/integration/providers/mongo/sensors/test_mongo.py similarity index 100% rename from tests/providers/mongo/sensors/test_mongo.py rename to tests/integration/providers/mongo/sensors/test_mongo.py diff --git a/tests/integration/providers/redis/__init__.py b/tests/integration/providers/redis/__init__.py new file mode 100644 index 0000000000000..13a83393a9124 --- /dev/null +++ b/tests/integration/providers/redis/__init__.py @@ -0,0 +1,16 @@ +# 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. diff --git a/tests/integration/providers/redis/hooks/__init__.py b/tests/integration/providers/redis/hooks/__init__.py new file mode 100644 index 0000000000000..13a83393a9124 --- /dev/null +++ b/tests/integration/providers/redis/hooks/__init__.py @@ -0,0 +1,16 @@ +# 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. diff --git a/tests/integration/providers/redis/hooks/test_redis.py b/tests/integration/providers/redis/hooks/test_redis.py new file mode 100644 index 0000000000000..eac17ee676edc --- /dev/null +++ b/tests/integration/providers/redis/hooks/test_redis.py @@ -0,0 +1,39 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +from __future__ import annotations + +import pytest + +from airflow.providers.redis.hooks.redis import RedisHook + + +@pytest.mark.integration("redis") +class TestRedisHook: + def test_real_ping(self): + hook = RedisHook(redis_conn_id="redis_default") + redis = hook.get_conn() + + assert redis.ping(), "Connection to Redis with PING works." + + def test_real_get_and_set(self): + hook = RedisHook(redis_conn_id="redis_default") + redis = hook.get_conn() + + assert redis.set("test_key", "test_value"), "Connection to Redis with SET works." + assert redis.get("test_key") == b"test_value", "Connection to Redis with GET works." + assert redis.delete("test_key") == 1, "Connection to Redis with DELETE works." diff --git a/tests/integration/providers/redis/operators/__init__.py b/tests/integration/providers/redis/operators/__init__.py new file mode 100644 index 0000000000000..13a83393a9124 --- /dev/null +++ b/tests/integration/providers/redis/operators/__init__.py @@ -0,0 +1,16 @@ +# 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. diff --git a/tests/providers/redis/operators/test_redis_publish.py b/tests/integration/providers/redis/operators/test_redis_publish.py similarity index 100% rename from tests/providers/redis/operators/test_redis_publish.py rename to tests/integration/providers/redis/operators/test_redis_publish.py diff --git a/tests/integration/providers/redis/sensors/__init__.py b/tests/integration/providers/redis/sensors/__init__.py new file mode 100644 index 0000000000000..13a83393a9124 --- /dev/null +++ b/tests/integration/providers/redis/sensors/__init__.py @@ -0,0 +1,16 @@ +# 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. diff --git a/tests/providers/redis/sensors/test_redis_key.py b/tests/integration/providers/redis/sensors/test_redis_key.py similarity index 100% rename from tests/providers/redis/sensors/test_redis_key.py rename to tests/integration/providers/redis/sensors/test_redis_key.py diff --git a/tests/integration/providers/redis/sensors/test_redis_pub_sub.py b/tests/integration/providers/redis/sensors/test_redis_pub_sub.py new file mode 100644 index 0000000000000..e99f2a38d9914 --- /dev/null +++ b/tests/integration/providers/redis/sensors/test_redis_pub_sub.py @@ -0,0 +1,80 @@ +# +# 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 __future__ import annotations + +from time import sleep +from unittest.mock import MagicMock, call + +import pytest + +from airflow.models.dag import DAG +from airflow.providers.redis.hooks.redis import RedisHook +from airflow.providers.redis.sensors.redis_pub_sub import RedisPubSubSensor +from airflow.utils import timezone + +DEFAULT_DATE = timezone.datetime(2017, 1, 1) + + +@pytest.mark.integration("redis") +class TestRedisPubSubSensor: + def setup_method(self): + args = {"owner": "airflow", "start_date": DEFAULT_DATE} + + self.dag = DAG("test_dag_id", default_args=args) + + self.mock_context = MagicMock() + + def test_poke_true(self): + sensor = RedisPubSubSensor( + task_id="test_task", dag=self.dag, channels="test", redis_conn_id="redis_default" + ) + + hook = RedisHook(redis_conn_id="redis_default") + redis = hook.get_conn() + redis.publish("test", "message") + + result = sensor.poke(self.mock_context) + assert not result + + for _ in range(1, 10): + result = sensor.poke(self.mock_context) + if result: + break + sleep(0.1) + assert result + context_calls = [ + call.xcom_push( + key="message", + value={"type": "message", "pattern": None, "channel": b"test", "data": b"message"}, + ) + ] + assert self.mock_context["ti"].method_calls == context_calls, "context calls should be same" + result = sensor.poke(self.mock_context) + assert not result + + def test_poke_false(self): + sensor = RedisPubSubSensor( + task_id="test_task", dag=self.dag, channels="test", redis_conn_id="redis_default" + ) + + result = sensor.poke(self.mock_context) + assert not result + assert self.mock_context["ti"].method_calls == [], "context calls should be same" + result = sensor.poke(self.mock_context) + assert not result + assert self.mock_context["ti"].method_calls == [], "context calls should be same" diff --git a/tests/integration/providers/trino/__init__.py b/tests/integration/providers/trino/__init__.py new file mode 100644 index 0000000000000..13a83393a9124 --- /dev/null +++ b/tests/integration/providers/trino/__init__.py @@ -0,0 +1,16 @@ +# 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. diff --git a/tests/integration/providers/trino/hooks/__init__.py b/tests/integration/providers/trino/hooks/__init__.py new file mode 100644 index 0000000000000..13a83393a9124 --- /dev/null +++ b/tests/integration/providers/trino/hooks/__init__.py @@ -0,0 +1,16 @@ +# 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. diff --git a/tests/integration/providers/trino/hooks/test_trino.py b/tests/integration/providers/trino/hooks/test_trino.py new file mode 100644 index 0000000000000..bb06d53887e66 --- /dev/null +++ b/tests/integration/providers/trino/hooks/test_trino.py @@ -0,0 +1,48 @@ +# +# 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 __future__ import annotations + +from unittest import mock + +import pytest + +from airflow.providers.trino.hooks.trino import TrinoHook + + +@pytest.mark.integration("trino") +class TestTrinoHookIntegration: + @mock.patch.dict("os.environ", AIRFLOW_CONN_TRINO_DEFAULT="trino://airflow@trino:8080/") + def test_should_record_records(self): + hook = TrinoHook() + sql = "SELECT name FROM tpch.sf1.customer ORDER BY custkey ASC LIMIT 3" + records = hook.get_records(sql) + assert [["Customer#000000001"], ["Customer#000000002"], ["Customer#000000003"]] == records + + @pytest.mark.integration("kerberos") + def test_should_record_records_with_kerberos_auth(self): + conn_url = ( + "trino://airflow@trino.example.com:7778/?" + "auth=kerberos&kerberos__service_name=HTTP&" + "verify=False&" + "protocol=https" + ) + with mock.patch.dict("os.environ", AIRFLOW_CONN_TRINO_DEFAULT=conn_url): + hook = TrinoHook() + sql = "SELECT name FROM tpch.sf1.customer ORDER BY custkey ASC LIMIT 3" + records = hook.get_records(sql) + assert [["Customer#000000001"], ["Customer#000000002"], ["Customer#000000003"]] == records diff --git a/tests/integration/security/__init__.py b/tests/integration/security/__init__.py new file mode 100644 index 0000000000000..13a83393a9124 --- /dev/null +++ b/tests/integration/security/__init__.py @@ -0,0 +1,16 @@ +# 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. diff --git a/tests/integration/security/test_kerberos.py b/tests/integration/security/test_kerberos.py new file mode 100644 index 0000000000000..033b455b56ea7 --- /dev/null +++ b/tests/integration/security/test_kerberos.py @@ -0,0 +1,75 @@ +# 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 __future__ import annotations + +import logging +import os +from contextlib import nullcontext +from unittest import mock + +import pytest + +from airflow.security import kerberos +from airflow.security.kerberos import renew_from_kt +from tests.test_utils.config import conf_vars + + +@pytest.mark.integration("kerberos") +class TestKerberosIntegration: + @classmethod + def setup_class(cls): + assert "KRB5_KTNAME" in os.environ, "Missing KRB5_KTNAME environment variable" + cls.keytab = os.environ["KRB5_KTNAME"] + + @pytest.mark.parametrize( + "kerberos_config", + [ + pytest.param({}, id="default-config"), + pytest.param({("kerberos", "include_ip"): "True"}, id="explicit-include-ip"), + pytest.param({("kerberos", "include_ip"): "False"}, id="explicit-not-include-ip"), + pytest.param({("kerberos", "forwardable"): "True"}, id="explicit-forwardable"), + pytest.param({("kerberos", "forwardable"): "False"}, id="explicit-not-forwardable"), + ], + ) + def test_renew_from_kt(self, kerberos_config): + """We expect return 0 (exit code) and successful run.""" + with conf_vars(kerberos_config): + assert renew_from_kt(principal=None, keytab=self.keytab) == 0 + + @pytest.mark.parametrize( + "exit_on_fail, expected_context", + [ + pytest.param(True, pytest.raises(SystemExit), id="exit-on-fail"), + pytest.param(False, nullcontext(), id="return-code-of-fail"), + ], + ) + def test_args_from_cli(self, exit_on_fail, expected_context, caplog): + """Test exit code if keytab not exist.""" + keytab = "/not/exists/keytab" + result = None + + with mock.patch.dict(os.environ, KRB5_KTNAME=keytab), conf_vars({("kerberos", "keytab"): keytab}): + with expected_context as ctx: + with caplog.at_level(logging.ERROR, logger=kerberos.log.name): + caplog.clear() + result = renew_from_kt(principal=None, keytab=keytab, exit_on_fail=exit_on_fail) + + # If `exit_on_fail` set to True than exit code in exception, otherwise in function return + exit_code = ctx.value.code if exit_on_fail else result + assert exit_code == 1 + assert caplog.record_tuples diff --git a/tests/providers/apache/pinot/hooks/test_pinot.py b/tests/providers/apache/pinot/hooks/test_pinot.py index 0ca2dfb1c581a..89dde59666b2b 100644 --- a/tests/providers/apache/pinot/hooks/test_pinot.py +++ b/tests/providers/apache/pinot/hooks/test_pinot.py @@ -271,13 +271,3 @@ def test_get_pandas_df(self): assert column == df.columns[0] for i, item in enumerate(result_sets): assert item[0] == df.values.tolist()[i][0] - - -@pytest.mark.integration("pinot") -class TestPinotDbApiHookIntegration: - @mock.patch.dict("os.environ", AIRFLOW_CONN_PINOT_BROKER_DEFAULT="pinot://pinot:8000/") - def test_should_return_records(self): - hook = PinotDbApiHook() - sql = "select playerName from baseballStats ORDER BY playerName limit 5" - records = hook.get_records(sql) - assert [["A. Harry"], ["A. Harry"], ["Aaron"], ["Aaron Albert"], ["Aaron Albert"]] == records diff --git a/tests/providers/redis/hooks/test_redis.py b/tests/providers/redis/hooks/test_redis.py index 0eb4edd76fc0a..119964fd3b22c 100644 --- a/tests/providers/redis/hooks/test_redis.py +++ b/tests/providers/redis/hooks/test_redis.py @@ -19,8 +19,6 @@ from unittest import mock -import pytest - from airflow.models import Connection from airflow.providers.redis.hooks.redis import RedisHook @@ -76,19 +74,3 @@ def test_get_conn_password_stays_none(self): hook = RedisHook(redis_conn_id="redis_default") hook.get_conn() assert hook.password is None - - @pytest.mark.integration("redis") - def test_real_ping(self): - hook = RedisHook(redis_conn_id="redis_default") - redis = hook.get_conn() - - assert redis.ping(), "Connection to Redis with PING works." - - @pytest.mark.integration("redis") - def test_real_get_and_set(self): - hook = RedisHook(redis_conn_id="redis_default") - redis = hook.get_conn() - - assert redis.set("test_key", "test_value"), "Connection to Redis with SET works." - assert redis.get("test_key") == b"test_value", "Connection to Redis with GET works." - assert redis.delete("test_key") == 1, "Connection to Redis with DELETE works." diff --git a/tests/providers/redis/sensors/test_redis_pub_sub.py b/tests/providers/redis/sensors/test_redis_pub_sub.py index 5ed0c40db71bd..dae08797ba69d 100644 --- a/tests/providers/redis/sensors/test_redis_pub_sub.py +++ b/tests/providers/redis/sensors/test_redis_pub_sub.py @@ -17,13 +17,9 @@ # under the License. from __future__ import annotations -from time import sleep from unittest.mock import MagicMock, call, patch -import pytest - from airflow.models.dag import DAG -from airflow.providers.redis.hooks.redis import RedisHook from airflow.providers.redis.sensors.redis_pub_sub import RedisPubSubSensor from airflow.utils import timezone @@ -76,45 +72,3 @@ def test_poke_mock_false(self, mock_redis_conn): context_calls = [] assert self.mock_context["ti"].method_calls == context_calls, "context calls should be same" - - @pytest.mark.integration("redis") - def test_poke_true(self): - sensor = RedisPubSubSensor( - task_id="test_task", dag=self.dag, channels="test", redis_conn_id="redis_default" - ) - - hook = RedisHook(redis_conn_id="redis_default") - redis = hook.get_conn() - redis.publish("test", "message") - - result = sensor.poke(self.mock_context) - assert not result - - for _ in range(1, 10): - result = sensor.poke(self.mock_context) - if result: - break - sleep(0.1) - assert result - context_calls = [ - call.xcom_push( - key="message", - value={"type": "message", "pattern": None, "channel": b"test", "data": b"message"}, - ) - ] - assert self.mock_context["ti"].method_calls == context_calls, "context calls should be same" - result = sensor.poke(self.mock_context) - assert not result - - @pytest.mark.integration("redis") - def test_poke_false(self): - sensor = RedisPubSubSensor( - task_id="test_task", dag=self.dag, channels="test", redis_conn_id="redis_default" - ) - - result = sensor.poke(self.mock_context) - assert not result - assert self.mock_context["ti"].method_calls == [], "context calls should be same" - result = sensor.poke(self.mock_context) - assert not result - assert self.mock_context["ti"].method_calls == [], "context calls should be same" diff --git a/tests/providers/trino/hooks/test_trino.py b/tests/providers/trino/hooks/test_trino.py index d4560a50ace12..4a0f2e6d2f680 100644 --- a/tests/providers/trino/hooks/test_trino.py +++ b/tests/providers/trino/hooks/test_trino.py @@ -311,27 +311,3 @@ def test_connection_failure(self, mock_conn): def test_serialize_cell(self): assert "foo" == self.db_hook._serialize_cell("foo", None) assert 1 == self.db_hook._serialize_cell(1, None) - - -@pytest.mark.integration("trino") -class TestTrinoHookIntegration: - @mock.patch.dict("os.environ", AIRFLOW_CONN_TRINO_DEFAULT="trino://airflow@trino:8080/") - def test_should_record_records(self): - hook = TrinoHook() - sql = "SELECT name FROM tpch.sf1.customer ORDER BY custkey ASC LIMIT 3" - records = hook.get_records(sql) - assert [["Customer#000000001"], ["Customer#000000002"], ["Customer#000000003"]] == records - - @pytest.mark.integration("kerberos") - def test_should_record_records_with_kerberos_auth(self): - conn_url = ( - "trino://airflow@trino.example.com:7778/?" - "auth=kerberos&kerberos__service_name=HTTP&" - "verify=False&" - "protocol=https" - ) - with mock.patch.dict("os.environ", AIRFLOW_CONN_TRINO_DEFAULT=conn_url): - hook = TrinoHook() - sql = "SELECT name FROM tpch.sf1.customer ORDER BY custkey ASC LIMIT 3" - records = hook.get_records(sql) - assert [["Customer#000000001"], ["Customer#000000002"], ["Customer#000000003"]] == records diff --git a/tests/security/test_kerberos.py b/tests/security/test_kerberos.py index 81f139c78206a..1a582220e0f74 100644 --- a/tests/security/test_kerberos.py +++ b/tests/security/test_kerberos.py @@ -18,9 +18,7 @@ from __future__ import annotations import logging -import os import shlex -from contextlib import nullcontext from unittest import mock import pytest @@ -30,52 +28,6 @@ from tests.test_utils.config import conf_vars -@pytest.mark.integration("kerberos") -class TestKerberosIntegration: - @classmethod - def setup_class(cls): - assert "KRB5_KTNAME" in os.environ, "Missing KRB5_KTNAME environment variable" - cls.keytab = os.environ["KRB5_KTNAME"] - - @pytest.mark.parametrize( - "kerberos_config", - [ - pytest.param({}, id="default-config"), - pytest.param({("kerberos", "include_ip"): "True"}, id="explicit-include-ip"), - pytest.param({("kerberos", "include_ip"): "False"}, id="explicit-not-include-ip"), - pytest.param({("kerberos", "forwardable"): "True"}, id="explicit-forwardable"), - pytest.param({("kerberos", "forwardable"): "False"}, id="explicit-not-forwardable"), - ], - ) - def test_renew_from_kt(self, kerberos_config): - """We expect return 0 (exit code) and successful run.""" - with conf_vars(kerberos_config): - assert renew_from_kt(principal=None, keytab=self.keytab) == 0 - - @pytest.mark.parametrize( - "exit_on_fail, expected_context", - [ - pytest.param(True, pytest.raises(SystemExit), id="exit-on-fail"), - pytest.param(False, nullcontext(), id="return-code-of-fail"), - ], - ) - def test_args_from_cli(self, exit_on_fail, expected_context, caplog): - """Test exit code if keytab not exist.""" - keytab = "/not/exists/keytab" - result = None - - with mock.patch.dict(os.environ, KRB5_KTNAME=keytab), conf_vars({("kerberos", "keytab"): keytab}): - with expected_context as ctx: - with caplog.at_level(logging.ERROR, logger=kerberos.log.name): - caplog.clear() - result = renew_from_kt(principal=None, keytab=keytab, exit_on_fail=exit_on_fail) - - # If `exit_on_fail` set to True than exit code in exception, otherwise in function return - exit_code = ctx.value.code if exit_on_fail else result - assert exit_code == 1 - assert caplog.record_tuples - - class TestKerberos: @pytest.mark.parametrize( "kerberos_config, expected_cmd", From f78d2851f899a670e915760347cbfe22db531b21 Mon Sep 17 00:00:00 2001 From: Jarek Potiuk Date: Thu, 8 Dec 2022 01:17:59 +0100 Subject: [PATCH 155/158] Merge redis and rabbitmq integration into celery (#28205) Previously we had separate integrations per service, but since we are moving the integration tests into separate job, it will be easier if the celery integration is a single one - this way we will have 1-1 relationship between tests to run and enabled integrations. The checks for various integrations were not really working recently and this change is introducing them back. (cherry picked from commit 68217f5df872a0098496cf75937dbf3d994d0549) --- .../src/airflow_breeze/global_constants.py | 6 +- images/breeze/output-commands-hash.txt | 10 +- images/breeze/output-commands.svg | 78 +++--- images/breeze/output_shell.svg | 240 ++++++++-------- images/breeze/output_start-airflow.svg | 264 +++++++++--------- images/breeze/output_testing_tests.svg | 68 +++-- ...ation-redis.yml => integration-celery.yml} | 17 +- .../docker-compose/integration-rabbitmq.yml | 40 --- scripts/in_container/check_environment.sh | 75 +++-- .../cli/commands/test_celery_command.py | 3 +- .../executors/test_celery_executor.py | 6 +- .../providers/redis/hooks/test_redis.py | 2 +- .../redis/operators/test_redis_publish.py | 2 +- .../providers/redis/sensors/test_redis_key.py | 2 +- .../redis/sensors/test_redis_pub_sub.py | 2 +- 15 files changed, 384 insertions(+), 431 deletions(-) rename scripts/ci/docker-compose/{integration-redis.yml => integration-celery.yml} (75%) delete mode 100644 scripts/ci/docker-compose/integration-rabbitmq.yml diff --git a/dev/breeze/src/airflow_breeze/global_constants.py b/dev/breeze/src/airflow_breeze/global_constants.py index 1c92fbdc232f6..0caf050e32fd1 100644 --- a/dev/breeze/src/airflow_breeze/global_constants.py +++ b/dev/breeze/src/airflow_breeze/global_constants.py @@ -49,8 +49,7 @@ "mongo", "openldap", "pinot", - "rabbitmq", - "redis", + "celery", "statsd", "trino", ] @@ -199,8 +198,7 @@ def get_airflow_extras(): "mongo", "openldap", "pinot", - "rabbitmq", - "redis", + "celery", "statsd", "trino", ] diff --git a/images/breeze/output-commands-hash.txt b/images/breeze/output-commands-hash.txt index ef8d5e0c86140..acd07f9b13591 100644 --- a/images/breeze/output-commands-hash.txt +++ b/images/breeze/output-commands-hash.txt @@ -1,7 +1,7 @@ # This file is automatically generated by pre-commit. If you have a conflict with this file # Please do not solve it but run `breeze setup regenerate-command-images`. # This command should fix the conflict and regenerate help images that you have conflict with. -main:11fedd2637e0e3b31936c4f015ab0d19 +main:0e7677cd7035e01b5260b9ac4889fd90 build-docs:80555245ea1142991ce1d63c3bf8ce74 ci:find-newer-dependencies:9fbe32c529ae96d751a34be093ae86e3 ci:fix-ownership:fee2c9ec9ef19686792002ae054fecdd @@ -48,11 +48,11 @@ setup:regenerate-command-images:20016a5ea492f214692c4b57c4fa9c06 setup:self-upgrade:d02f70c7a230eae3463ceec2056b63fa setup:version:123b462a421884dc2320ffc5e54b2478 setup:a3bd246c3a425f3e586d11bbdc8937cb -shell:f810c333d2c179343bbbd99872131cf0 -start-airflow:8ae982d2717006e6b73fe1f7928a19a9 +shell:e5bd8bb5b26d2f1e0748f7c65c92aefc +start-airflow:8276b0992eb3e52f001c214525551913 static-checks:7a39e28c87fbca0a9fae0ebfe1591b71 stop:8969537ccdd799f692ccb8600a7bbed6 testing:docker-compose-tests:b86c044b24138af0659a05ed6331576c testing:helm-tests:94a442e7f3f63b34c4831a84d165690a -testing:tests:6f6b7f18cde20255fe988f3a26f340cf -testing:2a93cd42229622d0eaa445c96fd1246d +testing:tests:d2961459b8b39377dd2bc6a06a1bacf0 +testing:9d83bc79d12d10f15402cb848bebcc01 diff --git a/images/breeze/output-commands.svg b/images/breeze/output-commands.svg index 763f7d8c8051d..2aab9ace34b54 100644 --- a/images/breeze/output-commands.svg +++ b/images/breeze/output-commands.svg @@ -1,4 +1,4 @@ - +
    diff --git a/images/breeze/output_shell.svg b/images/breeze/output_shell.svg index 59ee6c24f33b6..0c98aad3010ec 100644 --- a/images/breeze/output_shell.svg +++ b/images/breeze/output_shell.svg @@ -1,4 +1,4 @@ - + - - + + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - - - - Command: shell + Command: shell - + - - -Usage: breeze shell [OPTIONS] [EXTRA_ARGS]... - -Enter breeze environment. this is the default command use when no other is selected. - -╭─ Basic flags ────────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---python-pPython major/minor version used in Airflow image for images.(>3.7< | 3.8 | 3.9 | 3.10) -[default: 3.7]                                               ---backend-bDatabase backend to use.(>sqlite< | mysql | postgres | mssql)[default: sqlite] ---postgres-version-PVersion of Postgres used.(>11< | 12 | 13 | 14 | 15)[default: 11] ---mysql-version-MVersion of MySQL used.(>5.7< | 8)[default: 5.7] ---mssql-version-SVersion of MsSQL used.(>2017-latest< | 2019-latest)[default: 2017-latest] ---integrationIntegration(s) to enable when running (can be more than one).                             -(cassandra | kerberos | mongo | openldap | pinot | rabbitmq | redis | statsd | trino |    -all)                                                                                      ---forward-credentials-fForward local credentials to container when running. ---db-reset-dReset DB when entering the container. ---github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Advanced flag for running ──────────────────────────────────────────────────────────────────────────────────────────╮ ---use-airflow-versionUse (reinstall at entry) Airflow version from PyPI. It can also be `none`,      -`wheel`, or `sdist` if Airflow should be removed, installed from wheel packages -or sdist packages available in dist folder respectively. Implies                ---mount-sources `remove`.                                                       -(none | wheel | sdist | <airflow_version>)                                      ---airflow-constraints-referenceConstraint reference to use. Useful with --use-airflow-version parameter to     -specify constraints for the installed version and to find newer dependencies    -(TEXT)                                                                          ---platformPlatform for Airflow image.(linux/amd64 | linux/arm64) ---airflow-extrasAirflow extras to install when --use-airflow-version is used(TEXT) ---use-packages-from-distInstall all found packages (--package-format determines type) from 'dist'       -folder when entering breeze.                                                    ---package-formatFormat of packages that should be installed from dist.(wheel | sdist) -[default: wheel]                                       ---force-buildForce image build no matter if it is determined as needed. ---image-tag-tTag of the image which is used to run the image (implies --mount-sources=skip). -(TEXT)                                                                          -[default: latest]                                                               ---mount-sourcesChoose scope of local sources that should be mounted, skipped, or removed       -(default = selected).                                                           -(selected | all | skip | remove)                                                -[default: selected]                                                             ---include-mypy-volumeWhether to include mounting of the mypy volume (useful for debugging mypy). ---max-timeMaximum time that the command should take - if it takes longer, the command     -will fail.                                                                      -(INTEGER RANGE)                                                                 -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---verbose-vPrint verbose information about performed steps. ---dry-run-DIf dry-run is set, commands are only printed, not executed. ---answer-aForce answer to questions.(y | n | q | yes | no | quit) ---help-hShow this message and exit. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + +Usage: breeze shell [OPTIONS] [EXTRA_ARGS]... + +Enter breeze environment. this is the default command use when no other is selected. + +╭─ Basic flags ────────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--python-pPython major/minor version used in Airflow image for images.(>3.7< | 3.8 | 3.9 | 3.10) +[default: 3.7]                                               +--backend-bDatabase backend to use.(>sqlite< | mysql | postgres | mssql)[default: sqlite] +--postgres-version-PVersion of Postgres used.(>11< | 12 | 13 | 14 | 15)[default: 11] +--mysql-version-MVersion of MySQL used.(>5.7< | 8)[default: 5.7] +--mssql-version-SVersion of MsSQL used.(>2017-latest< | 2019-latest)[default: 2017-latest] +--integrationIntegration(s) to enable when running (can be more than one).                     +(cassandra | kerberos | mongo | openldap | pinot | celery | statsd | trino | all) +--forward-credentials-fForward local credentials to container when running. +--db-reset-dReset DB when entering the container. +--github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Advanced flag for running ──────────────────────────────────────────────────────────────────────────────────────────╮ +--use-airflow-versionUse (reinstall at entry) Airflow version from PyPI. It can also be `none`,      +`wheel`, or `sdist` if Airflow should be removed, installed from wheel packages +or sdist packages available in dist folder respectively. Implies                +--mount-sources `remove`.                                                       +(none | wheel | sdist | <airflow_version>)                                      +--airflow-constraints-referenceConstraint reference to use. Useful with --use-airflow-version parameter to     +specify constraints for the installed version and to find newer dependencies    +(TEXT)                                                                          +--platformPlatform for Airflow image.(linux/amd64 | linux/arm64) +--airflow-extrasAirflow extras to install when --use-airflow-version is used(TEXT) +--use-packages-from-distInstall all found packages (--package-format determines type) from 'dist'       +folder when entering breeze.                                                    +--package-formatFormat of packages that should be installed from dist.(wheel | sdist) +[default: wheel]                                       +--force-buildForce image build no matter if it is determined as needed. +--image-tag-tTag of the image which is used to run the image (implies --mount-sources=skip). +(TEXT)                                                                          +[default: latest]                                                               +--mount-sourcesChoose scope of local sources that should be mounted, skipped, or removed       +(default = selected).                                                           +(selected | all | skip | remove)                                                +[default: selected]                                                             +--include-mypy-volumeWhether to include mounting of the mypy volume (useful for debugging mypy). +--max-timeMaximum time that the command should take - if it takes longer, the command     +will fail.                                                                      +(INTEGER RANGE)                                                                 +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--answer-aForce answer to questions.(y | n | q | yes | no | quit) +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ diff --git a/images/breeze/output_start-airflow.svg b/images/breeze/output_start-airflow.svg index 958737b9fe22d..b31689a6b2177 100644 --- a/images/breeze/output_start-airflow.svg +++ b/images/breeze/output_start-airflow.svg @@ -1,4 +1,4 @@ - + - - + + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - - - - Command: start-airflow + Command: start-airflow - + - - -Usage: breeze start-airflow [OPTIONS] [EXTRA_ARGS]... - -Enter breeze environment and starts all Airflow components in the tmux session. Compile assets if contents of www  -directory changed. - -╭─ Basic flags ────────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---python-pPython major/minor version used in Airflow image for images. -(>3.7< | 3.8 | 3.9 | 3.10)                                   -[default: 3.7]                                               ---load-example-dags-eEnable configuration to load example DAGs when starting Airflow. ---load-default-connections-cEnable configuration to load default connections when starting Airflow. ---backend-bDatabase backend to use.(>sqlite< | mysql | postgres | mssql)[default: sqlite] ---platformPlatform for Airflow image.(linux/amd64 | linux/arm64) ---postgres-version-PVersion of Postgres used.(>11< | 12 | 13 | 14 | 15)[default: 11] ---mysql-version-MVersion of MySQL used.(>5.7< | 8)[default: 5.7] ---mssql-version-SVersion of MsSQL used.(>2017-latest< | 2019-latest)[default: 2017-latest] ---integrationIntegration(s) to enable when running (can be more than one).                        -(cassandra | kerberos | mongo | openldap | pinot | rabbitmq | redis | statsd | trino -| all)                                                                               ---forward-credentials-fForward local credentials to container when running. ---db-reset-dReset DB when entering the container. ---github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Asset compilation options ──────────────────────────────────────────────────────────────────────────────────────────╮ ---skip-asset-compilationSkips compilation of assets when starting airflow even if the content of www changed     -(mutually exclusive with --dev-mode).                                                    ---dev-modeStarts webserver in dev mode (assets are always recompiled in this case when starting)   -(mutually exclusive with --skip-asset-compilation).                                      -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Advanced flag for running ──────────────────────────────────────────────────────────────────────────────────────────╮ ---use-airflow-versionUse (reinstall at entry) Airflow version from PyPI. It can also be `none`,      -`wheel`, or `sdist` if Airflow should be removed, installed from wheel packages -or sdist packages available in dist folder respectively. Implies                ---mount-sources `remove`.                                                       -(none | wheel | sdist | <airflow_version>)                                      ---airflow-constraints-referenceConstraint reference to use. Useful with --use-airflow-version parameter to     -specify constraints for the installed version and to find newer dependencies    -(TEXT)                                                                          ---airflow-extrasAirflow extras to install when --use-airflow-version is used(TEXT) ---use-packages-from-distInstall all found packages (--package-format determines type) from 'dist'       -folder when entering breeze.                                                    ---package-formatFormat of packages that should be installed from dist.(wheel | sdist) -[default: wheel]                                       ---force-buildForce image build no matter if it is determined as needed. ---image-tag-tTag of the image which is used to run the image (implies --mount-sources=skip). -(TEXT)                                                                          -[default: latest]                                                               ---mount-sourcesChoose scope of local sources that should be mounted, skipped, or removed       -(default = selected).                                                           -(selected | all | skip | remove)                                                -[default: selected]                                                             -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---verbose-vPrint verbose information about performed steps. ---dry-run-DIf dry-run is set, commands are only printed, not executed. ---answer-aForce answer to questions.(y | n | q | yes | no | quit) ---help-hShow this message and exit. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + +Usage: breeze start-airflow [OPTIONS] [EXTRA_ARGS]... + +Enter breeze environment and starts all Airflow components in the tmux session. Compile assets if contents of www  +directory changed. + +╭─ Basic flags ────────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--python-pPython major/minor version used in Airflow image for images. +(>3.7< | 3.8 | 3.9 | 3.10)                                   +[default: 3.7]                                               +--load-example-dags-eEnable configuration to load example DAGs when starting Airflow. +--load-default-connections-cEnable configuration to load default connections when starting Airflow. +--backend-bDatabase backend to use.(>sqlite< | mysql | postgres | mssql)[default: sqlite] +--platformPlatform for Airflow image.(linux/amd64 | linux/arm64) +--postgres-version-PVersion of Postgres used.(>11< | 12 | 13 | 14 | 15)[default: 11] +--mysql-version-MVersion of MySQL used.(>5.7< | 8)[default: 5.7] +--mssql-version-SVersion of MsSQL used.(>2017-latest< | 2019-latest)[default: 2017-latest] +--integrationIntegration(s) to enable when running (can be more than one).                     +(cassandra | kerberos | mongo | openldap | pinot | celery | statsd | trino | all) +--forward-credentials-fForward local credentials to container when running. +--db-reset-dReset DB when entering the container. +--github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Asset compilation options ──────────────────────────────────────────────────────────────────────────────────────────╮ +--skip-asset-compilationSkips compilation of assets when starting airflow even if the content of www changed     +(mutually exclusive with --dev-mode).                                                    +--dev-modeStarts webserver in dev mode (assets are always recompiled in this case when starting)   +(mutually exclusive with --skip-asset-compilation).                                      +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Advanced flag for running ──────────────────────────────────────────────────────────────────────────────────────────╮ +--use-airflow-versionUse (reinstall at entry) Airflow version from PyPI. It can also be `none`,      +`wheel`, or `sdist` if Airflow should be removed, installed from wheel packages +or sdist packages available in dist folder respectively. Implies                +--mount-sources `remove`.                                                       +(none | wheel | sdist | <airflow_version>)                                      +--airflow-constraints-referenceConstraint reference to use. Useful with --use-airflow-version parameter to     +specify constraints for the installed version and to find newer dependencies    +(TEXT)                                                                          +--airflow-extrasAirflow extras to install when --use-airflow-version is used(TEXT) +--use-packages-from-distInstall all found packages (--package-format determines type) from 'dist'       +folder when entering breeze.                                                    +--package-formatFormat of packages that should be installed from dist.(wheel | sdist) +[default: wheel]                                       +--force-buildForce image build no matter if it is determined as needed. +--image-tag-tTag of the image which is used to run the image (implies --mount-sources=skip). +(TEXT)                                                                          +[default: latest]                                                               +--mount-sourcesChoose scope of local sources that should be mounted, skipped, or removed       +(default = selected).                                                           +(selected | all | skip | remove)                                                +[default: selected]                                                             +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--answer-aForce answer to questions.(y | n | q | yes | no | quit) +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ diff --git a/images/breeze/output_testing_tests.svg b/images/breeze/output_testing_tests.svg index 1b23af7398977..0fbf2186b49af 100644 --- a/images/breeze/output_testing_tests.svg +++ b/images/breeze/output_testing_tests.svg @@ -1,4 +1,4 @@ - + diff --git a/scripts/ci/docker-compose/integration-redis.yml b/scripts/ci/docker-compose/integration-celery.yml similarity index 75% rename from scripts/ci/docker-compose/integration-redis.yml rename to scripts/ci/docker-compose/integration-celery.yml index abefa80ff3387..4c3f4a07229b5 100644 --- a/scripts/ci/docker-compose/integration-redis.yml +++ b/scripts/ci/docker-compose/integration-celery.yml @@ -17,6 +17,17 @@ --- version: "3.7" services: + rabbitmq: + image: rabbitmq:3.7 + volumes: + - /dev/urandom:/dev/random # Required to get non-blocking entropy source + - rabbitmq-db-volume:/var/lib/rabbitmq + healthcheck: + test: rabbitmq-diagnostics -q ping + interval: 5s + timeout: 30s + retries: 50 + restart: "on-failure" redis: image: redis:5.0.1 volumes: @@ -30,13 +41,15 @@ services: timeout: 30s retries: 50 restart: "on-failure" - airflow: environment: - - INTEGRATION_REDIS=true + - INTEGRATION_CELERY=true depends_on: redis: condition: service_healthy + rabbitmq: + condition: service_healthy volumes: + rabbitmq-db-volume: redis-db-volume: diff --git a/scripts/ci/docker-compose/integration-rabbitmq.yml b/scripts/ci/docker-compose/integration-rabbitmq.yml deleted file mode 100644 index bf04b4e84fac2..0000000000000 --- a/scripts/ci/docker-compose/integration-rabbitmq.yml +++ /dev/null @@ -1,40 +0,0 @@ -# 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. ---- -version: "3.7" -services: - rabbitmq: - image: rabbitmq:3.7 - volumes: - - /dev/urandom:/dev/random # Required to get non-blocking entropy source - - rabbitmq-db-volume:/var/lib/rabbitmq - healthcheck: - test: rabbitmq-diagnostics -q ping - interval: 5s - timeout: 30s - retries: 50 - restart: "on-failure" - - airflow: - environment: - - INTEGRATION_RABBITMQ=true - depends_on: - rabbitmq: - condition: service_healthy - -volumes: - rabbitmq-db-volume: diff --git a/scripts/in_container/check_environment.sh b/scripts/in_container/check_environment.sh index de9f45cdef47c..5f18ad24eff19 100755 --- a/scripts/in_container/check_environment.sh +++ b/scripts/in_container/check_environment.sh @@ -19,8 +19,6 @@ # shellcheck source=scripts/in_container/_in_container_script_init.sh EXIT_CODE=0 -DISABLED_INTEGRATIONS="" - # We want to avoid misleading messages and perform only forward lookup of the service IP address. # Netcat when run without -n performs both forward and reverse lookup and fails if the reverse # lookup name does not match the original name even if the host is reachable via IP. This happens @@ -76,23 +74,6 @@ function check_service { fi } -function check_integration { - local integration_label=$1 - local integration_name=$2 - local call=$3 - local max_check=${4:=1} - - local env_var_name - env_var_name=INTEGRATION_${integration_name^^} - if [[ ${!env_var_name:=} != "true" || ${!env_var_name} != "True" ]]; then - if [[ ! ${DISABLED_INTEGRATIONS} == *" ${integration_name}"* ]]; then - DISABLED_INTEGRATIONS="${DISABLED_INTEGRATIONS} ${integration_name}" - fi - return - fi - check_service "${integration_label}" "${call}" "${max_check}" -} - function check_db_backend { local max_check=${1:=1} @@ -144,30 +125,44 @@ function startairflow_if_requested() { } echo -echo "${COLOR_BLUE}Checking integrations and backends.${COLOR_RESET}" +echo "${COLOR_BLUE}Checking backend and integrations.${COLOR_RESET}" echo if [[ -n ${BACKEND=} ]]; then check_db_backend 50 fi echo -check_integration "Kerberos" "kerberos" "run_nc kdc-server-example-com 88" 50 -check_integration "MongoDB" "mongo" "run_nc mongo 27017" 50 -check_integration "Redis" "redis" "run_nc redis 6379" 50 -check_integration "Cassandra" "cassandra" "run_nc cassandra 9042" 50 -check_integration "OpenLDAP" "openldap" "run_nc openldap 389" 50 -check_integration "Trino (HTTP)" "trino" "run_nc trino 8080" 50 -check_integration "Trino (HTTPS)" "trino" "run_nc trino 7778" 50 -check_integration "Trino (API)" "trino" \ - "curl --max-time 1 http://trino:8080/v1/info/ | grep '\"starting\":false'" 50 -check_integration "Pinot (HTTP)" "pinot" "run_nc pinot 9000" 50 -CMD="curl --max-time 1 -X GET 'http://pinot:9000/health' -H 'accept: text/plain' | grep OK" -check_integration "Pinot (Controller API)" "pinot" "${CMD}" 50 -CMD="curl --max-time 1 -X GET 'http://pinot:9000/pinot-controller/admin' -H 'accept: text/plain' | grep GOOD" -check_integration "Pinot (Controller API)" "pinot" "${CMD}" 50 -CMD="curl --max-time 1 -X GET 'http://pinot:8000/health' -H 'accept: text/plain' | grep OK" -check_integration "Pinot (Broker API)" "pinot" "${CMD}" 50 -check_integration "RabbitMQ" "rabbitmq" "run_nc rabbitmq 5672" 50 + +if [[ ${INTEGRATION_KERBEROS} == "true" ]]; then + check_service "Kerberos" "run_nc kdc-server-example-com 88" 50 +fi +if [[ ${INTEGRATION_MONGO} == "true" ]]; then + check_service "MongoDB" "run_nc mongo 27017" 50 +fi +if [[ ${INTEGRATION_CELERY} == "true" ]]; then + check_service "Redis" "run_nc redis 6379" 50 + check_service "RabbitMQ" "run_nc rabbitmq 5672" 50 +fi +if [[ ${INTEGRATION_CASSANDRA} == "true" ]]; then + check_service "Cassandra" "run_nc cassandra 9042" 50 +fi +if [[ ${INTEGRATION_OPENLDAP} == "true" ]]; then + check_service "OpenLDAP" "run_nc openldap 389" 50 +fi +if [[ ${INTEGRATION_TRINO} == "true" ]]; then + check_service "Trino (HTTP)" "run_nc trino 8080" 50 + check_service "Trino (HTTPS)" "run_nc trino 7778" 50 + check_service "Trino (API)" "curl --max-time 1 http://trino:8080/v1/info/ | grep '\"starting\":false'" 50 +fi +if [[ ${INTEGRATION_PINOT} == "true" ]]; then + check_service "Pinot (HTTP)" "run_nc pinot 9000" 50 + CMD="curl --max-time 1 -X GET 'http://pinot:9000/health' -H 'accept: text/plain' | grep OK" + check_service "Pinot (Controller API)" "${CMD}" 50 + CMD="curl --max-time 1 -X GET 'http://pinot:9000/pinot-controller/admin' -H 'accept: text/plain' | grep GOOD" + check_service "Pinot (Controller API)" "${CMD}" 50 + CMD="curl --max-time 1 -X GET 'http://pinot:8000/health' -H 'accept: text/plain' | grep OK" + check_service "Pinot (Broker API)" "${CMD}" 50 +fi if [[ ${EXIT_CODE} != 0 ]]; then echo @@ -180,9 +175,3 @@ fi resetdb_if_requested startairflow_if_requested - -if [[ -n ${DISABLED_INTEGRATIONS=} && (${VERBOSE=} == "true" || ${VERBOSE} == "True") ]]; then - echo - echo "${COLOR_BLUE}Those integrations are disabled: ${DISABLED_INTEGRATIONS}${COLOR_RESET}" - echo -fi diff --git a/tests/integration/cli/commands/test_celery_command.py b/tests/integration/cli/commands/test_celery_command.py index 3cc9da0a13473..306d5fcfd88d9 100644 --- a/tests/integration/cli/commands/test_celery_command.py +++ b/tests/integration/cli/commands/test_celery_command.py @@ -26,8 +26,7 @@ from tests.test_utils.config import conf_vars -@pytest.mark.integration("redis") -@pytest.mark.integration("rabbitmq") +@pytest.mark.integration("celery") @pytest.mark.backend("mysql", "postgres") class TestWorkerServeLogs: @classmethod diff --git a/tests/integration/executors/test_celery_executor.py b/tests/integration/executors/test_celery_executor.py index f7f69fcfce789..2c0e578750f45 100644 --- a/tests/integration/executors/test_celery_executor.py +++ b/tests/integration/executors/test_celery_executor.py @@ -93,8 +93,7 @@ def _prepare_app(broker_url=None, execute=None): set_event_loop(None) -@pytest.mark.integration("redis") -@pytest.mark.integration("rabbitmq") +@pytest.mark.integration("celery") @pytest.mark.backend("mysql", "postgres") class TestCeleryExecutor: def setup_method(self) -> None: @@ -260,8 +259,7 @@ def __ne__(self, other): return not self.__eq__(other) -@pytest.mark.integration("redis") -@pytest.mark.integration("rabbitmq") +@pytest.mark.integration("celery") @pytest.mark.backend("mysql", "postgres") class TestBulkStateFetcher(unittest.TestCase): @mock.patch( diff --git a/tests/integration/providers/redis/hooks/test_redis.py b/tests/integration/providers/redis/hooks/test_redis.py index eac17ee676edc..d7760933f2143 100644 --- a/tests/integration/providers/redis/hooks/test_redis.py +++ b/tests/integration/providers/redis/hooks/test_redis.py @@ -22,7 +22,7 @@ from airflow.providers.redis.hooks.redis import RedisHook -@pytest.mark.integration("redis") +@pytest.mark.integration("celery") class TestRedisHook: def test_real_ping(self): hook = RedisHook(redis_conn_id="redis_default") diff --git a/tests/integration/providers/redis/operators/test_redis_publish.py b/tests/integration/providers/redis/operators/test_redis_publish.py index c5ea8a65bd626..44678369124d6 100644 --- a/tests/integration/providers/redis/operators/test_redis_publish.py +++ b/tests/integration/providers/redis/operators/test_redis_publish.py @@ -29,7 +29,7 @@ DEFAULT_DATE = timezone.datetime(2017, 1, 1) -@pytest.mark.integration("redis") +@pytest.mark.integration("celery") class TestRedisPublishOperator: def setup_method(self): args = {"owner": "airflow", "start_date": DEFAULT_DATE} diff --git a/tests/integration/providers/redis/sensors/test_redis_key.py b/tests/integration/providers/redis/sensors/test_redis_key.py index f54e816c29a50..76034a1be77dc 100644 --- a/tests/integration/providers/redis/sensors/test_redis_key.py +++ b/tests/integration/providers/redis/sensors/test_redis_key.py @@ -27,7 +27,7 @@ DEFAULT_DATE = timezone.datetime(2017, 1, 1) -@pytest.mark.integration("redis") +@pytest.mark.integration("celery") class TestRedisSensor: def setup_method(self): args = {"owner": "airflow", "start_date": DEFAULT_DATE} diff --git a/tests/integration/providers/redis/sensors/test_redis_pub_sub.py b/tests/integration/providers/redis/sensors/test_redis_pub_sub.py index e99f2a38d9914..e50967328f3ef 100644 --- a/tests/integration/providers/redis/sensors/test_redis_pub_sub.py +++ b/tests/integration/providers/redis/sensors/test_redis_pub_sub.py @@ -30,7 +30,7 @@ DEFAULT_DATE = timezone.datetime(2017, 1, 1) -@pytest.mark.integration("redis") +@pytest.mark.integration("celery") class TestRedisPubSubSensor: def setup_method(self): args = {"owner": "airflow", "start_date": DEFAULT_DATE} From d879ddefd7e2e2072dcbbc1ef639568bb5253c02 Mon Sep 17 00:00:00 2001 From: Jarek Potiuk Date: Thu, 8 Dec 2022 09:43:49 +0100 Subject: [PATCH 156/158] Integration tests are separated into separate command and CI job (#28207) Integration tests so far were a separate test type among the unit tests, however we have to start them differently. This PR introduces new command in Breeze: breeze testing integration-tests The `--integration` option has been removed from the regular unit tests, and now it is used to the integration-tests command. The integration-tests command has no parallel option. (cherry picked from commit b37452e0a2aea5065bb08fd9211089814c07b804) --- .github/workflows/ci.yml | 108 ++++- BREEZE.rst | 30 +- Dockerfile.ci | 8 +- TESTING.rst | 437 +++++++++--------- .../commands/testing_commands.py | 108 +++-- .../commands/testing_commands_config.py | 25 +- .../src/airflow_breeze/global_constants.py | 5 - .../src/airflow_breeze/params/shell_params.py | 13 +- .../utils/docker_command_utils.py | 2 - .../airflow_breeze/utils/selective_checks.py | 6 - dev/breeze/tests/test_selective_checks.py | 26 +- images/breeze/output-commands-hash.txt | 15 +- images/breeze/output-commands.svg | 4 +- ...output_setup_regenerate-command-images.svg | 28 +- images/breeze/output_shell.svg | 4 +- images/breeze/output_start-airflow.svg | 4 +- images/breeze/output_testing.svg | 16 +- .../output_testing_integration-tests.svg | 184 ++++++++ images/breeze/output_testing_tests.svg | 96 ++-- scripts/ci/docker-compose/_docker.env | 2 - scripts/ci/docker-compose/base.yml | 2 - scripts/ci/docker-compose/devcontainer.env | 2 - .../docker-compose/integration-openldap.yml | 46 -- .../openldap/ldif/01-users.example.com.ldif | 48 -- .../openldap/ldif/02-groups.example.com.ldif | 44 -- .../openldap/ldif/03-manager.example.com.ldif | 21 - scripts/ci/openldap/ldif/04-rootdn.ldif | 24 - scripts/ci/openldap/slapd.conf | 64 --- scripts/docker/entrypoint_ci.sh | 8 +- scripts/in_container/check_environment.sh | 3 - tests/conftest.py | 2 +- 31 files changed, 738 insertions(+), 647 deletions(-) create mode 100644 images/breeze/output_testing_integration-tests.svg delete mode 100644 scripts/ci/docker-compose/integration-openldap.yml delete mode 100644 scripts/ci/openldap/ldif/01-users.example.com.ldif delete mode 100644 scripts/ci/openldap/ldif/02-groups.example.com.ldif delete mode 100644 scripts/ci/openldap/ldif/03-manager.example.com.ldif delete mode 100644 scripts/ci/openldap/ldif/04-rootdn.ldif delete mode 100644 scripts/ci/openldap/slapd.conf diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index 28038a9298ca8..d4163671bde51 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -826,7 +826,7 @@ jobs: uses: ./.github/actions/prepare_breeze_and_image - name: "Migration Tests: ${{matrix.python-version}}:${{needs.build-info.outputs.test-types}}" uses: ./.github/actions/migration_tests - - name: "Tests: ${{matrix.python-version}}:${{needs.build-info.outputs.test-types}}" + - name: "Tests: ${{matrix.python-version}}:${{needs.build-info.outputs.test-types}} (w/Kerberos)" run: breeze testing tests --run-in-parallel - name: "Post Tests: ${{matrix.python-version}}:${{needs.build-info.outputs.test-types}}" uses: ./.github/actions/post_tests @@ -954,6 +954,104 @@ jobs: - name: "Post Tests: ${{matrix.python-version}}:${{needs.build-info.outputs.test-types}}" uses: ./.github/actions/post_tests + tests-integration-postgres: + timeout-minutes: 130 + name: Integration Tests Postgres + runs-on: "${{needs.build-info.outputs.runs-on}}" + needs: [build-info, wait-for-ci-images] + env: + RUNS_ON: "${{needs.build-info.outputs.runs-on}}" + TEST_TYPES: "${{needs.build-info.outputs.test-types}}" + PR_LABELS: "${{needs.build-info.outputs.pull-request-labels}}" + FULL_TESTS_NEEDED: "${{needs.build-info.outputs.full-tests-needed}}" + DEBUG_RESOURCES: "${{needs.build-info.outputs.debug-resources}}" + BACKEND: "postgres" + PYTHON_MAJOR_MINOR_VERSION: "${{needs.build-info.outputs.default-python-version}}" + POSTGRES_VERSION: "${{needs.build-info.outputs.default-postgres-version}}" + BACKEND_VERSION: "${{needs.build-info.outputs.default-python-version}}" + JOB_ID: "integration" + COVERAGE: "${{needs.build-info.outputs.run-coverage}}" + if: needs.build-info.outputs.run-tests == 'true' && needs.build-info.outputs.default-branch == 'main' + steps: + - name: Cleanup repo + shell: bash + run: docker run -v "${GITHUB_WORKSPACE}:/workspace" -u 0:0 bash -c "rm -rf /workspace/*" + - name: "Checkout ${{ github.ref }} ( ${{ github.sha }} )" + uses: actions/checkout@v3 + with: + persist-credentials: false + - name: "Prepare breeze & CI image: ${{env.PYTHON_MAJOR_MINOR_VERSION}}:${{env.IMAGE_TAG}}" + uses: ./.github/actions/prepare_breeze_and_image + - name: "Integration Tests Postgres: cassandra" + run: | + breeze testing integration-tests --integration cassandra + breeze stop + if: needs.build-info.outputs.runs-on != 'self-hosted' + - name: "Integration Tests Postgres: mongo" + run: | + breeze testing integration-tests --integration mongo + breeze stop + if: needs.build-info.outputs.runs-on != 'self-hosted' + - name: "Integration Tests Postgres: pinot" + run: | + breeze testing integration-tests --integration pinot + breeze stop + if: needs.build-info.outputs.runs-on != 'self-hosted' + - name: "Integration Tests Postgres: celery" + run: | + breeze testing integration-tests --integration celery + breeze stop + if: needs.build-info.outputs.runs-on != 'self-hosted' + - name: "Integration Tests Postgres: trino, kerberos" + run: | + breeze testing integration-tests --integration trino --integration kerberos + breeze stop + if: needs.build-info.outputs.runs-on != 'self-hosted' + - name: "Integration Tests Postgres: all" + run: breeze testing integration-tests --integration all + if: needs.build-info.outputs.runs-on == 'self-hosted' + - name: "Post Tests: ${{matrix.python-version}}:${{needs.build-info.outputs.test-types}}" + uses: ./.github/actions/post_tests + + tests-integration-mysql: + timeout-minutes: 130 + name: Integration Tests MySQL + runs-on: "${{needs.build-info.outputs.runs-on}}" + needs: [build-info, wait-for-ci-images] + env: + RUNS_ON: "${{needs.build-info.outputs.runs-on}}" + TEST_TYPES: "${{needs.build-info.outputs.test-types}}" + PR_LABELS: "${{needs.build-info.outputs.pull-request-labels}}" + FULL_TESTS_NEEDED: "${{needs.build-info.outputs.full-tests-needed}}" + DEBUG_RESOURCES: "${{needs.build-info.outputs.debug-resources}}" + BACKEND: "postgres" + PYTHON_MAJOR_MINOR_VERSION: "${{needs.build-info.outputs.default-python-version}}" + POSTGRES_VERSION: "${{needs.build-info.outputs.default-postgres-version}}" + BACKEND_VERSION: "${{needs.build-info.outputs.default-python-version}}" + JOB_ID: "integration" + COVERAGE: "${{needs.build-info.outputs.run-coverage}}" + if: needs.build-info.outputs.run-tests == 'true' && needs.build-info.outputs.default-branch == 'main' + steps: + - name: Cleanup repo + shell: bash + run: docker run -v "${GITHUB_WORKSPACE}:/workspace" -u 0:0 bash -c "rm -rf /workspace/*" + if: needs.build-info.outputs.runs-on == 'self-hosted' + - name: "Checkout ${{ github.ref }} ( ${{ github.sha }} )" + uses: actions/checkout@v3 + with: + persist-credentials: false + if: needs.build-info.outputs.runs-on == 'self-hosted' + - name: "Prepare breeze & CI image: ${{env.PYTHON_MAJOR_MINOR_VERSION}}:${{env.IMAGE_TAG}}" + uses: ./.github/actions/prepare_breeze_and_image + if: needs.build-info.outputs.runs-on == 'self-hosted' + - name: "Integration Tests MySQL: all" + run: breeze testing integration-tests --integration all + if: needs.build-info.outputs.runs-on == 'self-hosted' + - name: "Post Tests: ${{matrix.python-version}}:${{needs.build-info.outputs.test-types}}" + uses: ./.github/actions/post_tests + if: needs.build-info.outputs.runs-on == 'self-hosted' + + tests-quarantined: timeout-minutes: 60 name: "Quarantined tests" @@ -1001,6 +1099,8 @@ jobs: - tests-mysql - tests-mssql - tests-quarantined + - tests-integration-postgres + - tests-integration-mysql env: RUNS_ON: "${{needs.build-info.outputs.runs-on}}" # Only upload coverage on merges to main @@ -1035,6 +1135,8 @@ jobs: - tests-mysql - tests-mssql - tests-quarantined + - tests-integration-postgres + - tests-integration-mysql env: RUNS_ON: "${{needs.build-info.outputs.runs-on}}" steps: @@ -1198,6 +1300,8 @@ jobs: - tests-mysql - tests-mssql - tests-postgres + - tests-integration-postgres + - tests-integration-mysql - push-early-buildx-cache-to-github-registry env: RUNS_ON: "${{needs.build-info.outputs.runs-on}}" @@ -1358,6 +1462,8 @@ jobs: - tests-mysql - tests-mssql - tests-postgres + - tests-integration-postgres + - tests-integration-mysql env: DEFAULT_BRANCH: ${{ needs.build-info.outputs.default-branch }} DEFAULT_CONSTRAINTS_BRANCH: ${{ needs.build-info.outputs.default-constraints-branch }} diff --git a/BREEZE.rst b/BREEZE.rst index 202a89d2518ae..341f8b8dbeede 100644 --- a/BREEZE.rst +++ b/BREEZE.rst @@ -687,12 +687,12 @@ on how to run them. This applies to all kind of tests - all our tests can be run using pytest. -Running unit/integration tests in groups -........................................ +Running unit tests +.................. -Another option you have is that you can also run tests via built-in ``breeze testing`` command. +Another option you have is that you can also run tests via built-in ``breeze testing tests`` command. The iterative ``pytest`` command allows to run test individually, or by class or in any other way -pytest allows to test them and run them interactively, but ``breeze testing`` command allows to +pytest allows to test them and run them interactively, but ``breeze testing tests`` command allows to run the tests in the same test "types" that are used to run the tests in CI: for example Core, Always API, Providers. This how our CI runs them - running each group in parallel to other groups and you can replicate this behaviour. @@ -724,6 +724,28 @@ Here is the detailed set of options for the ``breeze testing tests`` command. :width: 100% :alt: Breeze testing tests +Running integration tests +......................... + +You can also run integration tests via built-in ``breeze testing integration-tests`` command. Some of our +tests require additional integrations to be started in docker-compose. The integration tests command will +run the expected integration and tests that need that integration. + +For example this will only run kerberos tests: + +.. code-block:: bash + + breeze testing integration-tests --integration Kerberos + + +Here is the detailed set of options for the ``breeze testing integration-tests`` command. + +.. image:: ./images/breeze/output_testing_integration-tests.svg + :target: https://raw.githubusercontent.com/apache/airflow/main/images/breeze/output_testing_integration_tests.svg + :width: 100% + :alt: Breeze testing integration-tests + + Running Helm tests .................. diff --git a/Dockerfile.ci b/Dockerfile.ci index e645b332426ca..cc28c355d916d 100644 --- a/Dockerfile.ci +++ b/Dockerfile.ci @@ -971,13 +971,7 @@ fi readonly SELECTED_TESTS CLI_TESTS API_TESTS PROVIDERS_TESTS CORE_TESTS WWW_TESTS \ ALL_TESTS ALL_PRESELECTED_TESTS -if [[ -n ${LIST_OF_INTEGRATION_TESTS_TO_RUN=} ]]; then - # Integration tests - for INT in ${LIST_OF_INTEGRATION_TESTS_TO_RUN} - do - EXTRA_PYTEST_ARGS+=("--integration" "${INT}") - done -elif [[ ${TEST_TYPE:=""} == "Long" ]]; then +if [[ ${TEST_TYPE:=""} == "Long" ]]; then EXTRA_PYTEST_ARGS+=( "-m" "long_running" "--include-long-running" diff --git a/TESTING.rst b/TESTING.rst index 7520d58811618..7cbb9d49c9057 100644 --- a/TESTING.rst +++ b/TESTING.rst @@ -58,6 +58,41 @@ Follow the guidelines when writing unit tests: **NOTE:** We plan to convert all unit tests to standard "asserts" semi-automatically, but this will be done later in Airflow 2.0 development phase. That will include setUp/tearDown/context managers and decorators. +Airflow test types +------------------ + +Airflow tests in the CI environment are split into several test types: + +* Always - those are tests that should be always executed (always folder) +* Core - for the core Airflow functionality (core folder) +* API - Tests for the Airflow API (api and api_connexion folders) +* CLI - Tests for the Airflow CLI (cli folder) +* WWW - Tests for the Airflow webserver (www folder) +* Providers - Tests for all Providers of Airflow (providers folder) +* Other - all other tests (all other folders that are not part of any of the above) + +This is done for three reasons: + +1. in order to selectively run only subset of the test types for some PRs +2. in order to allow parallel execution of the tests on Self-Hosted runners + +For case 2. We can utilise memory and CPUs available on both CI and local development machines to run +test in parallel. This way we can decrease the time of running all tests in self-hosted runners from +60 minutes to ~15 minutes. + +.. note:: + + We need to split tests manually into separate suites rather than utilise + ``pytest-xdist`` or ``pytest-parallel`` which could be a simpler and much more "native" parallelization + mechanism. Unfortunately, we cannot utilise those tools because our tests are not truly ``unit`` tests that + can run in parallel. A lot of our tests rely on shared databases - and they update/reset/cleanup the + databases while they are executing. They are also exercising features of the Database such as locking which + further increases cross-dependency between tests. Until we make all our tests truly unit tests (and not + touching the database or until we isolate all such tests to a separate test type, we cannot really rely on + frameworks that run tests in parallel. In our solution each of the test types is run in parallel with its + own database (!) so when we have 8 test types running in parallel, there are in fact 8 databases run + behind the scenes to support them and each of the test types executes its own tests sequentially. + Running Unit Tests from PyCharm IDE ----------------------------------- @@ -111,8 +146,8 @@ To run unit tests from the Visual Studio Code: :align: center :alt: Running tests -Running Unit Tests ------------------- +Running Unit Tests in local virtualenv +-------------------------------------- To run unit, integration, and system tests from the Breeze and your virtualenv, you can use the `pytest `_ framework. @@ -162,8 +197,8 @@ for debugging purposes, enter: pytest --log-cli-level=DEBUG tests/core/test_core.py::TestCore -Running Tests for a Specified Target Using Breeze from the Host ---------------------------------------------------------------- +Running Tests using Breeze from the Host +---------------------------------------- If you wish to only run tests and not to drop into the shell, apply the ``tests`` command. You can add extra targets and pytest flags after the ``--`` command. Note that @@ -204,7 +239,6 @@ You can also limit the set of providers you would like to run tests of breeze testing tests --test-type "Providers[airbyte,http]" - Running Tests of a specified type from the Host ----------------------------------------------- @@ -231,7 +265,7 @@ kinds of test types: breeze testing tests --test-type Providers --db-reset tests -* Special kinds of tests - Integration, Quarantined, Postgres, MySQL, which are marked with pytest +* Special kinds of tests Quarantined, Postgres, MySQL, which are marked with pytest marks and for those you need to select the type using test-type switch. If you want to run such tests using breeze, you need to pass appropriate ``--test-type`` otherwise the test will be skipped. Similarly to the per-directory tests if you do not specify the test or tests to run, @@ -249,99 +283,160 @@ kinds of test types: breeze testing tests --test-type Quarantined tests --db-reset -Helm Unit Tests -=============== -On the Airflow Project, we have decided to stick with pythonic testing for our Helm chart. This makes our chart -easier to test, easier to modify, and able to run with the same testing infrastructure. To add Helm unit tests -add them in ``tests/charts``. +Running full Airflow unit test suite in parallel +------------------------------------------------ + +If you run ``breeze testing tests --run-in-parallel`` tests run in parallel +on your development machine - maxing out the number of parallel runs at the number of cores you +have available in your Docker engine. + +In case you do not have enough memory available to your Docker (8 GB), the ``Integration``. ``Provider`` +and ``Core`` test type are executed sequentially with cleaning the docker setup in-between. This +allows to print + +This allows for massive speedup in full test execution. On 8 CPU machine with 16 cores and 64 GB memory +and fast SSD disk, the whole suite of tests completes in about 5 minutes (!). Same suite of tests takes +more than 30 minutes on the same machine when tests are run sequentially. + +.. note:: + + On MacOS you might have less CPUs and less memory available to run the tests than you have in the host, + simply because your Docker engine runs in a Linux Virtual Machine under-the-hood. If you want to make + use of the parallelism and memory usage for the CI tests you might want to increase the resources available + to your docker engine. See the `Resources `_ chapter + in the ``Docker for Mac`` documentation on how to do it. + +You can also limit the parallelism by specifying the maximum number of parallel jobs via +MAX_PARALLEL_TEST_JOBS variable. If you set it to "1", all the test types will be run sequentially. + +.. code-block:: bash + + MAX_PARALLEL_TEST_JOBS="1" ./scripts/ci/testing/ci_run_airflow_testing.sh + +.. note:: + + In case you would like to cleanup after execution of such tests you might have to cleanup + some of the docker containers running in case you use ctrl-c to stop execution. You can easily do it by + running this command (it will kill all docker containers running so do not use it if you want to keep some + docker containers running): + + .. code-block:: bash + + docker kill $(docker ps -q) + +Running Backend-Specific Tests +------------------------------ + +Tests that are using a specific backend are marked with a custom pytest marker ``pytest.mark.backend``. +The marker has a single parameter - the name of a backend. It corresponds to the ``--backend`` switch of +the Breeze environment (one of ``mysql``, ``sqlite``, or ``postgres``). Backend-specific tests only run when +the Breeze environment is running with the right backend. If you specify more than one backend +in the marker, the test runs for all specified backends. + +Example of the ``postgres`` only test: .. code-block:: python - class TestBaseChartTest: + @pytest.mark.backend("postgres") + def test_copy_expert(self): ... -To render the chart create a YAML string with the nested dictionary of options you wish to test. You can then -use our ``render_chart`` function to render the object of interest into a testable Python dictionary. Once the chart -has been rendered, you can use the ``render_k8s_object`` function to create a k8s model object. It simultaneously -ensures that the object created properly conforms to the expected resource spec and allows you to use object values -instead of nested dictionaries. -Example test here: +Example of the ``postgres,mysql`` test (they are skipped with the ``sqlite`` backend): .. code-block:: python - from tests.charts.helm_template_generator import render_chart, render_k8s_object + @pytest.mark.backend("postgres", "mysql") + def test_celery_executor(self): + ... - git_sync_basic = """ - dags: - gitSync: - enabled: true - """ +You can use the custom ``--backend`` switch in pytest to only run tests specific for that backend. +Here is an example of running only postgres-specific backend tests: - class TestGitSyncScheduler: - def test_basic(self): - helm_settings = yaml.safe_load(git_sync_basic) - res = render_chart( - "GIT-SYNC", - helm_settings, - show_only=["templates/scheduler/scheduler-deployment.yaml"], - ) - dep: k8s.V1Deployment = render_k8s_object(res[0], k8s.V1Deployment) - assert "dags" == dep.spec.template.spec.volumes[1].name +.. code-block:: bash + pytest --backend postgres -To execute all Helm tests using breeze command and utilize parallel pytest tests, you can run the -following command (but it takes quite a long time even in a multi-processor machine). +Running Long-running tests +-------------------------- -.. code-block:: bash +Some of the tests rung for a long time. Such tests are marked with ``@pytest.mark.long_running`` annotation. +Those tests are skipped by default. You can enable them with ``--include-long-running`` flag. You +can also decide to only run tests with ``-m long-running`` flags to run only those tests. - breeze testing helm-tests +Running Quarantined tests +------------------------- -You can also run Helm tests individually via the usual ``breeze`` command. Just enter breeze and run the -tests with pytest as you would do with regular unit tests (you can add ``-n auto`` command to run Helm -tests in parallel - unlike most of the regular unit tests of ours that require a database, the Helm tests are -perfectly safe to be run in parallel (and if you have multiple processors, you can gain significant -speedups when using parallel runs): +Some of our tests are quarantined. This means that this test will be run in isolation and that it will be +re-run several times. Also when quarantined tests fail, the whole test suite will not fail. The quarantined +tests are usually flaky tests that need some attention and fix. -.. code-block:: bash +Those tests are marked with ``@pytest.mark.quarantined`` annotation. +Those tests are skipped by default. You can enable them with ``--include-quarantined`` flag. You +can also decide to only run tests with ``-m quarantined`` flag to run only those tests. - breeze +Running Tests with provider packages +------------------------------------ -This enters breeze container. +Airflow 2.0 introduced the concept of splitting the monolithic Airflow package into separate +providers packages. The main "apache-airflow" package contains the bare Airflow implementation, +and additionally we have 70+ providers that we can install additionally to get integrations with +external services. Those providers live in the same monorepo as Airflow, but we build separate +packages for them and the main "apache-airflow" package does not contain the providers. -.. code-block:: bash +Most of the development in Breeze happens by iterating on sources and when you run +your tests during development, you usually do not want to build packages and install them separately. +Therefore by default, when you enter Breeze airflow and all providers are available directly from +sources rather than installed from packages. This is for example to test the "provider discovery" +mechanism available that reads provider information from the package meta-data. - pytest tests/charts -n auto +When Airflow is run from sources, the metadata is read from provider.yaml +files, but when Airflow is installed from packages, it is read via the package entrypoint +``apache_airflow_provider``. -This runs all chart tests using all processors you have available. +By default, all packages are prepared in wheel format. To install Airflow from packages you +need to run the following steps: + +1. Prepare provider packages .. code-block:: bash - pytest tests/charts/test_airflow_common.py -n auto + breeze release-management prepare-provider-packages [PACKAGE ...] -This will run all tests from ``tests_airflow_common.py`` file using all processors you have available. +If you run this command without packages, you will prepare all packages. However, You can specify +providers that you would like to build if you just want to build few provider packages. +The packages are prepared in ``dist`` folder. Note that this command cleans up the ``dist`` folder +before running, so you should run it before generating ``apache-airflow`` package. + +2. Prepare airflow packages .. code-block:: bash - pytest tests/charts/test_airflow_common.py + breeze release-management prepare-airflow-package -This will run all tests from ``tests_airflow_common.py`` file sequentially. +This prepares airflow .whl package in the dist folder. + +3. Enter breeze installing both airflow and providers from the dist packages + +.. code-block:: bash + + breeze --use-airflow-version wheel --use-packages-from-dist --skip-mounting-local-sources Airflow Integration Tests ========================= Some of the tests in Airflow are integration tests. These tests require ``airflow`` Docker -image and extra images with integrations (such as ``redis``, ``mongodb``, etc.). - +image and extra images with integrations (such as ``celery``, ``mongodb``, etc.). +The integration tests are all stored in the ``tests/integration`` folder. Enabling Integrations --------------------- Airflow integration tests cannot be run in the local virtualenv. They can only run in the Breeze -environment with enabled integrations and in the CI. See `<.github/workflows/ci.yml>`_ for details about Airflow CI. +environment with enabled integrations and in the CI. See `CI `_ for details about Airflow CI. When you are in the Breeze environment, by default, all integrations are disabled. This enables only true unit tests to be executed in Breeze. You can enable the integration by passing the ``--integration `` @@ -350,8 +445,7 @@ or using the ``--integration all`` switch that enables all integrations. NOTE: Every integration requires a separate container with the corresponding integration image. These containers take precious resources on your PC, mainly the memory. The started integrations are not stopped -until you stop the Breeze environment with the ``stop`` command and restart it -via ``restart`` command. +until you stop the Breeze environment with the ``stop`` command and started with the ``start`` command. The following integrations are available: @@ -367,13 +461,9 @@ The following integrations are available: - Integration that provides Kerberos authentication * - mongo - Integration required for MongoDB hooks - * - openldap - - Integration required for OpenLDAP hooks * - pinot - Integration required for Apache Pinot hooks - * - rabbitmq - - Integration required for Celery executor tests - * - redis + * - celery - Integration required for Celery executor tests * - trino - Integration required for Trino hooks @@ -396,10 +486,6 @@ To start all integrations, enter: breeze --integration all -In the CI environment, integrations can be enabled by specifying the ``ENABLED_INTEGRATIONS`` variable -storing a space-separated list of integrations to start. Thanks to that, we can run integration and -integration-less tests separately in different jobs, which is desired from the memory usage point of view. - Note that Kerberos is a special kind of integration. Some tests run differently when Kerberos integration is enabled (they retrieve and use a Kerberos authentication token) and differently when the Kerberos integration is disabled (they neither retrieve nor use the token). Therefore, one of the test jobs @@ -411,11 +497,11 @@ Running Integration Tests All tests using an integration are marked with a custom pytest marker ``pytest.mark.integration``. The marker has a single parameter - the name of integration. -Example of the ``redis`` integration test: +Example of the ``celery`` integration test: .. code-block:: python - @pytest.mark.integration("redis") + @pytest.mark.integration("celery") def test_real_ping(self): hook = RedisHook(redis_conn_id="redis_default") redis = hook.get_conn() @@ -439,204 +525,121 @@ To run only ``mongo`` integration tests: .. code-block:: bash - pytest --integration mongo - -To run integration tests for ``mongo`` and ``rabbitmq``: - -.. code-block:: bash - - pytest --integration mongo --integration rabbitmq - -Note that collecting all tests takes some time. So, if you know where your tests are located, you can -speed up the test collection significantly by providing the folder where the tests are located. + pytest --integration mongo tests/integration -Here is an example of the collection limited to the ``providers/apache`` directory: +To run integration tests for ``mongo`` and ``celery``: .. code-block:: bash - pytest --integration cassandra tests/providers/apache/ - -Running Backend-Specific Tests ------------------------------- - -Tests that are using a specific backend are marked with a custom pytest marker ``pytest.mark.backend``. -The marker has a single parameter - the name of a backend. It corresponds to the ``--backend`` switch of -the Breeze environment (one of ``mysql``, ``sqlite``, or ``postgres``). Backend-specific tests only run when -the Breeze environment is running with the right backend. If you specify more than one backend -in the marker, the test runs for all specified backends. - -Example of the ``postgres`` only test: - -.. code-block:: python - - @pytest.mark.backend("postgres") - def test_copy_expert(self): - ... - - -Example of the ``postgres,mysql`` test (they are skipped with the ``sqlite`` backend): - -.. code-block:: python - - @pytest.mark.backend("postgres", "mysql") - def test_celery_executor(self): - ... + pytest --integration mongo --integration celery tests/integration -You can use the custom ``--backend`` switch in pytest to only run tests specific for that backend. -Here is an example of running only postgres-specific backend tests: +Here is an example of the collection limited to the ``providers/apache`` sub-directory: .. code-block:: bash - pytest --backend postgres - -Running Long-running tests --------------------------- + pytest --integration cassandra tests/integrations/providers/apache -Some of the tests rung for a long time. Such tests are marked with ``@pytest.mark.long_running`` annotation. -Those tests are skipped by default. You can enable them with ``--include-long-running`` flag. You -can also decide to only run tests with ``-m long-running`` flags to run only those tests. +Running Integration Tests from the Host +--------------------------------------- -Quarantined tests ------------------ +You can also run integration tests using Breeze from the host. -Some of our tests are quarantined. This means that this test will be run in isolation and that it will be -re-run several times. Also when quarantined tests fail, the whole test suite will not fail. The quarantined -tests are usually flaky tests that need some attention and fix. +Runs all integration tests: -Those tests are marked with ``@pytest.mark.quarantined`` annotation. -Those tests are skipped by default. You can enable them with ``--include-quarantined`` flag. You -can also decide to only run tests with ``-m quarantined`` flag to run only those tests. + .. code-block:: bash + breeze testing integration-tests --db-reset --integration all -Airflow test types -================== +Runs all mongo DB tests: -Airflow tests in the CI environment are split into several test types: + .. code-block:: bash -* Always - those are tests that should be always executed (always folder) -* Core - for the core Airflow functionality (core folder) -* API - Tests for the Airflow API (api and api_connexion folders) -* CLI - Tests for the Airflow CLI (cli folder) -* WWW - Tests for the Airflow webserver (www folder) -* Providers - Tests for all Providers of Airflow (providers folder) -* Other - all other tests (all other folders that are not part of any of the above) + breeze testing integration-tests --db-reset --integration mongo -This is done for three reasons: +Helm Unit Tests +=============== -1. in order to selectively run only subset of the test types for some PRs -2. in order to allow parallel execution of the tests on Self-Hosted runners +On the Airflow Project, we have decided to stick with pythonic testing for our Helm chart. This makes our chart +easier to test, easier to modify, and able to run with the same testing infrastructure. To add Helm unit tests +add them in ``tests/charts``. -For case 2. We can utilise memory and CPUs available on both CI and local development machines to run -test in parallel. This way we can decrease the time of running all tests in self-hosted runners from -60 minutes to ~15 minutes. +.. code-block:: python -.. note:: + class TestBaseChartTest: + ... - We need to split tests manually into separate suites rather than utilise - ``pytest-xdist`` or ``pytest-parallel`` which could be a simpler and much more "native" parallelization - mechanism. Unfortunately, we cannot utilise those tools because our tests are not truly ``unit`` tests that - can run in parallel. A lot of our tests rely on shared databases - and they update/reset/cleanup the - databases while they are executing. They are also exercising features of the Database such as locking which - further increases cross-dependency between tests. Until we make all our tests truly unit tests (and not - touching the database or until we isolate all such tests to a separate test type, we cannot really rely on - frameworks that run tests in parallel. In our solution each of the test types is run in parallel with its - own database (!) so when we have 8 test types running in parallel, there are in fact 8 databases run - behind the scenes to support them and each of the test types executes its own tests sequentially. +To render the chart create a YAML string with the nested dictionary of options you wish to test. You can then +use our ``render_chart`` function to render the object of interest into a testable Python dictionary. Once the chart +has been rendered, you can use the ``render_k8s_object`` function to create a k8s model object. It simultaneously +ensures that the object created properly conforms to the expected resource spec and allows you to use object values +instead of nested dictionaries. +Example test here: -Running full Airflow test suite in parallel -=========================================== +.. code-block:: python -If you run ``breeze testing tests --run-in-parallel`` tests run in parallel -on your development machine - maxing out the number of parallel runs at the number of cores you -have available in your Docker engine. + from tests.charts.helm_template_generator import render_chart, render_k8s_object -In case you do not have enough memory available to your Docker (8 GB), the ``Integration``. ``Provider`` -and ``Core`` test type are executed sequentially with cleaning the docker setup in-between. This -allows to print + git_sync_basic = """ + dags: + gitSync: + enabled: true + """ -This allows for massive speedup in full test execution. On 8 CPU machine with 16 cores and 64 GB memory -and fast SSD disk, the whole suite of tests completes in about 5 minutes (!). Same suite of tests takes -more than 30 minutes on the same machine when tests are run sequentially. -.. note:: + class TestGitSyncScheduler: + def test_basic(self): + helm_settings = yaml.safe_load(git_sync_basic) + res = render_chart( + "GIT-SYNC", + helm_settings, + show_only=["templates/scheduler/scheduler-deployment.yaml"], + ) + dep: k8s.V1Deployment = render_k8s_object(res[0], k8s.V1Deployment) + assert "dags" == dep.spec.template.spec.volumes[1].name - On MacOS you might have less CPUs and less memory available to run the tests than you have in the host, - simply because your Docker engine runs in a Linux Virtual Machine under-the-hood. If you want to make - use of the parallelism and memory usage for the CI tests you might want to increase the resources available - to your docker engine. See the `Resources `_ chapter - in the ``Docker for Mac`` documentation on how to do it. -You can also limit the parallelism by specifying the maximum number of parallel jobs via -MAX_PARALLEL_TEST_JOBS variable. If you set it to "1", all the test types will be run sequentially. +To execute all Helm tests using breeze command and utilize parallel pytest tests, you can run the +following command (but it takes quite a long time even in a multi-processor machine). .. code-block:: bash - MAX_PARALLEL_TEST_JOBS="1" ./scripts/ci/testing/ci_run_airflow_testing.sh - -.. note:: - - In case you would like to cleanup after execution of such tests you might have to cleanup - some of the docker containers running in case you use ctrl-c to stop execution. You can easily do it by - running this command (it will kill all docker containers running so do not use it if you want to keep some - docker containers running): - - .. code-block:: bash - - docker kill $(docker ps -q) - - -Running Tests with provider packages -==================================== - -Airflow 2.0 introduced the concept of splitting the monolithic Airflow package into separate -providers packages. The main "apache-airflow" package contains the bare Airflow implementation, -and additionally we have 70+ providers that we can install additionally to get integrations with -external services. Those providers live in the same monorepo as Airflow, but we build separate -packages for them and the main "apache-airflow" package does not contain the providers. + breeze testing helm-tests -Most of the development in Breeze happens by iterating on sources and when you run -your tests during development, you usually do not want to build packages and install them separately. -Therefore by default, when you enter Breeze airflow and all providers are available directly from -sources rather than installed from packages. This is for example to test the "provider discovery" -mechanism available that reads provider information from the package meta-data. +You can also run Helm tests individually via the usual ``breeze`` command. Just enter breeze and run the +tests with pytest as you would do with regular unit tests (you can add ``-n auto`` command to run Helm +tests in parallel - unlike most of the regular unit tests of ours that require a database, the Helm tests are +perfectly safe to be run in parallel (and if you have multiple processors, you can gain significant +speedups when using parallel runs): -When Airflow is run from sources, the metadata is read from provider.yaml -files, but when Airflow is installed from packages, it is read via the package entrypoint -``apache_airflow_provider``. +.. code-block:: bash -By default, all packages are prepared in wheel format. To install Airflow from packages you -need to run the following steps: + breeze -1. Prepare provider packages +This enters breeze container. .. code-block:: bash - breeze release-management prepare-provider-packages [PACKAGE ...] - -If you run this command without packages, you will prepare all packages. However, You can specify -providers that you would like to build if you just want to build few provider packages. -The packages are prepared in ``dist`` folder. Note that this command cleans up the ``dist`` folder -before running, so you should run it before generating ``apache-airflow`` package. + pytest tests/charts -n auto -2. Prepare airflow packages +This runs all chart tests using all processors you have available. .. code-block:: bash - breeze release-management prepare-airflow-package - -This prepares airflow .whl package in the dist folder. + pytest tests/charts/test_airflow_common.py -n auto -3. Enter breeze installing both airflow and providers from the dist packages +This will run all tests from ``tests_airflow_common.py`` file using all processors you have available. .. code-block:: bash - breeze --use-airflow-version wheel --use-packages-from-dist --skip-mounting-local-sources + pytest tests/charts/test_airflow_common.py + +This will run all tests from ``tests_airflow_common.py`` file sequentially. -Running Tests with Kubernetes -============================= +Kubernetes tests +================ Airflow has tests that are run against real Kubernetes cluster. We are using `Kind `_ to create and run the cluster. We integrated the tools to start/stop/ diff --git a/dev/breeze/src/airflow_breeze/commands/testing_commands.py b/dev/breeze/src/airflow_breeze/commands/testing_commands.py index d3f80fd21b0dc..1dcfa1196d71a 100644 --- a/dev/breeze/src/airflow_breeze/commands/testing_commands.py +++ b/dev/breeze/src/airflow_breeze/commands/testing_commands.py @@ -23,11 +23,7 @@ import click from click import IntRange -from airflow_breeze.global_constants import ( - ALL_INTEGRATIONS, - ALLOWED_TEST_TYPE_CHOICES, - all_selective_test_types, -) +from airflow_breeze.global_constants import ALLOWED_TEST_TYPE_CHOICES, all_selective_test_types from airflow_breeze.params.build_prod_params import BuildProdParams from airflow_breeze.params.shell_params import ShellParams from airflow_breeze.utils.ci_group import ci_group @@ -123,6 +119,7 @@ def _run_test( db_reset: bool, output: Output | None, test_timeout: int, + output_outside_the_group: bool = False, ) -> tuple[int, str]: env_variables = get_env_variables_for_docker_commands(exec_shell_params) env_variables["RUN_TESTS"] = "true" @@ -137,13 +134,6 @@ def _run_test( "[error]Only 'Providers' test type can specify actual tests with \\[\\][/]" ) sys.exit(1) - if exec_shell_params.integration: - integration = exec_shell_params.integration - if "trino" in integration and "kerberos" not in integration: - int_list = list(integration) - int_list.append("kerberos") - integration = tuple(int_list) - env_variables["LIST_OF_INTEGRATION_TESTS_TO_RUN"] = " ".join(list(integration)) project_name = _file_name_from_test_type(exec_shell_params.test_type) down_cmd = [ *DOCKER_COMPOSE_COMMAND, @@ -165,7 +155,13 @@ def _run_test( ] run_cmd.extend(list(extra_pytest_args)) try: - result = run_command(run_cmd, env=env_variables, output=output, check=False) + result = run_command( + run_cmd, + env=env_variables, + output=output, + check=False, + output_outside_the_group=output_outside_the_group, + ) if os.environ.get("CI") == "true" and result.returncode != 0: ps_result = run_command( ["docker", "ps", "--all", "--format", "{{.Names}}"], @@ -226,17 +222,14 @@ def _run_tests_in_pool( progress_matcher=GenericRegexpProgressMatcher( regexp=TEST_PROGRESS_REGEXP, regexp_for_joined_line=PERCENT_TEST_PROGRESS_REGEXP, - lines_to_search=200, + lines_to_search=400, ), ) as (pool, outputs): results = [ pool.apply_async( _run_test, kwds={ - "exec_shell_params": exec_shell_params.clone_with_test( - test_type=test_type, - integration=ALL_INTEGRATIONS if test_type == "Integration" else (), - ), + "exec_shell_params": exec_shell_params.clone_with_test(test_type=test_type), "extra_pytest_args": extra_pytest_args, "db_reset": db_reset, "output": outputs[index], @@ -271,20 +264,9 @@ def run_tests_in_parallel( import psutil memory_available = psutil.virtual_memory() - if ( - memory_available.available < LOW_MEMORY_CONDITION - and not full_tests_needed - and "Integration" in test_types_list - and exec_shell_params.backend != "sqlite" - ): - get_console().print( - f"[warning]Integration tests are skipped on {exec_shell_params.backend} when " - "memory is constrained as we are in non full-test-mode!" - ) - test_types_list.remove("Integration") if memory_available.available < LOW_MEMORY_CONDITION and exec_shell_params.backend in ["mssql", "mysql"]: # Run heavy tests sequentially - heavy_test_types_to_run = {"Core", "Integration", "Providers"} & set(test_types_list) + heavy_test_types_to_run = {"Core", "Providers"} & set(test_types_list) if heavy_test_types_to_run: # some of those are requested get_console().print( @@ -437,6 +419,72 @@ def tests( sys.exit(returncode) +@testing.command( + name="integration-tests", + help="Run the specified integratio tests.", + context_settings=dict( + ignore_unknown_options=True, + allow_extra_args=True, + ), +) +@option_python +@option_backend +@option_postgres_version +@option_mysql_version +@option_mssql_version +@option_image_tag_for_running +@option_mount_sources +@option_integration +@click.option( + "--test-timeout", + help="Test timeout. Set the pytest setup, execution and teardown timeouts to this value", + default=60, + type=IntRange(min=0), + show_default=True, +) +@option_db_reset +@option_verbose +@option_dry_run +@click.argument("extra_pytest_args", nargs=-1, type=click.UNPROCESSED) +def integration_tests( + python: str, + backend: str, + postgres_version: str, + mysql_version: str, + mssql_version: str, + integration: tuple, + test_timeout: int, + db_reset: bool, + image_tag: str | None, + mount_sources: str, + extra_pytest_args: tuple, +): + docker_filesystem = get_filesystem_type("/var/lib/docker") + get_console().print(f"Docker filesystem: {docker_filesystem}") + exec_shell_params = ShellParams( + python=python, + backend=backend, + integration=integration, + postgres_version=postgres_version, + mysql_version=mysql_version, + mssql_version=mssql_version, + image_tag=image_tag, + mount_sources=mount_sources, + forward_ports=False, + test_type="Integration", + ) + cleanup_python_generated_files() + returncode, _ = _run_test( + exec_shell_params=exec_shell_params, + extra_pytest_args=extra_pytest_args, + db_reset=db_reset, + output=None, + test_timeout=test_timeout, + output_outside_the_group=True, + ) + sys.exit(returncode) + + @testing.command( name="helm-tests", help="Run Helm chart tests.", diff --git a/dev/breeze/src/airflow_breeze/commands/testing_commands_config.py b/dev/breeze/src/airflow_breeze/commands/testing_commands_config.py index 737fdb5c26c23..84e71ca246249 100644 --- a/dev/breeze/src/airflow_breeze/commands/testing_commands_config.py +++ b/dev/breeze/src/airflow_breeze/commands/testing_commands_config.py @@ -18,14 +18,13 @@ TESTING_COMMANDS: dict[str, str | list[str]] = { "name": "Testing", - "commands": ["tests", "helm-tests", "docker-compose-tests"], + "commands": ["tests", "integration-tests", "helm-tests", "docker-compose-tests"], } TESTING_PARAMETERS: dict[str, list[dict[str, str | list[str]]]] = { "breeze testing tests": [ { "name": "Basic flag for tests command", "options": [ - "--integration", "--test-type", "--test-timeout", "--db-reset", @@ -57,6 +56,28 @@ ], }, ], + "breeze testing integration-tests": [ + { + "name": "Basic flag for integration tests command", + "options": [ + "--integration", + "--test-timeout", + "--db-reset", + "--backend", + "--python", + "--postgres-version", + "--mysql-version", + "--mssql-version", + ], + }, + { + "name": "Advanced flag for integration tests command", + "options": [ + "--image-tag", + "--mount-sources", + ], + }, + ], "breeze testing helm-tests": [ { "name": "Advanced flag for helms-tests command", diff --git a/dev/breeze/src/airflow_breeze/global_constants.py b/dev/breeze/src/airflow_breeze/global_constants.py index 0caf050e32fd1..6079f2d92852a 100644 --- a/dev/breeze/src/airflow_breeze/global_constants.py +++ b/dev/breeze/src/airflow_breeze/global_constants.py @@ -47,10 +47,8 @@ "cassandra", "kerberos", "mongo", - "openldap", "pinot", "celery", - "statsd", "trino", ] ALLOWED_INTEGRATIONS = [ @@ -87,7 +85,6 @@ class SelectiveUnitTestTypes(Enum): CLI = "CLI" CORE = "Core" OTHER = "Other" - INTEGRATION = "Integration" PROVIDERS = "Providers" WWW = "WWW" @@ -196,13 +193,11 @@ def get_airflow_extras(): "cassandra", "kerberos", "mongo", - "openldap", "pinot", "celery", "statsd", "trino", ] -ENABLED_INTEGRATIONS = "" ALL_PROVIDER_YAML_FILES = Path(AIRFLOW_SOURCES_ROOT).glob("airflow/providers/**/provider.yaml") # Initialize files for rebuild check FILES_FOR_REBUILD_CHECK = [ diff --git a/dev/breeze/src/airflow_breeze/params/shell_params.py b/dev/breeze/src/airflow_breeze/params/shell_params.py index 9a1813e780bfe..f69a950389dfa 100644 --- a/dev/breeze/src/airflow_breeze/params/shell_params.py +++ b/dev/breeze/src/airflow_breeze/params/shell_params.py @@ -111,10 +111,9 @@ class ShellParams: dry_run: bool = False verbose: bool = False - def clone_with_test(self, test_type: str, integration: tuple[str, ...]) -> ShellParams: + def clone_with_test(self, test_type: str) -> ShellParams: new_params = deepcopy(self) new_params.test_type = test_type - new_params.integration = integration if test_type == "Integration" else () return new_params @property @@ -153,16 +152,6 @@ def airflow_image_kubernetes(self) -> str: def airflow_sources(self): return AIRFLOW_SOURCES_ROOT - @property - def enabled_integrations(self) -> str: - if "all" in self.integration: - enabled_integration = " ".join(AVAILABLE_INTEGRATIONS) - elif len(self.integration) > 0: - enabled_integration = " ".join(self.integration) - else: - enabled_integration = "" - return enabled_integration - @property def image_type(self) -> str: return "CI" diff --git a/dev/breeze/src/airflow_breeze/utils/docker_command_utils.py b/dev/breeze/src/airflow_breeze/utils/docker_command_utils.py index 7771811e86db1..7670b5bbddf91 100644 --- a/dev/breeze/src/airflow_breeze/utils/docker_command_utils.py +++ b/dev/breeze/src/airflow_breeze/utils/docker_command_utils.py @@ -607,7 +607,6 @@ def update_expected_environment_variables(env: dict[str, str]) -> None: set_value_to_default_if_not_set(env, "INIT_SCRIPT_FILE", "init.sh") set_value_to_default_if_not_set(env, "INSTALL_PACKAGES_FROM_CONTEXT", "false") set_value_to_default_if_not_set(env, "INSTALL_PROVIDERS_FROM_SOURCES", "true") - set_value_to_default_if_not_set(env, "LIST_OF_INTEGRATION_TESTS_TO_RUN", "") set_value_to_default_if_not_set(env, "LOAD_DEFAULT_CONNECTIONS", "false") set_value_to_default_if_not_set(env, "LOAD_EXAMPLES", "false") set_value_to_default_if_not_set(env, "PACKAGE_FORMAT", ALLOWED_PACKAGE_FORMATS[0]) @@ -642,7 +641,6 @@ def update_expected_environment_variables(env: dict[str, str]) -> None: "DB_RESET": "db_reset", "DEV_MODE": "dev_mode", "DEFAULT_CONSTRAINTS_BRANCH": "default_constraints_branch", - "ENABLED_INTEGRATIONS": "enabled_integrations", "GITHUB_ACTIONS": "github_actions", "INSTALL_AIRFLOW_VERSION": "install_airflow_version", "INSTALL_PROVIDERS_FROM_SOURCES": "install_providers_from_sources", diff --git a/dev/breeze/src/airflow_breeze/utils/selective_checks.py b/dev/breeze/src/airflow_breeze/utils/selective_checks.py index 91f311e21f7bb..d978434b3c083 100644 --- a/dev/breeze/src/airflow_breeze/utils/selective_checks.py +++ b/dev/breeze/src/airflow_breeze/utils/selective_checks.py @@ -565,12 +565,6 @@ def test_types(self) -> str: f"is {self._default_branch} and not main[/]" ) test_types_to_remove.add(test_type) - if "Integration" in current_test_types: - get_stderr_console().print( - "[warning]Removing 'Integration' because the target branch " - f"is {self._default_branch} and not main[/]" - ) - test_types_to_remove.add("Integration") current_test_types = current_test_types - test_types_to_remove return " ".join(sorted(current_test_types)) diff --git a/dev/breeze/tests/test_selective_checks.py b/dev/breeze/tests/test_selective_checks.py index 026a7ef45b50a..27333dd5c08cf 100644 --- a/dev/breeze/tests/test_selective_checks.py +++ b/dev/breeze/tests/test_selective_checks.py @@ -233,7 +233,7 @@ def assert_outputs_are_printed(expected_outputs: dict[str, str], stderr: str): "run-tests": "true", "docs-build": "true", "upgrade-to-newer-dependencies": "true", - "test-types": "API Always CLI Core Integration Other Providers WWW", + "test-types": "API Always CLI Core Other Providers WWW", }, id="Everything should run - including all providers and upgrading to " "newer requirements as setup.py changed and all Python versions", @@ -252,7 +252,7 @@ def assert_outputs_are_printed(expected_outputs: dict[str, str], stderr: str): "run-tests": "true", "docs-build": "true", "upgrade-to-newer-dependencies": "true", - "test-types": "API Always CLI Core Integration Other Providers WWW", + "test-types": "API Always CLI Core Other Providers WWW", }, id="Everything should run and upgrading to newer requirements as dependencies change", ) @@ -292,7 +292,7 @@ def test_expected_output_pull_request_main( "full-tests-needed": "true", "providers-package-format-exclude": "[]", "upgrade-to-newer-dependencies": "false", - "test-types": "API Always CLI Core Integration Other Providers WWW", + "test-types": "API Always CLI Core Other Providers WWW", }, id="Everything should run including all providers when full tests are needed", ) @@ -316,7 +316,7 @@ def test_expected_output_pull_request_main( "full-tests-needed": "true", "providers-package-format-exclude": "[]", "upgrade-to-newer-dependencies": "false", - "test-types": "API Always CLI Core Integration Other Providers WWW", + "test-types": "API Always CLI Core Other Providers WWW", }, id="Everything should run including full providers when full " "tests are needed even with different label set as well", @@ -338,7 +338,7 @@ def test_expected_output_pull_request_main( "full-tests-needed": "true", "upgrade-to-newer-dependencies": "false", "providers-package-format-exclude": "[]", - "test-types": "API Always CLI Core Integration Other Providers WWW", + "test-types": "API Always CLI Core Other Providers WWW", }, id="Everything should run including full providers when" "full tests are needed even if no files are changed", @@ -362,8 +362,7 @@ def test_expected_output_pull_request_main( "providers-package-format-exclude": "[]", "test-types": "API Always CLI Core Other WWW", }, - id="Everything should run except Providers and Integration " - "when full tests are needed for non-main branch", + id="Everything should run except Providers when full tests are needed for non-main branch", ) ), ], @@ -462,8 +461,7 @@ def test_expected_output_full_tests_needed( "upgrade-to-newer-dependencies": "false", "test-types": "API Always CLI Core Other WWW", }, - id="All tests except providers and Integration should " - "run if core file changed in non-main branch", + id="All tests except Providers and should run if core file changed in non-main branch", ), ], ) @@ -531,7 +529,7 @@ def test_expected_output_pull_request_v2_3( "docs-build": "true", "run-kubernetes-tests": "false", "upgrade-to-newer-dependencies": "false", - "test-types": "API Always CLI Core Integration Other Providers WWW", + "test-types": "API Always CLI Core Other Providers WWW", }, id="All tests except should run if core file changed", ), @@ -566,7 +564,7 @@ def test_expected_output_pull_request_target( "run-tests": "true", "docs-build": "true", "upgrade-to-newer-dependencies": "true", - "test-types": "API Always CLI Core Integration Other Providers WWW", + "test-types": "API Always CLI Core Other Providers WWW", }, id="All tests run on push even if unimportant file changed", ), @@ -584,7 +582,7 @@ def test_expected_output_pull_request_target( "upgrade-to-newer-dependencies": "true", "test-types": "API Always CLI Core Other WWW", }, - id="All tests except Providers Integration and Helm run on push" + id="All tests except Providers and Helm run on push" " even if unimportant file changed in non-main branch", ), pytest.param( @@ -599,7 +597,7 @@ def test_expected_output_pull_request_target( "run-tests": "true", "docs-build": "true", "upgrade-to-newer-dependencies": "true", - "test-types": "API Always CLI Core Integration Other Providers WWW", + "test-types": "API Always CLI Core Other Providers WWW", }, id="All tests run on push if core file changed", ), @@ -650,7 +648,7 @@ def test_no_commit_provided_trigger_full_build_for_any_event_type(github_event): "upgrade-to-newer-dependencies": "true" if github_event in [GithubEvents.PUSH, GithubEvents.SCHEDULE] else "false", - "test-types": "API Always CLI Core Integration Other Providers WWW", + "test-types": "API Always CLI Core Other Providers WWW", }, str(stderr), ) diff --git a/images/breeze/output-commands-hash.txt b/images/breeze/output-commands-hash.txt index acd07f9b13591..efec015ba0090 100644 --- a/images/breeze/output-commands-hash.txt +++ b/images/breeze/output-commands-hash.txt @@ -1,7 +1,7 @@ # This file is automatically generated by pre-commit. If you have a conflict with this file # Please do not solve it but run `breeze setup regenerate-command-images`. # This command should fix the conflict and regenerate help images that you have conflict with. -main:0e7677cd7035e01b5260b9ac4889fd90 +main:50fe93e07fbe182f6d374ea305841240 build-docs:80555245ea1142991ce1d63c3bf8ce74 ci:find-newer-dependencies:9fbe32c529ae96d751a34be093ae86e3 ci:fix-ownership:fee2c9ec9ef19686792002ae054fecdd @@ -44,15 +44,16 @@ release-management:verify-provider-packages:8d3c6362657d76bb3cd064fed5596e84 release-management:c4e840ca22b11a431f5bc7b118dc061f setup:autocomplete:03343478bf1d0cf9c101d454cdb63b68 setup:config:3ffcd35dd24b486ddf1d08b797e3d017 -setup:regenerate-command-images:20016a5ea492f214692c4b57c4fa9c06 +setup:regenerate-command-images:255746830d7b5d1337d13b8e101f7f83 setup:self-upgrade:d02f70c7a230eae3463ceec2056b63fa setup:version:123b462a421884dc2320ffc5e54b2478 -setup:a3bd246c3a425f3e586d11bbdc8937cb -shell:e5bd8bb5b26d2f1e0748f7c65c92aefc -start-airflow:8276b0992eb3e52f001c214525551913 +setup:2e9e4ab1729c5420b7a2b78cbee7539a +shell:affbf6f7f469408d0af47f75c6a38f6c +start-airflow:109728919a0dd5c5ff5640ae86ba9e90 static-checks:7a39e28c87fbca0a9fae0ebfe1591b71 stop:8969537ccdd799f692ccb8600a7bbed6 testing:docker-compose-tests:b86c044b24138af0659a05ed6331576c testing:helm-tests:94a442e7f3f63b34c4831a84d165690a -testing:tests:d2961459b8b39377dd2bc6a06a1bacf0 -testing:9d83bc79d12d10f15402cb848bebcc01 +testing:integration-tests:4c8321c60b2112b12c866beebeb6c61b +testing:tests:1b67704a3f8eae3a6db8d7c8d6cc162a +testing:0614ca49c1b4c2b0b1d92c31278dc4f9 diff --git a/images/breeze/output-commands.svg b/images/breeze/output-commands.svg index 2aab9ace34b54..22001729e6f7f 100644 --- a/images/breeze/output-commands.svg +++ b/images/breeze/output-commands.svg @@ -196,8 +196,8 @@ --postgres-version-PVersion of Postgres used.(>11< | 12 | 13 | 14 | 15)[default: 11] --mysql-version-MVersion of MySQL used.(>5.7< | 8)[default: 5.7] --mssql-version-SVersion of MsSQL used.(>2017-latest< | 2019-latest)[default: 2017-latest] ---integrationIntegration(s) to enable when running (can be more than one).                     -(cassandra | kerberos | mongo | openldap | pinot | celery | statsd | trino | all) +--integrationIntegration(s) to enable when running (can be more than one). +(cassandra | kerberos | mongo | pinot | celery | trino | all) --forward-credentials-fForward local credentials to container when running. --db-reset-dReset DB when entering the container. --max-timeMaximum time that the command should take - if it takes longer, the command will fail. diff --git a/images/breeze/output_setup_regenerate-command-images.svg b/images/breeze/output_setup_regenerate-command-images.svg index 11cc8d3d0bdc9..dc74dc78529d4 100644 --- a/images/breeze/output_setup_regenerate-command-images.svg +++ b/images/breeze/output_setup_regenerate-command-images.svg @@ -1,4 +1,4 @@ - + release-management:release-prod-images | release-management:verify-provider-packages |               release-management | setup:autocomplete | setup:config | setup:regenerate-command-images |           setup:self-upgrade | setup:version | setup | shell | start-airflow | static-checks | stop |          -testing:docker-compose-tests | testing:helm-tests | testing:tests | testing)                         ---check-onlyOnly check if some images need to be regenerated. Return 0 if no need or 1 if needed. Cannot be used -together with --command flag or --force.                                                             -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---verbose-vPrint verbose information about performed steps. ---dry-run-DIf dry-run is set, commands are only printed, not executed. ---help-hShow this message and exit. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +testing:docker-compose-tests | testing:helm-tests | testing:integration-tests | testing:tests |      +testing)                                                                                             +--check-onlyOnly check if some images need to be regenerated. Return 0 if no need or 1 if needed. Cannot be used +together with --command flag or --force.                                                             +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ diff --git a/images/breeze/output_shell.svg b/images/breeze/output_shell.svg index 0c98aad3010ec..b2df94179525d 100644 --- a/images/breeze/output_shell.svg +++ b/images/breeze/output_shell.svg @@ -222,8 +222,8 @@ --postgres-version-PVersion of Postgres used.(>11< | 12 | 13 | 14 | 15)[default: 11] --mysql-version-MVersion of MySQL used.(>5.7< | 8)[default: 5.7] --mssql-version-SVersion of MsSQL used.(>2017-latest< | 2019-latest)[default: 2017-latest] ---integrationIntegration(s) to enable when running (can be more than one).                     -(cassandra | kerberos | mongo | openldap | pinot | celery | statsd | trino | all) +--integrationIntegration(s) to enable when running (can be more than one). +(cassandra | kerberos | mongo | pinot | celery | trino | all) --forward-credentials-fForward local credentials to container when running. --db-reset-dReset DB when entering the container. --github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] diff --git a/images/breeze/output_start-airflow.svg b/images/breeze/output_start-airflow.svg index b31689a6b2177..2f7285314f3ed 100644 --- a/images/breeze/output_start-airflow.svg +++ b/images/breeze/output_start-airflow.svg @@ -245,8 +245,8 @@ --postgres-version-PVersion of Postgres used.(>11< | 12 | 13 | 14 | 15)[default: 11] --mysql-version-MVersion of MySQL used.(>5.7< | 8)[default: 5.7] --mssql-version-SVersion of MsSQL used.(>2017-latest< | 2019-latest)[default: 2017-latest] ---integrationIntegration(s) to enable when running (can be more than one).                     -(cassandra | kerberos | mongo | openldap | pinot | celery | statsd | trino | all) +--integrationIntegration(s) to enable when running (can be more than one). +(cassandra | kerberos | mongo | pinot | celery | trino | all) --forward-credentials-fForward local credentials to container when running. --db-reset-dReset DB when entering the container. --github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] diff --git a/images/breeze/output_testing.svg b/images/breeze/output_testing.svg index bedaaf4ac7086..db5690e0ad3e1 100644 --- a/images/breeze/output_testing.svg +++ b/images/breeze/output_testing.svg @@ -1,4 +1,4 @@ - + ╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ ╭─ Testing ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ tests                                    Run the specified unit test targets.                                      -helm-tests                               Run Helm chart tests.                                                     -docker-compose-tests                     Run docker-compose tests.                                                 -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +integration-tests                        Run the specified integratio tests.                                       +helm-tests                               Run Helm chart tests.                                                     +docker-compose-tests                     Run docker-compose tests.                                                 +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
    diff --git a/images/breeze/output_testing_integration-tests.svg b/images/breeze/output_testing_integration-tests.svg new file mode 100644 index 0000000000000..42fe4cc6bbe65 --- /dev/null +++ b/images/breeze/output_testing_integration-tests.svg @@ -0,0 +1,184 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Command: testing integration-tests + + + + + + + + + + +Usage: breeze testing integration-tests [OPTIONS] [EXTRA_PYTEST_ARGS]... + +Run the specified integratio tests. + +╭─ Basic flag for integration tests command ───────────────────────────────────────────────────────────────────────────╮ +--integrationIntegration(s) to enable when running (can be more than one). +(cassandra | kerberos | mongo | pinot | celery | trino | all) +--test-timeoutTest timeout. Set the pytest setup, execution and teardown timeouts to this value +(INTEGER RANGE)                                                                   +[default: 60; x>=0]                                                               +--db-reset-dReset DB when entering the container. +--backend-bDatabase backend to use.(>sqlite< | mysql | postgres | mssql)[default: sqlite] +--python-pPython major/minor version used in Airflow image for images.(>3.7< | 3.8 | 3.9 | 3.10) +[default: 3.7]                                               +--postgres-version-PVersion of Postgres used.(>11< | 12 | 13 | 14 | 15)[default: 11] +--mysql-version-MVersion of MySQL used.(>5.7< | 8)[default: 5.7] +--mssql-version-SVersion of MsSQL used.(>2017-latest< | 2019-latest)[default: 2017-latest] +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Advanced flag for integration tests command ────────────────────────────────────────────────────────────────────────╮ +--image-tag-tTag of the image which is used to run the image (implies --mount-sources=skip).(TEXT) +[default: latest]                                                               +--mount-sourcesChoose scope of local sources that should be mounted, skipped, or removed (default = selected). +(selected | all | skip | remove)                                                                +[default: selected]                                                                             +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + + + diff --git a/images/breeze/output_testing_tests.svg b/images/breeze/output_testing_tests.svg index 0fbf2186b49af..5b14c22176888 100644 --- a/images/breeze/output_testing_tests.svg +++ b/images/breeze/output_testing_tests.svg @@ -1,4 +1,4 @@ - +
    diff --git a/scripts/ci/docker-compose/_docker.env b/scripts/ci/docker-compose/_docker.env index 78b6755fac939..34b9381f814ed 100644 --- a/scripts/ci/docker-compose/_docker.env +++ b/scripts/ci/docker-compose/_docker.env @@ -33,7 +33,6 @@ DB_RESET DEFAULT_BRANCH DEFAULT_CONSTRAINTS_BRANCH DEV_MODE -ENABLED_INTEGRATIONS ENABLED_SYSTEMS ENABLE_TEST_COVERAGE GITHUB_ACTIONS @@ -56,7 +55,6 @@ POSTGRES_VERSION PYTHONDONTWRITEBYTECODE PYTHON_MAJOR_MINOR_VERSION RUN_TESTS -LIST_OF_INTEGRATION_TESTS_TO_RUN RUN_SYSTEM_TESTS START_AIRFLOW SKIP_CONSTRAINTS diff --git a/scripts/ci/docker-compose/base.yml b/scripts/ci/docker-compose/base.yml index a9c8037ed21f8..ea95ede5f5493 100644 --- a/scripts/ci/docker-compose/base.yml +++ b/scripts/ci/docker-compose/base.yml @@ -46,7 +46,6 @@ services: - DEFAULT_BRANCH=${DEFAULT_BRANCH} - DEFAULT_CONSTRAINTS_BRANCH=${DEFAULT_CONSTRAINTS_BRANCH} - DEV_MODE=${DEV_MODE} - - ENABLED_INTEGRATIONS=${ENABLED_INTEGRATIONS} - ENABLED_SYSTEMS=${ENABLED_SYSTEMS} - ENABLE_TEST_COVERAGE=${ENABLE_TEST_COVERAGE} - GITHUB_ACTIONS=${GITHUB_ACTIONS} @@ -69,7 +68,6 @@ services: - PYTHONDONTWRITEBYTECODE=${PYTHONDONTWRITEBYTECODE} - PYTHON_MAJOR_MINOR_VERSION=${PYTHON_MAJOR_MINOR_VERSION} - RUN_TESTS=${RUN_TESTS} - - LIST_OF_INTEGRATION_TESTS_TO_RUN=${LIST_OF_INTEGRATION_TESTS_TO_RUN} - RUN_SYSTEM_TESTS=${RUN_SYSTEM_TESTS} - START_AIRFLOW=${START_AIRFLOW} - SKIP_CONSTRAINTS=${SKIP_CONSTRAINTS} diff --git a/scripts/ci/docker-compose/devcontainer.env b/scripts/ci/docker-compose/devcontainer.env index 9d6fde58df497..e0f326612b179 100644 --- a/scripts/ci/docker-compose/devcontainer.env +++ b/scripts/ci/docker-compose/devcontainer.env @@ -33,7 +33,6 @@ DB_RESET="false" DEFAULT_BRANCH="main" DEFAULT_CONSTRAINTS_BRANCH="constraints-main" DEV_MODE="true" -ENABLED_INTEGRATIONS= ENABLED_SYSTEMS= ENABLE_TEST_COVERAGE="false" GITHUB_ACTIONS="false" @@ -55,7 +54,6 @@ PACKAGE_FORMAT= POSTGRES_VERSION=10 PYTHONDONTWRITEBYTECODE="true" RUN_TESTS="false" -LIST_OF_INTEGRATION_TESTS_TO_RUN="" RUN_SYSTEM_TESTS="" START_AIRFLOW="false" SKIP_CONSTRAINTS="false" diff --git a/scripts/ci/docker-compose/integration-openldap.yml b/scripts/ci/docker-compose/integration-openldap.yml deleted file mode 100644 index 1afe1d21e01e5..0000000000000 --- a/scripts/ci/docker-compose/integration-openldap.yml +++ /dev/null @@ -1,46 +0,0 @@ -# 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. ---- -version: "3.7" -services: - openldap: - image: ghcr.io/apache/airflow-openldap:2.4.50-2021.07.04 - command: "--copy-service" - environment: - - LDAP_DOMAIN=example.com - - LDAP_ADMIN_PASSWORD=insecure - - LDAP_CONFIG_PASSWORD=insecure - volumes: - - ../openldap/ldif:/container/service/slapd/assets/config/bootstrap/ldif/custom:ro - - /dev/urandom:/dev/random # Required to get non-blocking entropy source - - openldap-db-volume:/var/lib/ldap - healthcheck: - test: 'ss -ltp | grep 389' - interval: 5s - timeout: 30s - retries: 50 - restart: "on-failure" - - airflow: - environment: - - INTEGRATION_OPENLDAP=true - depends_on: - openldap: - condition: service_healthy - -volumes: - openldap-db-volume: diff --git a/scripts/ci/openldap/ldif/01-users.example.com.ldif b/scripts/ci/openldap/ldif/01-users.example.com.ldif deleted file mode 100644 index bf5baf5ad8a18..0000000000000 --- a/scripts/ci/openldap/ldif/01-users.example.com.ldif +++ /dev/null @@ -1,48 +0,0 @@ -# -# 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. - -# create all users - -dn: uid=user1,dc=example,dc=com -objectClass: account -objectClass: simpleSecurityObject -uid: user1 -userPassword: user1 -memberOf: cn=group1,dc=example,dc=com -memberOf: cn=group3,dc=example,dc=com - -dn: uid=user2,dc=example,dc=com -objectClass: account -objectClass: simpleSecurityObject -uid: user2 -userPassword: user2 -memberOf: cn=group2,dc=example,dc=com - -dn: uid=dataprofiler,dc=example,dc=com -objectClass: account -objectClass: simpleSecurityObject -uid: dataprofiler -userPassword: dataprofiler -description: dataprofiler - -dn: uid=superuser,dc=example,dc=com -objectClass: account -objectClass: simpleSecurityObject -uid: superuser -userPassword: superuser -description: superuser diff --git a/scripts/ci/openldap/ldif/02-groups.example.com.ldif b/scripts/ci/openldap/ldif/02-groups.example.com.ldif deleted file mode 100644 index 9d1deb349fa05..0000000000000 --- a/scripts/ci/openldap/ldif/02-groups.example.com.ldif +++ /dev/null @@ -1,44 +0,0 @@ -# -# 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. - -# create all groups - -dn: ou=groups,dc=example,dc=com -objectClass: organizationalunit -ou: groups -description: generic groups branch - -dn: cn=group1,ou=groups,dc=example,dc=com -objectclass: groupofnames -cn: group1 -description: Group 1 of users -# add the group members all of which are -# assumed to exist under example -member: cn=user1,dc=example,dc=com - -dn: cn=group2,ou=groups,dc=example,dc=com -objectclass: groupofnames -cn: group2 -description: Group 2 of users -member: cn=user2,dc=example,dc=com - -dn: cn=group3,ou=groups,dc=example,dc=com -objectclass: groupofnames -cn: group3 -description: Group 3 of users -member: cn=user1,dc=example,dc=com diff --git a/scripts/ci/openldap/ldif/03-manager.example.com.ldif b/scripts/ci/openldap/ldif/03-manager.example.com.ldif deleted file mode 100644 index d4b90b73b8850..0000000000000 --- a/scripts/ci/openldap/ldif/03-manager.example.com.ldif +++ /dev/null @@ -1,21 +0,0 @@ -# -# 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. - -dn: cn=Manager,dc=example,dc=com -cn: Manager -objectClass: organizationalRole diff --git a/scripts/ci/openldap/ldif/04-rootdn.ldif b/scripts/ci/openldap/ldif/04-rootdn.ldif deleted file mode 100644 index 59bd3e554db90..0000000000000 --- a/scripts/ci/openldap/ldif/04-rootdn.ldif +++ /dev/null @@ -1,24 +0,0 @@ -# -# 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. - -dn: cn=config -changetype: modify - -dn: olcDatabase={1}{{ LDAP_BACKEND }},cn=config -replace: olcRootDN -olcRootDN: cn=Manager,{{ LDAP_BASE_DN }} diff --git a/scripts/ci/openldap/slapd.conf b/scripts/ci/openldap/slapd.conf deleted file mode 100644 index a404530b8d285..0000000000000 --- a/scripts/ci/openldap/slapd.conf +++ /dev/null @@ -1,64 +0,0 @@ -# -# 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. - -## -# Global Directives -## - -# Schema and objectClass definitions -include /etc/ldap/schema/core.schema -include /etc/ldap/schema/cosine.schema -include /etc/ldap/schema/nis.schema -include /etc/ldap/schema/inetorgperson.schema - -moduleload back_hdb -moduleload memberof.la - -disallow bind_anon - -## -# Test DB -## - -database hdb - -suffix "dc=example,dc=com" -rootdn "cn=Manager,dc=example,dc=com" -rootpw insecure - -# The database directory MUST exist prior to running slapd AND -# change path as necessary -directory /tmp/ldap_db/ - -## -# ACL -## - -# The userPassword by default can be changed -# by the entry owning it if they are authenticated. -# Others should not be able to see it, except the -# admin entry below -# These access lines apply to database #1 only -access to attrs=userPassword - by self write - by anonymous auth - by users none - -access to * by * read - -overlay memberof diff --git a/scripts/docker/entrypoint_ci.sh b/scripts/docker/entrypoint_ci.sh index 298420f2c9a43..190995590d1e4 100755 --- a/scripts/docker/entrypoint_ci.sh +++ b/scripts/docker/entrypoint_ci.sh @@ -423,13 +423,7 @@ fi readonly SELECTED_TESTS CLI_TESTS API_TESTS PROVIDERS_TESTS CORE_TESTS WWW_TESTS \ ALL_TESTS ALL_PRESELECTED_TESTS -if [[ -n ${LIST_OF_INTEGRATION_TESTS_TO_RUN=} ]]; then - # Integration tests - for INT in ${LIST_OF_INTEGRATION_TESTS_TO_RUN} - do - EXTRA_PYTEST_ARGS+=("--integration" "${INT}") - done -elif [[ ${TEST_TYPE:=""} == "Long" ]]; then +if [[ ${TEST_TYPE:=""} == "Long" ]]; then EXTRA_PYTEST_ARGS+=( "-m" "long_running" "--include-long-running" diff --git a/scripts/in_container/check_environment.sh b/scripts/in_container/check_environment.sh index 5f18ad24eff19..858f804061b5a 100755 --- a/scripts/in_container/check_environment.sh +++ b/scripts/in_container/check_environment.sh @@ -146,9 +146,6 @@ fi if [[ ${INTEGRATION_CASSANDRA} == "true" ]]; then check_service "Cassandra" "run_nc cassandra 9042" 50 fi -if [[ ${INTEGRATION_OPENLDAP} == "true" ]]; then - check_service "OpenLDAP" "run_nc openldap 389" 50 -fi if [[ ${INTEGRATION_TRINO} == "true" ]]; then check_service "Trino (HTTP)" "run_nc trino 8080" 50 check_service "Trino (HTTPS)" "run_nc trino 7778" 50 diff --git a/tests/conftest.py b/tests/conftest.py index 559bf20f24a34..0d4d1170f00c6 100644 --- a/tests/conftest.py +++ b/tests/conftest.py @@ -158,7 +158,7 @@ def pytest_addoption(parser): action="append", metavar="INTEGRATIONS", help="only run tests matching integration specified: " - "[cassandra,kerberos,mongo,openldap,rabbitmq,redis,statsd,trino]. ", + "[cassandra,kerberos,mongo,celery,statsd,trino]. ", ) group.addoption( "--backend", From 4ae8f42aac2b34de81744e4badaed7d4bcf10d04 Mon Sep 17 00:00:00 2001 From: Ephraim Anierobi Date: Thu, 12 Jan 2023 10:34:40 +0100 Subject: [PATCH 157/158] Update Airflow version to 2.5.1 --- README.md | 14 +++++++------- airflow/api_connexion/openapi/v1.yaml | 2 +- airflow/utils/db.py | 1 + .../installation/supported-versions.rst | 2 +- docs/docker-stack/README.md | 10 +++++----- .../extending/add-apt-packages/Dockerfile | 2 +- .../add-build-essential-extend/Dockerfile | 2 +- .../extending/add-providers/Dockerfile | 2 +- .../extending/add-pypi-packages/Dockerfile | 2 +- .../extending/add-requirement-packages/Dockerfile | 2 +- .../extending/custom-providers/Dockerfile | 2 +- .../extending/embedding-dags/Dockerfile | 2 +- .../extending/writable-directory/Dockerfile | 2 +- docs/docker-stack/entrypoint.rst | 14 +++++++------- .../ci/pre_commit/pre_commit_supported_versions.py | 2 +- setup.py | 2 +- 16 files changed, 32 insertions(+), 31 deletions(-) diff --git a/README.md b/README.md index 650372e1120c1..3b5e704e5f086 100644 --- a/README.md +++ b/README.md @@ -86,7 +86,7 @@ Airflow is not a streaming solution, but it is often used to process real-time d Apache Airflow is tested with: -| | Main version (dev) | Stable version (2.5.0) | +| | Main version (dev) | Stable version (2.5.1) | |---------------------|------------------------------|------------------------------| | Python | 3.7, 3.8, 3.9, 3.10 | 3.7, 3.8, 3.9, 3.10 | | Platform | AMD64/ARM64(\*) | AMD64/ARM64(\*) | @@ -158,15 +158,15 @@ them to the appropriate format and workflow that your tool requires. ```bash -pip install 'apache-airflow==2.5.0' \ - --constraint "https://raw.githubusercontent.com/apache/airflow/constraints-2.5.0/constraints-3.7.txt" +pip install 'apache-airflow==2.5.1' \ + --constraint "https://raw.githubusercontent.com/apache/airflow/constraints-2.5.1/constraints-3.7.txt" ``` 2. Installing with extras (i.e., postgres, google) ```bash -pip install 'apache-airflow[postgres,google]==2.5.0' \ - --constraint "https://raw.githubusercontent.com/apache/airflow/constraints-2.5.0/constraints-3.7.txt" +pip install 'apache-airflow[postgres,google]==2.5.1' \ + --constraint "https://raw.githubusercontent.com/apache/airflow/constraints-2.5.1/constraints-3.7.txt" ``` For information on installing provider packages, check @@ -271,7 +271,7 @@ Apache Airflow version life cycle: | Version | Current Patch/Minor | State | First Release | Limited Support | EOL/Terminated | |-----------|-----------------------|-----------|-----------------|-------------------|------------------| -| 2 | 2.5.0 | Supported | Dec 17, 2020 | TBD | TBD | +| 2 | 2.5.1 | Supported | Dec 17, 2020 | TBD | TBD | | 1.10 | 1.10.15 | EOL | Aug 27, 2018 | Dec 17, 2020 | June 17, 2021 | | 1.9 | 1.9.0 | EOL | Jan 03, 2018 | Aug 27, 2018 | Aug 27, 2018 | | 1.8 | 1.8.2 | EOL | Mar 19, 2017 | Jan 03, 2018 | Jan 03, 2018 | @@ -301,7 +301,7 @@ They are based on the official release schedule of Python and Kubernetes, nicely 2. The "oldest" supported version of Python/Kubernetes is the default one until we decide to switch to later version. "Default" is only meaningful in terms of "smoke tests" in CI PRs, which are run using this default version and the default reference image available. Currently `apache/airflow:latest` - and `apache/airflow:2.5.0` images are Python 3.7 images. This means that default reference image will + and `apache/airflow:2.5.1` images are Python 3.7 images. This means that default reference image will become the default at the time when we start preparing for dropping 3.7 support which is few months before the end of life for Python 3.7. diff --git a/airflow/api_connexion/openapi/v1.yaml b/airflow/api_connexion/openapi/v1.yaml index 9db4e7647dce0..3f25b54640741 100644 --- a/airflow/api_connexion/openapi/v1.yaml +++ b/airflow/api_connexion/openapi/v1.yaml @@ -229,7 +229,7 @@ info: This means that the server encountered an unexpected condition that prevented it from fulfilling the request. - version: '2.4.0' + version: '2.5.1' license: name: Apache 2.0 url: http://www.apache.org/licenses/LICENSE-2.0.html diff --git a/airflow/utils/db.py b/airflow/utils/db.py index 8f22bc1dfbb55..04f6789d90bfe 100644 --- a/airflow/utils/db.py +++ b/airflow/utils/db.py @@ -76,6 +76,7 @@ "2.4.2": "b0d31815b5a6", "2.4.3": "e07f49787c9d", "2.5.0": "290244fb8b83", + "2.5.1": "290244fb8b83", } diff --git a/docs/apache-airflow/installation/supported-versions.rst b/docs/apache-airflow/installation/supported-versions.rst index 56385ebfd8a12..38ace709db585 100644 --- a/docs/apache-airflow/installation/supported-versions.rst +++ b/docs/apache-airflow/installation/supported-versions.rst @@ -29,7 +29,7 @@ Apache Airflow version life cycle: ========= ===================== ========= =============== ================= ================ Version Current Patch/Minor State First Release Limited Support EOL/Terminated ========= ===================== ========= =============== ================= ================ -2 2.5.0 Supported Dec 17, 2020 TBD TBD +2 2.5.1 Supported Dec 17, 2020 TBD TBD 1.10 1.10.15 EOL Aug 27, 2018 Dec 17, 2020 June 17, 2021 1.9 1.9.0 EOL Jan 03, 2018 Aug 27, 2018 Aug 27, 2018 1.8 1.8.2 EOL Mar 19, 2017 Jan 03, 2018 Jan 03, 2018 diff --git a/docs/docker-stack/README.md b/docs/docker-stack/README.md index 60150cb0b1693..8df4f59cc7c18 100644 --- a/docs/docker-stack/README.md +++ b/docs/docker-stack/README.md @@ -31,12 +31,12 @@ Every time a new version of Airflow is released, the images are prepared in the [apache/airflow DockerHub](https://hub.docker.com/r/apache/airflow) for all the supported Python versions. -You can find the following images there (Assuming Airflow version `2.5.0`): +You can find the following images there (Assuming Airflow version `2.5.1`): * `apache/airflow:latest` - the latest released Airflow image with default Python version (3.7 currently) * `apache/airflow:latest-pythonX.Y` - the latest released Airflow image with specific Python version -* `apache/airflow:2.5.0` - the versioned Airflow image with default Python version (3.7 currently) -* `apache/airflow:2.5.0-pythonX.Y` - the versioned Airflow image with specific Python version +* `apache/airflow:2.5.1` - the versioned Airflow image with default Python version (3.7 currently) +* `apache/airflow:2.5.1-pythonX.Y` - the versioned Airflow image with specific Python version Those are "reference" regular images. They contain the most common set of extras, dependencies and providers that are often used by the users and they are good to "try-things-out" when you want to just take Airflow for a spin, @@ -47,8 +47,8 @@ via [Building the image](https://airflow.apache.org/docs/docker-stack/build.html * `apache/airflow:slim-latest` - the latest released Airflow image with default Python version (3.7 currently) * `apache/airflow:slim-latest-pythonX.Y` - the latest released Airflow image with specific Python version -* `apache/airflow:slim-2.5.0` - the versioned Airflow image with default Python version (3.7 currently) -* `apache/airflow:slim-2.5.0-pythonX.Y` - the versioned Airflow image with specific Python version +* `apache/airflow:slim-2.5.1` - the versioned Airflow image with default Python version (3.7 currently) +* `apache/airflow:slim-2.5.1-pythonX.Y` - the versioned Airflow image with specific Python version The Apache Airflow image provided as convenience package is optimized for size, and it provides just a bare minimal set of the extras and dependencies installed and in most cases diff --git a/docs/docker-stack/docker-examples/extending/add-apt-packages/Dockerfile b/docs/docker-stack/docker-examples/extending/add-apt-packages/Dockerfile index 03a63de56242e..ab034d218fa47 100644 --- a/docs/docker-stack/docker-examples/extending/add-apt-packages/Dockerfile +++ b/docs/docker-stack/docker-examples/extending/add-apt-packages/Dockerfile @@ -15,7 +15,7 @@ # This is an example Dockerfile. It is not intended for PRODUCTION use # [START Dockerfile] -FROM apache/airflow:2.5.0 +FROM apache/airflow:2.5.1 USER root RUN apt-get update \ && apt-get install -y --no-install-recommends \ diff --git a/docs/docker-stack/docker-examples/extending/add-build-essential-extend/Dockerfile b/docs/docker-stack/docker-examples/extending/add-build-essential-extend/Dockerfile index 2790fd6bd3555..af9cefb613dec 100644 --- a/docs/docker-stack/docker-examples/extending/add-build-essential-extend/Dockerfile +++ b/docs/docker-stack/docker-examples/extending/add-build-essential-extend/Dockerfile @@ -15,7 +15,7 @@ # This is an example Dockerfile. It is not intended for PRODUCTION use # [START Dockerfile] -FROM apache/airflow:2.5.0 +FROM apache/airflow:2.5.1 USER root RUN apt-get update \ && apt-get install -y --no-install-recommends \ diff --git a/docs/docker-stack/docker-examples/extending/add-providers/Dockerfile b/docs/docker-stack/docker-examples/extending/add-providers/Dockerfile index fddb5b8e23214..01939b83d0009 100644 --- a/docs/docker-stack/docker-examples/extending/add-providers/Dockerfile +++ b/docs/docker-stack/docker-examples/extending/add-providers/Dockerfile @@ -15,7 +15,7 @@ # This is an example Dockerfile. It is not intended for PRODUCTION use # [START Dockerfile] -FROM apache/airflow:2.5.0 +FROM apache/airflow:2.5.1 USER root RUN apt-get update \ && apt-get install -y --no-install-recommends \ diff --git a/docs/docker-stack/docker-examples/extending/add-pypi-packages/Dockerfile b/docs/docker-stack/docker-examples/extending/add-pypi-packages/Dockerfile index 5a10c877b5528..7f3d150bb1c17 100644 --- a/docs/docker-stack/docker-examples/extending/add-pypi-packages/Dockerfile +++ b/docs/docker-stack/docker-examples/extending/add-pypi-packages/Dockerfile @@ -15,6 +15,6 @@ # This is an example Dockerfile. It is not intended for PRODUCTION use # [START Dockerfile] -FROM apache/airflow:2.5.0 +FROM apache/airflow:2.5.1 RUN pip install --no-cache-dir lxml # [END Dockerfile] diff --git a/docs/docker-stack/docker-examples/extending/add-requirement-packages/Dockerfile b/docs/docker-stack/docker-examples/extending/add-requirement-packages/Dockerfile index dd29971cf3a9b..bdd31c610eca1 100644 --- a/docs/docker-stack/docker-examples/extending/add-requirement-packages/Dockerfile +++ b/docs/docker-stack/docker-examples/extending/add-requirement-packages/Dockerfile @@ -15,7 +15,7 @@ # This is an example Dockerfile. It is not intended for PRODUCTION use # [START Dockerfile] -FROM apache/airflow:2.5.0 +FROM apache/airflow:2.5.1 COPY requirements.txt / RUN pip install --no-cache-dir -r /requirements.txt # [END Dockerfile] diff --git a/docs/docker-stack/docker-examples/extending/custom-providers/Dockerfile b/docs/docker-stack/docker-examples/extending/custom-providers/Dockerfile index b43d7ee3afa71..8ae4a23365ae2 100644 --- a/docs/docker-stack/docker-examples/extending/custom-providers/Dockerfile +++ b/docs/docker-stack/docker-examples/extending/custom-providers/Dockerfile @@ -15,6 +15,6 @@ # This is an example Dockerfile. It is not intended for PRODUCTION use # [START Dockerfile] -FROM apache/airflow:2.5.0 +FROM apache/airflow:2.5.1 RUN pip install --no-cache-dir apache-airflow-providers-docker==2.5.1 # [END Dockerfile] diff --git a/docs/docker-stack/docker-examples/extending/embedding-dags/Dockerfile b/docs/docker-stack/docker-examples/extending/embedding-dags/Dockerfile index cee3e95699c83..e21fea1257b2a 100644 --- a/docs/docker-stack/docker-examples/extending/embedding-dags/Dockerfile +++ b/docs/docker-stack/docker-examples/extending/embedding-dags/Dockerfile @@ -15,7 +15,7 @@ # This is an example Dockerfile. It is not intended for PRODUCTION use # [START Dockerfile] -FROM apache/airflow:2.5.0 +FROM apache/airflow:2.5.1 COPY --chown=airflow:root test_dag.py /opt/airflow/dags diff --git a/docs/docker-stack/docker-examples/extending/writable-directory/Dockerfile b/docs/docker-stack/docker-examples/extending/writable-directory/Dockerfile index 09e17f5910ea3..378339974dd89 100644 --- a/docs/docker-stack/docker-examples/extending/writable-directory/Dockerfile +++ b/docs/docker-stack/docker-examples/extending/writable-directory/Dockerfile @@ -15,7 +15,7 @@ # This is an example Dockerfile. It is not intended for PRODUCTION use # [START Dockerfile] -FROM apache/airflow:2.5.0 +FROM apache/airflow:2.5.1 RUN umask 0002; \ mkdir -p ~/writeable-directory # [END Dockerfile] diff --git a/docs/docker-stack/entrypoint.rst b/docs/docker-stack/entrypoint.rst index 8ec9674d52899..9122c28a1a2bc 100644 --- a/docs/docker-stack/entrypoint.rst +++ b/docs/docker-stack/entrypoint.rst @@ -132,7 +132,7 @@ if you specify extra arguments. For example: .. code-block:: bash - docker run -it apache/airflow:2.5.0-python3.6 bash -c "ls -la" + docker run -it apache/airflow:2.5.1-python3.6 bash -c "ls -la" total 16 drwxr-xr-x 4 airflow root 4096 Jun 5 18:12 . drwxr-xr-x 1 root root 4096 Jun 5 18:12 .. @@ -144,7 +144,7 @@ you pass extra parameters. For example: .. code-block:: bash - > docker run -it apache/airflow:2.5.0-python3.6 python -c "print('test')" + > docker run -it apache/airflow:2.5.1-python3.6 python -c "print('test')" test If first argument equals to "airflow" - the rest of the arguments is treated as an airflow command @@ -152,13 +152,13 @@ to execute. Example: .. code-block:: bash - docker run -it apache/airflow:2.5.0-python3.6 airflow webserver + docker run -it apache/airflow:2.5.1-python3.6 airflow webserver If there are any other arguments - they are simply passed to the "airflow" command .. code-block:: bash - > docker run -it apache/airflow:2.5.0-python3.6 help + > docker run -it apache/airflow:2.5.1-python3.6 help usage: airflow [-h] GROUP_OR_COMMAND ... positional arguments: @@ -363,7 +363,7 @@ database and creating an ``admin/admin`` Admin user with the following command: --env "_AIRFLOW_DB_UPGRADE=true" \ --env "_AIRFLOW_WWW_USER_CREATE=true" \ --env "_AIRFLOW_WWW_USER_PASSWORD=admin" \ - apache/airflow:2.5.0-python3.8 webserver + apache/airflow:2.5.1-python3.8 webserver .. code-block:: bash @@ -372,7 +372,7 @@ database and creating an ``admin/admin`` Admin user with the following command: --env "_AIRFLOW_DB_UPGRADE=true" \ --env "_AIRFLOW_WWW_USER_CREATE=true" \ --env "_AIRFLOW_WWW_USER_PASSWORD_CMD=echo admin" \ - apache/airflow:2.5.0-python3.8 webserver + apache/airflow:2.5.1-python3.8 webserver The commands above perform initialization of the SQLite database, create admin user with admin password and Admin role. They also forward local port ``8080`` to the webserver port and finally start the webserver. @@ -412,6 +412,6 @@ Example: --env "_AIRFLOW_DB_UPGRADE=true" \ --env "_AIRFLOW_WWW_USER_CREATE=true" \ --env "_AIRFLOW_WWW_USER_PASSWORD_CMD=echo admin" \ - apache/airflow:2.5.0-python3.8 webserver + apache/airflow:2.5.1-python3.8 webserver This method is only available starting from Docker image of Airflow 2.1.1 and above. diff --git a/scripts/ci/pre_commit/pre_commit_supported_versions.py b/scripts/ci/pre_commit/pre_commit_supported_versions.py index 5692e27d7d160..4c28a71f0d07b 100755 --- a/scripts/ci/pre_commit/pre_commit_supported_versions.py +++ b/scripts/ci/pre_commit/pre_commit_supported_versions.py @@ -27,7 +27,7 @@ HEADERS = ("Version", "Current Patch/Minor", "State", "First Release", "Limited Support", "EOL/Terminated") SUPPORTED_VERSIONS = ( - ("2", "2.5.0", "Supported", "Dec 17, 2020", "TBD", "TBD"), + ("2", "2.5.1", "Supported", "Dec 17, 2020", "TBD", "TBD"), ("1.10", "1.10.15", "EOL", "Aug 27, 2018", "Dec 17, 2020", "June 17, 2021"), ("1.9", "1.9.0", "EOL", "Jan 03, 2018", "Aug 27, 2018", "Aug 27, 2018"), ("1.8", "1.8.2", "EOL", "Mar 19, 2017", "Jan 03, 2018", "Jan 03, 2018"), diff --git a/setup.py b/setup.py index c7af47ffcc55f..390682722aac4 100644 --- a/setup.py +++ b/setup.py @@ -49,7 +49,7 @@ logger = logging.getLogger(__name__) -version = "2.5.0" +version = "2.5.1" AIRFLOW_SOURCES_ROOT = Path(__file__).parent.resolve() PROVIDERS_ROOT = AIRFLOW_SOURCES_ROOT / "airflow" / "providers" From 2ce4b56ed6e61e5729e2d1d9d282749f34406d60 Mon Sep 17 00:00:00 2001 From: Ephraim Anierobi Date: Thu, 12 Jan 2023 11:35:30 +0100 Subject: [PATCH 158/158] Add Changelog for 2.5.1 --- RELEASE_NOTES.rst | 111 +++++++++++++++++++++++++++++++++++ newsfragments/08212.misc.rst | 1 - 2 files changed, 111 insertions(+), 1 deletion(-) delete mode 100644 newsfragments/08212.misc.rst diff --git a/RELEASE_NOTES.rst b/RELEASE_NOTES.rst index e963142a69d64..e879df02c328a 100644 --- a/RELEASE_NOTES.rst +++ b/RELEASE_NOTES.rst @@ -21,6 +21,117 @@ .. towncrier release notes start +Airflow 2.5.1 (2023-01-16) +-------------------------- + +Significant Changes +^^^^^^^^^^^^^^^^^^^ + +Trigger gevent ``monkeypatching`` via environment variable (#28283) +""""""""""""""""""""""""""""""""""""""""""""""""""""""""""""""""""" + + If you are using gevent for your webserver deployment and used local settings to ``monkeypatch`` gevent, + you might want to replace local settings patching with an ``_AIRFLOW_PATCH_GEVENT`` environment variable + set to 1 in your webserver. This ensures gevent patching is done as early as possible. (#8212) + +Bug Fixes +^^^^^^^^^ +- Fix masking of non-sensitive environment variables (#28802) +- Remove swagger-ui extra from connexion and install ``swagger-ui-dist`` via npm package (#28788) +- Fix ``UIAlert`` should_show when ``AUTH_ROLE_PUBLIC`` set (#28781) +- Only patch single label when adopting pod (#28776) +- Update CSRF token to expire with session (#28730) +- Fix "airflow tasks render" cli command for mapped task instances (#28698) +- Allow XComArgs for ``external_task_ids`` of ExternalTaskSensor (#28692) +- Row-lock TIs to be removed during mapped task expansion (#28689) +- Handle ConnectionReset exception in Executor cleanup (#28685) +- Fix description of output redirection for access_log for gunicorn (#28672) +- Add back join to zombie query that was dropped in #28198 (#28544) +- Fix calendar view for CronTriggerTimeTable dags (#28411) +- After running the DAG the employees table is empty. (#28353) +- Fix ``DetachedInstanceError`` when finding zombies in Dag Parsing process (#28198) +- Nest header blocks in ``divs`` to fix ``dagid`` copy nit on dag.html (#28643) +- Fix UI caret direction (#28624) +- Guard not-yet-expanded ti in trigger rule dep (#28592) +- Move TI ``setNote`` endpoints under TaskInstance in OpenAPI (#28566) +- Consider previous run in ``CronTriggerTimetable`` (#28532) +- Ensure correct log dir in file task handler (#28477) +- Fix bad pods pickled in executor_config (#28454) +- Add ``ensure_ascii=False`` in trigger dag run API (#28451) +- Add setters to MappedOperator on_*_callbacks (#28313) +- Fix ``ti._try_number`` for deferred and up_for_reschedule tasks (#26993) +- separate ``callModal`` from dag.js (#28410) +- A manual run can't look like a scheduled one (#28397) +- Dont show task/run durations when there is no start_date (#28395) +- Maintain manual scroll position in task logs (#28386) +- Correctly select a mapped task's "previous" task (#28379) +- Trigger gevent ``monkeypatching`` via environment variable (#28283) +- Fix db clean warnings (#28243) +- Make arguments 'offset' and 'length' not required (#28234) +- Make live logs reading work for "other" k8s executors (#28213) +- Add custom pickling hooks to ``LazyXComAccess`` (#28191) +- fix next run datasets error (#28165) +- Ensure that warnings from ``@dag`` decorator are reported in dag file (#28153) +- Do not warn when airflow dags tests command is used (#28138) +- Ensure the ``dagbag_size`` metric decreases when files are deleted (#28135) +- Improve run/task grid view actions (#28130) +- Make BaseJob.most_recent_job favor "running" jobs (#28119) +- Don't emit FutureWarning when code not calling old key (#28109) +- Add ``airflow.api.auth.backend.session`` to backend sessions in compose (#28094) +- Resolve false warning about calling conf.get on moved item (#28075) +- Return list of tasks that will be changed (#28066) +- Handle bad zip files nicely when parsing DAGs. (#28011) +- Prevent double loading of providers from local paths (#27988) +- Fix deadlock when chaining multiple empty mapped tasks (#27964) +- fix: current_state method on TaskInstance doesn't filter by map_index (#27898) +- Don't log CLI actions if db not initialized (#27851) +- Make sure we can get out of a faulty scheduler state (#27834) +- dagrun, ``next_dagruns_to_examine``, add MySQL index hint (#27821) +- Handle DAG disappearing mid-flight when dag verification happens (#27720) +- fix: continue checking sla (#26968) +- Allow generation of connection URI to work when no conn type (#26765) + +Misc/Internal +^^^^^^^^^^^^^ +- Add automated version replacement in example dag indexes (#28090) +- Cleanup and do housekeeping with plugin examples (#28537) +- Limit ``SQLAlchemy`` to below ``2.0`` (#28725) +- Bump ``json5`` from ``1.0.1`` to ``1.0.2`` in ``/airflow/www`` (#28715) +- Fix some docs on using sensors with taskflow (#28708) +- Change Architecture and OperatingSystem classes into ``Enums`` (#28627) +- Add doc-strings and small improvement to email util (#28634) +- Fix ``Connection.get_extra`` type (#28594) +- navbar, cap dropdown size, and add scroll bar (#28561) +- Emit warnings for ``conf.get*`` from the right source location (#28543) +- Move MyPY plugins of ours to dev folder (#28498) +- Add retry to ``purge_inactive_dag_warnings`` (#28481) +- Re-enable Plyvel on ARM as it now builds cleanly (#28443) +- Add SIGUSR2 handler for LocalTaskJob and workers to aid debugging (#28309) +- Convert ``test_task_command`` to Pytest and ``unquarantine`` tests in it (#28247) +- Make invalid characters exception more readable (#28181) +- Bump decode-uri-component from ``0.2.0`` to ``0.2.2`` in ``/airflow/www`` (#28080) +- Use asserts instead of exceptions for executor not started (#28019) +- Simplify dataset ``subgraph`` logic (#27987) +- Order TIs by ``map_index`` (#27904) +- Additional info about Segmentation Fault in ``LocalTaskJob`` (#27381) + +Doc Only Changes +^^^^^^^^^^^^^^^^ +- Mention mapped operator in cluster policy doc (#28885) +- Slightly improve description of Dynamic DAG generation preamble (#28650) +- Restructure Docs (#27235) +- Update scheduler docs about low priority tasks (#28831) +- Clarify that versioned constraints are fixed at release time (#28762) +- Clarify about docker compose (#28729) +- Adding an example dag for dynamic task mapping (#28325) +- Use docker compose v2 command (#28605) +- Add AIRFLOW_PROJ_DIR to docker-compose example (#28517) +- Remove outdated Optional Provider Feature outdated documentation (#28506) +- Add documentation for [core] mp_start_method config (#27993) +- Documentation for the LocalTaskJob return code counter (#27972) +- Note which versions of Python are supported (#27798) + + Airflow 2.5.0 (2022-12-02) -------------------------- diff --git a/newsfragments/08212.misc.rst b/newsfragments/08212.misc.rst deleted file mode 100644 index acce074f103ec..0000000000000 --- a/newsfragments/08212.misc.rst +++ /dev/null @@ -1 +0,0 @@ -If you are using gevent for your webserver deployment and used local settings to monkeypatch gevent, you might want to replace local settings patching with an ``_AIRFLOW_PATCH_GEVENT`` environment variable set to 1 in your webserver. This ensures gevent patching is done as early as possible.