def get_execution_dates(dag: DAG, execution_date: datetime, future: bool, past: bool, *, session: SASession = NEW_SESSION) -> List[datetime]: """Returns dates of DAG execution""" latest_execution_date = dag.get_latest_execution_date(session=session) if latest_execution_date is None: raise ValueError(f"Received non-localized date {execution_date}") execution_date = timezone.coerce_datetime(execution_date) # determine date range of dag runs and tasks to consider end_date = latest_execution_date if future else execution_date if dag.start_date: start_date = dag.start_date else: start_date = execution_date start_date = execution_date if not past else start_date if not dag.timetable.can_run: # If the DAG never schedules, need to look at existing DagRun if the user wants future or # past runs. dag_runs = dag.get_dagruns_between(start_date=start_date, end_date=end_date) dates = sorted({d.execution_date for d in dag_runs}) elif not dag.timetable.periodic: dates = [start_date] else: dates = [ info.logical_date for info in dag.iter_dagrun_infos_between( start_date, end_date, align=False) ] return dates
def __init__(self, *, target_time, **kwargs): super().__init__(**kwargs) self.target_time = target_time self.target_datetime = timezone.coerce_datetime( datetime.datetime.combine(datetime.datetime.today(), self.target_time))
def run_cleanup( *, clean_before_timestamp: DateTime, table_names: Optional[List[str]] = None, dry_run: bool = False, verbose: bool = False, confirm: bool = True, skip_archive: bool = False, session: 'Session' = NEW_SESSION, ): """ Purges old records in airflow metadata database. The last non-externally-triggered dag run will always be kept in order to ensure continuity of scheduled dag runs. Where there are foreign key relationships, deletes will cascade, so that for example if you clean up old dag runs, the associated task instances will be deleted. :param clean_before_timestamp: The timestamp before which data should be purged :param table_names: Optional. List of table names to perform maintenance on. If list not provided, will perform maintenance on all tables. :param dry_run: If true, print rows meeting deletion criteria :param verbose: If true, may provide more detailed output. :param confirm: Require user input to confirm before processing deletions. :param skip_archive: Set to True if you don't want the purged rows preservied in an archive table. :param session: Session representing connection to the metadata database. """ clean_before_timestamp = timezone.coerce_datetime(clean_before_timestamp) effective_table_names = table_names if table_names else list(config_dict.keys()) effective_config_dict = {k: v for k, v in config_dict.items() if k in effective_table_names} if dry_run: print('Performing dry run for db cleanup.') print( f"Data prior to {clean_before_timestamp} would be purged " f"from tables {effective_table_names} with the following config:\n" ) _print_config(configs=effective_config_dict) if not dry_run and confirm: _confirm_delete(date=clean_before_timestamp, tables=list(effective_config_dict.keys())) existing_tables = reflect_tables(tables=None, session=session).tables for table_name, table_config in effective_config_dict.items(): if table_name not in existing_tables: logger.warning("Table %s not found. Skipping.", table_name) continue with _suppress_with_logging(table_name, session): _cleanup_table( clean_before_timestamp=clean_before_timestamp, dry_run=dry_run, verbose=verbose, **table_config.__dict__, skip_archive=skip_archive, session=session, ) session.commit()
def _execute(self, session=None): """ Initializes all components required to run a dag for a specified date range and calls helper method to execute the tasks. """ ti_status = BackfillJob._DagRunTaskStatus() start_date = self.bf_start_date # Get DagRun schedule between the start/end dates, which will turn into dag runs. dagrun_start_date = timezone.coerce_datetime(start_date) if self.bf_end_date is None: dagrun_end_date = pendulum.now(timezone.utc) else: dagrun_end_date = pendulum.instance(self.bf_end_date) dagrun_infos = list( self.dag.iter_dagrun_infos_between(dagrun_start_date, dagrun_end_date)) if self.run_backwards: tasks_that_depend_on_past = [ t.task_id for t in self.dag.task_dict.values() if t.depends_on_past ] if tasks_that_depend_on_past: raise AirflowException( f'You cannot backfill backwards because one or more ' f'tasks depend_on_past: {",".join(tasks_that_depend_on_past)}' ) dagrun_infos = dagrun_infos[::-1] if not dagrun_infos: if not self.run_at_least_once: self.log.info( "No run dates were found for the given dates and dag interval." ) return dagrun_infos = [ DagRunInfo.interval(dagrun_start_date, dagrun_end_date) ] # picklin' pickle_id = None if not self.donot_pickle and self.executor_class not in ( executor_constants.LOCAL_EXECUTOR, executor_constants.SEQUENTIAL_EXECUTOR, executor_constants.DASK_EXECUTOR, ): pickle = DagPickle(self.dag) session.add(pickle) session.commit() pickle_id = pickle.id executor = self.executor executor.job_id = "backfill" executor.start() ti_status.total_runs = len(dagrun_infos) # total dag runs in backfill try: remaining_dates = ti_status.total_runs while remaining_dates > 0: dagrun_infos_to_process = [ dagrun_info for dagrun_info in dagrun_infos if dagrun_info.logical_date not in ti_status.executed_dag_run_dates ] self._execute_dagruns( dagrun_infos=dagrun_infos_to_process, ti_status=ti_status, executor=executor, pickle_id=pickle_id, start_date=start_date, session=session, ) remaining_dates = ti_status.total_runs - len( ti_status.executed_dag_run_dates) err = self._collect_errors(ti_status=ti_status, session=session) if err: raise BackfillUnfinished(err, ti_status) if remaining_dates > 0: self.log.info( "max_active_runs limit for dag %s has been reached " " - waiting for other dag runs to finish", self.dag_id, ) time.sleep(self.delay_on_limit_secs) except (KeyboardInterrupt, SystemExit): self.log.warning("Backfill terminated by user.") # TODO: we will need to terminate running task instances and set the # state to failed. self._set_unfinished_dag_runs_to_failed(ti_status.active_runs) finally: session.commit() executor.end() self.log.info("Backfill done. Exiting.")
def _execute(self, session=None): """ Initializes all components required to run a dag for a specified date range and calls helper method to execute the tasks. """ ti_status = BackfillJob._DagRunTaskStatus() start_date = self.bf_start_date # Get DagRun schedule between the start/end dates, which will turn into dag runs. dagrun_start_date = timezone.coerce_datetime(start_date) if self.bf_end_date is None: dagrun_end_date = pendulum.now(timezone.utc) else: dagrun_end_date = pendulum.instance(self.bf_end_date) dagrun_infos = list( self.dag.iter_dagrun_infos_between(dagrun_start_date, dagrun_end_date)) if self.run_backwards: tasks_that_depend_on_past = [ t.task_id for t in self.dag.task_dict.values() if t.depends_on_past ] if tasks_that_depend_on_past: raise AirflowException( f'You cannot backfill backwards because one or more ' f'tasks depend_on_past: {",".join(tasks_that_depend_on_past)}' ) dagrun_infos = dagrun_infos[::-1] if not dagrun_infos: if not self.run_at_least_once: self.log.info( "No run dates were found for the given dates and dag interval." ) return dagrun_infos = [ DagRunInfo.interval(dagrun_start_date, dagrun_end_date) ] dag_with_subdags_ids = [d.dag_id for d in self._get_dag_with_subdags()] running_dagruns = DagRun.find( dag_id=dag_with_subdags_ids, execution_start_date=self.bf_start_date, execution_end_date=self.bf_end_date, no_backfills=True, state=DagRunState.RUNNING, ) if running_dagruns: for run in running_dagruns: self.log.error( "Backfill cannot be created for DagRun %s in %s, as there's already %s in a RUNNING " "state.", run.run_id, run.execution_date.strftime("%Y-%m-%dT%H:%M:%S"), run.run_type, ) self.log.error( "Changing DagRun into BACKFILL would cause scheduler to lose track of executing " "tasks. Not changing DagRun type into BACKFILL, and trying insert another DagRun into " "database would cause database constraint violation for dag_id + execution_date " "combination. Please adjust backfill dates or wait for this DagRun to finish.", ) return # picklin' pickle_id = None if not self.donot_pickle and self.executor_class not in ( executor_constants.LOCAL_EXECUTOR, executor_constants.SEQUENTIAL_EXECUTOR, executor_constants.DASK_EXECUTOR, ): pickle = DagPickle(self.dag) session.add(pickle) session.commit() pickle_id = pickle.id executor = self.executor executor.job_id = "backfill" executor.start() ti_status.total_runs = len(dagrun_infos) # total dag runs in backfill try: remaining_dates = ti_status.total_runs while remaining_dates > 0: dagrun_infos_to_process = [ dagrun_info for dagrun_info in dagrun_infos if dagrun_info.logical_date not in ti_status.executed_dag_run_dates ] self._execute_dagruns( dagrun_infos=dagrun_infos_to_process, ti_status=ti_status, executor=executor, pickle_id=pickle_id, start_date=start_date, session=session, ) remaining_dates = ti_status.total_runs - len( ti_status.executed_dag_run_dates) err = self._collect_errors(ti_status=ti_status, session=session) if err: if not self.continue_on_failures or ti_status.deadlocked: raise BackfillUnfinished(err, ti_status) if remaining_dates > 0: self.log.info( "max_active_runs limit for dag %s has been reached " " - waiting for other dag runs to finish", self.dag_id, ) time.sleep(self.delay_on_limit_secs) except (KeyboardInterrupt, SystemExit): self.log.warning("Backfill terminated by user.") # TODO: we will need to terminate running task instances and set the # state to failed. self._set_unfinished_dag_runs_to_failed(ti_status.active_runs) finally: session.commit() executor.end() self.log.info("Backfill done for DAG %s. Exiting.", self.dag)
def execute(self, context: dict) -> None: # Define intervals and prefixes. try: timespan_start = context["data_interval_start"] timespan_end = context["data_interval_end"] except KeyError: # Data interval context variables are only available in Airflow 2.2+ timespan_start = timezone.coerce_datetime( context["execution_date"]) timespan_end = timezone.coerce_datetime( context["dag"].following_schedule(timespan_start)) if timespan_end is None: # Only possible in Airflow before 2.2. self.log.warning( "No following schedule found, setting timespan end to max %s", timespan_end) timespan_end = DateTime.max elif timespan_start >= timespan_end: # Airflow 2.2 sets start == end for non-perodic schedules. self.log.warning( "DAG schedule not periodic, setting timespan end to max %s", timespan_end) timespan_end = DateTime.max timespan_start = timespan_start.in_timezone(timezone.utc) timespan_end = timespan_end.in_timezone(timezone.utc) source_prefix_interp = GCSTimeSpanFileTransformOperator.interpolate_prefix( self.source_prefix, timespan_start, ) destination_prefix_interp = GCSTimeSpanFileTransformOperator.interpolate_prefix( self.destination_prefix, timespan_start, ) source_hook = GCSHook( gcp_conn_id=self.source_gcp_conn_id, impersonation_chain=self.source_impersonation_chain, ) destination_hook = GCSHook( gcp_conn_id=self.destination_gcp_conn_id, impersonation_chain=self.destination_impersonation_chain, ) # Fetch list of files. blobs_to_transform = source_hook.list_by_timespan( bucket_name=self.source_bucket, prefix=source_prefix_interp, timespan_start=timespan_start, timespan_end=timespan_end, ) with TemporaryDirectory() as temp_input_dir, TemporaryDirectory( ) as temp_output_dir: temp_input_dir = Path(temp_input_dir) temp_output_dir = Path(temp_output_dir) # TODO: download in parallel. for blob_to_transform in blobs_to_transform: destination_file = temp_input_dir / blob_to_transform destination_file.parent.mkdir(parents=True, exist_ok=True) try: source_hook.download( bucket_name=self.source_bucket, object_name=blob_to_transform, filename=str(destination_file), chunk_size=self.chunk_size, num_max_attempts=self.download_num_attempts, ) except GoogleCloudError: if self.download_continue_on_fail: continue raise self.log.info("Starting the transformation") cmd = [self.transform_script] if isinstance( self.transform_script, str) else self.transform_script cmd += [ str(temp_input_dir), str(temp_output_dir), timespan_start.replace(microsecond=0).isoformat(), timespan_end.replace(microsecond=0).isoformat(), ] with subprocess.Popen(args=cmd, stdout=subprocess.PIPE, stderr=subprocess.STDOUT, close_fds=True) as process: self.log.info("Process output:") if process.stdout: for line in iter(process.stdout.readline, b''): self.log.info( line.decode(self.output_encoding).rstrip()) process.wait() if process.returncode: raise AirflowException( f"Transform script failed: {process.returncode}") self.log.info( "Transformation succeeded. Output temporarily located at %s", temp_output_dir) files_uploaded = [] # TODO: upload in parallel. for upload_file in temp_output_dir.glob("**/*"): if upload_file.is_dir(): continue upload_file_name = str( upload_file.relative_to(temp_output_dir)) if self.destination_prefix is not None: upload_file_name = f"{destination_prefix_interp}/{upload_file_name}" self.log.info("Uploading file %s to %s", upload_file, upload_file_name) try: destination_hook.upload( bucket_name=self.destination_bucket, object_name=upload_file_name, filename=str(upload_file), chunk_size=self.chunk_size, num_max_attempts=self.upload_num_attempts, ) files_uploaded.append(str(upload_file_name)) except GoogleCloudError: if self.upload_continue_on_fail: continue raise return files_uploaded
def _trigger_dag( dag_id: str, dag_bag: DagBag, run_id: Optional[str] = None, conf: Optional[Union[dict, str]] = None, execution_date: Optional[datetime] = None, replace_microseconds: bool = True, ) -> List[Optional[DagRun]]: """Triggers DAG run. :param dag_id: DAG ID :param dag_bag: DAG Bag model :param run_id: ID of the dag_run :param conf: configuration :param execution_date: date of execution :param replace_microseconds: whether microseconds should be zeroed :return: list of triggered dags """ dag = dag_bag.get_dag(dag_id) # prefetch dag if it is stored serialized if dag is None or dag_id not in dag_bag.dags: raise DagNotFound(f"Dag id {dag_id} not found") execution_date = execution_date if execution_date else timezone.utcnow() if not timezone.is_localized(execution_date): raise ValueError("The execution_date should be localized") if replace_microseconds: execution_date = execution_date.replace(microsecond=0) if dag.default_args and 'start_date' in dag.default_args: min_dag_start_date = dag.default_args["start_date"] if min_dag_start_date and execution_date < min_dag_start_date: raise ValueError( f"The execution_date [{execution_date.isoformat()}] should be >= start_date " f"[{min_dag_start_date.isoformat()}] from DAG's default_args" ) logical_date = timezone.coerce_datetime(execution_date) data_interval = dag.timetable.infer_manual_data_interval(run_after=logical_date) run_id = run_id or dag.timetable.generate_run_id( run_type=DagRunType.MANUAL, logical_date=logical_date, data_interval=data_interval ) dag_run = DagRun.find_duplicate(dag_id=dag_id, execution_date=execution_date, run_id=run_id) if dag_run: raise DagRunAlreadyExists( f"A Dag Run already exists for dag id {dag_id} at {execution_date} with run id {run_id}" ) run_conf = None if conf: run_conf = conf if isinstance(conf, dict) else json.loads(conf) dag_runs = [] dags_to_run = [dag] + dag.subdags for _dag in dags_to_run: dag_run = _dag.create_dagrun( run_id=run_id, execution_date=execution_date, state=DagRunState.QUEUED, conf=run_conf, external_trigger=True, dag_hash=dag_bag.dags_hash.get(dag_id), data_interval=data_interval, ) dag_runs.append(dag_run) return dag_runs