techcodie opened a new pull request, #63775:
URL: https://github.com/apache/airflow/pull/63775
## Summary
When using any Databricks operator with `deferrable=True` in Airflow 3, the
trigger fails immediately with:
```
RuntimeError: You cannot use AsyncToSync in the same thread as an async
event loop
- just await the async function directly.
```
## Root Cause
The `databricks_conn` `@cached_property` lazily fetches the connection on
first access via the **sync** `get_connection()` path. In Airflow 3, when a
trigger runs inside the triggerer's async event loop, that sync path goes
through `SUPERVISOR_COMMS.send()` which calls `async_to_sync(self.asend)(msg)`
— forbidden inside a running event loop.
**Full call chain:**
```
DatabricksExecutionTrigger.run() # async, inside triggerer event
loop
→ hook.a_get_run_state() # async
→ _a_do_api_call() # async
→ _endpoint_url() # sync — accesses
self.databricks_conn
→ databricks_conn @cached_property # first access, not yet cached
→ get_connection()
→ Connection.get_connection_from_secrets()
→ TaskSDKConnection.get()
→ SUPERVISOR_COMMS.send()
→ async_to_sync(self.asend)(msg) ← CRASH
```
## Fix
Added `_a_get_databricks_conn()` — an async method that uses the
already-existing `_async_get_connection()` from
`airflow.sdk.execution_time.context`, which uses `await comms_decoder.asend()`
directly and is safe inside an event loop.
`_a_do_api_call()` now calls `await self._a_get_databricks_conn()` at the
top, populating `self.__dict__["databricks_conn"]` (satisfying the
`@cached_property` cache) before any sync attribute access occurs.
No behaviour change for non-async (sync) code paths.
## Changes
| File | Change |
|------|--------|
|
`providers/databricks/src/airflow/providers/databricks/hooks/databricks_base.py`
| Added `_a_get_databricks_conn()` async method; added one `await` call at the
top of `_a_do_api_call()` |
## How to reproduce
```python
from datetime import datetime, timezone
from airflow import DAG
from airflow.providers.databricks.operators.databricks import
DatabricksTaskOperator
with DAG(
dag_id="test_deferrable_databricks",
start_date=datetime(2024, 1, 1, tzinfo=timezone.utc),
schedule=None,
catchup=False,
):
DatabricksTaskOperator(
task_id="test_task",
databricks_conn_id="databricks_default",
deferrable=True,
task_config={"spark_python_task": {"python_file":
"dbfs:/some/script.py"}},
new_cluster={
"spark_version": "13.3.x-scala2.12",
"node_type_id": "i3.xlarge",
"num_workers": 1,
},
)
```
---
##### Was generative AI tooling used to co-author this PR?
- [X] Yes (Amazon Q Developer)
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]