Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Deserialization of old xcom data fails after upgrade to 2.6.1 from 2.5.2 when calling /xcom/list/ [GET] #31769

Closed
1 of 2 tasks
jherrmannNetfonds opened this issue Jun 7, 2023 · 6 comments · Fixed by #31866
Assignees
Labels
area:core kind:bug This is a clearly a bug
Milestone

Comments

@jherrmannNetfonds
Copy link

jherrmannNetfonds commented Jun 7, 2023

Apache Airflow version

2.6.1

What happened

After upgrading from airflow 2.5.2 to 2.6.1 calling the endpoint xcom/list/ we get the following exception:

[2023-06-07T12:16:50.050+0000] {app.py:1744} ERROR - Exception on /xcom/list/ [GET]
Traceback (most recent call last):
  File "/home/airflow/.local/lib/python3.10/site-packages/flask/app.py", line 2529, in wsgi_app
    response = self.full_dispatch_request()
  File "/home/airflow/.local/lib/python3.10/site-packages/flask/app.py", line 1825, in full_dispatch_request
    rv = self.handle_user_exception(e)
  File "/home/airflow/.local/lib/python3.10/site-packages/flask/app.py", line 1823, in full_dispatch_request
    rv = self.dispatch_request()
  File "/home/airflow/.local/lib/python3.10/site-packages/flask/app.py", line 1799, in dispatch_request
    return self.ensure_sync(self.view_functions[rule.endpoint])(**view_args)
  File "/home/airflow/.local/lib/python3.10/site-packages/flask_appbuilder/security/decorators.py", line 139, in wraps
    return f(self, *args, **kwargs)
  File "/home/airflow/.local/lib/python3.10/site-packages/flask_appbuilder/views.py", line 554, in list
    widgets = self._list()
  File "/home/airflow/.local/lib/python3.10/site-packages/flask_appbuilder/baseviews.py", line 1177, in _list
    widgets = self._get_list_widget(
  File "/home/airflow/.local/lib/python3.10/site-packages/flask_appbuilder/baseviews.py", line 1076, in _get_list_widget
    count, lst = self.datamodel.query(
  File "/home/airflow/.local/lib/python3.10/site-packages/flask_appbuilder/models/sqla/interface.py", line 500, in query
    query_results = query.all()
  File "/home/airflow/.local/lib/python3.10/site-packages/sqlalchemy/orm/query.py", line 2773, in all
    return self._iter().all()
  File "/home/airflow/.local/lib/python3.10/site-packages/sqlalchemy/engine/result.py", line 1476, in all
    return self._allrows()
  File "/home/airflow/.local/lib/python3.10/site-packages/sqlalchemy/engine/result.py", line 401, in _allrows
    rows = self._fetchall_impl()
  File "/home/airflow/.local/lib/python3.10/site-packages/sqlalchemy/engine/result.py", line 1389, in _fetchall_impl
    return self._real_result._fetchall_impl()
  File "/home/airflow/.local/lib/python3.10/site-packages/sqlalchemy/engine/result.py", line 1813, in _fetchall_impl
    return list(self.iterator)
  File "/home/airflow/.local/lib/python3.10/site-packages/sqlalchemy/orm/loading.py", line 151, in chunks
    rows = [proc(row) for row in fetch]
  File "/home/airflow/.local/lib/python3.10/site-packages/sqlalchemy/orm/loading.py", line 151, in <listcomp>
    rows = [proc(row) for row in fetch]
  File "/home/airflow/.local/lib/python3.10/site-packages/sqlalchemy/orm/loading.py", line 984, in _instance
    state.manager.dispatch.load(state, context)
  File "/home/airflow/.local/lib/python3.10/site-packages/sqlalchemy/event/attr.py", line 334, in __call__
    fn(*args, **kw)
  File "/home/airflow/.local/lib/python3.10/site-packages/sqlalchemy/orm/mapper.py", line 3702, in _event_on_load
    instrumenting_mapper._reconstructor(state.obj())
  File "/home/airflow/.local/lib/python3.10/site-packages/airflow/models/xcom.py", line 128, in init_on_load
    self.value = self.orm_deserialize_value()
  File "/home/airflow/.local/lib/python3.10/site-packages/airflow/models/xcom.py", line 677, in orm_deserialize_value
    return BaseXCom._deserialize_value(self, True)
  File "/home/airflow/.local/lib/python3.10/site-packages/airflow/models/xcom.py", line 659, in _deserialize_value
    return json.loads(result.value.decode("UTF-8"), cls=XComDecoder, object_hook=object_hook)
  File "/usr/local/lib/python3.10/json/__init__.py", line 359, in loads
    return cls(**kw).decode(s)
  File "/usr/local/lib/python3.10/json/decoder.py", line 337, in decode
    obj, end = self.raw_decode(s, idx=_w(s, 0).end())
  File "/usr/local/lib/python3.10/json/decoder.py", line 353, in raw_decode
    obj, end = self.scan_once(s, idx)
  File "/home/airflow/.local/lib/python3.10/site-packages/airflow/utils/json.py", line 126, in orm_object_hook
    return deserialize(dct, False)
  File "/home/airflow/.local/lib/python3.10/site-packages/airflow/serialization/serde.py", line 209, in deserialize
    o = _convert(o)
  File "/home/airflow/.local/lib/python3.10/site-packages/airflow/serialization/serde.py", line 273, in _convert
    return {CLASSNAME: old[OLD_TYPE], VERSION: DEFAULT_VERSION, DATA: old[OLD_DATA][OLD_DATA]}
KeyError: '__var'

Some xcom entries from previous airflow versions seem to be incompatible with the new refactored serialization from #28067

What you think should happen instead

xcom entries should be displayed

How to reproduce

Add an entry to your the xcom table where value contains:
[{"__classname__": "airflow.datasets.Dataset", "__version__": 1, "__data__": {"__var": {"uri": "bq://google_cloud_default@?table=table_name&schema=schema_name", "extra": null}, "__type": "dict"}}]

Operating System

Debian GNU/Linux 11 (bullseye)

Versions of Apache Airflow Providers

apache-airflow-providers-amazon==8.0.0
apache-airflow-providers-apache-kafka==1.1.0
apache-airflow-providers-celery==3.1.0
apache-airflow-providers-cncf-kubernetes==6.1.0
apache-airflow-providers-common-sql==1.4.0
apache-airflow-providers-docker==3.6.0
apache-airflow-providers-elasticsearch==4.4.0
apache-airflow-providers-ftp==3.3.1
apache-airflow-providers-google==10.1.1
apache-airflow-providers-grpc==3.1.0
apache-airflow-providers-hashicorp==3.3.1
apache-airflow-providers-http==4.3.0
apache-airflow-providers-imap==3.1.1
apache-airflow-providers-microsoft-azure==6.0.0
apache-airflow-providers-mysql==5.0.0
apache-airflow-providers-odbc==3.2.1
apache-airflow-providers-postgres==5.4.0
apache-airflow-providers-redis==3.1.0
apache-airflow-providers-sendgrid==3.1.0
apache-airflow-providers-sftp==4.2.4
apache-airflow-providers-slack==7.2.0
apache-airflow-providers-snowflake==4.0.5
apache-airflow-providers-sqlite==3.3.2
apache-airflow-providers-ssh==3.6.0

Deployment

Other 3rd-party Helm chart

Deployment details

No response

Anything else

The double old[OLD_DATA][OLD_DATA] looks suspicious to me in

return {CLASSNAME: old[OLD_TYPE], VERSION: DEFAULT_VERSION, DATA: old[OLD_DATA][OLD_DATA]}

Are you willing to submit PR?

  • Yes I am willing to submit a PR!

Code of Conduct

@jherrmannNetfonds jherrmannNetfonds added area:core kind:bug This is a clearly a bug needs-triage label for new issues that we didn't triage yet labels Jun 7, 2023
@boring-cyborg
Copy link

boring-cyborg bot commented Jun 7, 2023

Thanks for opening your first issue here! Be sure to follow the issue template! If you are willing to raise PR to address this issue please do so, no need to wait for approval.

@jherrmannNetfonds
Copy link
Author

I can change the code at

return {CLASSNAME: old[OLD_TYPE], VERSION: DEFAULT_VERSION, DATA: old[OLD_DATA][OLD_DATA]}
and open a pull request. Since I cannot create a dev environment on my work PC, I wont be able to add a test for the change.
@bolkedebruin since you are the author of that code, plz advice.

@phanikumv phanikumv removed the needs-triage label for new issues that we didn't triage yet label Jun 13, 2023
@utkarsharma2
Copy link
Contributor

@phanikumv Please assign this issue to me.

@phanikumv
Copy link
Contributor

I was just thinking whether adding this to 2.6.2 make sense? @jedcunningham @ephraimbuddy

@phanikumv phanikumv added this to the Airlfow 2.6.3 milestone Jun 13, 2023
@bolkedebruin
Copy link
Contributor

Can you tell me how you got:

[{"__classname__": "airflow.datasets.Dataset", "__version__": 1, "__data__": {"__var": {"uri": "bq://google_cloud_default@?table=table_name&schema=schema_name", "extra": null}, "__type": "dict"}}]

As that is mixing the new style serialized form with the old style form: __classname__ is new style, but __var is old style.

@jherrmannNetfonds
Copy link
Author

that output was created with airflow 2.5.x (1 or 2) on a task with:

from airflow import Dataset

dbt_test = DbtTestOperator(
   task_id="dbt_test",
   ... omitted ...
   outlets=Dataset("bq://google_cloud_default@?table=table_name&schema=schema_name"),
)

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
area:core kind:bug This is a clearly a bug
Projects
None yet
Development

Successfully merging a pull request may close this issue.

4 participants