You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by GitBox <gi...@apache.org> on 2021/11/18 04:19:51 UTC

[GitHub] [airflow] humit0 opened a new issue #19671: Scheduler raise exception while running adopt_or_reset_orphaned_tasks method

humit0 opened a new issue #19671:
URL: https://github.com/apache/airflow/issues/19671


   ### Apache Airflow version
   
   2.2.0
   
   ### Operating System
   
   CentOS 7
   
   ### Versions of Apache Airflow Providers
   
   _No response_
   
   ### Deployment
   
   Other
   
   ### Deployment details
   
   _No response_
   
   ### What happened
   
   Scheduler process is killed with errors.
   ```
   [2021-11-18 11:41:49,826] {scheduler_job.py:603} ERROR - Exception when executing SchedulerJob._run_scheduler_loop
   Traceback (most recent call last):
     File "/opt/app-root/lib64/python3.8/site-packages/airflow/jobs/scheduler_job.py", line 587, in _execute
       self._run_scheduler_loop()
     File "/opt/app-root/lib64/python3.8/site-packages/airflow/jobs/scheduler_job.py", line 680, in _run_scheduler_loop
       next_event = timers.run(blocking=False)
     File "/opt/rh/rh-python38/root/usr/lib64/python3.8/sched.py", line 151, in run
       action(*argument, **kwargs)
     File "/opt/app-root/lib64/python3.8/site-packages/airflow/utils/event_scheduler.py", line 36, in repeat
       action(*args, **kwargs)
     File "/opt/app-root/lib64/python3.8/site-packages/airflow/utils/session.py", line 70, in wrapper
       return func(*args, session=session, **kwargs)
     File "/opt/app-root/lib64/python3.8/site-packages/airflow/jobs/scheduler_job.py", line 1069, in adopt_or_reset_orphaned_tasks
       for attempt in run_with_db_retries(logger=self.log):
     File "/opt/app-root/lib64/python3.8/site-packages/tenacity/__init__.py", line 382, in __iter__
       do = self.iter(retry_state=retry_state)
     File "/opt/app-root/lib64/python3.8/site-packages/tenacity/__init__.py", line 349, in iter
       return fut.result()
     File "/opt/rh/rh-python38/root/usr/lib64/python3.8/concurrent/futures/_base.py", line 432, in result
       return self.__get_result()
     File "/opt/rh/rh-python38/root/usr/lib64/python3.8/concurrent/futures/_base.py", line 388, in __get_result
       raise self._exception
     File "/opt/app-root/lib64/python3.8/site-packages/airflow/jobs/scheduler_job.py", line 1113, in adopt_or_reset_orphaned_tasks
       to_reset = self.executor.try_adopt_task_instances(tis_to_reset_or_adopt)
     File "/opt/app-root/lib64/python3.8/site-packages/airflow/executors/celery_executor.py", line 485, in try_adopt_task_instances
       self.adopted_task_timeouts[ti.key] = ti.queued_dttm + self.task_adoption_timeout
   TypeError: unsupported operand type(s) for +: 'NoneType' and 'datetime.timedelta'
   [2021-11-18 11:41:50,850] {process_utils.py:100} INFO - Sending Signals.SIGTERM to GPID 41
   [2021-11-18 11:41:51,103] {process_utils.py:66} INFO - Process psutil.Process(pid=41, status='terminated', exitcode=0, started='11:00:42') (41) terminated with exit code 0
   [2021-11-18 11:41:51,104] {scheduler_job.py:614} INFO - Exited execute loop
   Traceback (most recent call last):
     File "/opt/app-root/bin/airflow", line 8, in <module>
       sys.exit(main())
     File "/opt/app-root/lib64/python3.8/site-packages/airflow/__main__.py", line 40, in main
       args.func(args)
     File "/opt/app-root/lib64/python3.8/site-packages/airflow/cli/cli_parser.py", line 48, in command
       return func(*args, **kwargs)
     File "/opt/app-root/lib64/python3.8/site-packages/airflow/utils/cli.py", line 92, in wrapper
       return f(*args, **kwargs)
     File "/opt/app-root/lib64/python3.8/site-packages/airflow/cli/commands/scheduler_command.py", line 75, in scheduler
       _run_scheduler_job(args=args)
     File "/opt/app-root/lib64/python3.8/site-packages/airflow/cli/commands/scheduler_command.py", line 46, in _run_scheduler_job
       job.run()
     File "/opt/app-root/lib64/python3.8/site-packages/airflow/jobs/base_job.py", line 245, in run
       self._execute()
     File "/opt/app-root/lib64/python3.8/site-packages/airflow/jobs/scheduler_job.py", line 587, in _execute
       self._run_scheduler_loop()
     File "/opt/app-root/lib64/python3.8/site-packages/airflow/jobs/scheduler_job.py", line 680, in _run_scheduler_loop
       next_event = timers.run(blocking=False)
     File "/opt/rh/rh-python38/root/usr/lib64/python3.8/sched.py", line 151, in run
       action(*argument, **kwargs)
     File "/opt/app-root/lib64/python3.8/site-packages/airflow/utils/event_scheduler.py", line 36, in repeat
       action(*args, **kwargs)
     File "/opt/app-root/lib64/python3.8/site-packages/airflow/utils/session.py", line 70, in wrapper
       return func(*args, session=session, **kwargs)
     File "/opt/app-root/lib64/python3.8/site-packages/airflow/jobs/scheduler_job.py", line 1069, in adopt_or_reset_orphaned_tasks
       for attempt in run_with_db_retries(logger=self.log):
     File "/opt/app-root/lib64/python3.8/site-packages/tenacity/__init__.py", line 382, in __iter__
       do = self.iter(retry_state=retry_state)
     File "/opt/app-root/lib64/python3.8/site-packages/tenacity/__init__.py", line 349, in iter
       return fut.result()
     File "/opt/rh/rh-python38/root/usr/lib64/python3.8/concurrent/futures/_base.py", line 432, in result
       return self.__get_result()
     File "/opt/rh/rh-python38/root/usr/lib64/python3.8/concurrent/futures/_base.py", line 388, in __get_result
       raise self._exception
     File "/opt/app-root/lib64/python3.8/site-packages/airflow/jobs/scheduler_job.py", line 1113, in adopt_or_reset_orphaned_tasks
       to_reset = self.executor.try_adopt_task_instances(tis_to_reset_or_adopt)
     File "/opt/app-root/lib64/python3.8/site-packages/airflow/executors/celery_executor.py", line 485, in try_adopt_task_instances
       self.adopted_task_timeouts[ti.key] = ti.queued_dttm + self.task_adoption_timeout
   TypeError: unsupported operand type(s) for +: 'NoneType' and 'datetime.timedelta'
   ```
   
   ### What you expected to happen
   
   _No response_
   
   ### How to reproduce
   
   Environment Variables
   ```bash
   export AIRFLOW_HOME="/home/deploy/bi-airflow"
   export AIRFLOW__CORE__SQL_ALCHEMY_CONN="mysql://****/airflow?charset=utf8mb4"
   export AIRFLOW__CORE__LOAD_EXAMPLES="False"
   export AIRFLOW__CORE__LOAD_DEFAULT_CONNECTIONS="False"
   export AIRFLOW__CORE__DEFAULT_TIMEZONE="Asia/Seoul"
   export AIRFLOW__WEBSERVER__DEFAULT_UI_TIMEZONE="Asia/Seoul"
   export AIRFLOW__API__AUTH_BACKEND="airflow.api.auth.backend.basic_auth"
   export AIRFLOW__WEBSERVER__RELOAD_ON_PLUGIN_CHANGE="True"
   export AIRFLOW__CELERY__BROKER_URL="pyamqp://******/airflow"
   export AIRFLOW__CELERY__RESULT_BACKEND="db+mysql://*****/airflow?charset=utf8mb4"
   export AIRFLOW__CELERY__FLOWER_URL_PREFIX="/flower"
   export AIRFLOW__CORE__FERNET_KEY="*******"
   
   # Required parts for reproduce!
   export AIRFLOW__CORE__EXECUTOR="CeleryExecutor"
   export AIRFLOW__SCHEDULER__ORPHANED_TASKS_CHECK_INTERVAL="30.0"
   export AIRFLOW__SCHEDULER__DAG_DIR_LIST_INTERVAL="10"
   ```
   
   Dag files
   ```python
   from datetime import datetime, timedelta
   import time
    
   from airflow import DAG
   from airflow.operators.python import PythonOperator
    
   default_args = {
       "owner": "deploy",
       "start_date": datetime(2021, 10, 1),
       "end_date": datetime(2021, 10, 25),
       "retries": 2,
       "retry_delay": timedelta(minutes=20),
       "depends_on_past": False,
   }
    
   def sleep_func(t: float):
       print(f"sleeping {t} sec...")
       time.sleep(t)
       print("sleep end...")
    
    
   with DAG(
       dag_id="active_run_test",
       default_args=default_args,
       schedule_interval="5 0 * * *",
       max_active_runs=32,
       catchup=True,
   ) as dag:
       t1 = PythonOperator(task_id="sleep_task1", python_callable=sleep_func, op_kwargs={"t": 120})
       t2 = PythonOperator(task_id="sleep_task2", python_callable=sleep_func, op_kwargs={"t": 70})
       t3 = PythonOperator(task_id="sleep_task3", python_callable=sleep_func, op_kwargs={"t": 80})
    
       t1 >> t2 >> t3
   ```
   
   1. Go to `active_run_test` dag site.
   2. Turn on `active_run_test` dag
   3. Run `sleep_task1` task (which run_id: scheduled__2021-10-22T15:05:00+00:00) with "Ignore All Deps" 
   
   When the state of `sleep_task1` task changes `running`, scheduler processer raise exception at `adopt_or_reset_orphaned_tasks` method.
   
   ### Anything else
   
   _No response_
   
   ### Are you willing to submit PR?
   
   - [X] Yes I am willing to submit a PR!
   
   ### Code of Conduct
   
   - [X] I agree to follow this project's [Code of Conduct](https://github.com/apache/airflow/blob/main/CODE_OF_CONDUCT.md)
   


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] uranusjr commented on issue #19671: Scheduler raise exception while running adopt_or_reset_orphaned_tasks method

Posted by GitBox <gi...@apache.org>.
uranusjr commented on issue #19671:
URL: https://github.com/apache/airflow/issues/19671#issuecomment-972597390


   A none-checking will fix the exception, but not the issue. When `ti.queued_dttm` is set when the `ti` (task instance) is being queued for execution, and should not be None here because the fact this `ti` appears in the executor in the first place means it should have been queud for execution. So the problem is deeper than the sympton and needs a more involved fix than simply none-checking.


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] humit0 commented on issue #19671: Scheduler raise exception while running adopt_or_reset_orphaned_tasks method

Posted by GitBox <gi...@apache.org>.
humit0 commented on issue #19671:
URL: https://github.com/apache/airflow/issues/19671#issuecomment-972677071


   @uranusjr 
   How about setting `ti.queued_dttm` when run task in website or cli. (`airflow tasks run active_run_test sleep_task1 scheduled__2021-10-23T15:05:00+00:00 -A`)
   
   Or add ignore `queued_dttm` value logic when run task by user.


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] uranusjr commented on issue #19671: Scheduler raise exception while running adopt_or_reset_orphaned_tasks method

Posted by GitBox <gi...@apache.org>.
uranusjr commented on issue #19671:
URL: https://github.com/apache/airflow/issues/19671#issuecomment-972705755


   Maybe we could just default to `now()` if the TI is not “actually queued” i.e. running ad-hoc.


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] potiuk commented on issue #19671: Scheduler raise exception while running adopt_or_reset_orphaned_tasks method

Posted by GitBox <gi...@apache.org>.
potiuk commented on issue #19671:
URL: https://github.com/apache/airflow/issues/19671#issuecomment-1030912142


   I think there was a similar error on using some non-posix compliant FS or where client configuration of your deployment does not match the server side of your file-system volume. For example this one https://github.com/apache/airflow/discussions/20908#discussioncomment-2121029 turned out to be cause by upgrade of NFS3 to NFS4 without upgrading clients. 
   
   I willl turn it into a discussion, because this looks awfully like similar issue


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ITJamie commented on issue #19671: Scheduler raise exception while running adopt_or_reset_orphaned_tasks method

Posted by GitBox <gi...@apache.org>.
ITJamie commented on issue #19671:
URL: https://github.com/apache/airflow/issues/19671#issuecomment-1014487656


   still experiencing this as of 2.2.3.
   It also seems to be able to nuke the entirty of the scheduler
   
   ```
   airflow-scheduler_1  | [2022-01-17 12:46:03,705] {scheduler_job.py:1114} INFO - Resetting orphaned tasks for active dag runs
   airflow-scheduler_1  | [2022-01-17 12:46:03,820] {scheduler_job.py:644} ERROR - Exception when executing SchedulerJob._run_scheduler_loop
   airflow-scheduler_1  | Traceback (most recent call last):
   airflow-scheduler_1  |   File "/opt/bitnami/airflow/venv/lib/python3.8/site-packages/airflow/jobs/scheduler_job.py", line 628, in _execute
   airflow-scheduler_1  |     self._run_scheduler_loop()
   airflow-scheduler_1  |   File "/opt/bitnami/airflow/venv/lib/python3.8/site-packages/airflow/jobs/scheduler_job.py", line 721, in _run_scheduler_loop
   airflow-scheduler_1  |     next_event = timers.run(blocking=False)
   airflow-scheduler_1  |   File "/opt/bitnami/python/lib/python3.8/sched.py", line 151, in run
   airflow-scheduler_1  |     action(*argument, **kwargs)
   airflow-scheduler_1  |   File "/opt/bitnami/airflow/venv/lib/python3.8/site-packages/airflow/utils/event_scheduler.py", line 36, in repeat
   airflow-scheduler_1  |     action(*args, **kwargs)
   airflow-scheduler_1  |   File "/opt/bitnami/airflow/venv/lib/python3.8/site-packages/airflow/utils/session.py", line 70, in wrapper
   airflow-scheduler_1  |     return func(*args, session=session, **kwargs)
   airflow-scheduler_1  |   File "/opt/bitnami/airflow/venv/lib/python3.8/site-packages/airflow/jobs/scheduler_job.py", line 1117, in adopt_or_reset_orphaned_tasks
   airflow-scheduler_1  |     for attempt in run_with_db_retries(logger=self.log):
   airflow-scheduler_1  |   File "/opt/bitnami/airflow/venv/lib/python3.8/site-packages/tenacity/__init__.py", line 382, in __iter__
   airflow-scheduler_1  |     do = self.iter(retry_state=retry_state)
   airflow-scheduler_1  |   File "/opt/bitnami/airflow/venv/lib/python3.8/site-packages/tenacity/__init__.py", line 349, in iter
   airflow-scheduler_1  |     return fut.result()
   airflow-scheduler_1  |   File "/opt/bitnami/python/lib/python3.8/concurrent/futures/_base.py", line 437, in result
   airflow-scheduler_1  |     return self.__get_result()
   airflow-scheduler_1  |   File "/opt/bitnami/python/lib/python3.8/concurrent/futures/_base.py", line 389, in __get_result
   airflow-scheduler_1  |     raise self._exception
   airflow-scheduler_1  |   File "/opt/bitnami/airflow/venv/lib/python3.8/site-packages/airflow/jobs/scheduler_job.py", line 1162, in adopt_or_reset_orphaned_tasks
   airflow-scheduler_1  |     to_reset = self.executor.try_adopt_task_instances(tis_to_reset_or_adopt)
   airflow-scheduler_1  |   File "/opt/bitnami/airflow/venv/lib/python3.8/site-packages/airflow/executors/celery_executor.py", line 485, in try_adopt_task_instances
   airflow-scheduler_1  |     self.adopted_task_timeouts[ti.key] = ti.queued_dttm + self.task_adoption_timeout
   airflow-scheduler_1  | TypeError: unsupported operand type(s) for +: 'NoneType' and 'datetime.timedelta'
   airflow-scheduler_1  | [2022-01-17 12:46:04,890] {process_utils.py:120} INFO - Sending Signals.SIGTERM to group 142. PIDs of all processes in the group: [1244, 1245, 1246, 1247, 1248, 142]
   airflow-scheduler_1  | [2022-01-17 12:46:04,890] {process_utils.py:75} INFO - Sending the signal Signals.SIGTERM to group 142
   airflow-scheduler_1  | --- Logging error ---
   airflow-scheduler_1  | Traceback (most recent call last):
   airflow-scheduler_1  |   File "/opt/bitnami/python/lib/python3.8/logging/handlers.py", line 69, in emit
   airflow-scheduler_1  |     if self.shouldRollover(record):
   airflow-scheduler_1  |   File "/opt/bitnami/python/lib/python3.8/logging/handlers.py", line 186, in shouldRollover
   airflow-scheduler_1  |     self.stream.seek(0, 2)  #due to non-posix-compliant Windows feature
   airflow-scheduler_1  | ValueError: I/O operation on closed file.
   airflow-scheduler_1  | Call stack:
   airflow-scheduler_1  |   File "/opt/bitnami/airflow/venv/bin/airflow", line 8, in <module>
   airflow-scheduler_1  |     sys.exit(main())
   airflow-scheduler_1  |   File "/opt/bitnami/airflow/venv/lib/python3.8/site-packages/airflow/__main__.py", line 48, in main
   airflow-scheduler_1  |     args.func(args)
   airflow-scheduler_1  |   File "/opt/bitnami/airflow/venv/lib/python3.8/site-packages/airflow/cli/cli_parser.py", line 48, in command
   airflow-scheduler_1  |     return func(*args, **kwargs)
   airflow-scheduler_1  |   File "/opt/bitnami/airflow/venv/lib/python3.8/site-packages/airflow/utils/cli.py", line 92, in wrapper
   airflow-scheduler_1  |     return f(*args, **kwargs)
   airflow-scheduler_1  |   File "/opt/bitnami/airflow/venv/lib/python3.8/site-packages/airflow/cli/commands/scheduler_command.py", line 75, in scheduler
   airflow-scheduler_1  |     _run_scheduler_job(args=args)
   airflow-scheduler_1  |   File "/opt/bitnami/airflow/venv/lib/python3.8/site-packages/airflow/cli/commands/scheduler_command.py", line 46, in _run_scheduler_job
   airflow-scheduler_1  |     job.run()
   airflow-scheduler_1  |   File "/opt/bitnami/airflow/venv/lib/python3.8/site-packages/airflow/jobs/base_job.py", line 245, in run
   airflow-scheduler_1  |     self._execute()
   airflow-scheduler_1  |   File "/opt/bitnami/airflow/venv/lib/python3.8/site-packages/airflow/jobs/scheduler_job.py", line 624, in _execute
   airflow-scheduler_1  |     self.processor_agent.start()
   airflow-scheduler_1  |   File "/opt/bitnami/airflow/venv/lib/python3.8/site-packages/airflow/dag_processing/manager.py", line 161, in start
   airflow-scheduler_1  |     process.start()
   airflow-scheduler_1  |   File "/opt/bitnami/python/lib/python3.8/multiprocessing/process.py", line 121, in start
   airflow-scheduler_1  |     self._popen = self._Popen(self)
   airflow-scheduler_1  |   File "/opt/bitnami/python/lib/python3.8/multiprocessing/context.py", line 277, in _Popen
   airflow-scheduler_1  |     return Popen(process_obj)
   airflow-scheduler_1  |   File "/opt/bitnami/python/lib/python3.8/multiprocessing/popen_fork.py", line 19, in __init__
   airflow-scheduler_1  |     self._launch(process_obj)
   airflow-scheduler_1  |   File "/opt/bitnami/python/lib/python3.8/multiprocessing/popen_fork.py", line 75, in _launch
   airflow-scheduler_1  |     code = process_obj._bootstrap(parent_sentinel=child_r)
   airflow-scheduler_1  |   File "/opt/bitnami/python/lib/python3.8/multiprocessing/process.py", line 315, in _bootstrap
   airflow-scheduler_1  |     self.run()
   airflow-scheduler_1  |   File "/opt/bitnami/python/lib/python3.8/multiprocessing/process.py", line 108, in run
   airflow-scheduler_1  |     self._target(*self._args, **self._kwargs)
   airflow-scheduler_1  |   File "/opt/bitnami/airflow/venv/lib/python3.8/site-packages/airflow/dag_processing/manager.py", line 287, in _run_processor_manager
   airflow-scheduler_1  |     processor_manager.start()
   airflow-scheduler_1  |   File "/opt/bitnami/airflow/venv/lib/python3.8/site-packages/airflow/dag_processing/manager.py", line 520, in start
   airflow-scheduler_1  |     return self._run_parsing_loop()
   airflow-scheduler_1  |   File "/opt/bitnami/airflow/venv/lib/python3.8/site-packages/airflow/dag_processing/manager.py", line 584, in _run_parsing_loop
   airflow-scheduler_1  |     self._refresh_dag_dir()
   airflow-scheduler_1  |   File "/opt/bitnami/airflow/venv/lib/python3.8/site-packages/airflow/dag_processing/manager.py", line 663, in _refresh_dag_dir
   airflow-scheduler_1  |     self.log.info("Searching for files in %s", self._dag_directory)
   airflow-scheduler_1  | Message: 'Searching for files in %s'
   airflow-scheduler_1  | Arguments: ('/opt/bitnami/airflow/dags',)
   airflow-scheduler_1  | --- Logging error ---
   airflow-scheduler_1  | Traceback (most recent call last):
   airflow-scheduler_1  |   File "/opt/bitnami/python/lib/python3.8/logging/handlers.py", line 69, in emit
   airflow-scheduler_1  |     if self.shouldRollover(record):
   airflow-scheduler_1  |   File "/opt/bitnami/python/lib/python3.8/logging/handlers.py", line 186, in shouldRollover
   airflow-scheduler_1  |     self.stream.seek(0, 2)  #due to non-posix-compliant Windows feature
   airflow-scheduler_1  | ValueError: I/O operation on closed file.
   airflow-scheduler_1  | Call stack:
   airflow-scheduler_1  |   File "/opt/bitnami/airflow/venv/bin/airflow", line 8, in <module>
   airflow-scheduler_1  |     sys.exit(main())
   airflow-scheduler_1  |   File "/opt/bitnami/airflow/venv/lib/python3.8/site-packages/airflow/__main__.py", line 48, in main
   airflow-scheduler_1  |     args.func(args)
   airflow-scheduler_1  |   File "/opt/bitnami/airflow/venv/lib/python3.8/site-packages/airflow/cli/cli_parser.py", line 48, in command
   airflow-scheduler_1  |     return func(*args, **kwargs)
   airflow-scheduler_1  |   File "/opt/bitnami/airflow/venv/lib/python3.8/site-packages/airflow/utils/cli.py", line 92, in wrapper
   airflow-scheduler_1  |     return f(*args, **kwargs)
   airflow-scheduler_1  |   File "/opt/bitnami/airflow/venv/lib/python3.8/site-packages/airflow/cli/commands/scheduler_command.py", line 75, in scheduler
   airflow-scheduler_1  |     _run_scheduler_job(args=args)
   airflow-scheduler_1  |   File "/opt/bitnami/airflow/venv/lib/python3.8/site-packages/airflow/cli/commands/scheduler_command.py", line 46, in _run_scheduler_job
   airflow-scheduler_1  |     job.run()
   airflow-scheduler_1  |   File "/opt/bitnami/airflow/venv/lib/python3.8/site-packages/airflow/jobs/base_job.py", line 245, in run
   airflow-scheduler_1  |     self._execute()
   airflow-scheduler_1  |   File "/opt/bitnami/airflow/venv/lib/python3.8/site-packages/airflow/jobs/scheduler_job.py", line 624, in _execute
   airflow-scheduler_1  |     self.processor_agent.start()
   airflow-scheduler_1  |   File "/opt/bitnami/airflow/venv/lib/python3.8/site-packages/airflow/dag_processing/manager.py", line 161, in start
   airflow-scheduler_1  |     process.start()
   airflow-scheduler_1  |   File "/opt/bitnami/python/lib/python3.8/multiprocessing/process.py", line 121, in start
   airflow-scheduler_1  |     self._popen = self._Popen(self)
   airflow-scheduler_1  |   File "/opt/bitnami/python/lib/python3.8/multiprocessing/context.py", line 277, in _Popen
   airflow-scheduler_1  |     return Popen(process_obj)
   airflow-scheduler_1  |   File "/opt/bitnami/python/lib/python3.8/multiprocessing/popen_fork.py", line 19, in __init__
   airflow-scheduler_1  |     self._launch(process_obj)
   airflow-scheduler_1  |   File "/opt/bitnami/python/lib/python3.8/multiprocessing/popen_fork.py", line 75, in _launch
   airflow-scheduler_1  |     code = process_obj._bootstrap(parent_sentinel=child_r)
   airflow-scheduler_1  |   File "/opt/bitnami/python/lib/python3.8/multiprocessing/process.py", line 315, in _bootstrap
   airflow-scheduler_1  |     self.run()
   airflow-scheduler_1  |   File "/opt/bitnami/python/lib/python3.8/multiprocessing/process.py", line 108, in run
   airflow-scheduler_1  |     self._target(*self._args, **self._kwargs)
   airflow-scheduler_1  |   File "/opt/bitnami/airflow/venv/lib/python3.8/site-packages/airflow/dag_processing/manager.py", line 287, in _run_processor_manager
   airflow-scheduler_1  |     processor_manager.start()
   airflow-scheduler_1  |   File "/opt/bitnami/airflow/venv/lib/python3.8/site-packages/airflow/dag_processing/manager.py", line 520, in start
   airflow-scheduler_1  |     return self._run_parsing_loop()
   airflow-scheduler_1  |   File "/opt/bitnami/airflow/venv/lib/python3.8/site-packages/airflow/dag_processing/manager.py", line 584, in _run_parsing_loop
   airflow-scheduler_1  |     self._refresh_dag_dir()
   airflow-scheduler_1  |   File "/opt/bitnami/airflow/venv/lib/python3.8/site-packages/airflow/dag_processing/manager.py", line 666, in _refresh_dag_dir
   airflow-scheduler_1  |     self.log.info("There are %s files in %s", len(self._file_paths), self._dag_directory)
   airflow-scheduler_1  | Message: 'There are %s files in %s'
   airflow-scheduler_1  | Arguments: (4359, '/opt/bitnami/airflow/dags')
   airflow-scheduler_1  | --- Logging error ---
   airflow-scheduler_1  | Traceback (most recent call last):
   airflow-scheduler_1  |   File "/opt/bitnami/python/lib/python3.8/logging/handlers.py", line 69, in emit
   airflow-scheduler_1  |     if self.shouldRollover(record):
   airflow-scheduler_1  |   File "/opt/bitnami/python/lib/python3.8/logging/handlers.py", line 186, in shouldRollover
   airflow-scheduler_1  |     self.stream.seek(0, 2)  #due to non-posix-compliant Windows feature
   airflow-scheduler_1  | ValueError: I/O operation on closed file.
   airflow-scheduler_1  | Call stack:
   airflow-scheduler_1  |   File "/opt/bitnami/airflow/venv/bin/airflow", line 8, in <module>
   airflow-scheduler_1  |     sys.exit(main())
   airflow-scheduler_1  |   File "/opt/bitnami/airflow/venv/lib/python3.8/site-packages/airflow/__main__.py", line 48, in main
   airflow-scheduler_1  |     args.func(args)
   airflow-scheduler_1  |   File "/opt/bitnami/airflow/venv/lib/python3.8/site-packages/airflow/cli/cli_parser.py", line 48, in command
   airflow-scheduler_1  |     return func(*args, **kwargs)
   airflow-scheduler_1  |   File "/opt/bitnami/airflow/venv/lib/python3.8/site-packages/airflow/utils/cli.py", line 92, in wrapper
   airflow-scheduler_1  |     return f(*args, **kwargs)
   airflow-scheduler_1  |   File "/opt/bitnami/airflow/venv/lib/python3.8/site-packages/airflow/cli/commands/scheduler_command.py", line 75, in scheduler
   airflow-scheduler_1  |     _run_scheduler_job(args=args)
   airflow-scheduler_1  |   File "/opt/bitnami/airflow/venv/lib/python3.8/site-packages/airflow/cli/commands/scheduler_command.py", line 46, in _run_scheduler_job
   airflow-scheduler_1  |     job.run()
   airflow-scheduler_1  |   File "/opt/bitnami/airflow/venv/lib/python3.8/site-packages/airflow/jobs/base_job.py", line 245, in run
   airflow-scheduler_1  |     self._execute()
   airflow-scheduler_1  |   File "/opt/bitnami/airflow/venv/lib/python3.8/site-packages/airflow/jobs/scheduler_job.py", line 624, in _execute
   airflow-scheduler_1  |     self.processor_agent.start()
   airflow-scheduler_1  |   File "/opt/bitnami/airflow/venv/lib/python3.8/site-packages/airflow/dag_processing/manager.py", line 161, in start
   airflow-scheduler_1  |     process.start()
   airflow-scheduler_1  |   File "/opt/bitnami/python/lib/python3.8/multiprocessing/process.py", line 121, in start
   airflow-scheduler_1  |     self._popen = self._Popen(self)
   airflow-scheduler_1  |   File "/opt/bitnami/python/lib/python3.8/multiprocessing/context.py", line 277, in _Popen
   airflow-scheduler_1  |     return Popen(process_obj)
   airflow-scheduler_1  | [2022-01-17 12:46:05,337] {process_utils.py:237} INFO - Waiting up to 5 seconds for processes to exit...
   airflow-scheduler_1  |   File "/opt/bitnami/python/lib/python3.8/multiprocessing/popen_fork.py", line 19, in __init__
   airflow-scheduler_1  |     self._launch(process_obj)
   airflow-scheduler_1  |   File "/opt/bitnami/python/lib/python3.8/multiprocessing/popen_fork.py", line 75, in _launch
   airflow-scheduler_1  |     code = process_obj._bootstrap(parent_sentinel=child_r)
   airflow-scheduler_1  |   File "/opt/bitnami/python/lib/python3.8/multiprocessing/process.py", line 315, in _bootstrap
   airflow-scheduler_1  |     self.run()
   airflow-scheduler_1  |   File "/opt/bitnami/python/lib/python3.8/multiprocessing/process.py", line 108, in run
   airflow-scheduler_1  |     self._target(*self._args, **self._kwargs)
   airflow-scheduler_1  |   File "/opt/bitnami/airflow/venv/lib/python3.8/site-packages/airflow/dag_processing/manager.py", line 287, in _run_processor_manager
   airflow-scheduler_1  |     processor_manager.start()
   airflow-scheduler_1  |   File "/opt/bitnami/airflow/venv/lib/python3.8/site-packages/airflow/dag_processing/manager.py", line 520, in start
   airflow-scheduler_1  |     return self._run_parsing_loop()
   airflow-scheduler_1  |   File "/opt/bitnami/airflow/venv/lib/python3.8/site-packages/airflow/dag_processing/manager.py", line 584, in _run_parsing_loop
   airflow-scheduler_1  |     self._refresh_dag_dir()
   airflow-scheduler_1  |   File "/opt/bitnami/airflow/venv/lib/python3.8/site-packages/airflow/dag_processing/manager.py", line 697, in _refresh_dag_dir
   airflow-scheduler_1  |     DagCode.remove_deleted_code(dag_filelocs)
   airflow-scheduler_1  |   File "/opt/bitnami/airflow/venv/lib/python3.8/site-packages/airflow/utils/session.py", line 70, in wrapper
   airflow-scheduler_1  |     return func(*args, session=session, **kwargs)
   airflow-scheduler_1  |   File "/opt/bitnami/airflow/venv/lib/python3.8/site-packages/airflow/models/dagcode.py", line 134, in remove_deleted_code
   airflow-scheduler_1  |     session.query(cls).filter(
   airflow-scheduler_1  |   File "/opt/bitnami/airflow/venv/lib/python3.8/site-packages/sqlalchemy/orm/query.py", line 3926, in delete
   airflow-scheduler_1  |     delete_op.exec_()
   airflow-scheduler_1  |   File "/opt/bitnami/airflow/venv/lib/python3.8/site-packages/sqlalchemy/orm/persistence.py", line 1697, in exec_
   airflow-scheduler_1  |     self._do_exec()
   airflow-scheduler_1  |   File "/opt/bitnami/airflow/venv/lib/python3.8/site-packages/sqlalchemy/orm/persistence.py", line 1930, in _do_exec
   airflow-scheduler_1  |     self._execute_stmt(delete_stmt)
   airflow-scheduler_1  |   File "/opt/bitnami/airflow/venv/lib/python3.8/site-packages/sqlalchemy/orm/persistence.py", line 1702, in _execute_stmt
   airflow-scheduler_1  |     self.result = self.query._execute_crud(stmt, self.mapper)
   airflow-scheduler_1  |   File "/opt/bitnami/airflow/venv/lib/python3.8/site-packages/sqlalchemy/orm/query.py", line 3568, in _execute_crud
   airflow-scheduler_1  |     return conn.execute(stmt, self._params)
   airflow-scheduler_1  |   File "/opt/bitnami/airflow/venv/lib/python3.8/site-packages/sqlalchemy/engine/base.py", line 1011, in execute
   airflow-scheduler_1  |     return meth(self, multiparams, params)
   airflow-scheduler_1  |   File "/opt/bitnami/airflow/venv/lib/python3.8/site-packages/sqlalchemy/sql/elements.py", line 298, in _execute_on_connection
   airflow-scheduler_1  |     return connection._execute_clauseelement(self, multiparams, params)
   airflow-scheduler_1  |   File "/opt/bitnami/airflow/venv/lib/python3.8/site-packages/sqlalchemy/engine/base.py", line 1124, in _execute_clauseelement
   airflow-scheduler_1  |     ret = self._execute_context(
   airflow-scheduler_1  |   File "/opt/bitnami/airflow/venv/lib/python3.8/site-packages/sqlalchemy/engine/base.py", line 1276, in _execute_context
   airflow-scheduler_1  |     self.dialect.do_execute(
   airflow-scheduler_1  |   File "/opt/bitnami/airflow/venv/lib/python3.8/site-packages/sqlalchemy/engine/default.py", line 608, in do_execute
   airflow-scheduler_1  |     cursor.execute(statement, parameters)
   airflow-scheduler_1  |   File "/opt/bitnami/python/lib/python3.8/encodings/utf_8.py", line 15, in decode
   airflow-scheduler_1  |     def decode(input, errors='strict'):
   airflow-scheduler_1  |   File "/opt/bitnami/airflow/venv/lib/python3.8/site-packages/airflow/dag_processing/manager.py", line 495, in _exit_gracefully
   airflow-scheduler_1  |     self.log.info("Exiting gracefully upon receiving signal %s", signum)
   airflow-scheduler_1  | Message: 'Exiting gracefully upon receiving signal %s'
   airflow-scheduler_1  | Arguments: (15,)
   airflow-scheduler_1  | [2022-01-17 12:46:05,368] {process_utils.py:70} INFO - Process psutil.Process(pid=142, status='terminated', exitcode=0, started='12:40:57') (142) terminated with exit code 0
   airflow-scheduler_1  | [2022-01-17 12:46:05,369] {process_utils.py:70} INFO - Process psutil.Process(pid=1246, status='terminated', started='12:45:56') (1246) terminated with exit code None
   airflow-scheduler_1  | [2022-01-17 12:46:05,370] {process_utils.py:70} INFO - Process psutil.Process(pid=1247, status='terminated', started='12:45:56') (1247) terminated with exit code None
   airflow-scheduler_1  | [2022-01-17 12:46:05,370] {process_utils.py:70} INFO - Process psutil.Process(pid=1245, status='terminated', started='12:45:56') (1245) terminated with exit code None
   airflow-scheduler_1  | [2022-01-17 12:46:05,370] {process_utils.py:70} INFO - Process psutil.Process(pid=1244, status='terminated', started='12:45:56') (1244) terminated with exit code None
   airflow-scheduler_1  | [2022-01-17 12:46:05,371] {process_utils.py:70} INFO - Process psutil.Process(pid=1248, status='terminated', started='12:45:56') (1248) terminated with exit code None
   airflow-scheduler_1  | [2022-01-17 12:46:05,371] {scheduler_job.py:655} INFO - Exited execute loop
   airflow-scheduler_1  | Traceback (most recent call last):
   airflow-scheduler_1  |   File "/opt/bitnami/airflow/venv/bin/airflow", line 8, in <module>
   airflow-scheduler_1  |     sys.exit(main())
   airflow-scheduler_1  |   File "/opt/bitnami/airflow/venv/lib/python3.8/site-packages/airflow/__main__.py", line 48, in main
   airflow-scheduler_1  |     args.func(args)
   airflow-scheduler_1  |   File "/opt/bitnami/airflow/venv/lib/python3.8/site-packages/airflow/cli/cli_parser.py", line 48, in command
   airflow-scheduler_1  |     return func(*args, **kwargs)
   airflow-scheduler_1  |   File "/opt/bitnami/airflow/venv/lib/python3.8/site-packages/airflow/utils/cli.py", line 92, in wrapper
   airflow-scheduler_1  |     return f(*args, **kwargs)
   airflow-scheduler_1  |   File "/opt/bitnami/airflow/venv/lib/python3.8/site-packages/airflow/cli/commands/scheduler_command.py", line 75, in scheduler
   airflow-scheduler_1  |     _run_scheduler_job(args=args)
   airflow-scheduler_1  |   File "/opt/bitnami/airflow/venv/lib/python3.8/site-packages/airflow/cli/commands/scheduler_command.py", line 46, in _run_scheduler_job
   airflow-scheduler_1  |     job.run()
   airflow-scheduler_1  |   File "/opt/bitnami/airflow/venv/lib/python3.8/site-packages/airflow/jobs/base_job.py", line 245, in run
   airflow-scheduler_1  |     self._execute()
   airflow-scheduler_1  |   File "/opt/bitnami/airflow/venv/lib/python3.8/site-packages/airflow/jobs/scheduler_job.py", line 628, in _execute
   airflow-scheduler_1  |     self._run_scheduler_loop()
   airflow-scheduler_1  |   File "/opt/bitnami/airflow/venv/lib/python3.8/site-packages/airflow/jobs/scheduler_job.py", line 721, in _run_scheduler_loop
   airflow-scheduler_1  |     next_event = timers.run(blocking=False)
   airflow-scheduler_1  |   File "/opt/bitnami/python/lib/python3.8/sched.py", line 151, in run
   airflow-scheduler_1  |     action(*argument, **kwargs)
   airflow-scheduler_1  |   File "/opt/bitnami/airflow/venv/lib/python3.8/site-packages/airflow/utils/event_scheduler.py", line 36, in repeat
   airflow-scheduler_1  |     action(*args, **kwargs)
   airflow-scheduler_1  |   File "/opt/bitnami/airflow/venv/lib/python3.8/site-packages/airflow/utils/session.py", line 70, in wrapper
   airflow-scheduler_1  |     return func(*args, session=session, **kwargs)
   airflow-scheduler_1  |   File "/opt/bitnami/airflow/venv/lib/python3.8/site-packages/airflow/jobs/scheduler_job.py", line 1117, in adopt_or_reset_orphaned_tasks
   airflow-scheduler_1  |     for attempt in run_with_db_retries(logger=self.log):
   airflow-scheduler_1  |   File "/opt/bitnami/airflow/venv/lib/python3.8/site-packages/tenacity/__init__.py", line 382, in __iter__
   airflow-scheduler_1  |     do = self.iter(retry_state=retry_state)
   airflow-scheduler_1  |   File "/opt/bitnami/airflow/venv/lib/python3.8/site-packages/tenacity/__init__.py", line 349, in iter
   airflow-scheduler_1  |     return fut.result()
   airflow-scheduler_1  |   File "/opt/bitnami/python/lib/python3.8/concurrent/futures/_base.py", line 437, in result
   airflow-scheduler_1  |     return self.__get_result()
   airflow-scheduler_1  |   File "/opt/bitnami/python/lib/python3.8/concurrent/futures/_base.py", line 389, in __get_result
   airflow-scheduler_1  |     raise self._exception
   airflow-scheduler_1  |   File "/opt/bitnami/airflow/venv/lib/python3.8/site-packages/airflow/jobs/scheduler_job.py", line 1162, in adopt_or_reset_orphaned_tasks
   airflow-scheduler_1  |     to_reset = self.executor.try_adopt_task_instances(tis_to_reset_or_adopt)
   airflow-scheduler_1  |   File "/opt/bitnami/airflow/venv/lib/python3.8/site-packages/airflow/executors/celery_executor.py", line 485, in try_adopt_task_instances
   airflow-scheduler_1  |     self.adopted_task_timeouts[ti.key] = ti.queued_dttm + self.task_adoption_timeout
   airflow-scheduler_1  | TypeError: unsupported operand type(s) for +: 'NoneType' and 'datetime.timedelta'
   ```
   
   Im not sure if the " #due to non-posix-compliant Windows feature" error is related or new 


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org