Source code for airflow.providers.celery.sensors.celery_queue
## Licensed to the Apache Software Foundation (ASF) under one# or more contributor license agreements. See the NOTICE file# distributed with this work for additional information# regarding copyright ownership. The ASF licenses this file# to you under the Apache License, Version 2.0 (the# "License"); you may not use this file except in compliance# with the License. You may obtain a copy of the License at## http://www.apache.org/licenses/LICENSE-2.0## Unless required by applicable law or agreed to in writing,# software distributed under the License is distributed on an# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY# KIND, either express or implied. See the License for the# specific language governing permissions and limitations# under the License.fromtypingimportTYPE_CHECKING,Optionalfromcelery.appimportcontrolfromairflow.sensors.baseimportBaseSensorOperatorifTYPE_CHECKING:fromairflow.utils.contextimportContext
[docs]classCeleryQueueSensor(BaseSensorOperator):""" Waits for a Celery queue to be empty. By default, in order to be considered empty, the queue must not have any tasks in the ``reserved``, ``scheduled`` or ``active`` states. :param celery_queue: The name of the Celery queue to wait for. :param target_task_id: Task id for checking """def__init__(self,*,celery_queue:str,target_task_id:Optional[str]=None,**kwargs)->None:super().__init__(**kwargs)self.celery_queue=celery_queueself.target_task_id=target_task_iddef_check_task_id(self,context:'Context')->bool:""" Gets the returned Celery result from the Airflow task ID provided to the sensor, and returns True if the celery result has been finished execution. :param context: Airflow's execution context :return: True if task has been executed, otherwise False :rtype: bool """ti=context['ti']celery_result=ti.xcom_pull(task_ids=self.target_task_id)returncelery_result.ready()
[docs]defpoke(self,context:'Context')->bool:ifself.target_task_id:returnself._check_task_id(context)inspect_result=control.Inspect()reserved=inspect_result.reserved()scheduled=inspect_result.scheduled()active=inspect_result.active()try:reserved=len(reserved[self.celery_queue])scheduled=len(scheduled[self.celery_queue])active=len(active[self.celery_queue])self.log.info('Checking if celery queue %s is empty.',self.celery_queue)returnreserved==0andscheduled==0andactive==0exceptKeyError:raiseKeyError(f'Could not locate Celery queue {self.celery_queue}')