def colored_console_logger(init_context): """This logger provides support for sending Dagster logs to stdout in a colored format. It is included by default on jobs which do not otherwise specify loggers. """ level = coerce_valid_log_level(init_context.logger_config["log_level"]) name = init_context.logger_config["name"] klass = logging.getLoggerClass() logger_ = klass(name, level=level) coloredlogs.install( logger=logger_, level=level, fmt=default_format_string(), datefmt=default_date_format_string(), field_styles={ "levelname": { "color": "blue" }, "asctime": { "color": "green" } }, level_styles={ "debug": {}, "error": { "color": "red" } }, ) return logger_
def launch_scheduled_runs_for_schedule( instance, logger, schedule_state, workspace, repo_location, end_datetime_utc, max_catchup_runs, max_tick_retries, debug_crash_flags=None, log_verbose_checks=True, ): check.inst_param(instance, "instance", DagsterInstance) check.inst_param(schedule_state, "schedule_state", InstigatorState) check.inst_param(end_datetime_utc, "end_datetime_utc", datetime.datetime) check.inst_param(repo_location, "repo_location", RepositoryLocation) latest_tick = instance.get_latest_job_tick(schedule_state.job_origin_id) start_timestamp_utc = schedule_state.job_specific_data.start_timestamp if latest_tick: if latest_tick.status == TickStatus.STARTED: # Scheduler was interrupted while performing this tick, re-do it start_timestamp_utc = max(start_timestamp_utc, latest_tick.timestamp) elif (latest_tick.status == TickStatus.FAILURE and latest_tick.failure_count <= max_tick_retries): # Tick failed and hasn't reached its retry limit, retry it start_timestamp_utc = max(start_timestamp_utc, latest_tick.timestamp) else: start_timestamp_utc = max(start_timestamp_utc, latest_tick.timestamp + 1) schedule_name = schedule_state.job_name repo_name = schedule_state.origin.external_repository_origin.repository_name if not repo_location.has_repository(repo_name): raise DagsterSchedulerError( f"Could not find repository {repo_name} in location {repo_location.name} to " + f"run schedule {schedule_name}. If this repository no longer exists, you can " + "turn off the schedule in the Dagit UI.", ) external_repo = repo_location.get_repository(repo_name) if not external_repo.has_external_schedule(schedule_name): raise DagsterSchedulerError( f"Could not find schedule {schedule_name} in repository {repo_name}. If this " "schedule no longer exists, you can turn it off in the Dagit UI.", ) external_schedule = external_repo.get_external_schedule(schedule_name) timezone_str = external_schedule.execution_timezone if not timezone_str: timezone_str = "UTC" if log_verbose_checks: logger.warn( f"Using UTC as the timezone for {external_schedule.name} as it did not specify " "an execution_timezone in its definition.") tick_times = [] for next_time in external_schedule.execution_time_iterator( start_timestamp_utc): if next_time.timestamp() > end_datetime_utc.timestamp(): break tick_times.append(next_time) if not tick_times: if log_verbose_checks: logger.info(f"No new runs for {schedule_name}") return if not external_schedule.partition_set_name and len(tick_times) > 1: logger.warning( f"{schedule_name} has no partition set, so not trying to catch up") tick_times = tick_times[-1:] elif len(tick_times) > max_catchup_runs: logger.warning( f"{schedule_name} has fallen behind, only launching {max_catchup_runs} runs" ) tick_times = tick_times[-max_catchup_runs:] if len(tick_times) == 1: tick_time = tick_times[0].strftime(default_date_format_string()) logger.info(f"Evaluating schedule `{schedule_name}` at {tick_time}") else: times = ", ".join([ time.strftime(default_date_format_string()) for time in tick_times ]) logger.info( f"Evaluating schedule `{schedule_name}` at the following times: {times}" ) for schedule_time in tick_times: schedule_timestamp = schedule_time.timestamp() schedule_time_str = schedule_time.strftime( default_date_format_string()) if latest_tick and latest_tick.timestamp == schedule_timestamp: tick = latest_tick if latest_tick.status == TickStatus.FAILURE: logger.info( f"Retrying previously failed schedule execution at {schedule_time_str}" ) else: logger.info( f"Resuming previously interrupted schedule execution at {schedule_time_str}" ) else: tick = instance.create_job_tick( TickData( job_origin_id=external_schedule.get_external_origin_id(), job_name=schedule_name, job_type=InstigatorType.SCHEDULE, status=TickStatus.STARTED, timestamp=schedule_timestamp, )) _check_for_debug_crash(debug_crash_flags, "TICK_CREATED") with _ScheduleLaunchContext(tick, instance, logger) as tick_context: try: _check_for_debug_crash(debug_crash_flags, "TICK_HELD") yield from _schedule_runs_at_time( instance, logger, workspace, repo_location, external_repo, external_schedule, schedule_time, tick_context, debug_crash_flags, ) except Exception as e: if isinstance(e, DagsterUserCodeUnreachableError): try: raise DagsterSchedulerError( f"Unable to reach the user code server for schedule {schedule_name}. Schedule will resume execution once the server is available." ) from e except: error_data = serializable_error_info_from_exc_info( sys.exc_info()) tick_context.update_state( TickStatus.FAILURE, error=error_data, # don't increment the failure count - retry forever until the server comes back up # or the schedule is turned off failure_count=tick_context.failure_count, ) raise # Raise the wrapped DagsterSchedulerError exception else: error_data = serializable_error_info_from_exc_info( sys.exc_info()) tick_context.update_state( TickStatus.FAILURE, error=error_data, failure_count=tick_context.failure_count + 1, ) raise
def launch_scheduled_runs_for_schedule( instance, logger, external_schedule: ExternalSchedule, schedule_state: InstigatorState, workspace, end_datetime_utc: datetime.datetime, max_catchup_runs, max_tick_retries, debug_crash_flags=None, log_verbose_checks=True, ): instance = check.inst_param(instance, "instance", DagsterInstance) schedule_state = check.opt_inst_param(schedule_state, "schedule_state", InstigatorState) end_datetime_utc = check.inst_param(end_datetime_utc, "end_datetime_utc", datetime.datetime) instigator_origin_id = external_schedule.get_external_origin_id() ticks = instance.get_ticks(instigator_origin_id, external_schedule.selector_id, limit=1) latest_tick = ticks[0] if ticks else None instigator_data = cast(ScheduleInstigatorData, schedule_state.instigator_data) start_timestamp_utc = instigator_data.start_timestamp if schedule_state else None if latest_tick: if latest_tick.status == TickStatus.STARTED or ( latest_tick.status == TickStatus.FAILURE and latest_tick.failure_count <= max_tick_retries ): # Scheduler was interrupted while performing this tick, re-do it start_timestamp_utc = ( max(start_timestamp_utc, latest_tick.timestamp) if start_timestamp_utc else latest_tick.timestamp ) else: start_timestamp_utc = ( max(start_timestamp_utc, latest_tick.timestamp + 1) if start_timestamp_utc else latest_tick.timestamp + 1 ) else: start_timestamp_utc = instigator_data.start_timestamp schedule_name = external_schedule.name timezone_str = external_schedule.execution_timezone if not timezone_str: timezone_str = "UTC" if log_verbose_checks: logger.warn( f"Using UTC as the timezone for {external_schedule.name} as it did not specify " "an execution_timezone in its definition." ) tick_times = [] for next_time in external_schedule.execution_time_iterator(start_timestamp_utc): if next_time.timestamp() > end_datetime_utc.timestamp(): break tick_times.append(next_time) if not tick_times: if log_verbose_checks: logger.info(f"No new runs for {schedule_name}") return if not external_schedule.partition_set_name and len(tick_times) > 1: logger.warning(f"{schedule_name} has no partition set, so not trying to catch up") tick_times = tick_times[-1:] elif len(tick_times) > max_catchup_runs: logger.warning(f"{schedule_name} has fallen behind, only launching {max_catchup_runs} runs") tick_times = tick_times[-max_catchup_runs:] if len(tick_times) == 1: tick_time = tick_times[0].strftime(default_date_format_string()) logger.info(f"Evaluating schedule `{schedule_name}` at {tick_time}") else: times = ", ".join([time.strftime(default_date_format_string()) for time in tick_times]) logger.info(f"Evaluating schedule `{schedule_name}` at the following times: {times}") for schedule_time in tick_times: schedule_timestamp = schedule_time.timestamp() schedule_time_str = schedule_time.strftime(default_date_format_string()) if latest_tick and latest_tick.timestamp == schedule_timestamp: tick = latest_tick if latest_tick.status == TickStatus.FAILURE: logger.info(f"Retrying previously failed schedule execution at {schedule_time_str}") else: logger.info( f"Resuming previously interrupted schedule execution at {schedule_time_str}" ) else: tick = instance.create_tick( TickData( instigator_origin_id=instigator_origin_id, instigator_name=schedule_name, instigator_type=InstigatorType.SCHEDULE, status=TickStatus.STARTED, timestamp=schedule_timestamp, selector_id=external_schedule.selector_id, ) ) _check_for_debug_crash(debug_crash_flags, "TICK_CREATED") with _ScheduleLaunchContext(tick, instance, logger) as tick_context: try: _check_for_debug_crash(debug_crash_flags, "TICK_HELD") yield from _schedule_runs_at_time( instance, logger, workspace, external_schedule, schedule_time, tick_context, debug_crash_flags, ) except Exception as e: if isinstance(e, DagsterUserCodeUnreachableError): try: raise DagsterSchedulerError( f"Unable to reach the user code server for schedule {schedule_name}. Schedule will resume execution once the server is available." ) from e except: error_data = serializable_error_info_from_exc_info(sys.exc_info()) tick_context.update_state( TickStatus.FAILURE, error=error_data, # don't increment the failure count - retry forever until the server comes back up # or the schedule is turned off failure_count=tick_context.failure_count, ) raise # Raise the wrapped DagsterSchedulerError exception else: error_data = serializable_error_info_from_exc_info(sys.exc_info()) tick_context.update_state( TickStatus.FAILURE, error=error_data, failure_count=tick_context.failure_count + 1, ) raise