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

Executor reports task instance (...) finished (failed) although the task says it's queued #39717

Closed
1 of 2 tasks
andreyvital opened this issue May 20, 2024 · 89 comments · Fixed by #41260
Closed
1 of 2 tasks
Assignees
Labels
affected_version:2.9 Issues Reported for 2.9 area:core area:Scheduler including HA (high availability) scheduler kind:bug This is a clearly a bug

Comments

@andreyvital
Copy link

Apache Airflow version

2.9.1

If "Other Airflow 2 version" selected, which one?

No response

What happened?

[2024-05-20T12:03:24.184+0000] {task_context_logger.py:91} ERROR - Executor reports task instance
<TaskInstance: (...) scheduled__2024-05-20T11:00:00+00:00 map_index=15 [queued]> 
finished (failed) although the task says it's queued. (Info: None) Was the task killed externally?

What you think should happen instead?

No response

How to reproduce

I am not sure, unfortunately. But every day I see my tasks being killed randomly without good reasoning behind why it got killed/failed.

Operating System

Ubuntu 22.04.4 LTS

Versions of Apache Airflow Providers

apache-airflow==2.9.1
apache-airflow-providers-amazon==8.20.0
apache-airflow-providers-celery==3.6.2
apache-airflow-providers-cncf-kubernetes==8.1.1
apache-airflow-providers-common-io==1.3.1
apache-airflow-providers-common-sql==1.12.0
apache-airflow-providers-docker==3.10.0
apache-airflow-providers-elasticsearch==5.3.4
apache-airflow-providers-fab==1.0.4
apache-airflow-providers-ftp==3.8.0
apache-airflow-providers-google==10.17.0
apache-airflow-providers-grpc==3.4.1
apache-airflow-providers-hashicorp==3.6.4
apache-airflow-providers-http==4.10.1
apache-airflow-providers-imap==3.5.0
apache-airflow-providers-microsoft-azure==10.0.0
apache-airflow-providers-mongo==4.0.0
apache-airflow-providers-mysql==5.5.4
apache-airflow-providers-odbc==4.5.0
apache-airflow-providers-openlineage==1.7.0
apache-airflow-providers-postgres==5.10.2
apache-airflow-providers-redis==3.6.1
apache-airflow-providers-sendgrid==3.4.0
apache-airflow-providers-sftp==4.9.1
apache-airflow-providers-slack==8.6.2
apache-airflow-providers-smtp==1.6.1
apache-airflow-providers-snowflake==5.4.0
apache-airflow-providers-sqlite==3.7.1
apache-airflow-providers-ssh==3.10.1

Deployment

Docker-Compose

Deployment details

Client: Docker Engine - Community
 Version:    26.1.3
 Context:    default
 Debug Mode: false
 Plugins:
  buildx: Docker Buildx (Docker Inc.)
    Version:  v0.14.0
    Path:     /usr/libexec/docker/cli-plugins/docker-buildx
  compose: Docker Compose (Docker Inc.)
    Version:  v2.27.0
    Path:     /usr/libexec/docker/cli-plugins/docker-compose
  scan: Docker Scan (Docker Inc.)
    Version:  v0.23.0
    Path:     /usr/libexec/docker/cli-plugins/docker-scan

Server:
 Containers: 30
  Running: 25
  Paused: 0
  Stopped: 5
 Images: 36
 Server Version: 26.1.3
 Storage Driver: overlay2
  Backing Filesystem: btrfs
  Supports d_type: true
  Using metacopy: false
  Native Overlay Diff: true
  userxattr: false
 Logging Driver: json-file
 Cgroup Driver: systemd
 Cgroup Version: 2
 Plugins:
  Volume: local
  Network: bridge host ipvlan macvlan null overlay
  Log: awslogs fluentd gcplogs gelf journald json-file local splunk syslog
 Swarm: inactive
 Runtimes: io.containerd.runc.v2 runc
 Default Runtime: runc
 Init Binary: docker-init
 containerd version: e377cd56a71523140ca6ae87e30244719194a521
 runc version: v1.1.12-0-g51d5e94
 init version: de40ad0
 Security Options:
  apparmor
  seccomp
   Profile: builtin
  cgroupns
 Kernel Version: 5.15.0-107-generic
 Operating System: Ubuntu 22.04.4 LTS
 OSType: linux
 Architecture: x86_64
 CPUs: 80
 Total Memory: 62.33GiB
 Name: troy
 ID: UFMO:HODB:7MRE:7O2C:FLWN:HE4Y:EZDF:ZGNF:OZRW:BUTZ:DBQK:MFR2
 Docker Root Dir: /var/lib/docker
 Debug Mode: false
 Experimental: false
 Insecure Registries:
  127.0.0.0/8
 Live Restore Enabled: false
OS: Ubuntu 22.04.4 LTS x86_64
Kernel: 5.15.0-107-generic
Uptime: 1 day, 23 hours, 12 mins
Packages: 847 (dpkg), 4 (snap)
Shell: fish 3.7.1
Resolution: 1024x768
Terminal: /dev/pts/0
CPU: Intel Xeon Silver 4316 (80) @ 3.400GHz
GPU: 03:00.0 Matrox Electronics Systems Ltd. Integrated
Memory: 24497MiB / 63830MiB

Anything else?

No response

Are you willing to submit PR?

  • Yes I am willing to submit a PR!

Code of Conduct

@andreyvital andreyvital added area:core kind:bug This is a clearly a bug needs-triage label for new issues that we didn't triage yet labels May 20, 2024
@nathadfield
Copy link
Collaborator

I'm not sure there's an Airflow issue here.

My initial thought is that you are experiencing issues related to your workers and perhaps they are falling over due to resource issues, i.e. disk, ram?

I can see that you are using dynamic task mapping which, depending on what you are asking the workers to do, how many parallel tasks and the number of workers you have, could be overloading your capacity.

@andreyvital
Copy link
Author

andreyvital commented May 21, 2024

Not sure...it seems related to redis? I have seen other people report similar issues:

Also, a lot of DAGs are failing within the same reason, so that's not entirely tied to Task Mapping at all. Some tasks fail very early...also this server has a lot of RAM, of which I've granted ~12gb to each worker and the task is very simple, just HTTP requests, all of them run in less than 2 minutes when they don't fail.

@RNHTTR RNHTTR removed the needs-triage label for new issues that we didn't triage yet label May 21, 2024
@RNHTTR
Copy link
Contributor

RNHTTR commented May 21, 2024

I think the log you shared (source) erroneously replaced the "stuck in queued" log somehow. Can you check your scheduler logs for "stuck in queued"?

@andreyvital
Copy link
Author

@RNHTTR there's nothing stating "stuck in queued" on scheduler logs.

@nghilethanh-atherlabs
Copy link

nghilethanh-atherlabs commented May 27, 2024

same issue here

@mikoloay
Copy link

I had the same issue when running hundreds of sensors on reschedule mode - a lot of the times they got stuck in the queued status and raised the same error that you posted. It turned out that our redis pod used by Celery restarted quite often and lost the info about queued tasks. Adding persistence to redis seems to have helped. Do you have persistence enabled?

@nghilethanh-atherlabs
Copy link

I had the same issue when running hundreds of sensors on reschedule mode - a lot of the times they got stuck in the queued status and raised the same error that you posted. It turned out that our redis pod used by Celery restarted quite often and lost the info about queued tasks. Adding persistence to redis seems to have helped. Do you have persistence enabled?

Can you help me how to add this persistence?

@andreyvital
Copy link
Author

andreyvital commented May 27, 2024

Hi @nghilethanh-atherlabs I've been experimenting with those configs as well:

# airflow.cfg


# https://airflow.apache.org/docs/apache-airflow-providers-celery/stable/configurations-ref.html#task-acks-late
# https://github.com/apache/airflow/issues/16163#issuecomment-1563704852
task_acks_late = False
# https://github.com/apache/airflow/blob/2b6f8ffc69b5f34a1c4ab7463418b91becc61957/airflow/providers/celery/executors/default_celery.py#L52
# https://github.com/celery/celery/discussions/7276#discussioncomment-8720263
# https://github.com/celery/celery/issues/4627#issuecomment-396907957
[celery_broker_transport_options]
visibility_timeout = 300
max_retries = 120
interval_start = 0
interval_step = 0.2
interval_max = 0.5
# sentinel_kwargs = {}

For the redis persistency, you can refer to their config file to enable persistency. Not sure it will sort out. But let's keep trying folks.

# redis.conf
bind 0.0.0.0

protected-mode no

requirepass REDACTED

maxmemory 6gb
# https://redis.io/docs/manual/eviction/
maxmemory-policy noeviction

port 6379

tcp-backlog 511

timeout 0

tcp-keepalive 300

daemonize no
supervised no

pidfile /var/run/redis.pid

loglevel notice

logfile ""

databases 16

always-show-logo no

save 900 1
save 300 10
save 60 10000

stop-writes-on-bgsave-error yes

rdbcompression yes
rdbchecksum yes

dbfilename dump.rdb

dir /bitnami/redis/data

appendonly no
appendfilename "appendonly.aof"
appendfsync everysec
# appendfsync no
no-appendfsync-on-rewrite no
auto-aof-rewrite-percentage 100
auto-aof-rewrite-min-size 64mb
aof-load-truncated yes
aof-use-rdb-preamble no
aof-rewrite-incremental-fsync yes

lua-time-limit 5000

slowlog-log-slower-than 10000
slowlog-max-len 128

latency-monitor-threshold 0
notify-keyspace-events ""

hash-max-ziplist-entries 512
hash-max-ziplist-value 64

list-max-ziplist-size -2
list-compress-depth 0

set-max-intset-entries 512

zset-max-ziplist-entries 128
zset-max-ziplist-value 64

hll-sparse-max-bytes 3000

activerehashing yes

client-output-buffer-limit normal 0 0 0
client-output-buffer-limit slave 256mb 64mb 60
client-output-buffer-limit pubsub 32mb 8mb 60

hz 10
# docker-compose.yml
redis:
  image: bitnami/redis:7.2.5
  container_name: redis
  environment:
    - REDIS_DISABLE_COMMANDS=CONFIG
    # The password will come from the config file, but we need to bypass the validation
    - ALLOW_EMPTY_PASSWORD=yes
  ports:
    - 6379:6379
  # command: /opt/bitnami/scripts/redis/run.sh --maxmemory 2gb
  command: /opt/bitnami/scripts/redis/run.sh
  volumes:
    - ./redis/redis.conf:/opt/bitnami/redis/mounted-etc/redis.conf
    - redis:/bitnami/redis/data
  restart: always
  healthcheck:
    test:
      - CMD
      - redis-cli
      - ping
    interval: 5s
    timeout: 30s
    retries: 10

@seanmuth
Copy link

Seeing this issue on 2.9.1 as well, also only with sensors.

We've found that the DAG is timing out trying to fill up the Dagbag on the worker. Even with debug logs enabled I don't have a hint about where in the import it's hanging.

[2024-05-31 18:00:01,335: INFO/ForkPoolWorker-63] Filling up the DagBag from <redacted dag file path>
[2024-05-31 18:00:01,350: DEBUG/ForkPoolWorker-63] Importing <redacted dag file path>
[2024-05-31 18:00:31,415: ERROR/ForkPoolWorker-63] Process timed out, PID: 314

On the scheduler the DAG imports in less than a second.

and not all the tasks from this DAG fail to import, many import just fine, at the same time on the same celery worker. below is the same dag file as above, importing fine:

[2024-05-31 18:01:52,911: INFO/ForkPoolWorker-3] Filling up the DagBag from <redacted dag file path>
[2024-05-31 18:01:52,913: DEBUG/ForkPoolWorker-3] Importing <redacted dag file path>
[2024-05-31 18:01:54,232: WARNING/ForkPoolWorker-3] /usr/local/lib/python3.11/site-packages/airflow/models/baseoperator.py:484: RemovedInAirflow3Warning: The 'task_concurrency' parameter is deprecated. Please use 'max_active_tis_per_dag'.
  result = func(self, **kwargs, default_args=default_args)

[2024-05-31 18:01:54,272: DEBUG/ForkPoolWorker-3] Loaded DAG <DAG: redacted dag>

one caveat/note is that it looks like the 2nd run/retry of each sensor is what runs just fine.

We've also confirmed this behavior was not present on Airflow 2.7.3, and only started occurring since upgrading to 2.9.1.

@nghilethanh-atherlabs
Copy link

@andreyvital thank you so much for your response. I have setup and it works really great :)

@petervanko
Copy link

I was working on the issue with @seanmuth and increasing parsing time solved the issue.
It does not fix the root cause, but as a workaround it can save your night...

AIRFLOW__CORE__DAGBAG_IMPORT_TIMEOUT = 120

@Lee-W
Copy link
Member

Lee-W commented Jun 5, 2024

Hello everyone,

I'm currently investigating this issue, but I haven't been able to replicate it yet. Could you please try setting AIRFLOW__CORE__EXECUTE_TASKS_NEW_PYTHON_INTERPRETER=True [1] to see if we can generate more error logs? It seems that _execute_in_subprocess generates more error logs compared to _execute_in_fork, which might provide us with some additional clues.

log.exception("[%s] execute_command encountered a CalledProcessError", celery_task_id)
log.error(e.output)

[1] https://airflow.apache.org/docs/apache-airflow/stable/configurations-ref.html#execute-tasks-new-python-interpreter

@niegowic
Copy link

niegowic commented Jun 7, 2024

Spotted same problem with Airflow 2.9.1 - problem didn't occur earlier so it's strictly related with this version. It happens randomly on random task execution. Restarting scheduler and triggerer helps - but this is our temp workaround.

@eladkal eladkal added area:Scheduler including HA (high availability) scheduler affected_version:2.9 Issues Reported for 2.9 labels Jun 8, 2024
@Lee-W
Copy link
Member

Lee-W commented Jun 11, 2024

Spotted same problem with Airflow 2.9.1 - problem didn't occur earlier so it's strictly related with this version. It happens randomly on random task execution. Restarting scheduler and triggerer helps - but this is our temp workaround.

We've released apache-airflow-providers-celery 3.7.2 with enhanced logging. Could you please update the provider version and check the debug log for any clues? Additionally, what I mentioned in #39717 (comment) might give us some club as well. Thanks!

@trlopes1974
Copy link

Following... and adding some spice.

We have just upgraded to Airflow 2.9.2 and also have the (same) issue. Yet we have seen the problem in Airflow 2.8 (in our case the celery task airflow.exceptions.AirflowException: Celery command failed on host: slautop02 with celery_task_id 5d7f577d-3e89-4867-8481-24df778346ae (PID: 815333, Return Code: 256) but the Airflow tasks did not fail.

After reading this issue I also caugth this on shceduler logs:
[2024-06-20T17:45:58.167+0100] {processor.py:161} INFO - Started process (PID=830424) to work on /home/ttauto/airflow/dags/ITSM_DAILY_INFORM_CLOSE_TASKS.py
[2024-06-20T17:45:58.169+0100] {processor.py:830} INFO - Processing file /home/ttauto/airflow/dags/ITSM_DAILY_INFORM_CLOSE_TASKS.py for tasks to queue
[2024-06-20T17:45:58.170+0100] {logging_mixin.py:188} INFO - [2024-06-20T17:45:58.170+0100] {dagbag.py:545} INFO - Filling up the DagBag from /home/ttauto/airflow/dags/ITSM_DAILY_INFORM_CLOSE_TASKS.py
[2024-06-20T17:46:28.174+0100] {logging_mixin.py:188} INFO - [2024-06-20T17:46:28.173+0100] {timeout.py:68} ERROR - Process timed out, PID: 830424

Despite that these timeouts apear on several dags, we see no errors on the airflow ui neither on the airflow tasks
We also cannot match the Pid in this logs with the pid mentioned on the celery tasks (pid XXX return code 256)

We are experiencing Celery tasks failures with the following stack trace:
Traceback (most recent call last):
File "/opt/tkapp/env_airflow/lib64/python3.9/site-packages/celery/app/trace.py", line 453, in trace_task
R = retval = fun(*args, **kwargs)
File "/opt/tkapp/env_airflow/lib64/python3.9/site-packages/celery/app/trace.py", line 736, in protected_call
return self.run(*args, **kwargs)
File "/opt/tkapp/env_airflow/lib64/python3.9/site-packages/airflow/providers/celery/executors/celery_executor_utils.py", line 136, in execute_command
_execute_in_fork(command_to_exec, celery_task_id)
File "/opt/tkapp/env_airflow/lib64/python3.9/site-packages/airflow/providers/celery/executors/celery_executor_utils.py", line 151, in _execute_in_fork
raise AirflowException(msg)
airflow.exceptions.AirflowException: Celery command failed on host: slautop02 with celery_task_id 5d7f577d-3e89-4867-8481-24df778346ae (PID: 815333, Return Code: 256)

Most of the times, this does not raise any issues and the dags tasks complete successfully without problems, even if the CELERY task is marked as failed, the airflow tasks completes successefully.
Today we had a dag failure on the very first task ( an emptyoperator ) wit the exact same problem in the celery task. So the problem is now a real issue for us.

*** Found local files:
*** * /opt/tkapp/airflow/logs/dag_id=CSDISPATCHER_SIMPLES/run_id=scheduled__2024-06-20T16:34:00+00:00/task_id=Start/attempt=1.log.SchedulerJob.log
[2024-06-20, 17:39:30 WEST] {scheduler_job_runner.py:843} ERROR - Executor reports task instance <TaskInstance: CSDISPATCHER_SIMPLES.Start scheduled__2024-06-20T16:34:00+00:00 [queued]> finished (failed) although the task says it's queued. (Info: None) Was the task killed externally?
[2024-06-20, 17:50:47 WEST] {event_scheduler.py:40} WARNING - Marking task instance <TaskInstance: CSDISPATCHER_SIMPLES.Start scheduled__2024-06-20T16:34:00+00:00 [queued]> stuck in queued as failed. If the task instance has available retries, it will be retried.
[2024-06-20, 17:50:48 WEST] {scheduler_job_runner.py:843} ERROR - Executor reports task instance <TaskInstance: CSDISPATCHER_SIMPLES.Start scheduled__2024-06-20T16:34:00+00:00 [queued]> finished (failed) although the task says it's queued. (Info: None) Was the task killed externally?

We have investigated the (Return Code: 256) but without success, the "best" reason would be memory contention on the server but we also do not observe that.

Our server status, no exhaustion of resources.
image

version in use:
apache-airflow==2.9.2
apache-airflow-providers-celery==3.7.2
apache-airflow-providers-common-io==1.3.2
apache-airflow-providers-common-sql==1.14.0
apache-airflow-providers-fab==1.1.1
apache-airflow-providers-ftp==3.9.1
apache-airflow-providers-hashicorp==3.7.1
apache-airflow-providers-http==4.11.1
apache-airflow-providers-imap==3.6.1
apache-airflow-providers-postgres==5.11.1
apache-airflow-providers-sftp==4.10.1
apache-airflow-providers-smtp==1.7.1
apache-airflow-providers-sqlite==3.8.1
apache-airflow-providers-ssh==3.11.1

We have just changed the
AIRFLOW__CORE__EXECUTE_TASKS_NEW_PYTHON_INTERPRETER=True to try to get some more info.

@potiuk
Copy link
Member

potiuk commented Jun 21, 2024

@vizeit
Copy link
Contributor

vizeit commented Jun 27, 2024

I see the same issue, with dynamic task mapping in multiple instances of a DAG. All the pods have enough cpu-memory

Executor: CeleryKubernetes
Airflow version: 2.9.1
Redis persistence: enabled
DAG: Dynamic task group with dynamic tasks and multiple instances of the DAG may run at a time

when I re-run the failed tasks with this error, it goes through and finishes successfully

@potiuk
Copy link
Member

potiuk commented Jun 27, 2024

@vizeit and anyone looking here and tempted to report "I have the same issue". PLEASE before doing it upgrade to 2.9.2 and latest celery provider. And when you do, report it here whether things are fixed, and if not, add logs from the celery executor.

If you actually look at the discussion - some of related issues were fixed in 2.9.2 and Celery logging has been improved in latest provider to add more information. So the best thing you can do - is not really post "i have the same issue" but upgrade and let us know if it helped, and second best thing is to upgrade celery provider and post relevant logs.

Just posting "I have the same issue in 2.9.1" is not moving a needle when it comes to investigating and fixing such problem.

@vizeit
Copy link
Contributor

vizeit commented Jun 27, 2024

Sure, I can upgrade and check. I believe others here already tested on 2.9.2 reporting the same issue

@trlopes1974
Copy link

trlopes1974 commented Aug 2, 2024

@NBardelot
I was talking about HTTP proxy.
Has we have different services that need different proxys, I had reimplemented part of Hashicorp api using python requests where I can set the proxy without messing with os proxy settings and have to manage no_proxy.
This is something that bothers me quite has many apis do not implement setting proxy params and only use the OS proxy or environment proxy settings. In our case we have at least 3 services that use different proxy servers... it's a pain!!!

But going back to the point, we saw that some task errors occurred while some secret retrieval was requested. further investigation lead us to proxy-related dns issues when accessing vault that where causing the failures.

@trlopes1974
Copy link

OK. I think we are getting closer to the root cause thanks to that stacktrace. No airlfow picking is involved here (pickling is done internally by deepcopy).

What happens there:

  1. Mini-scheduler is enabled
  2. During that mini-scheduling - partial_subset() of the dag is prepared (including deepcopy of the task + all the downstream tasks it has.
  3. deepcopying one of the tasks fails - because the tasks apparently uses CFFI to interact with a C code and the implementation of whatever you have there does not allow for deepcopying the task

I think a good solution would be @ashb @ephraimbuddy - following what you've done in #27506 - to just skip mini-scheduler when something like that happens. In this case that would mean exiting mini-scheduler without raising any error if anything wrong happens during partial_subset.

WDYT @ashb @ephraimbuddy ?

Maybe this is related to SSH /SFTP operators ? I did found a similar issue refering paramiko ( used in SSH / SFTP)

for instance, one of the failing tasks:
ssh_command_remove_operator

        ssh_command_remove_operator = SSHOperator(
            task_id='ssh_command_remove_operator',
            command='/usr/bin/sudo -s --  eval \'/usr/bin/su - orauser -c "/home/oradb/correcao_dados/correcao_dados.sh {{ ti.xcom_pull(task_ids="prepare_task",key="WORKORDERID") }} REMOVE"\'' ,
            ssh_hook=ssh_hook,
            cmd_timeout = 60, 
            conn_timeout= 90, 
            get_pty=True,      
            do_xcom_push = True    
            )

@potiuk
Copy link
Member

potiuk commented Aug 2, 2024

Maybe this is related to SSH /SFTP operators ? I did found a similar issue refering paramiko ( used in SSH / SFTP)

Quite likely

@potiuk
Copy link
Member

potiuk commented Aug 5, 2024

I think a good solution would be @ashb @ephraimbuddy - following what you've done in #27506 - to just skip mini-scheduler when something like that happens. In this case that would mean exiting mini-scheduler without raising any error if anything wrong happens during partial_subset.

WDYT @ashb @ephraimbuddy ?

Any thoughts @ashb , @ephraimbuddy - since you were involved in similar cases - I think this is a small fix (and happy to submit it - it's merely catching and logging all exceptions and skipping mini-scheduler when it happens) - but I am not 100% sure if that is a good idea.

@ephraimbuddy
Copy link
Contributor

I think a good solution would be @ashb @ephraimbuddy - following what you've done in #27506 - to just skip mini-scheduler when something like that happens. In this case that would mean exiting mini-scheduler without raising any error if anything wrong happens during partial_subset.

WDYT @ashb @ephraimbuddy ?

Any thoughts @ashb , @ephraimbuddy - since you were involved in similar cases - I think this is a small fix (and happy to submit it - it's merely catching and logging all exceptions and skipping mini-scheduler when it happens) - but I am not 100% sure if that is a good idea.

It's a good idea and would solve at least one of the issues that can lead to that log message. I'm okay with the solution. Other issues can also lead to the scheduler sending this log.

@potiuk
Copy link
Member

potiuk commented Aug 5, 2024

It's a good idea and would solve at least one of the issues that can lead to that log message. I'm okay with the solution. Other issues can also lead to the scheduler sending this log.

Yeah - but one less is good :)

@scaoupgrade
Copy link

scaoupgrade commented Aug 5, 2024

I have been following this thread recently since we also experienced this issue on airlfow 2.8.4. We have been running on this version for over two months and this is the first time I see this error. this may suggest that this issue happens less often on 2.8.X?

I see two issues being discussed in this thread:

  1. The airflow scheduler complains about: Executor reports task instance <TaskInstance: (...)> finished (failed) although the task says it's queued. (Info: None) Was the task killed externally?

  2. The airflow worker throws error on: airflow.exceptions.AirflowException: Celery command failed on host: xxxx with celery_task_id xxxxx

Based on my observation on the logs when the issue happened the other day, these two are not the same issue.

Issue 2 happens frequently, I can see about 1600 messages of such error on daily basis, and the number of errors I observe everyday are stable.

Thanks @potiuk for providing a fix. https://github.com/apache/airflow/pull/41260/files could address issue 2, but issue 1 should be something else.

Because the day the incident happened on our platform, I see a burst of messages like: Executor reports task instance <TaskInstance: (...)> finished (failed) although the task says it's queued. (Info: None) Was the task killed externally?, while the error in worker log saying celery command failed remains stable (around 1600 messages).

by looking at the scheduler log when the issue happened, I notice this pattern being repeated for the same task multiple times for a given dag:

{""log"":""\t<TaskInstance: xxxxx scheduled__2024-07-30T20:43:00+00:00 [scheduled]>"",""stream"":""stdout"",""timestamp"":1722380423388}"
{""log"":""\t<TaskInstance: xxxxx scheduled__2024-07-30T20:43:00+00:00 [scheduled]>"",""stream"":""stdout"",""timestamp"":1722380423388}"
{""log"":""\t<TaskInstance: xxxxx scheduled__2024-07-30T20:43:00+00:00 [scheduled]>"",""stream"":""stdout"",""timestamp"":1722380423388}"
{""log"":""\t<TaskInstance: xxxxx scheduled__2024-07-30T20:43:00+00:00 [scheduled]>"",""stream"":""stdout"",""timestamp"":1722380423388}"
{""log"":""\t<TaskInstance: xxxxx scheduled__2024-07-30T20:43:00+00:00 [scheduled]>"",""stream"":""stdout"",""timestamp"":1722380423388}"
{""log"":""\t<TaskInstance: xxxxx scheduled__2024-07-30T20:43:00+00:00 [scheduled]>"",""stream"":""stdout"",""timestamp"":1722380423388}"

The same line is repeated for the each task in the that dag hundreds of times, which seems to be abnormal.

Looks like scheduler dag processor runs into some issue and something failed during the scheduling phase. When this happens, all workers are still online and redis (celery borker) is also healthy. But all workers stopped picking up tasks from
celery and just remains idle.

This is not a recurring issue, I have only observed it once after running on 2.8.4 for months.

@trlopes1974
Copy link

I tend to agree with @scaoupgrade .
I also believe that there are 2 issues on this threads. I've mentioned it in here
And I also believe that te dag processor is causing the other one, some timeout that is mitigated by messing around with airflow's configuration....

@potiuk
Copy link
Member

potiuk commented Aug 5, 2024

@scaoupgrade - @trlopes1974 . Yes. We actually discuseed it few comments above in case you missed it:

It's a good idea and would solve at least one of the issues that can lead to that log message. I'm okay with the solution. Other issues can also lead to the scheduler sending thise log.

Yeah - but one less is good :)

And yes - as long as we have more details that we can diagnose it, we might also in the future address other similar issues - one thing at a time. I am actually proposing to close that issue here and if someone can open another similar issue with details that explain other issues of the same type with "related to THIS ISSUE" - it would be great. It's extremely hard to discuss and reason when multiple different issues are mixed in a single huge thread. Closing that one and opening new one seeing that it happens after fixing part of the issue seems like the best thing that we can dol

Generally speaking - unless we see enough evidences that point to some issue that can be diagnosed and/or repreoduced, there is not much anyone can do about it. With the stack-trace from @trlopes1974, it was quite easy to figure out.

PROPOSAL: maybe anyone who experiences this one, applies the patch from my PR and then see if they can still see the issue and if they do - open a new issue - hopefully with some details that will allow someone to diagnose it/ reproduce, once it's known that at last this one is already patched. The patch should be asy to apply on any version of Airflow.

@potiuk
Copy link
Member

potiuk commented Aug 5, 2024

I can easily close the issue and add simple instructions what anyone who sees similar issue should do (apply patch and if they see similar issue - report all the details there).

@trlopes1974
Copy link

trlopes1974 commented Aug 5, 2024 via email

@scaoupgrade
Copy link

scaoupgrade commented Aug 5, 2024

I'm a noob, but I can follow instructions 😂 Unfortuntly I only have tomorrow to make ot happen has I'm going on vacation!💪💪💪 A segunda, 5/08/2024, 20:45, Jarek Potiuk @.> escreveu:

I can easily close the issue and add simple instructions what anyone who sees similar issue should do (apply patch and if they see similar issue - report all the details there). — Reply to this email directly, view it on GitHub <#39717 (comment)>, or unsubscribe https://github.com/notifications/unsubscribe-auth/AD7TGQFC4UH7ONDXR4QFE43ZP7I7BAVCNFSM6AAAAABH7TVE5OVHI2DSMVQWIX3LMV43OSLTON2WKQ3PNVWWK3TUHMZDENRZG44DSNRQHE . You are receiving this because you were mentioned.Message ID: @.
>

Thanks @trlopes1974 . You have provided a log of useful information on this issue. I was following your logs and trying to understand what exactly happened. This is not a reproducible issue on my side. There could be a combination of different things to happen together to lead to this bug.

The main component involved could be:
Scheduler side: mainly the interaction with executor to check task status and re-queue task:

  1. scheduler heartbeat executor to process event: https://github.com/apache/airflow/blame/main/airflow/jobs/scheduler_job_runner.py#L879-L904
  2. scheduler timer to fail task which has been queued long enough: https://github.com/apache/airflow/blob/main/airflow/jobs/scheduler_job_runner.py#L1091-L1094

Worker side:
I don't notice anything abnormal on the worker side when the scheduler error happens for now yet.

@trlopes1974
Copy link

trlopes1974 commented Aug 5, 2024 via email

potiuk added a commit to potiuk/airflow that referenced this issue Aug 6, 2024
When mini-scheduler raises an exception, it has a bit weird side
effect - the task succeeds but it is seen as failed and scheduler
gets confused. Also flower celery worker in this case shows an error.

This happens for example when DAG contains non-serializable tasks.

This PR swallows any exceptions raised in mini-scheduler and simply
logs them as error rather than fail the process. Mini-scheduler
is generally optional and we are also already sometimes skipping
it already so occasional skipping is not a big problem.

Fixes: apache#39717
@trlopes1974
Copy link

just got one...

Dagrun Running | Task instance's dagrun was not in the 'running' state but in the state 'failed'. -- | -- Task Instance State | Task is in the 'failed' state.

external_executor_id a01b358f-ad34-4b16-81b9-fd69218f613e does not exist in flower / celery

look at the timestamps in the logs:
image

image

there is a gap of 10minutes betwwen the Start ( dummy task) and the dispatch_restores task.
And this behaviour is recurrent, (the 10m gap)

and in the task log:
'attempt=1.log.SchedulerJob.log'
(tkapp) ttauto@slautop02$ cat attempt=1.log.SchedulerJob.log
[2024-08-05T21:02:15.585+0100] {event_scheduler.py:40} WARNING - Marking task instance <TaskInstance: CSDISPATCHER_OTHERS.dispatch_restores scheduled__2024-08-05T19:47:00+00:00 [queued]> stuck in queued as failed. If the task instance has available retries, it will be retried.
[2024-08-05T21:02:16.750+0100] {scheduler_job_runner.py:843} ERROR - Executor reports task instance <TaskInstance: CSDISPATCHER_OTHERS.dispatch_restores scheduled__2024-08-05T19:47:00+00:00 [queued]> finished (failed) although the task says it's queued. (Info: None) Was the task killed externally?

@trlopes1974
Copy link

#41276 to take care of the "2" cenario has the "1" has a solution now.

@trlopes1974
Copy link

@potiuk , if you guide-me, I can deploy the fix to our production env and see if "1" goes away

@potiuk
Copy link
Member

potiuk commented Aug 6, 2024

@trlopes1974 -> I see you opened a new issue (cool) - for testing just apply the patch from #41260 to your installation - it might be building your own image with the change applied (git patch might be useful to generate patch that can be applied) or just manually modify the code in running venv/container.

@trlopes1974
Copy link

trlopes1974 commented Aug 6, 2024 via email

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
affected_version:2.9 Issues Reported for 2.9 area:core area:Scheduler including HA (high availability) scheduler kind:bug This is a clearly a bug
Projects
None yet
Development

Successfully merging a pull request may close this issue.