amoghrajesh opened a new pull request, #52693:
URL: https://github.com/apache/airflow/pull/52693
<!--
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.
-->
<!--
Thank you for contributing! Please make sure that your code changes
are covered with tests. And in case of new features or big changes
remember to adjust the documentation.
Feel free to ping committers for the review!
In case of an existing issue, reference it using one of the following:
closes: #ISSUE
related: #ISSUE
How to write a good git commit message:
http://chris.beams.io/posts/git-commit/
-->
closes: https://github.com/apache/airflow/issues/51809
## What?
https://github.com/apache/airflow/pull/50359/commits/74e44ba6712d1c1bfb63fb6c18fb668347875222
adds support to store the trigger kwargs properly in the DB (serialized
format) to fail early in cases when the kwargs aren't serialisable and / or
aren't jsonable.
This introduces a regression when trigger is executing due and leads to an
error like:
```
Trigger ID 12 exited with error apply_function() got an unexpected keyword
argument '__var' [airflow.jobs.triggerer_job_runner]
error_detail=[{'exc_type': 'TypeError', 'exc_value': "apply_function() got
an unexpected keyword argument '__var'",
'exc_notes': [], 'syntax_error': None, 'is_cause': False, 'frames': [
{'filename':
'/home/airflow/.local/lib/python3.12/site-packages/airflow/jobs/triggerer_job_runner.py',
'lineno': 901, 'name': 'cleanup_finished_triggers'},
{'filename':
'/home/airflow/.local/lib/python3.12/site-packages/airflow/jobs/triggerer_job_runner.py',
'lineno': 1014, 'name': 'run_trigger'},
{'filename':
'/home/airflow/.local/lib/python3.12/site-packages/airflow/providers/apache/kafka/triggers/await_message.py',
'lineno': 111, 'name': 'run'},
{'filename':
'/home/airflow/.local/lib/python3.12/site-packages/asgiref/sync.py', 'lineno':
468, 'name': '__call__'}, {'filename':
'/usr/local/lib/python3.12/concurrent/futures/thread.py', 'lineno': 59, 'name':
'run'},
{'filename':
'/home/airflow/.local/lib/python3.12/site-packages/asgiref/sync.py', 'lineno':
522, 'name': 'thread_handler'}],
'is_group': False, 'exceptions': []}]
```
The issue is that the triggers are encoded and stored in the database but
they aren't deserialised properly in the assetwatcher execution path. The
kwargs end up coming in their serialised form leading to failures.
## Fix
- Fixing this by introducing the same deserialisation path as used in
`decode_asset` for display.
- I have not added the fix to `Trigger._decrypt_kwargs` because that is used
in here
https://github.com/apache/airflow/blob/main/airflow-core/src/airflow/models/trigger.py#L121-L124
due to which there could be issues in `add_asset_trigger_references` while
writing to DB due to no triggers being found due to different hashes.
## Testing
### Step 1: Run breeze with kafka integration
```
breeze start-airflow --backend postgres --executor CeleryExecutor --dev-mode
--db-reset --load-default-connections --integration kafka
```
It will have a kafka setup installed with it at default host and ports, good
enough for us.
### Step 2: Add the dag:
```
import json
from airflow.providers.common.messaging.triggers.msg_queue import
MessageQueueTrigger
from airflow.providers.standard.operators.empty import EmptyOperator
from airflow.sdk import dag, Asset, AssetWatcher
def apply_function(message, **kwargs):
print("Kwargs are", kwargs)
val = json.loads(message.value())
print(f"Value in message is {val}")
return True
trigger = MessageQueueTrigger(
queue="kafka://localhost:9092/test",
apply_function="kafka_example_dag.apply_function",
kafka_config_id="kafka_default"
)
asset = Asset("kafka_queue_asset_2",
watchers=[AssetWatcher(name="kafka_watcher_2", trigger=trigger)])
@dag(dag_id="example_kafka_watcher_2", schedule=[asset])
def example_kafka_watcher():
EmptyOperator(task_id="task")
example_kafka_watcher()
```
### Step 3: Enable the dag and try running it normally
<img width="1722" alt="image"
src="https://github.com/user-attachments/assets/3cdb3652-0edd-40a2-a0b4-b89674907afd"
/>
### Step 4: Send a kafka message in the "test" topic
Wrote a script to do so:
```
import json
from kafka import KafkaProducer
producer = KafkaProducer(
bootstrap_servers=["localhost:9092"], value_serializer=lambda v:
json.dumps(v).encode("utf-8")
)
message = {"foo": "bar"}
future = producer.send("test", message)
result = future.get(timeout=10)
print(f"Message sent to topic 'test'")
print(f"Message: {message}")
producer.close()
```
Run the script:
```
➜ airflow git:(fixing-trigger-regression) ✗ python send_message.py
Message sent to topic 'test' at offset 4
Message: {'foo': 'bar'}
```
Your dag should start running but from that asset trigger:
<img width="1722" alt="image"
src="https://github.com/user-attachments/assets/7005eea2-d815-4052-b1f4-f38017e3d46d"
/>
<img width="1722" alt="image"
src="https://github.com/user-attachments/assets/9dc42bf8-2edf-4cc4-8c6b-cd3817ae3257"
/>
Observe triggerer logs:
```
[2025-07-02T08:03:46.081+0000] {_client.py:1026} INFO - HTTP Request: GET
http://in-process.invalid./connections/kafka_default "HTTP/1.1 200 OK"
2025-07-02 08:03:46 [info ] Connection Retrieved 'kafka_default'
[airflow.hooks.base] [airflow.jobs.triggerer_job_runner]
2025-07-02 08:03:46 [info ] Triggerer's async thread was blocked for
0.30 seconds, likely by a badly-written trigger. Set PYTHONASYNCIODEBUG=1 to
get more information on overrunning coroutines.
[airflow.jobs.triggerer_job_runner]
2025-07-02 08:04:01 [info ] Kwargs are {}
[airflow.jobs.triggerer_job_runner] chan=stdout
2025-07-02 08:04:01 [info ] Value in message is {'foo': 'bar'}
[airflow.jobs.triggerer_job_runner] chan=stdout
2025-07-02 08:04:01 [info ] Trigger fired event
[airflow.jobs.triggerer_job_runner] name='ID 1' result=TriggerEvent<True>
2025-07-02 08:04:01 [info ] trigger completed
[airflow.jobs.triggerer_job_runner] name='ID 1'
[2025-07-02T08:04:02.545+0000] {_client.py:1026} INFO - HTTP Request: GET
http://in-process.invalid./connections/kafka_default "HTTP/1.1 200 OK"
2025-07-02 08:04:02 [info ] Before the deser kwargs are {'topics':
['test'], 'apply_function': 'kafka_example_dag.apply_function',
'apply_function_args': {'__var': [], '__type': 'tuple'},
'apply_function_kwargs': {'__var': {}, '__type': 'dict'}, 'kafka_config_id':
'kafka_default', 'poll_timeout': 1, 'poll_interval': 5}
[airflow.jobs.triggerer_job_runner] chan=stdout
2025-07-02 08:04:02 [info ] After the deser kwargs are {'topics':
['test'], 'apply_function': 'kafka_example_dag.apply_function',
'apply_function_args': (), 'apply_function_kwargs': {}, 'kafka_config_id':
'kafka_default', 'poll_timeout': 1, 'poll_interval': 5}
[airflow.jobs.triggerer_job_runner] chan=stdout
2025-07-02 08:04:02 [info ] trigger ID 1 starting
[airflow.jobs.triggerer_job_runner]
```
<!-- Please keep an empty line above the dashes. -->
---
**^ Add meaningful description above**
Read the **[Pull Request
Guidelines](https://github.com/apache/airflow/blob/main/contributing-docs/05_pull_requests.rst#pull-request-guidelines)**
for more information.
In case of fundamental code changes, an Airflow Improvement Proposal
([AIP](https://cwiki.apache.org/confluence/display/AIRFLOW/Airflow+Improvement+Proposals))
is needed.
In case of a new dependency, check compliance with the [ASF 3rd Party
License Policy](https://www.apache.org/legal/resolved.html#category-x).
In case of backwards incompatible changes please leave a note in a
newsfragment file, named `{pr_number}.significant.rst` or
`{issue_number}.significant.rst`, in
[airflow-core/newsfragments](https://github.com/apache/airflow/tree/main/airflow-core/newsfragments).
--
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]