Граф коммитов

506 Коммитов

Автор SHA1 Сообщение Дата
Joshua Carp a9ac2b040b
Switch to f-strings using flynt. (#13732) 2021-01-23 06:19:38 +01:00
QP Hou f1d4f54b34
Fix race conditions in task callback invocations (#10917)
This race condition resulted in task success and failure callbacks being
called more than once. Here is the order of events that could lead to
this issue:

* task started running within process 2
* (process 1) local_task_job checked for task return code, returns None
* (process 2) task exited with failure state, task state updated as failed in DB
* (process 2) task failure callback invoked through taskinstance.handle_failure method
* (process 1) local_task_job heartbeat noticed task state set to
  failure, mistoken it as state bing updated externally, also invoked task
  failure callback

To avoid this race condition, we need to make sure task callbacks are
only invoked within a single process.
2021-01-18 23:39:41 +00:00
Kaxil Naik c128aa744e
BugFix: Dag-level Callback Requests were not run (#13651)
In https://github.com/apache/airflow/pull/13163 - I attempted to only run
Callback requests when they are defined on DAG. But I just found out
that while we were storing the task-level callbacks as string in Serialized
JSON, we were not storing DAG level callbacks and hence it default to None
when the Serialized DAG was deserialized which meant that the DAG callbacks
were not run.

This PR fixes it, we don't need to store DAG level callbacks as string, as
we don't display them in the Webserver and the actual contents are not used anywhere
in the Scheduler itself. Scheduler just checks if the callbacks are defined and sends
it to DagFileProcessorProcess to run with the actual DAG file. So instead of storing
the actual callback as string which would have resulted in larger JSON blob, I have
added properties to determine whether a callback is defined or not.

(`dag.has_on_success_callback` and `dag.has_on_failure_callback`)

Note: SLA callbacks don't have issue, as we currently check that SLAs are defined on
any tasks are not, if yes, we send it to DagFileProcessorProcess which then executes
the SLA callback defined on DAG.
2021-01-14 15:46:58 +00:00
Kamil Breguła 5954ef5f41
Warn about precedence of env var when getting variables (#13501) 2021-01-10 10:35:09 +01:00
Andreas Franzén 52339a55c0
[AIRFLOW-7044] Host key can be specified via SSH connection extras. (#12944) 2021-01-08 12:02:53 +01:00
Xiaodong DENG 6ef23aff80
Streamline & simplify __eq__ methods in models Dag and BaseOperator (#13449)
- Use getattr() instead of __dict__ as __dict__ doesn't return
  correct values for properties.
- Avoid unnecessary condition checks (the removed condition checks are covered by _comps)
2021-01-04 04:12:31 +01:00
Xiaodong DENG 4f494d4d92
Fix few typos (#13450) 2021-01-03 14:56:50 +00:00
Kaxil Naik 2c9c55781e
Stop sending Callback Requests if no callbacks are defined on DAG (#13163)
If no on_*_callback are defined on DAG, Callbacks should not be registered
and sent to DAG Processor.

This will reduce the KeyError mentioned in https://github.com/apache/airflow/issues/13047
2020-12-19 12:10:18 +00:00
Joshua Carp ccaca0af39
Annotate DagRun methods with return types (#11486) 2020-12-16 13:27:21 +00:00
dstandish 62a5b2dfa4
Fix return type in prev-date context variables (#12910) 2020-12-09 15:10:03 +00:00
Kaxil Naik a075b6df99
Rename remaining Sensors to match AIP-21 (#12927)
As discussed in AIP-21

* Rename airflow.sensors.external_task_sensor to airflow.sensors.external_task
* Rename airflow.sensors.sql_sensor to airflow.sensors.sql
* Rename airflow.contrib.sensors.weekday_sensor to airflow.sensors.weekday
2020-12-09 00:09:08 +00:00
Kishore Vancheeshwaran bfbd4bbb70
Moved subdag_operator.py to subdag.py (#11307)
Part of #11178
2020-12-08 16:26:39 +00:00
Jarek Potiuk 9b39f24780
Add support for dynamic connection form fields per provider (#12558)
Connection form behaviour depends on the connection type. Since we've
separated providers into separate packages, the connection form should
be extendable by each provider. This PR implements both:

  * extra fields added by provider
  * configurable behaviour per provider

This PR will be followed by separate documentation on how to write your
provider.

Also this change triggers (in tests only) the snowflake annoyance
described in #12881 so we had to xfail presto test where monkeypatching
of snowflake causes the test to fail.

Part of #11429
2020-12-08 16:00:37 +01:00
Kaxil Naik ff25bd6ffe
Make xcom_pull results order deterministic (#12905)
closes https://github.com/apache/airflow/issues/11858
2020-12-08 13:00:06 +00:00
Jarek Potiuk f66a46db88
Changes the type of source_code field in DagCode to MEDIUMTEXT (#12890)
This change increases the maximum amount of code one can store
in dag_code in MySQL. The limit for TEXT is 64KB where for
MEDIUMTEXT is 16MB.

Fixes #12776
2020-12-07 23:17:22 +01:00
Daniel Imberman 190066cf20
Kubernetes worker pod doesn't use docker container entrypoint (#12766)
* Kubernetes worker pod doesn't use docker container entrypoint

Fixes issue on openshift caused by KubernetesExecutor pods not running
via the entrypoint script

* fix

* Update UPGRADING_TO_2.0.md

Co-authored-by: Ash Berlin-Taylor <ash_github@firemirror.com>

* fix UPDGRADING

* @ashb comments

Co-authored-by: Ash Berlin-Taylor <ash_github@firemirror.com>
2020-12-07 13:54:45 -08:00
Ash Berlin-Taylor 5d328a2f7e
Show DAG serialization errors in the UI. (#12866)
The previous behaviour led to "bad" data being written in the DB -- for
example:

```json
    "dag": {
        "tasks": [
            "serialization_failed"
        ],
```

(`tasks` should be a list of dictionaries. It clearly isn't.)

Instead of doing this we throw an error, that is captured and showing
using the existing import_error mechanism for DAGs. This almost
certainly happens because a user has done "something interesting".
2020-12-07 12:28:12 +00:00
Ash Berlin-Taylor c045ff335e
Store per-task TIDeps in serialized blob (#12858)
Without this change sensors in "reschedule" mode were being instantly
rescheduled because they didn't have the extra dep that
BaseSensorOperator added.

To fix that we need to include deps in the serialization format (but to
save space only when they are not the default list). As of this PR right
now, only built-in deps are allowed -- a custom dep will result in a DAG
parse error.

We can fix that for 2.0.x, as I think it is a _very_ uncommon thing to
do.

Fixes #12783
2020-12-06 21:55:53 +00:00
Ash Berlin-Taylor 4a02e0a287
Don't emit first_task_scheduling_delay metric for only-once dags (#12835)
Dags with a schedule interval of None, or `@once` don't have a following
schedule, so we can't realistically calculate this metric.

Additionally, this changes the emitted metric from seconds to
milliseconds -- all timers to statsd should be in milliseconds -- this
is what Statsd and apps that consume data from there expect. See #10629
for more details.

This will be a "breaking" change from 1.10.14, where the metric was
back-ported to, but was (incorrectly) emitting seconds.
2020-12-05 21:56:51 +00:00
Xiaodong DENG 1f4152b551
Fix docstring for models.Variable.get() (#12828) 2020-12-05 17:08:28 +01:00
Ash Berlin-Taylor 3ff5a35494
Add paused column to `dags list` sub-command (#12830)
This can still show "None" if the dag is not yet in the metadata DB --
showing either True or False there would give a false impression
(especially False -- as if it doesn't exist in the DB it can't be
unpaused yet!)
2020-12-05 14:34:36 +00:00
yuqian90 12ce5be77f
Fix for empty Graph View when task does not have a DAG during relationship setting (#12829)
Closes #12757
2020-12-05 11:52:55 +00:00
Xiaodong DENG fbb8a4a151
Cleanup & improvements around scheduling (#12815)
* Cleanup & improvement around scheduling

- Remove unneeded code line
- Remove stale docstring
- Fix wrong docstring
- Fix stale doc image link in docstring
- avoid unnecessary loop in DagRun.schedule_tis()
- Minor improvement on DAG.deactivate_stale_dags()
  which is invoked inside SchedulerJob

* Revert one change, because we plan to have a dedicated project-wise PR for this issue

* One more fix: dagbag.read_dags_from_db = True in DagFileProcess.process_file() is not needed anymore
2020-12-05 07:12:59 +01:00
Kaxil Naik 101da213c5
Optimize subclasses of DummyOperator for Scheduling (#12745)
Custom operators inheriting from DummyOperator will now instead
 of going to a scheduled state will go set straight to success
 if they don't have callbacks set.

 closes https://github.com/apache/airflow/issues/11393
2020-12-02 12:49:17 +00:00
Kaxil Naik ac3a8bfb0c
Allow switching xcom_pickling to JSON/Pickle (#12724)
Without this commit, the Webserver throws an error when
enabling xcom_pickling in the airflow_config by setting `enable_xcom_pickling = True`
(the default is `False`).

Example error:

```
>           return pickle.loads(result.value)
E           _pickle.UnpicklingError: invalid load key, '{'.

airflow/models/xcom.py:250: UnpicklingError
--------------------------------------------------
```
2020-12-01 11:35:34 +00:00
HasanJ 4ac66cf8c4
Deprecate BaseHook.get_connections method (#10135) (#10192)
Co-authored-by: Ash Berlin-Taylor <ash_github@firemirror.com>
2020-11-30 16:29:30 +00:00
Ash Berlin-Taylor 5e13c37286
Remove deprecated dagbag metrics (#12695)
These were deprecated in 1.10.6 via #6157, so we should remove them
before 2.0 rolls around.
2020-11-30 09:14:15 +00:00
Jarek Potiuk 2037303eef
Adds support for Connection/Hook discovery from providers (#12466)
* Adds support for Hook discovery from providers

This PR extends providers discovery with the mechanism
of retrieving mapping of connections from type to hook.

Fixes #12456

* fixup! Adds support for Hook discovery from providers

* fixup! fixup! Adds support for Hook discovery from providers
2020-11-29 15:31:49 +01:00
Ash Berlin-Taylor 02d94349be
Don't use time.time() or timezone.utcnow() for duration calculations (#12353)
`time.time() - start`, or `timezone.utcnow() - start_dttm` will work
fine in 99% of cases, but it has one fatal flaw:

They both operate on system time, and that can go backwards.

While this might be surprising, it can happen -- usually due to clocks
being adjusted.

And while it is might seem rare, for long running processes it is more
common than we might expect. Most of these durations are harmless to get
wrong (just being logs) it is better to be safe than sorry.

Also the `utcnow()` style I have replaced will be much lighter weight -
creating a date time object is a comparatively expensive operation, and
computing a diff between two even more so, _especially_ when compared to
just subtracting two floats.

To make the "common" case easier of wanting to compute a duration for a
block, I have made `Stats.timer()` return an object that has a
`duration` field.
2020-11-29 10:12:30 +00:00
Ash Berlin-Taylor 8291fabaf9
Ensure that tasks set to up_for_retry have an end date (#12675)
If a task is "manually" set to up_for_retry (via the UI for instance) it
might not have an end date, and much of the logic about computing
retries assumes that it does.

Without this, manually setting a running task to up_for_retry will make
the make it impossible to view the TaskInstance details page (as it
tries to print the is_premature property), and also the NotInRetryPeriod
TIDep fails - both with an exception:

> File "airflow/models/taskinstance.py", line 882, in next_retry_datetime
>   return self.end_date + delay
> TypeError: unsupported operand type(s) for +: 'NoneType' and 'datetime.timedelta'
2020-11-29 10:11:50 +00:00
Kamil Breguła de3b1e687b
Move connection guides to provider documentation packages (#12653) 2020-11-28 08:09:53 +01:00
Jarek Potiuk cdaaff12c7
Fix Connection.description migration for MySQL8 (#12596)
Due to not executing MySQL8 tests Fixed in #12591 added
description for connection table was not compatible with
MySQL8 with utf8mb4 character set.

This change adds migration and fixes the previous migration
to make it compatible.
2020-11-25 14:30:31 +01:00
Ehsan Poursaeed 08251c145d
Remove foreign key constraint on SerializedDagModel's dag_runs field (#12586)
Issue: https://github.com/apache/airflow/issues/12448
2020-11-25 02:07:10 +00:00
Daniel Imberman 6caf2607e0
Don't set child tasks to schedulable in test runs (#12595)
Fixes a bug where Airflow will attempt to set child tasks to schedulable
for test tasks when users run `airflow task test.` This causes an error
as Airflow runs a DB seek for a task that has not been recorded.
2020-11-24 13:43:22 -08:00
QP Hou 01ff088dfb
Fix Dag Serialization crash caused by preset DagContext (#12530) 2020-11-23 20:42:50 +00:00
Abhilash Kishore bf6da166a9
Add description field to connection (#10873)
closes https://github.com/apache/airflow/issues/10840
2020-11-18 11:00:30 +00:00
Tomek Urbaszek a4aa32b875
Simplify using XComArg in jinja template string (#12405)
This changes XComArg string representation from 'task_instance.pull(...)'
to '{{ task_instance.xcom_pull(...) }}' so users can use XComArgs with
f-string (and other) in simpler way. Instead of doing
f'echo {{{{ {op.output} }}}}' they can simply do f'echo {op.output}'.
2020-11-17 19:35:12 +01:00
Tomek Urbaszek 1623df8721
Use different deserialization method in XCom init_on_load (#12327)
The init_on_load method used deserialize_value method which
in case of custom XCom backends may perform requests to external
services (for example downloading file from buckets).

This is problematic because wherever we query XCom the resuest would be
send (for example when listing XCom in webui). This PR proposes implementing
orm_deserialize_value which allows overriding this behavior. By default
we use BaseXCom.deserialize_value.

closes: #12315
2020-11-16 13:32:36 +01:00
Ace Haidrey f32497395a
Add success/failed sets to State class (#12359)
Co-authored-by: Ace Haidrey <ahaidrey@pinterest.com>
2020-11-14 09:22:32 +01:00
Ace Haidrey aac3877ec3
Add metric for scheduling delay between first run task & expected start time (#9544)
Co-authored-by: Ace Haidrey <ahaidrey@pinterest.com>
2020-11-13 23:03:42 +01:00
Daniel Imberman 4e362c1347
K8s yaml templates not rendered by k8sexecutor (#12303)
* K8s yaml templates not rendered by k8sexecutor

There is a bug in the yaml template rendering caused by the logic that
yaml templates are only generated when the current executor is the
k8sexecutor. This is a problem as the templates are generated by the
task pod, which is itself running a LocalExecutor. Also generates a
"base" template if this taskInstance has not run yet.

* fix tests

* fix taskinstance test

* fix taskinstance

* fix pod generator tests

* fix podgen

* Update tests/kubernetes/test_pod_generator.py

Co-authored-by: Ash Berlin-Taylor <ash_github@firemirror.com>

* @ashb comment

Co-authored-by: Ash Berlin-Taylor <ash_github@firemirror.com>
2020-11-13 12:06:29 -08:00
Ryan Hamilton e5e47dac47
Fix/Enhancement: Disable forms and communicate to user when no DAG Runs (#12320)
* Disable forms and communicate to user when no DAG runs yet

* Refactor method name to not use negation in name

* lint fix
2020-11-13 12:46:59 -05:00
Tomek Urbaszek 1222ebd4e1
Create DAG-level cluster policy (#12184)
This commit adds new concept of dag_policy which is checked
once for every DAG when creating DagBag. It also improves
documentation around cluster policies.

closes: #12179

Co-authored-by: Kaxil Naik <kaxilnaik@gmail.com>
Co-authored-by: Ash Berlin-Taylor <ash_github@firemirror.com>
2020-11-13 14:32:49 +01:00
Ephraim Anierobi 0d37c59669
Make dag_id, task_id, and execution_date nullable in event log schema (#12287) 2020-11-11 20:10:13 +01:00
Xiaodong DENG dd2095f4a8
Simplify string expressions & Use f-string (#12216)
* Simplify string expressions & Use f-string

This is a follow-up clean-up work for the minor issues caused in the process of introducing Black

* Fixup
2020-11-10 08:48:27 +01:00
Ephraim Anierobi f37c6e6fce
Add Compute Engine SSH hook (#9879) 2020-11-10 02:20:38 +01:00
Daniel Imberman 90a147813a
Render k8s yaml for tasks via the Airflow UI (#11815)
This function allows users of the k8s executor to get previews
of their tasks via the Airflow UI before they launch

Co-authored-by: Ryan Hamilton <ryan@ryanahamilton.com>
Co-authored-by: Kaxil Naik <kaxilnaik@gmail.com>
2020-11-09 19:55:48 +00:00
Ash Berlin-Taylor 55c401dbf9
Remove BaseDag and BaseDagBag classes (#12195)
Since #7694 these haven't really be needed, but we hadn't removed them
yet.

No UPDATING.md note for this as I think it's extremely unlikely anyone
was using this directly -- it's very much an implementation detail
relating to DAG/SimpleDag.
2020-11-09 15:34:27 +00:00
Ash Berlin-Taylor bdcb6f8d2a
Remove the ability to add hooks to airflow.hooks namespace (#12108)
Hooks do not need to live under "airflow.hooks" namespace for them to
work -- so remove the ability to create them under there in plugins.

Using them as normal python imports is good enough!

We still allow them to be "registered" to support dynamically populating
the connections list in the UI (which won't be done for 2.0)

Closes #9507
2020-11-06 13:24:10 +00:00
Kamil Breguła 41bf172c1d
Simplify string expressions (#12093) 2020-11-04 18:31:08 +01:00