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

Network instabilities are able to freeze KubernetesJobWatcher #12644

Closed
guillemborrell opened this issue Nov 26, 2020 · 9 comments
Closed

Network instabilities are able to freeze KubernetesJobWatcher #12644

guillemborrell opened this issue Nov 26, 2020 · 9 comments
Labels
affected_version:2.0 Issues Reported for 2.0 kind:bug This is a clearly a bug pending-response provider:cncf-kubernetes Kubernetes provider related issues stale Stale PRs per the .github/workflows/stale.yml policy file

Comments

@guillemborrell
Copy link

guillemborrell commented Nov 26, 2020

Apache Airflow version:
1.10.10 and 1.10.12 so far

Kubernetes version (if you are using kubernetes) (use kubectl version): v1.16.10

Environment:

Cloud provider or hardware configuration: Azure
OS (e.g. from /etc/os-release):
Kernel (e.g. uname -a):
Install tools:
Others:

What happened:
We are running Airflow with the KubernetesExecutor on a Kubernetes cluster with a somewhat unreliable network. The scheduler runs smoothly until it logs the following line:

WARNING - HTTPError when attempting to run task, re-queueing. Exception: ('Connection aborted.', OSError("(110, 'ETIMEDOUT')"))

From then on, the executor is able to schedule jobs, but the job watcher stops processing events. The resulting behaviour is that new jobs are scheduled, but all jobs with status Completed remain in the cluster instead of being terminated. We mitigate this condition by periodically restarting the scheduler, which cleans up all the completed tasks correctly.

What you expected to happen:
Either KubernetesJobWatcher opens a new connection and goes on processing events, or it raises an exception shutting the process down so it can be restarted.

How to reproduce the bug:
Take down the connection between the scheduler pod and the Kubernetes API.

@boring-cyborg
Copy link

boring-cyborg bot commented Nov 26, 2020

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

@atrbgithub
Copy link
Contributor

atrbgithub commented Dec 8, 2020

@guillemborrell We're also seeing this with kubernetes. We seem to hit:

[2020-11-19 06:02:00,227] {kubernetes_executor.py:837} WARNING - HTTPError when attempting to run task, re-queueing. Exception: HTTPSConnectionPool(host='some_ip', port=443): Max retries exceeded with url: /api/v1/namespaces/some-namespace/pods (Caused by NewConnectionError('<urllib3.connection.HTTPSConnection object at 0x7fcc36a8eee0>: Failed to establish a new connection: [Errno 111] Connection refused'))

We're on version 1.10.12. As you mention, pods then remain in the namespace with a status of completed and are not cleaned up. No more jobs appear to be submitted to k8s once this is hit.

We're seeing this regularly when kubernetes undergoes maintenance on the master node.

There is a similar issue here against the python k8s client, which I believe airflow is using:
kubernetes-client/python#1148

Separately we're seeing issues with the pod operator when using the kubernetes task executor. The parent pod appears to stop seeing events from the child pod it created. Thus it never sees that the child pod completes, and leaving the parent pod and hence the job hanging forever.

@atrbgithub
Copy link
Contributor

There is also this issue which seems to be what we are seeing when using the kubernetes task executor, launching child pods with the pod operator kubernetes-client/python#1234

@atrbgithub
Copy link
Contributor

atrbgithub commented Dec 9, 2020

Looking at our logs for when we see k8s maintenance, we also get this error once it comes back:

[2020-11-19 06:05:24,313] {kubernetes_executor.py:345} ERROR - Encountered Error response from k8s list namespaced pod stream => {'type': 'ERROR', 'object': {'api_version': 'v1',
 'kind': 'Status',
 'metadata': {'annotations': None,
              'cluster_name': None,
              'creation_timestamp': None,
              'deletion_grace_period_seconds': None,
              'deletion_timestamp': None,
              'finalizers': None,
              'generate_name': None,
              'generation': None,
              'initializers': None,
              'labels': None,
              'managed_fields': None,
              'name': None,
              'namespace': None,
              'owner_references': None,
              'resource_version': None,
              'self_link': None,
              'uid': None},
 'spec': None,
 'status': {'conditions': None,
            'container_statuses': None,
            'host_ip': None,
            'init_container_statuses': None,
            'message': None,
            'nominated_node_name': None,
            'phase': None,
            'pod_ip': None,
            'qos_class': None,
            'reason': None,
            'start_time': None}}, 'raw_object': {'kind': 'Status', 'apiVersion': 'v1', 'metadata': {}, 'status': 'Failure', 'message': 'too old resource version: 787074279 (787074339)', 'rea
son': 'Gone', 'code': 410}}
[2020-11-19 06:05:24,314] {kubernetes_executor.py:351} INFO - Kubernetes resource version is too old, must reset to 0 => ('too old resource version: 787074279 (787074339)',)
[2020-11-19 06:05:24,314] {kubernetes_executor.py:296} WARNING - Watch died gracefully, starting back up with: last resource_version: 0
[2020-11-19 06:05:24,314] {kubernetes_executor.py:300} INFO - Event: and now my watch begins starting at resource_version: 0

Not sure if this is somehow meaning that it no longer sees the events of pods created before maintenance started, which have since stopped?

Maintenance seems to take around 3 or 4 mins, but if pods stop during that time will airflow notice once it reconnects, or will it only see events from post reconnecting?

Anyhow it seems we get into the state where no new pods are launched and old completed pods are not cleaned up....

This only happens once the scheduler pod is terminated, at that point it does notice that the completed pods have finished and cleans them up.

There might be a fix for this already in master, or at least there seems to be a change to how orphaned pods are collected.

When we restart the scheduler we see

[2020-11-19 07:30:07,266] {scheduler_job.py:1438} INFO - Resetting orphaned tasks for active dag runs

This then identifies the completed pods. However it only runs on startup:
https://github.com/apache/airflow/blob/1.10.12/airflow/jobs/scheduler_job.py#L1438

In master this has been modified and now runs every 5 minutes by default:
https://github.com/apache/airflow/blob/master/airflow/jobs/scheduler_job.py#L1351

@jmullins
Copy link

jmullins commented Jan 11, 2021

We consistently experienced kubernetes executor slot starvation, as described above, where worker pods get stuck in a completed state and are never deleted due to indefinite blocking in the KubernetesJobWatcher watch:

https://github.com/apache/airflow/blob/1.10.14/airflow/executors/kubernetes_executor.py#L315-L322
https://github.com/apache/airflow/blob/1.10.14/airflow/executors/kubernetes_executor.py#L315-L322

The indefinite blocking is due to a lack of tcp keepalives or a default _request_timeout (socket timeout) in kube_client_request_args:
https://github.com/apache/airflow/blob/2.0.0/airflow/config_templates/default_airflow.cfg#L990

We were able to consistently reproduce this behavior by injecting network faults or clearing the conntrack state on the node where the scheduler was running as part of an overlay network.

Setting a socket timeout, _request_timeout in kube_client_request_args, prevents executor slot starvation since the KubernetesJobWatcher recovers once the timeout is reached and properly cleans up worker pods stuck in the completed state.

kube_client_request_args = { "_request_timeout": 600 }

We currently set the _request_timeout to 10 minutes so we won't see a timeout unless there's a network fault -- since the kubernetes watch itself will expire before this (after 5 min).

I think it makes sense to consider setting a default _request_timeout, even if the value is high, to protect against executor slot starvation and unavailability during network faults.

@joaquin-casanova
Copy link

Hello any update about this issue, I have the same problem in Airflow 2.0.0

@eladkal
Copy link
Contributor

eladkal commented Nov 17, 2022

Is this still reproducible in latest Airflow version?
I think #23521 should have resolved this issue?

@github-actions
Copy link

This issue has been automatically marked as stale because it has been open for 30 days with no response from the author. It will be closed in next 7 days if no further activity occurs from the issue author.

@github-actions github-actions bot added the stale Stale PRs per the .github/workflows/stale.yml policy file label Dec 19, 2022
@github-actions
Copy link

This issue has been closed because it has not received response from the issue author.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
affected_version:2.0 Issues Reported for 2.0 kind:bug This is a clearly a bug pending-response provider:cncf-kubernetes Kubernetes provider related issues stale Stale PRs per the .github/workflows/stale.yml policy file
Projects
None yet
Development

No branches or pull requests

6 participants