blob: 4fee53e73b58adf61caa9675358db03ee503da41 [file] [log] [blame]
# -*- coding: utf-8 -*-
#
# 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.
from airflow.sensors.base_sensor_operator import BaseSensorOperator
from airflow.utils import timezone
from airflow.utils.decorators import apply_defaults
class TimeDeltaSensor(BaseSensorOperator):
"""
Waits for a timedelta after the task's execution_date + schedule_interval.
In Airflow, the daily task stamped with ``execution_date``
2016-01-01 can only start running on 2016-01-02. The timedelta here
represents the time after the execution period has closed.
:param delta: time length to wait after execution_date before succeeding
:type delta: datetime.timedelta
"""
@apply_defaults
def __init__(self, delta, *args, **kwargs):
super().__init__(*args, **kwargs)
self.delta = delta
def poke(self, context):
dag = context['dag']
target_dttm = dag.following_schedule(context['execution_date'])
target_dttm += self.delta
self.log.info('Checking if the time (%s) has come', target_dttm)
return timezone.utcnow() > target_dttm