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

Airflow 2.3 scheduler error: 'V1Container' object has no attribute '_startup_probe' #23727

Closed
1 of 2 tasks
patryk126p opened this issue May 16, 2022 · 41 comments · Fixed by #24117
Closed
1 of 2 tasks
Assignees
Labels
area:core kind:bug This is a clearly a bug
Milestone

Comments

@patryk126p
Copy link

patryk126p commented May 16, 2022

Apache Airflow version

2.3.0 (latest released)

What happened

After migrating from Airflow 2.2.4 to 2.3.0 scheduler fell into crash loop throwing:

--- Logging error ---
Traceback (most recent call last):
  File "/home/airflow/.local/lib/python3.9/site-packages/airflow/jobs/scheduler_job.py", line 736, in _execute
    self._run_scheduler_loop()
  File "/home/airflow/.local/lib/python3.9/site-packages/airflow/jobs/scheduler_job.py", line 826, in _run_scheduler_loop
    self.executor.heartbeat()
  File "/home/airflow/.local/lib/python3.9/site-packages/airflow/executors/base_executor.py", line 171, in heartbeat
    self.sync()
  File "/home/airflow/.local/lib/python3.9/site-packages/airflow/executors/kubernetes_executor.py", line 613, in sync
    self.kube_scheduler.run_next(task)
  File "/home/airflow/.local/lib/python3.9/site-packages/airflow/executors/kubernetes_executor.py", line 300, in run_next
    self.log.info('Kubernetes job is %s', str(next_job).replace("\n", " "))
  File "/home/airflow/.local/lib/python3.9/site-packages/kubernetes/client/models/v1_pod.py", line 214, in __repr__
    return self.to_str()
  File "/home/airflow/.local/lib/python3.9/site-packages/kubernetes/client/models/v1_pod.py", line 210, in to_str
    return pprint.pformat(self.to_dict())
  File "/home/airflow/.local/lib/python3.9/site-packages/kubernetes/client/models/v1_pod.py", line 196, in to_dict
    result[attr] = value.to_dict()
  File "/home/airflow/.local/lib/python3.9/site-packages/kubernetes/client/models/v1_pod_spec.py", line 1070, in to_dict
    result[attr] = list(map(
  File "/home/airflow/.local/lib/python3.9/site-packages/kubernetes/client/models/v1_pod_spec.py", line 1071, in <lambda>
    lambda x: x.to_dict() if hasattr(x, "to_dict") else x,
  File "/home/airflow/.local/lib/python3.9/site-packages/kubernetes/client/models/v1_container.py", line 672, in to_dict
    value = getattr(self, attr)
  File "/home/airflow/.local/lib/python3.9/site-packages/kubernetes/client/models/v1_container.py", line 464, in startup_probe
    return self._startup_probe
AttributeError: 'V1Container' object has no attribute '_startup_probe'

During handling of the above exception, another exception occurred:

Traceback (most recent call last):
  File "/usr/local/lib/python3.9/logging/__init__.py", line 1083, in emit
    msg = self.format(record)
  File "/usr/local/lib/python3.9/logging/__init__.py", line 927, in format
    return fmt.format(record)
  File "/usr/local/lib/python3.9/logging/__init__.py", line 663, in format
    record.message = record.getMessage()
  File "/usr/local/lib/python3.9/logging/__init__.py", line 367, in getMessage
    msg = msg % self.args
  File "/home/airflow/.local/lib/python3.9/site-packages/kubernetes/client/models/v1_pod.py", line 214, in __repr__
    return self.to_str()
  File "/home/airflow/.local/lib/python3.9/site-packages/kubernetes/client/models/v1_pod.py", line 210, in to_str
    return pprint.pformat(self.to_dict())
  File "/home/airflow/.local/lib/python3.9/site-packages/kubernetes/client/models/v1_pod.py", line 196, in to_dict
    result[attr] = value.to_dict()
  File "/home/airflow/.local/lib/python3.9/site-packages/kubernetes/client/models/v1_pod_spec.py", line 1070, in to_dict
    result[attr] = list(map(
  File "/home/airflow/.local/lib/python3.9/site-packages/kubernetes/client/models/v1_pod_spec.py", line 1071, in <lambda>
    lambda x: x.to_dict() if hasattr(x, "to_dict") else x,
  File "/home/airflow/.local/lib/python3.9/site-packages/kubernetes/client/models/v1_container.py", line 672, in to_dict
    value = getattr(self, attr)
  File "/home/airflow/.local/lib/python3.9/site-packages/kubernetes/client/models/v1_container.py", line 464, in startup_probe
    return self._startup_probe
AttributeError: 'V1Container' object has no attribute '_startup_probe'
Call stack:
  File "/home/airflow/.local/bin/airflow", line 8, in <module>
    sys.exit(main())
  File "/home/airflow/.local/lib/python3.9/site-packages/airflow/__main__.py", line 38, in main
    args.func(args)
  File "/home/airflow/.local/lib/python3.9/site-packages/airflow/cli/cli_parser.py", line 51, in command
    return func(*args, **kwargs)
  File "/home/airflow/.local/lib/python3.9/site-packages/airflow/utils/cli.py", line 99, in wrapper
    return f(*args, **kwargs)
  File "/home/airflow/.local/lib/python3.9/site-packages/airflow/cli/commands/scheduler_command.py", line 75, in scheduler
    _run_scheduler_job(args=args)
  File "/home/airflow/.local/lib/python3.9/site-packages/airflow/cli/commands/scheduler_command.py", line 46, in _run_scheduler_job
    job.run()
  File "/home/airflow/.local/lib/python3.9/site-packages/airflow/jobs/base_job.py", line 244, in run
    self._execute()
  File "/home/airflow/.local/lib/python3.9/site-packages/airflow/jobs/scheduler_job.py", line 757, in _execute
    self.executor.end()
  File "/home/airflow/.local/lib/python3.9/site-packages/airflow/executors/kubernetes_executor.py", line 809, in end
    self._flush_task_queue()
  File "/home/airflow/.local/lib/python3.9/site-packages/airflow/executors/kubernetes_executor.py", line 767, in _flush_task_queue
    self.log.warning('Executor shutting down, will NOT run task=%s', task)
Unable to print the message and arguments - possible formatting error.
Use the traceback above to help find the error.

kubernetes python library version was exactly as specified in constraints file: https://raw.githubusercontent.com/apache/airflow/constraints-2.3.0/constraints-3.9.txt

What you think should happen instead

Scheduler should work

How to reproduce

Not 100% sure but:

  1. Run Airflow 2.2.4 using official Helm Chart
  2. Run some dags to have some records in DB
  3. Migrate to 2.3.0 (replace 2.2.4 image with 2.3.0 one)

Operating System

Debian GNU/Linux 11 (bullseye)

Versions of Apache Airflow Providers

irrelevant

Deployment

Official Apache Airflow Helm Chart

Deployment details

KubernetesExecutor
PostgreSQL (RDS) as Airflow DB
Python 3.9
Docker images build from apache/airflow:2.3.0-python3.9 (some additional libraries installed)

Anything else

No response

Are you willing to submit PR?

  • Yes I am willing to submit a PR!

Code of Conduct

@patryk126p patryk126p added area:core kind:bug This is a clearly a bug labels May 16, 2022
@boring-cyborg
Copy link

boring-cyborg bot commented May 16, 2022

Thanks for opening your first issue here! Be sure to follow the issue template!

@potiuk
Copy link
Member

potiuk commented May 18, 2022

There are few options where it could get wrong:

  • you might have a startup_probe defined in one of your pod template files (can you please take a look and let us know). @patryk126p ?
  • there are some left-over serialized objects in airflow DB that are left-over from 2.2.4 using the old version of kubernetes library - in which case we might want to catch such error and fix or remove the serialized data and reserialize it

@dstandish - if the second hypothesiss is confirmed, it might actually mean that we have to implement a fix for 2.3.1 as it might prevent the migration for a number of users.

@potiuk potiuk added this to the Airflow 2.3.1 milestone May 18, 2022
@patryk126p
Copy link
Author

We have no startup_probes defined and we are using the default pod template file that comes with the Helm chart - we only overwrite some options (resources limits and requests) via executor_config to match task needs

@potiuk
Copy link
Member

potiuk commented May 18, 2022

We have no startup_probes defined and we are using the default pod template file that comes with the Helm chart - we only overwrite some options (resources limits and requests) via executor_config to match task needs

So this is likely some serialization issue as I suspected.

@potiuk
Copy link
Member

potiuk commented May 18, 2022

I will wait for @dstandish comment - maybe this is a "known" issue already, but for me it looks like one that should be solved in 2.3.1 as it's quite likely caused by the k8s library version migration between 2.2 and 2.3 in general.

@dstandish
Copy link
Contributor

taking a look at this.

@dstandish dstandish self-assigned this May 18, 2022
@dstandish
Copy link
Contributor

@patryk126p can you provide a sample dag? i attempted to repro using your steps (ran a simple dag in 2.2.4, upgraded to 2.3.0, ran it again) and there was no issue. perhaps you are doing something with executor config? or perhaps specifying full_pod_spec in kubernetes pod operator?

@dstandish
Copy link
Contributor

actually yeah does look like you must be putting something in executor config. please share the code related to this. thanks.

@patryk126p
Copy link
Author

@dstandish we are not using kubernetes pod operators nor specifying full_pod_spec. When it comes to executor_config we use something like this:

{
    "pod_override": V1Pod(
        spec=V1PodSpec(
            containers=[
                V1Container(
                    name="base",
                    resources=V1ResourceRequirements(
                        limits={"cpu": "<X>", "memory": "<X>"},
                        requests={"cpu": "<X>", "memory": "<X>"},
                    ),
                )
            ]
        ),
#        metadata=V1ObjectMeta(
#            annotations={"cluster-autoscaler.kubernetes.io/safe-to-evict": "false"}
#        ),
    )
}

Metadata/annotations are added only in case of very long running and critical tasks, to be sure that kubernetes won't evict pods in the middle of processing.
Sample dag may be a little tricky to provide as in almost all dags we are using custom operators (mostly based directly on BaseOperator), I could provide simplified version using PythonOperators but I'm not sure if that would be of much help. In majority of dags the flow is simple: collect data from API, dump it to S3, process (if needed), load to stage tables, load to final tables.
In general we are not doing anything related to startup_probe and the only direct link between our dag/task definitions and kubernetes is the executor_config that I've provided

@dstandish
Copy link
Contributor

@patryk126p please show the imports for those classes

If you can establish repro steps too that would be really helpful too

@patryk126p
Copy link
Author

Here is the complete set of imports from all custom operators/sensors/hooks (somme of the imports are for type hints only):

import base64
import json
import logging
import math
import operator
import os
import pathlib
import re
import sys
import time
import typing
from collections import namedtuple
from contextlib import closing
from datetime import datetime, timedelta, timezone
from subprocess import PIPE, CompletedProcess, Popen, SubprocessError, run
from urllib.parse import urlencode

import backoff
import certifi
import google_auth_httplib2
import httplib2
import newrelic_telemetry_sdk
import pandas as pd
import psycopg2
import redis
import requests
import tableauserverclient
import yaml
from airflow.exceptions import AirflowException, AirflowSensorTimeout
from airflow.hooks.base import BaseHook
from airflow.models import DAG, BaseOperator, Connection, Variable
from airflow.providers.amazon.aws.hooks.s3 import S3Hook
from airflow.providers.microsoft.mssql.hooks.mssql import MsSqlHook
from airflow.providers.mysql.hooks.mysql import MySqlHook
from airflow.providers.postgres.hooks.postgres import PostgresHook
from airflow.providers.postgres.operators.postgres import PostgresOperator
from airflow.sensors.base import BaseSensorOperator
from airflow.sensors.external_task import ExternalTaskSensor
from airflow.utils import timezone
from airflow.utils.helpers import parse_template_string
from botocore.credentials import ReadOnlyCredentials
from google.api_core import retry
from google.oauth2 import service_account
from googleapiclient.discovery import Resource, build
from googleapiclient.http import set_user_agent
from jinja2 import Template
from tabulate import tabulate

@potiuk
Copy link
Member

potiuk commented May 19, 2022

Are you sure that the kubernetes library has been upgraded everywhere in your system ? For me it looks like you have some image that has the old version of library still installed.

@patryk126p
Copy link
Author

From inside container (same docker images were used for all components in helm chart):

$ pip freeze | grep -E "airflow|kubernetes"
apache-airflow==2.3.0
apache-airflow-providers-amazon==3.3.0
apache-airflow-providers-celery==2.1.4
apache-airflow-providers-cncf-kubernetes==4.0.1
apache-airflow-providers-docker==2.6.0
apache-airflow-providers-elasticsearch==3.0.3
apache-airflow-providers-ftp==2.1.2
apache-airflow-providers-google==6.8.0
apache-airflow-providers-grpc==2.0.4
apache-airflow-providers-hashicorp==2.2.0
apache-airflow-providers-http==2.1.2
apache-airflow-providers-imap==2.2.3
apache-airflow-providers-microsoft-azure==3.8.0
apache-airflow-providers-mysql==2.2.3
apache-airflow-providers-odbc==2.0.4
apache-airflow-providers-postgres==4.1.0
apache-airflow-providers-redis==2.0.4
apache-airflow-providers-sendgrid==2.0.4
apache-airflow-providers-sftp==2.6.0
apache-airflow-providers-slack==4.2.3
apache-airflow-providers-sqlite==2.1.3
apache-airflow-providers-ssh==2.4.3
google-cloud-orchestration-airflow==1.3.1
kubernetes==23.3.0

@potiuk
Copy link
Member

potiuk commented May 19, 2022

Which Kubernetes version do you run (cluster version) @patryk126p ?

@patryk126p
Copy link
Author

patryk126p commented May 20, 2022

@potiuk it's 1.21.9 on EKS

@dstandish
Copy link
Contributor

@patryk126p was hoping to see how you are importing the k8s objects used here:

{
    "pod_override": V1Pod(
        spec=V1PodSpec(
            containers=[
                V1Container(
                    name="base",
                    resources=V1ResourceRequirements(
                        limits={"cpu": "<X>", "memory": "<X>"},
                        requests={"cpu": "<X>", "memory": "<X>"},
                    ),
                )
            ]
        ),
#        metadata=V1ObjectMeta(
#            annotations={"cluster-autoscaler.kubernetes.io/safe-to-evict": "false"}
#        ),
    )
}

though i know it's a long shot that it leads us anywhere.
thanks

@patryk126p
Copy link
Author

@dstandish are you asking just for import statement? If yes then here it is:

from kubernetes.client.models import (
    V1Container,
    V1ObjectMeta,
    V1Pod,
    V1PodSpec,
    V1ResourceRequirements,
)

@tirkarthi
Copy link
Contributor

@potiuk
Copy link
Member

potiuk commented May 23, 2022

Yeah - it does look as the deserialisation issue as I suspected. Just a thought - we do have the new
airflow dags reserialize command in Airflow.

I wonder @patryk126p if that is something that you could run to see if it helps.

@potiuk
Copy link
Member

potiuk commented May 23, 2022

This command is exactly foreseen for this kind of cases where the dags were serialized using older airflow versions.

I suspect what could have happened is that the old dags were serialized when there was an old kubernetes library installed, and it lacked the_startup_probe, and the new version of library requires the _startup_probe to be there.

In this case airflow dags reserialize should help as it cleans all the serialized dags and re-serializes them using the current version of the library.

@patryk126p
Copy link
Author

Sounds like this may be it. Once we have a window when we can attempt moving to 2.3 again I will try this command. But in the meantime can such info be added to some migration docs or something?

@potiuk
Copy link
Member

potiuk commented May 31, 2022

Any news @patryk126p :) ?

@dstandish
Copy link
Contributor

@hterik @humbledude see #24478 for webserver fix

dstandish added a commit to astronomer/airflow that referenced this issue Jun 15, 2022
When UI unpickles executor_configs with outdated k8s objects it can run into the same issue as the scheduler does (see apache#23727).

Our JSON encoder therefore needs to suppress encoding errors arising from pod serialization, and fallback to a default value.
dstandish added a commit that referenced this issue Jun 15, 2022
When UI unpickles executor_configs with outdated k8s objects it can run into the same issue as the scheduler does (see #23727).

Our JSON encoder therefore needs to suppress encoding errors arising from pod serialization, and fallback to a default value.
@dstandish
Copy link
Contributor

ok @hterik @humbledude webserver fix merged so you are welcome to try patching your envs with those changes

a0x8o added a commit to a0x8o/airflow that referenced this issue Jun 15, 2022
When UI unpickles executor_configs with outdated k8s objects it can run into the same issue as the scheduler does (see apache/airflow#23727).

Our JSON encoder therefore needs to suppress encoding errors arising from pod serialization, and fallback to a default value.
ephraimbuddy pushed a commit to astronomer/airflow that referenced this issue Jun 16, 2022
From time to time k8s library objects change their attrs.  If executor config is stored with old version, and unpickled with new version, we can get attribute errors that can crash the scheduler (see apache#23727).

Here we update handling so that we fail the task but don't crash the scheduler.

(cherry picked from commit 0c41f43)
@humbledude
Copy link

@dstandish I patched with commits on #24478 and it seems work!

@dstandish
Copy link
Contributor

great, thanks

ephraimbuddy pushed a commit that referenced this issue Jun 29, 2022
When UI unpickles executor_configs with outdated k8s objects it can run into the same issue as the scheduler does (see #23727).

Our JSON encoder therefore needs to suppress encoding errors arising from pod serialization, and fallback to a default value.
ephraimbuddy pushed a commit that referenced this issue Jun 30, 2022
From time to time k8s library objects change their attrs.  If executor config is stored with old version, and unpickled with new version, we can get attribute errors that can crash the scheduler (see #23727).

Here we update handling so that we fail the task but don't crash the scheduler.

(cherry picked from commit 0c41f43)
@joshzana
Copy link

joshzana commented Aug 8, 2022

@dstandish I'm seeing this on Airflow 2.3.3 with a vanilla web server docker image, which should have #24478 in it. We use the REST API for an integration with another system of ours, and when it calls the get_task_instances_batch API it's hitting a similar stack:

   File "/home/airflow/.local/lib/python3.9/site-packages/airflow/api_connexion/endpoints/task_instance_endpoint.py", line 412, in get_task_instances_batch
     return task_instance_collection_schema.dump(
   File "/home/airflow/.local/lib/python3.9/site-packages/marshmallow/schema.py", line 557, in dump
     result = self._serialize(processed_obj, many=many)
   File "/home/airflow/.local/lib/python3.9/site-packages/marshmallow/schema.py", line 525, in _serialize
     value = field_obj.serialize(attr_name, obj, accessor=self.get_attribute)
   File "/home/airflow/.local/lib/python3.9/site-packages/marshmallow/fields.py", line 342, in serialize
     return self._serialize(value, attr, obj, **kwargs)
   File "/home/airflow/.local/lib/python3.9/site-packages/marshmallow/fields.py", line 774, in _serialize
     return [self.inner._serialize(each, attr, obj, **kwargs) for each in value]
   File "/home/airflow/.local/lib/python3.9/site-packages/marshmallow/fields.py", line 774, in <listcomp>
     return [self.inner._serialize(each, attr, obj, **kwargs) for each in value]
   File "/home/airflow/.local/lib/python3.9/site-packages/marshmallow/fields.py", line 643, in _serialize
     return schema.dump(nested_obj, many=many)
   File "/home/airflow/.local/lib/python3.9/site-packages/marshmallow/schema.py", line 557, in dump
     result = self._serialize(processed_obj, many=many)
   File "/home/airflow/.local/lib/python3.9/site-packages/marshmallow/schema.py", line 525, in _serialize
     value = field_obj.serialize(attr_name, obj, accessor=self.get_attribute)
   File "/home/airflow/.local/lib/python3.9/site-packages/marshmallow/fields.py", line 342, in serialize
     return self._serialize(value, attr, obj, **kwargs)
   File "/home/airflow/.local/lib/python3.9/site-packages/marshmallow/fields.py", line 893, in _serialize
     return utils.ensure_text_type(value)
   File "/home/airflow/.local/lib/python3.9/site-packages/marshmallow/utils.py", line 212, in ensure_text_type
     return str(val)
   File "/home/airflow/.local/lib/python3.9/site-packages/kubernetes/client/models/v1_pod.py", line 214, in __repr__
     return self.to_str()
   File "/home/airflow/.local/lib/python3.9/site-packages/kubernetes/client/models/v1_pod.py", line 210, in to_str
     return pprint.pformat(self.to_dict())
   File "/home/airflow/.local/lib/python3.9/site-packages/kubernetes/client/models/v1_pod.py", line 196, in to_dict
     result[attr] = value.to_dict()
   File "/home/airflow/.local/lib/python3.9/site-packages/kubernetes/client/models/v1_pod_spec.py", line 1058, in to_dict
     result[attr] = list(map(
   File "/home/airflow/.local/lib/python3.9/site-packages/kubernetes/client/models/v1_pod_spec.py", line 1059, in <lambda>
     lambda x: x.to_dict() if hasattr(x, "to_dict") else x,
   File "/home/airflow/.local/lib/python3.9/site-packages/kubernetes/client/models/v1_container.py", line 660, in to_dict
     value = getattr(self, attr)
   File "/home/airflow/.local/lib/python3.9/site-packages/kubernetes/client/models/v1_container.py", line 458, in startup_probe
     return self._startup_probe
 AttributeError: 'V1Container' object has no attribute '_startup_probe'

Is a separate fix needed for that?

@potiuk
Copy link
Member

potiuk commented Aug 21, 2022

Is a separate fix needed for that?

Can you try 2.3.4 (It will be out in 2 days or so) and if it is still there please open a new issue with detailed description of the problem and how you got there. You can refer to this issue in the new one but piggybacking on existing, closed issue is not going to make it "active" I am afraid.

leahecole pushed a commit to GoogleCloudPlatform/composer-airflow that referenced this issue Oct 4, 2022
When UI unpickles executor_configs with outdated k8s objects it can run into the same issue as the scheduler does (see apache/airflow#23727).

Our JSON encoder therefore needs to suppress encoding errors arising from pod serialization, and fallback to a default value.

GitOrigin-RevId: 4ab96bac198b27f7c558326e526be22e945e4c4d
leahecole pushed a commit to GoogleCloudPlatform/composer-airflow that referenced this issue Oct 4, 2022
From time to time k8s library objects change their attrs.  If executor config is stored with old version, and unpickled with new version, we can get attribute errors that can crash the scheduler (see apache/airflow#23727).

Here we update handling so that we fail the task but don't crash the scheduler.

(cherry picked from commit 0c41f437674f135fe7232a368bf9c198b0ecd2f0)

GitOrigin-RevId: b1be02473b2ad04dde8d1268a47f18a22eb89faa
leahecole pushed a commit to GoogleCloudPlatform/composer-airflow that referenced this issue Dec 7, 2022
When UI unpickles executor_configs with outdated k8s objects it can run into the same issue as the scheduler does (see apache/airflow#23727).

Our JSON encoder therefore needs to suppress encoding errors arising from pod serialization, and fallback to a default value.

GitOrigin-RevId: 4ab96bac198b27f7c558326e526be22e945e4c4d
leahecole pushed a commit to GoogleCloudPlatform/composer-airflow that referenced this issue Dec 7, 2022
From time to time k8s library objects change their attrs.  If executor config is stored with old version, and unpickled with new version, we can get attribute errors that can crash the scheduler (see apache/airflow#23727).

Here we update handling so that we fail the task but don't crash the scheduler.

(cherry picked from commit 0c41f437674f135fe7232a368bf9c198b0ecd2f0)

GitOrigin-RevId: b1be02473b2ad04dde8d1268a47f18a22eb89faa
@KulykDmytro
Copy link
Contributor

in 2.5.0 #28227

leahecole pushed a commit to GoogleCloudPlatform/composer-airflow that referenced this issue Jan 30, 2023
From time to time k8s library objects change their attrs.  If executor config is stored with old version, and unpickled with new version, we can get attribute errors that can crash the scheduler (see apache/airflow#23727).

Here we update handling so that we fail the task but don't crash the scheduler.

GitOrigin-RevId: 0c41f437674f135fe7232a368bf9c198b0ecd2f0
leahecole pushed a commit to GoogleCloudPlatform/composer-airflow that referenced this issue Jan 30, 2023
When UI unpickles executor_configs with outdated k8s objects it can run into the same issue as the scheduler does (see apache/airflow#23727).

Our JSON encoder therefore needs to suppress encoding errors arising from pod serialization, and fallback to a default value.

GitOrigin-RevId: fe5e689adfe3b2f9bcc37d3975ae1aea9b55e28a
kosteev pushed a commit to kosteev/composer-airflow-test-copybara that referenced this issue Sep 12, 2024
When UI unpickles executor_configs with outdated k8s objects it can run into the same issue as the scheduler does (see apache/airflow#23727).

Our JSON encoder therefore needs to suppress encoding errors arising from pod serialization, and fallback to a default value.

GitOrigin-RevId: 4ab96bac198b27f7c558326e526be22e945e4c4d
kosteev pushed a commit to kosteev/composer-airflow-test-copybara that referenced this issue Sep 12, 2024
From time to time k8s library objects change their attrs.  If executor config is stored with old version, and unpickled with new version, we can get attribute errors that can crash the scheduler (see apache/airflow#23727).

Here we update handling so that we fail the task but don't crash the scheduler.

(cherry picked from commit 0c41f437674f135fe7232a368bf9c198b0ecd2f0)

GitOrigin-RevId: b1be02473b2ad04dde8d1268a47f18a22eb89faa
kosteev pushed a commit to GoogleCloudPlatform/composer-airflow that referenced this issue Sep 18, 2024
From time to time k8s library objects change their attrs.  If executor config is stored with old version, and unpickled with new version, we can get attribute errors that can crash the scheduler (see apache/airflow#23727).

Here we update handling so that we fail the task but don't crash the scheduler.

GitOrigin-RevId: 0c41f437674f135fe7232a368bf9c198b0ecd2f0
kosteev pushed a commit to GoogleCloudPlatform/composer-airflow that referenced this issue Sep 18, 2024
When UI unpickles executor_configs with outdated k8s objects it can run into the same issue as the scheduler does (see apache/airflow#23727).

Our JSON encoder therefore needs to suppress encoding errors arising from pod serialization, and fallback to a default value.

GitOrigin-RevId: fe5e689adfe3b2f9bcc37d3975ae1aea9b55e28a
kosteev pushed a commit to GoogleCloudPlatform/composer-airflow that referenced this issue Nov 7, 2024
From time to time k8s library objects change their attrs.  If executor config is stored with old version, and unpickled with new version, we can get attribute errors that can crash the scheduler (see apache/airflow#23727).

Here we update handling so that we fail the task but don't crash the scheduler.

GitOrigin-RevId: 0c41f437674f135fe7232a368bf9c198b0ecd2f0
kosteev pushed a commit to GoogleCloudPlatform/composer-airflow that referenced this issue Nov 7, 2024
When UI unpickles executor_configs with outdated k8s objects it can run into the same issue as the scheduler does (see apache/airflow#23727).

Our JSON encoder therefore needs to suppress encoding errors arising from pod serialization, and fallback to a default value.

GitOrigin-RevId: fe5e689adfe3b2f9bcc37d3975ae1aea9b55e28a
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.