API Reference¶
Operators¶
Operators allow for generation of certain types of tasks that become nodes in
the DAG when instantiated. All operators derive from BaseOperator
and
inherit many attributes and methods that way. Refer to the BaseOperator
documentation for more details.
There are 3 main types of operators:
- Operators that performs an action, or tell another system to perform an action
- Transfer operators move data from one system to another
- Sensors are a certain type of operator that will keep running until a
certain criterion is met. Examples include a specific file landing in HDFS or
S3, a partition appearing in Hive, or a specific time of the day. Sensors
are derived from
BaseSensorOperator
and run a poke method at a specifiedpoke_interval
until it returnsTrue
.
BaseOperator¶
All operators are derived from BaseOperator
and acquire much
functionality through inheritance. Since this is the core of the engine,
it’s worth taking the time to understand the parameters of BaseOperator
to understand the primitive features that can be leveraged in your
DAGs.
-
class
airflow.models.
BaseOperator
(task_id, owner=’Airflow’, email=None, email_on_retry=True, email_on_failure=True, retries=0, retry_delay=datetime.timedelta(0, 300), retry_exponential_backoff=False, max_retry_delay=None, start_date=None, end_date=None, schedule_interval=None, depends_on_past=False, wait_for_downstream=False, dag=None, params=None, default_args=None, adhoc=False, priority_weight=1, queue=’default’, pool=None, sla=None, execution_timeout=None, on_failure_callback=None, on_success_callback=None, on_retry_callback=None, trigger_rule=’all_success’, resources=None, run_as_user=None, *args, **kwargs)[source]¶ Bases:
object
Abstract base class for all operators. Since operators create objects that become node in the dag, BaseOperator contains many recursive methods for dag crawling behavior. To derive this class, you are expected to override the constructor as well as the ‘execute’ method.
Operators derived from this task should perform or trigger certain tasks synchronously (wait for completion). Example of operators could be an operator the runs a Pig job (PigOperator), a sensor operator that waits for a partition to land in Hive (HiveSensorOperator), or one that moves data from Hive to MySQL (Hive2MySqlOperator). Instances of these operators (tasks) target specific operations, running specific scripts, functions or data transfers.
This class is abstract and shouldn’t be instantiated. Instantiating a class derived from this one results in the creation of a task object, which ultimately becomes a node in DAG objects. Task dependencies should be set by using the set_upstream and/or set_downstream methods.
Note that this class is derived from SQLAlchemy’s Base class, which allows us to push metadata regarding tasks to the database. Deriving this classes needs to implement the polymorphic specificities documented in SQLAlchemy. This should become clear while reading the code for other operators.
Parameters: - task_id (string) – a unique, meaningful id for the task
- owner (string) – the owner of the task, using the unix username is recommended
- retries (int) – the number of retries that should be performed before failing the task
- retry_delay (timedelta) – delay between retries
- retry_exponential_backoff (bool) – allow progressive longer waits between retries by using exponential backoff algorithm on retry delay (delay will be converted into seconds)
- max_retry_delay (timedelta) – maximum delay interval between retries
- start_date (datetime) – The
start_date
for the task, determines theexecution_date
for the first task instance. The best practice is to have the start_date rounded to your DAG’sschedule_interval
. Daily jobs have their start_date some day at 00:00:00, hourly jobs have their start_date at 00:00 of a specific hour. Note that Airflow simply looks at the latestexecution_date
and adds theschedule_interval
to determine the nextexecution_date
. It is also very important to note that different tasks’ dependencies need to line up in time. If task A depends on task B and their start_date are offset in a way that their execution_date don’t line up, A’s dependencies will never be met. If you are looking to delay a task, for example running a daily task at 2AM, look into theTimeSensor
andTimeDeltaSensor
. We advise against using dynamicstart_date
and recommend using fixed ones. Read the FAQ entry about start_date for more information. - end_date (datetime) – if specified, the scheduler won’t go beyond this date
- depends_on_past (bool) – when set to true, task instances will run sequentially while relying on the previous task’s schedule to succeed. The task instance for the start_date is allowed to run.
- wait_for_downstream (bool) – when set to true, an instance of task X will wait for tasks immediately downstream of the previous instance of task X to finish successfully before it runs. This is useful if the different instances of a task X alter the same asset, and this asset is used by tasks downstream of task X. Note that depends_on_past is forced to True wherever wait_for_downstream is used.
- queue (str) – which queue to target when running this job. Not all executors implement queue management, the CeleryExecutor does support targeting specific queues.
- dag (DAG) – a reference to the dag the task is attached to (if any)
- priority_weight (int) – priority weight of this task against other task. This allows the executor to trigger higher priority tasks before others when things get backed up.
- pool (str) – the slot pool this task should run in, slot pools are a way to limit concurrency for certain tasks
- sla (datetime.timedelta) – time by which the job is expected to succeed. Note that
this represents the
timedelta
after the period is closed. For example if you set an SLA of 1 hour, the scheduler would send dan email soon after 1:00AM on the2016-01-02
if the2016-01-01
instance has not succeeded yet. The scheduler pays special attention for jobs with an SLA and sends alert emails for sla misses. SLA misses are also recorded in the database for future reference. All tasks that share the same SLA time get bundled in a single email, sent soon after that time. SLA notification are sent once and only once for each task instance. - execution_timeout (datetime.timedelta) – max time allowed for the execution of this task instance, if it goes beyond it will raise and fail.
- on_failure_callback (callable) – a function to be called when a task instance of this task fails. a context dictionary is passed as a single parameter to this function. Context contains references to related objects to the task instance and is documented under the macros section of the API.
- on_retry_callback – much like the
on_failure_callback
excepts that it is executed when retries occur. - on_success_callback (callable) – much like the
on_failure_callback
excepts that it is executed when the task succeeds. - trigger_rule (str) – defines the rule by which dependencies are applied
for the task to get triggered. Options are:
{ all_success | all_failed | all_done | one_success | one_failed | dummy}
default isall_success
. Options can be set as string or using the constants defined in the static classairflow.utils.TriggerRule
- resources (dict) – A map of resource parameter names (the argument names of the Resources constructor) to their values.
- run_as_user (str) – unix username to impersonate while running the task
-
clear
(start_date=None, end_date=None, upstream=False, downstream=False)[source]¶ Clears the state of task instances associated with the task, following the parameters specified.
-
dag
¶ Returns the Operator’s DAG if set, otherwise raises an error
-
deps
¶ Returns the list of dependencies for the operator. These differ from execution context dependencies in that they are specific to tasks and can be extended/overriden by subclasses.
-
detect_downstream_cycle
(task=None)[source]¶ When invoked, this routine will raise an exception if a cycle is detected downstream from self. It is invoked when tasks are added to the DAG to detect cycles.
-
downstream_list
¶ @property: list of tasks directly downstream
-
execute
(context)[source]¶ This is the main method to derive when creating an operator. Context is the same dictionary used as when rendering jinja templates.
Refer to get_template_context for more context.
-
get_direct_relatives
(upstream=False)[source]¶ Get the direct relatives to the current task, upstream or downstream.
-
get_flat_relatives
(upstream=False, l=None)[source]¶ Get a flat list of relatives, either upstream or downstream.
-
get_task_instances
(session, start_date=None, end_date=None)[source]¶ Get a set of task instance related to this task for a specific date range.
-
on_kill
()[source]¶ Override this method to cleanup subprocesses when a task instance gets killed. Any use of the threading, subprocess or multiprocessing module within an operator needs to be cleaned up or it will leave ghost processes behind.
-
post_execute
(context)[source]¶ This is triggered right after self.execute, it’s mostly a hook for people deriving operators.
-
pre_execute
(context)[source]¶ This is triggered right before self.execute, it’s mostly a hook for people deriving operators.
-
prepare_template
()[source]¶ Hook that is triggered after the templated fields get replaced by their content. If you need your operator to alter the content of the file before the template is rendered, it should override this method to do so.
-
render_template
(attr, content, context)[source]¶ Renders a template either from a file or directly in a field, and returns the rendered result.
-
render_template_from_field
(attr, content, context, jinja_env)[source]¶ Renders a template from a field. If the field is a string, it will simply render the string and return the result. If it is a collection or nested set of collections, it will traverse the structure and render all strings in it.
-
run
(start_date=None, end_date=None, ignore_first_depends_on_past=False, ignore_ti_state=False, mark_success=False)[source]¶ Run a set of task instances for a date range.
-
schedule_interval
¶ The schedule interval of the DAG always wins over individual tasks so that tasks within a DAG always line up. The task still needs a schedule_interval as it may not be attached to a DAG.
-
set_downstream
(task_or_task_list)[source]¶ Set a task, or a task task to be directly downstream from the current task.
-
set_upstream
(task_or_task_list)[source]¶ Set a task, or a task task to be directly upstream from the current task.
-
upstream_list
¶ @property: list of tasks directly upstream
BaseSensorOperator¶
All sensors are derived from BaseSensorOperator
. All sensors inherit
the timeout
and poke_interval
on top of the BaseOperator
attributes.
Core Operators¶
Operators¶
-
class
airflow.operators.bash_operator.
BashOperator
(bash_command, xcom_push=False, env=None, output_encoding=’utf-8’, *args, **kwargs)[source]¶ Bases:
airflow.models.BaseOperator
Execute a Bash script, command or set of commands.
Parameters: - bash_command (string) – The command, set of commands or reference to a bash script (must be ‘.sh’) to be executed.
- xcom_push (bool) – If xcom_push is True, the last line written to stdout will also be pushed to an XCom when the bash command completes.
- env (dict) – If env is not None, it must be a mapping that defines the environment variables for the new process; these are used instead of inheriting the current process environment, which is the default behavior. (templated)
-
class
airflow.operators.python_operator.
BranchPythonOperator
(python_callable, op_args=None, op_kwargs=None, provide_context=False, templates_dict=None, templates_exts=None, *args, **kwargs)[source]¶ Bases:
airflow.operators.python_operator.PythonOperator
Allows a workflow to “branch” or follow a single path following the execution of this task.
It derives the PythonOperator and expects a Python function that returns the task_id to follow. The task_id returned should point to a task directly downstream from {self}. All other “branches” or directly downstream tasks are marked with a state of
skipped
so that these paths can’t move forward. Theskipped
states are propageted downstream to allow for the DAG state to fill up and the DAG run’s state to be inferred.Note that using tasks with
depends_on_past=True
downstream fromBranchPythonOperator
is logically unsound asskipped
status will invariably lead to block tasks that depend on their past successes.skipped
states propagates where all directly upstream tasks areskipped
.
-
class
airflow.operators.check_operator.
CheckOperator
(sql, conn_id=None, *args, **kwargs)[source]¶ Bases:
airflow.models.BaseOperator
Performs checks against a db. The
CheckOperator
expects a sql query that will return a single row. Each value on that first row is evaluated using pythonbool
casting. If any of the values returnFalse
the check is failed and errors out.Note that Python bool casting evals the following as
False
:False
0
- Empty string (
""
) - Empty list (
[]
) - Empty dictionary or set (
{}
)
Given a query like
SELECT COUNT(*) FROM foo
, it will fail only if the count== 0
. You can craft much more complex query that could, for instance, check that the table has the same number of rows as the source table upstream, or that the count of today’s partition is greater than yesterday’s partition, or that a set of metrics are less than 3 standard deviation for the 7 day average.This operator can be used as a data quality check in your pipeline, and depending on where you put it in your DAG, you have the choice to stop the critical path, preventing from publishing dubious data, or on the side and receive email alerts without stopping the progress of the DAG.
Note that this is an abstract class and get_db_hook needs to be defined. Whereas a get_db_hook is hook that gets a single record from an external source.
Parameters: sql (string) – the sql to be executed
-
class
airflow.operators.dummy_operator.
DummyOperator
(*args, **kwargs)[source]¶ Bases:
airflow.models.BaseOperator
Operator that does literally nothing. It can be used to group tasks in a DAG.
-
class
airflow.operators.email_operator.
EmailOperator
(to, subject, html_content, files=None, cc=None, bcc=None, mime_subtype=’mixed’, *args, **kwargs)[source]¶ Bases:
airflow.models.BaseOperator
Sends an email.
Parameters: - to (list or string (comma or semicolon delimited)) – list of emails to send the email to
- subject (string) – subject line for the email (templated)
- html_content (string) – content of the email (templated), html markup is allowed
- files (list) – file names to attach in email
- cc (list or string (comma or semicolon delimited)) – list of recipients to be added in CC field
- bcc (list or string (comma or semicolon delimited)) – list of recipients to be added in BCC field
-
class
airflow.operators.generic_transfer.
GenericTransfer
(sql, destination_table, source_conn_id, destination_conn_id, preoperator=None, *args, **kwargs)[source]¶ Bases:
airflow.models.BaseOperator
Moves data from a connection to another, assuming that they both provide the required methods in their respective hooks. The source hook needs to expose a get_records method, and the destination a insert_rows method.
This is mean to be used on small-ish datasets that fit in memory.
Parameters: - sql (str) – SQL query to execute against the source database
- destination_table (str) – target table
- source_conn_id (str) – source connection
- destination_conn_id (str) – source connection
- preoperator (str or list of str) – sql statement or list of statements to be executed prior to loading the data
-
class
airflow.operators.check_operator.
IntervalCheckOperator
(table, metrics_thresholds, date_filter_column=’ds’, days_back=-7, conn_id=None, *args, **kwargs)[source]¶ Bases:
airflow.models.BaseOperator
Checks that the values of metrics given as SQL expressions are within a certain tolerance of the ones from days_back before.
Note that this is an abstract class and get_db_hook needs to be defined. Whereas a get_db_hook is hook that gets a single record from an external source.
Parameters: - table (str) – the table name
- days_back (int) – number of days between ds and the ds we want to check against. Defaults to 7 days
- metrics_threshold (dict) – a dictionary of ratios indexed by metrics
-
class
airflow.operators.latest_only_operator.
LatestOnlyOperator
(task_id, owner=’Airflow’, email=None, email_on_retry=True, email_on_failure=True, retries=0, retry_delay=datetime.timedelta(0, 300), retry_exponential_backoff=False, max_retry_delay=None, start_date=None, end_date=None, schedule_interval=None, depends_on_past=False, wait_for_downstream=False, dag=None, params=None, default_args=None, adhoc=False, priority_weight=1, queue=’default’, pool=None, sla=None, execution_timeout=None, on_failure_callback=None, on_success_callback=None, on_retry_callback=None, trigger_rule=’all_success’, resources=None, run_as_user=None, *args, **kwargs)[source]¶ Bases:
airflow.models.BaseOperator
Allows a workflow to skip tasks that are not running during the most recent schedule interval.
If the task is run outside of the latest schedule interval, all directly downstream tasks will be skipped.
-
class
airflow.operators.mssql_operator.
MsSqlOperator
(sql, mssql_conn_id=’mssql_default’, parameters=None, autocommit=False, *args, **kwargs)[source]¶ Bases:
airflow.models.BaseOperator
Executes sql code in a specific Microsoft SQL database :param mssql_conn_id: reference to a specific mssql database :type mssql_conn_id: string :param sql: the sql code to be executed :type sql: string or string pointing to a template file. File must have a ‘.sql’ extensions.
-
class
airflow.operators.pig_operator.
PigOperator
(pig, pig_cli_conn_id=’pig_cli_default’, pigparams_jinja_translate=False, *args, **kwargs)[source]¶ Bases:
airflow.models.BaseOperator
Executes pig script.
Parameters: - pig (string) – the pig latin script to be executed
- pig_cli_conn_id (string) – reference to the Hive database
- pigparams_jinja_translate (boolean) – when True, pig params-type templating
${var} gets translated into jinja-type templating {{ var }}. Note that
you may want to use this along with the
DAG(user_defined_macros=myargs)
parameter. View the DAG object documentation for more details.
-
class
airflow.operators.python_operator.
PythonOperator
(python_callable, op_args=None, op_kwargs=None, provide_context=False, templates_dict=None, templates_exts=None, *args, **kwargs)[source]¶ Bases:
airflow.models.BaseOperator
Executes a Python callable
Parameters: - python_callable (python callable) – A reference to an object that is callable
- op_kwargs (dict) – a dictionary of keyword arguments that will get unpacked in your function
- op_args (list) – a list of positional arguments that will get unpacked when calling your callable
- provide_context (bool) – if set to true, Airflow will pass a set of keyword arguments that can be used in your function. This set of kwargs correspond exactly to what you can use in your jinja templates. For this to work, you need to define **kwargs in your function header.
- templates_dict (dict of str) – a dictionary where the values are templates that
will get templated by the Airflow engine sometime between
__init__
andexecute
takes place and are made available in your callable’s context after the template has been applied - templates_exts – a list of file extensions to resolve while
processing templated fields, for examples
['.sql', '.hql']
-
class
airflow.operators.python_operator.
ShortCircuitOperator
(python_callable, op_args=None, op_kwargs=None, provide_context=False, templates_dict=None, templates_exts=None, *args, **kwargs)[source]¶ Bases:
airflow.operators.python_operator.PythonOperator
Allows a workflow to continue only if a condition is met. Otherwise, the workflow “short-circuits” and downstream tasks are skipped.
The ShortCircuitOperator is derived from the PythonOperator. It evaluates a condition and short-circuits the workflow if the condition is False. Any downstream tasks are marked with a state of “skipped”. If the condition is True, downstream tasks proceed as normal.
The condition is determined by the result of python_callable.
-
class
airflow.operators.http_operator.
SimpleHttpOperator
(endpoint, method=’POST’, data=None, headers=None, response_check=None, extra_options=None, xcom_push=False, http_conn_id=’http_default’, *args, **kwargs)[source]¶ Bases:
airflow.models.BaseOperator
Calls an endpoint on an HTTP system to execute an action
Parameters: - http_conn_id (string) – The connection to run the sensor against
- endpoint (string) – The relative part of the full url
- method (string) – The HTTP method to use, default = “POST”
- data (For POST/PUT, depends on the content-type parameter, for GET a dictionary of key/value string pairs) – The data to pass. POST-data in POST/PUT and params in the URL for a GET request.
- headers (a dictionary of string key/value pairs) – The HTTP headers to be added to the GET request
- response_check (A lambda or defined function.) – A check against the ‘requests’ response object. Returns True for ‘pass’ and False otherwise.
- extra_options (A dictionary of options, where key is string and value depends on the option that's being modified.) – Extra options for the ‘requests’ library, see the ‘requests’ documentation (options to modify timeout, ssl, etc.)
-
class
airflow.operators.sqlite_operator.
SqliteOperator
(sql, sqlite_conn_id=’sqlite_default’, parameters=None, *args, **kwargs)[source]¶ Bases:
airflow.models.BaseOperator
Executes sql code in a specific Sqlite database
Parameters: - sqlite_conn_id (string) – reference to a specific sqlite database
- sql (string or string pointing to a template file. File must have a '.sql' extensions.) – the sql code to be executed
-
class
airflow.operators.subdag_operator.
SubDagOperator
(subdag, executor=<airflow.executors.sequential_executor.SequentialExecutor object>, *args, **kwargs)[source]¶ Bases:
airflow.models.BaseOperator
-
class
airflow.operators.dagrun_operator.
TriggerDagRunOperator
(trigger_dag_id, python_callable, *args, **kwargs)[source]¶ Bases:
airflow.models.BaseOperator
Triggers a DAG run for a specified
dag_id
if a criteria is metParameters: - trigger_dag_id (str) – the dag_id to trigger
- python_callable (python callable) – a reference to a python function that will be
called while passing it the
context
object and a placeholder objectobj
for your callable to fill and return if you want a DagRun created. Thisobj
object contains arun_id
andpayload
attribute that you can modify in your function. Therun_id
should be a unique identifier for that DAG run, and the payload has to be a picklable object that will be made available to your tasks while executing that DAG run. Your function header should look likedef foo(context, dag_run_obj):
-
class
airflow.operators.check_operator.
ValueCheckOperator
(sql, pass_value, tolerance=None, conn_id=None, *args, **kwargs)[source]¶ Bases:
airflow.models.BaseOperator
Performs a simple value check using sql code.
Note that this is an abstract class and get_db_hook needs to be defined. Whereas a get_db_hook is hook that gets a single record from an external source.
Parameters: sql (string) – the sql to be executed
Sensors¶
Community-contributed Operators¶
Operators¶
-
class
airflow.contrib.operators.hipchat_operator.
HipChatAPIOperator
(token, base_url=’https://api.hipchat.com/v2’, *args, **kwargs)[source]¶ Bases:
airflow.models.BaseOperator
Base HipChat Operator. All derived HipChat operators reference from HipChat’s official REST API documentation at https://www.hipchat.com/docs/apiv2. Before using any HipChat API operators you need to get an authentication token at https://www.hipchat.com/docs/apiv2/auth. In the future additional HipChat operators will be derived from this class as well. :param token: HipChat REST API authentication token :type token: str :param base_url: HipChat REST API base url. :type base_url: str
-
class
airflow.contrib.operators.hipchat_operator.
HipChatAPISendRoomNotificationOperator
(room_id, message, *args, **kwargs)[source]¶ Bases:
airflow.contrib.operators.hipchat_operator.HipChatAPIOperator
Send notification to a specific HipChat room. More info: https://www.hipchat.com/docs/apiv2/method/send_room_notification :param room_id: Room in which to send notification on HipChat :type room_id: str :param message: The message body :type message: str :param frm: Label to be shown in addition to sender’s name :type frm: str :param message_format: How the notification is rendered: html or text :type message_format: str :param color: Background color of the msg: yellow, green, red, purple, gray, or random :type color: str :param attach_to: The message id to attach this notification to :type attach_to: str :param notify: Whether this message should trigger a user notification :type notify: bool :param card: HipChat-defined card object :type card: dict
-
class
airflow.contrib.operators.spark_sql_operator.
SparkSqlOperator
(sql, conf=None, conn_id=’spark_sql_default’, executor_cores=None, executor_memory=None, keytab=None, master=’yarn’, name=’default-name’, num_executors=None, yarn_queue=’default’, *args, **kwargs)[source]¶ Bases:
airflow.models.BaseOperator
Execute Spark SQL query :param sql: The SQL query to execute :type sql: str :param conf: arbitrary Spark configuration property :type conf: str (format: PROP=VALUE) :param conn_id: connection_id string :type conn_id: str :param executor_cores: Number of cores per executor :type executor_cores: int :param executor_memory: Memory per executor (e.g. 1000M, 2G) (Default: 1G) :type executor_memory: str :param keytab: Full path to the file that contains the keytab :type keytab: str :param master: spark://host:port, mesos://host:port, yarn, or local :type master: str :param name: Name of the job :type name: str :param num_executors: Number of executors to launch :type num_executors: int :param verbose: Whether to pass the verbose flag to spark-sql :type verbose: bool :param yarn_queue: The YARN queue to submit to (Default: “default”) :type yarn_queue: str
-
class
airflow.contrib.operators.spark_submit_operator.
SparkSubmitOperator
(application=’‘, conf=None, conn_id=’spark_default’, files=None, py_files=None, jars=None, executor_cores=None, executor_memory=None, keytab=None, principal=None, name=’airflow-spark’, num_executors=None, verbose=False, *args, **kwargs)[source]¶ Bases:
airflow.models.BaseOperator
This hook is a wrapper around the spark-submit binary to kick off a spark-submit job. It requires that the “spark-submit” binary is in the PATH. :param application: The application that submitted as a job, either jar or py file. :type application: str :param conf: Arbitrary Spark configuration properties :type conf: dict :param conn_id: The connection id as configured in Airflow administration. When an
invalid connection_id is supplied, it will default to yarn.Parameters: - files (str) – Upload additional files to the container running the job, separated by a comma. For example hive-site.xml.
- py_files (str) – Additional python files used by the job, can be .zip, .egg or .py.
- jars (str) – Submit additional jars to upload and place them in executor classpath.
- executor_cores (int) – Number of cores per executor (Default: 2)
- executor_memory (str) – Memory per executor (e.g. 1000M, 2G) (Default: 1G)
- keytab (str) – Full path to the file that contains the keytab
- principal (str) – The name of the kerberos principal used for keytab
- name (str) – Name of the job (default airflow-spark)
- num_executors (int) – Number of executors to launch
- verbose (bool) – Whether to pass the verbose flag to spark-submit process for debugging
-
class
airflow.contrib.operators.sqoop_operator.
SqoopOperator
(conn_id=’sqoop_default’, type_cmd=’import’, table=’‘, target_dir=None, append=None, type=None, columns=None, num_mappers=‘1’, split_by=None, where=None, export_dir=None, *args, **kwargs)[source]¶ Bases:
airflow.models.BaseOperator
execute sqoop job
-
class
airflow.contrib.operators.vertica_operator.
VerticaOperator
(sql, vertica_conn_id=’vertica_default’, *args, **kwargs)[source]¶ Bases:
airflow.models.BaseOperator
Executes sql code in a specific Vertica database
Parameters: - vertica_conn_id (string) – reference to a specific Vertica database
- sql (Can receive a str representing a sql statement, a list of str (sql statements), or reference to a template file. Template reference are recognized by str ending in '.sql') – the sql code to be executed
-
class
airflow.contrib.operators.spark_submit_operator.
SparkSubmitOperator
(application=’‘, conf=None, conn_id=’spark_default’, files=None, py_files=None, jars=None, executor_cores=None, executor_memory=None, keytab=None, principal=None, name=’airflow-spark’, num_executors=None, verbose=False, *args, **kwargs)[source] Bases:
airflow.models.BaseOperator
This hook is a wrapper around the spark-submit binary to kick off a spark-submit job. It requires that the “spark-submit” binary is in the PATH. :param application: The application that submitted as a job, either jar or py file. :type application: str :param conf: Arbitrary Spark configuration properties :type conf: dict :param conn_id: The connection id as configured in Airflow administration. When an
invalid connection_id is supplied, it will default to yarn.Parameters: - files (str) – Upload additional files to the container running the job, separated by a comma. For example hive-site.xml.
- py_files (str) – Additional python files used by the job, can be .zip, .egg or .py.
- jars (str) – Submit additional jars to upload and place them in executor classpath.
- executor_cores (int) – Number of cores per executor (Default: 2)
- executor_memory (str) – Memory per executor (e.g. 1000M, 2G) (Default: 1G)
- keytab (str) – Full path to the file that contains the keytab
- principal (str) – The name of the kerberos principal used for keytab
- name (str) – Name of the job (default airflow-spark)
- num_executors (int) – Number of executors to launch
- verbose (bool) – Whether to pass the verbose flag to spark-submit process for debugging
-
execute
(context)[source] Call the SparkSubmitHook to run the provided spark job
Sensors¶
-
class
airflow.contrib.sensors.emr_base_sensor.
EmrBaseSensor
(aws_conn_id=’aws_default’, *args, **kwargs)[source]¶ Bases:
airflow.operators.sensors.BaseSensorOperator
Contains general sensor behavior for EMR. Subclasses should implement get_emr_response() and state_from_response() methods. Subclasses should also implment NON_TERMINAL_STATES and FAILED_STATE constants.
Macros¶
Here’s a list of variables and macros that can be used in templates
Default Variables¶
The Airflow engine passes a few variables by default that are accessible in all templates
Variable | Description |
---|---|
{{ ds }} |
the execution date as YYYY-MM-DD |
{{ ds_nodash }} |
the execution date as YYYYMMDD |
{{ yesterday_ds }} |
yesterday’s date as YYYY-MM-DD |
{{ yesterday_ds_nodash }} |
yesterday’s date as YYYYMMDD |
{{ tomorrow_ds }} |
tomorrow’s date as YYYY-MM-DD |
{{ tomorrow_ds_nodash }} |
tomorrow’s date as YYYYMMDD |
{{ ts }} |
same as execution_date.isoformat() |
{{ ts_nodash }} |
same as ts without - and : |
{{ execution_date }} |
the execution_date, (datetime.datetime) |
{{ prev_execution_date }} |
the previous execution date (if available) (datetime.datetime) |
{{ next_execution_date }} |
the next execution date (datetime.datetime) |
{{ dag }} |
the DAG object |
{{ task }} |
the Task object |
{{ macros }} |
a reference to the macros package, described below |
{{ task_instance }} |
the task_instance object |
{{ end_date }} |
same as {{ ds }} |
{{ latest_date }} |
same as {{ ds }} |
{{ ti }} |
same as {{ task_instance }} |
{{ params }} |
a reference to the user-defined params dictionary |
{{ var.value.my_var }} |
global defined variables represented as a dictionary |
{{ var.json.my_var.path }} |
global defined variables represented as a dictionary with deserialized JSON object, append the path to the key within the JSON object |
{{ task_instance_key_str }} |
a unique, human-readable key to the task instance
formatted {dag_id}_{task_id}_{ds} |
{{ conf }} |
the full configuration object located at
airflow.configuration.conf which
represents the content of your
airflow.cfg |
{{ run_id }} |
the run_id of the current DAG run |
{{ dag_run }} |
a reference to the DagRun object |
{{ test_mode }} |
whether the task instance was called using the CLI’s test subcommand |
Note that you can access the object’s attributes and methods with simple
dot notation. Here are some examples of what is possible:
{{ task.owner }}
, {{ task.task_id }}
, {{ ti.hostname }}
, …
Refer to the models documentation for more information on the objects’
attributes and methods.
The var
template variable allows you to access variables defined in Airflow’s
UI. You can access them as either plain-text or JSON. If you use JSON, you are
also able to walk nested structures, such as dictionaries like:
{{ var.json.my_dict_var.key1 }}
Macros¶
Macros are a way to expose objects to your templates and live under the
macros
namespace in your templates.
A few commonly used libraries and methods are made available.
Variable | Description |
---|---|
macros.datetime |
The standard lib’s datetime.datetime |
macros.timedelta |
The standard lib’s datetime.timedelta |
macros.dateutil |
A reference to the dateutil package |
macros.time |
The standard lib’s time |
macros.uuid |
The standard lib’s uuid |
macros.random |
The standard lib’s random |
Some airflow specific macros are also defined:
-
airflow.macros.
ds_add
(ds, days)[source]¶ Add or subtract days from a YYYY-MM-DD
Parameters: - ds (str) – anchor date in
YYYY-MM-DD
format to add to - days (int) – number of days to add to the ds, you can use negative values
>>> ds_add('2015-01-01', 5) '2015-01-06' >>> ds_add('2015-01-06', -5) '2015-01-01'
- ds (str) – anchor date in
-
airflow.macros.
ds_format
(ds, input_format, output_format)[source]¶ Takes an input string and outputs another string as specified in the output format
Parameters: - ds (str) – input string which contains a date
- input_format (str) – input string format. E.g. %Y-%m-%d
- output_format (str) – output string format E.g. %Y-%m-%d
>>> ds_format('2015-01-01', "%Y-%m-%d", "%m-%d-%y") '01-01-15' >>> ds_format('1/5/2015', "%m/%d/%Y", "%Y-%m-%d") '2015-01-05'
-
airflow.macros.
random
() → x in the interval [0, 1).¶
-
airflow.macros.hive.
closest_ds_partition
(table, ds, before=True, schema=’default’, metastore_conn_id=’metastore_default’)[source]¶ This function finds the date in a list closest to the target date. An optional parameter can be given to get the closest before or after.
Parameters: - table (str) – A hive table name
- ds (datetime.date list) – A datestamp
%Y-%m-%d
e.g.yyyy-mm-dd
- before (bool or None) – closest before (True), after (False) or either side of ds
Returns: The closest date
Return type: str or None
>>> tbl = 'airflow.static_babynames_partitioned' >>> closest_ds_partition(tbl, '2015-01-02') '2015-01-01'
-
airflow.macros.hive.
max_partition
(table, schema=’default’, field=None, filter=None, metastore_conn_id=’metastore_default’)[source]¶ Gets the max partition for a table.
Parameters: - schema (string) – The hive schema the table lives in
- table (string) – The hive table you are interested in, supports the dot notation as in “my_database.my_table”, if a dot is found, the schema param is disregarded
- hive_conn_id (string) – The hive connection you are interested in. If your default is set you don’t need to use this parameter.
- filter (string) – filter on a subset of partition as in sub_part=’specific_value’
- field – the field to get the max value from. If there’s only one partition field, this will be inferred
>>> max_partition('airflow.static_babynames_partitioned') '2015-01-01'
Models¶
Models are built on top of the SQLAlchemy ORM Base class, and instances are persisted in the database.
-
class
airflow.models.
BaseOperator
(task_id, owner=’Airflow’, email=None, email_on_retry=True, email_on_failure=True, retries=0, retry_delay=datetime.timedelta(0, 300), retry_exponential_backoff=False, max_retry_delay=None, start_date=None, end_date=None, schedule_interval=None, depends_on_past=False, wait_for_downstream=False, dag=None, params=None, default_args=None, adhoc=False, priority_weight=1, queue=’default’, pool=None, sla=None, execution_timeout=None, on_failure_callback=None, on_success_callback=None, on_retry_callback=None, trigger_rule=’all_success’, resources=None, run_as_user=None, *args, **kwargs)[source] Bases:
object
Abstract base class for all operators. Since operators create objects that become node in the dag, BaseOperator contains many recursive methods for dag crawling behavior. To derive this class, you are expected to override the constructor as well as the ‘execute’ method.
Operators derived from this task should perform or trigger certain tasks synchronously (wait for completion). Example of operators could be an operator the runs a Pig job (PigOperator), a sensor operator that waits for a partition to land in Hive (HiveSensorOperator), or one that moves data from Hive to MySQL (Hive2MySqlOperator). Instances of these operators (tasks) target specific operations, running specific scripts, functions or data transfers.
This class is abstract and shouldn’t be instantiated. Instantiating a class derived from this one results in the creation of a task object, which ultimately becomes a node in DAG objects. Task dependencies should be set by using the set_upstream and/or set_downstream methods.
Note that this class is derived from SQLAlchemy’s Base class, which allows us to push metadata regarding tasks to the database. Deriving this classes needs to implement the polymorphic specificities documented in SQLAlchemy. This should become clear while reading the code for other operators.
Parameters: - task_id (string) – a unique, meaningful id for the task
- owner (string) – the owner of the task, using the unix username is recommended
- retries (int) – the number of retries that should be performed before failing the task
- retry_delay (timedelta) – delay between retries
- retry_exponential_backoff (bool) – allow progressive longer waits between retries by using exponential backoff algorithm on retry delay (delay will be converted into seconds)
- max_retry_delay (timedelta) – maximum delay interval between retries
- start_date (datetime) – The
start_date
for the task, determines theexecution_date
for the first task instance. The best practice is to have the start_date rounded to your DAG’sschedule_interval
. Daily jobs have their start_date some day at 00:00:00, hourly jobs have their start_date at 00:00 of a specific hour. Note that Airflow simply looks at the latestexecution_date
and adds theschedule_interval
to determine the nextexecution_date
. It is also very important to note that different tasks’ dependencies need to line up in time. If task A depends on task B and their start_date are offset in a way that their execution_date don’t line up, A’s dependencies will never be met. If you are looking to delay a task, for example running a daily task at 2AM, look into theTimeSensor
andTimeDeltaSensor
. We advise against using dynamicstart_date
and recommend using fixed ones. Read the FAQ entry about start_date for more information. - end_date (datetime) – if specified, the scheduler won’t go beyond this date
- depends_on_past (bool) – when set to true, task instances will run sequentially while relying on the previous task’s schedule to succeed. The task instance for the start_date is allowed to run.
- wait_for_downstream (bool) – when set to true, an instance of task X will wait for tasks immediately downstream of the previous instance of task X to finish successfully before it runs. This is useful if the different instances of a task X alter the same asset, and this asset is used by tasks downstream of task X. Note that depends_on_past is forced to True wherever wait_for_downstream is used.
- queue (str) – which queue to target when running this job. Not all executors implement queue management, the CeleryExecutor does support targeting specific queues.
- dag (DAG) – a reference to the dag the task is attached to (if any)
- priority_weight (int) – priority weight of this task against other task. This allows the executor to trigger higher priority tasks before others when things get backed up.
- pool (str) – the slot pool this task should run in, slot pools are a way to limit concurrency for certain tasks
- sla (datetime.timedelta) – time by which the job is expected to succeed. Note that
this represents the
timedelta
after the period is closed. For example if you set an SLA of 1 hour, the scheduler would send dan email soon after 1:00AM on the2016-01-02
if the2016-01-01
instance has not succeeded yet. The scheduler pays special attention for jobs with an SLA and sends alert emails for sla misses. SLA misses are also recorded in the database for future reference. All tasks that share the same SLA time get bundled in a single email, sent soon after that time. SLA notification are sent once and only once for each task instance. - execution_timeout (datetime.timedelta) – max time allowed for the execution of this task instance, if it goes beyond it will raise and fail.
- on_failure_callback (callable) – a function to be called when a task instance of this task fails. a context dictionary is passed as a single parameter to this function. Context contains references to related objects to the task instance and is documented under the macros section of the API.
- on_retry_callback – much like the
on_failure_callback
excepts that it is executed when retries occur. - on_success_callback (callable) – much like the
on_failure_callback
excepts that it is executed when the task succeeds. - trigger_rule (str) – defines the rule by which dependencies are applied
for the task to get triggered. Options are:
{ all_success | all_failed | all_done | one_success | one_failed | dummy}
default isall_success
. Options can be set as string or using the constants defined in the static classairflow.utils.TriggerRule
- resources (dict) – A map of resource parameter names (the argument names of the Resources constructor) to their values.
- run_as_user (str) – unix username to impersonate while running the task
-
clear
(start_date=None, end_date=None, upstream=False, downstream=False)[source] Clears the state of task instances associated with the task, following the parameters specified.
-
dag
Returns the Operator’s DAG if set, otherwise raises an error
-
deps
Returns the list of dependencies for the operator. These differ from execution context dependencies in that they are specific to tasks and can be extended/overriden by subclasses.
-
detect_downstream_cycle
(task=None)[source] When invoked, this routine will raise an exception if a cycle is detected downstream from self. It is invoked when tasks are added to the DAG to detect cycles.
-
downstream_list
@property: list of tasks directly downstream
-
execute
(context)[source] This is the main method to derive when creating an operator. Context is the same dictionary used as when rendering jinja templates.
Refer to get_template_context for more context.
-
get_direct_relatives
(upstream=False)[source] Get the direct relatives to the current task, upstream or downstream.
-
get_flat_relatives
(upstream=False, l=None)[source] Get a flat list of relatives, either upstream or downstream.
-
get_task_instances
(session, start_date=None, end_date=None)[source] Get a set of task instance related to this task for a specific date range.
-
has_dag
()[source] Returns True if the Operator has been assigned to a DAG.
-
on_kill
()[source] Override this method to cleanup subprocesses when a task instance gets killed. Any use of the threading, subprocess or multiprocessing module within an operator needs to be cleaned up or it will leave ghost processes behind.
-
post_execute
(context)[source] This is triggered right after self.execute, it’s mostly a hook for people deriving operators.
-
pre_execute
(context)[source] This is triggered right before self.execute, it’s mostly a hook for people deriving operators.
-
prepare_template
()[source] Hook that is triggered after the templated fields get replaced by their content. If you need your operator to alter the content of the file before the template is rendered, it should override this method to do so.
-
render_template
(attr, content, context)[source] Renders a template either from a file or directly in a field, and returns the rendered result.
-
render_template_from_field
(attr, content, context, jinja_env)[source] Renders a template from a field. If the field is a string, it will simply render the string and return the result. If it is a collection or nested set of collections, it will traverse the structure and render all strings in it.
-
run
(start_date=None, end_date=None, ignore_first_depends_on_past=False, ignore_ti_state=False, mark_success=False)[source] Run a set of task instances for a date range.
-
schedule_interval
The schedule interval of the DAG always wins over individual tasks so that tasks within a DAG always line up. The task still needs a schedule_interval as it may not be attached to a DAG.
-
set_downstream
(task_or_task_list)[source] Set a task, or a task task to be directly downstream from the current task.
-
set_upstream
(task_or_task_list)[source] Set a task, or a task task to be directly upstream from the current task.
-
upstream_list
@property: list of tasks directly upstream
-
xcom_pull
(context, task_ids, dag_id=None, key=’return_value’, include_prior_dates=None)[source] See TaskInstance.xcom_pull()
-
xcom_push
(context, key, value, execution_date=None)[source] See TaskInstance.xcom_push()
-
class
airflow.models.
Connection
(conn_id=None, conn_type=None, host=None, login=None, password=None, schema=None, port=None, extra=None, uri=None)[source]¶ Bases:
sqlalchemy.ext.declarative.api.Base
Placeholder to store information about different database instances connection information. The idea here is that scripts use references to database instances (conn_id) instead of hard coding hostname, logins and passwords when using operators or hooks.
-
extra_dejson
¶ Returns the extra property by deserializing json.
-
-
class
airflow.models.
DAG
(dag_id, description=’‘, schedule_interval=datetime.timedelta(1), start_date=None, end_date=None, full_filepath=None, template_searchpath=None, user_defined_macros=None, default_args=None, concurrency=16, max_active_runs=16, dagrun_timeout=None, sla_miss_callback=None, orientation=’LR’, catchup=True, params=None)[source]¶ Bases:
airflow.dag.base_dag.BaseDag
,airflow.utils.logging.LoggingMixin
A dag (directed acyclic graph) is a collection of tasks with directional dependencies. A dag also has a schedule, a start end an end date (optional). For each schedule, (say daily or hourly), the DAG needs to run each individual tasks as their dependencies are met. Certain tasks have the property of depending on their own past, meaning that they can’t run until their previous schedule (and upstream tasks) are completed.
DAGs essentially act as namespaces for tasks. A task_id can only be added once to a DAG.
Parameters: - dag_id (string) – The id of the DAG
- description (string) – The description for the DAG to e.g. be shown on the webserver
- schedule_interval (datetime.timedelta or dateutil.relativedelta.relativedelta or str that acts as a cron expression) – Defines how often that DAG runs, this timedelta object gets added to your latest task instance’s execution_date to figure out the next schedule
- start_date (datetime.datetime) – The timestamp from which the scheduler will attempt to backfill
- end_date (datetime.datetime) – A date beyond which your DAG won’t run, leave to None for open ended scheduling
- template_searchpath (string or list of stings) – This list of folders (non relative) defines where jinja will look for your templates. Order matters. Note that jinja/airflow includes the path of your DAG file by default
- user_defined_macros (dict) – a dictionary of macros that will be exposed
in your jinja templates. For example, passing
dict(foo='bar')
to this argument allows you to{{ foo }}
in all jinja templates related to this DAG. Note that you can pass any type of object here. - default_args (dict) – A dictionary of default parameters to be used as constructor keyword parameters when initialising operators. Note that operators have the same hook, and precede those defined here, meaning that if your dict contains ‘depends_on_past’: True here and ‘depends_on_past’: False in the operator’s call default_args, the actual value will be False.
- params (dict) – a dictionary of DAG level parameters that are made accessible in templates, namespaced under params. These params can be overridden at the task level.
- concurrency (int) – the number of task instances allowed to run concurrently
- max_active_runs (int) – maximum number of active DAG runs, beyond this number of DAG runs in a running state, the scheduler won’t create new active DAG runs
- dagrun_timeout (datetime.timedelta) – specify how long a DagRun should be up before timing out / failing, so that new DagRuns can be created
- sla_miss_callback (types.FunctionType) – specify a function to call when reporting SLA timeouts.
- orientation (string) – Specify DAG orientation in graph view (LR, TB, RL, BT)
- catchup – Perform scheduler catchup (or only run latest)? Defaults to True
“type catchup: bool”
-
add_tasks
(tasks)[source]¶ Add a list of tasks to the DAG
Parameters: task (list of tasks) – a lit of tasks you want to add
-
clear
(start_date=None, end_date=None, only_failed=False, only_running=False, confirm_prompt=False, include_subdags=True, reset_dag_runs=True, dry_run=False)[source]¶ Clears a set of task instances associated with the current dag for a specified date range.
-
concurrency_reached
¶ Returns a boolean indicating whether the concurrency limit for this DAG has been reached
-
crawl_for_tasks
(objects)[source]¶ Typically called at the end of a script by passing globals() as a parameter. This allows to not explicitly add every single task to the dag explicitly.
-
create_dagrun
(run_id, state, execution_date=None, start_date=None, external_trigger=False, conf=None, session=None)[source]¶ Creates a dag run from this dag including the tasks associated with this dag. Returns the dag run.
Parameters: - run_id (string) – defines the the run id for this dag run
- execution_date (datetime) – the execution date of this dag run
- state (State) – the state of the dag run
- start_date (datetime) – the date this dag run should be evaluated
- external_trigger (bool) – whether this dag run is externally triggered
- session (Session) – database session
-
static
deactivate_stale_dags
(expiration_date, session=None)[source]¶ Deactivate any DAGs that were last touched by the scheduler before the expiration date. These DAGs were likely deleted.
Parameters: expiration_date – set inactive DAGs that were touched before this time :type expiration_date: datetime :return: None
-
static
deactivate_unknown_dags
(active_dag_ids, session=None)[source]¶ Given a list of known DAGs, deactivate any other DAGs that are marked as active in the ORM
Parameters: active_dag_ids (list[unicode]) – list of DAG IDs that are active Returns: None
-
filepath
¶ File location of where the dag object is instantiated
-
folder
¶ Folder location of where the dag object is instantiated
-
get_active_runs
(session=None)[source]¶ Returns a list of “running” tasks :param session: :return: List of execution dates
-
get_dagrun
(execution_date, session=None)[source]¶ Returns the dag run for a given execution date if it exists, otherwise none. :param execution_date: The execution date of the DagRun to find. :param session: :return: The DagRun if found, otherwise None.
-
get_last_dagrun
(session=None, include_externally_triggered=False)[source]¶ Returns the last dag run for this dag, None if there was none. Last dag run can be any type of run eg. scheduled or backfilled. Overriden DagRuns are ignored
-
get_template_env
()[source]¶ Returns a jinja2 Environment while taking into account the DAGs template_searchpath and user_defined_macros
-
is_paused
¶ Returns a boolean indicating whether this DAG is paused
-
latest_execution_date
¶ Returns the latest date for which at least one dag run exists
-
normalize_schedule
(dttm)[source]¶ Returns dttm + interval unless dttm is first interval then it returns dttm
-
run
(start_date=None, end_date=None, mark_success=False, include_adhoc=False, local=False, executor=None, donot_pickle=False, ignore_task_deps=False, ignore_first_depends_on_past=False, pool=None)[source]¶ Runs the DAG.
-
set_dependency
(upstream_task_id, downstream_task_id)[source]¶ Simple utility method to set dependency between two tasks that already have been added to the DAG using add_task()
-
sub_dag
(task_regex, include_downstream=False, include_upstream=True)[source]¶ Returns a subset of the current dag as a deep copy of the current dag based on a regex that should match one or many tasks, and includes upstream and downstream neighbours based on the flag passed.
-
subdags
¶ Returns a list of the subdag objects associated to this DAG
-
static
sync_to_db
(dag, owner, sync_time, session=None)[source]¶ Save attributes about this DAG to the DB. Note that this method can be called for both DAGs and SubDAGs. A SubDag is actually a SubDagOperator.
Parameters: dag (DAG) – the DAG object to save to the DB :own :param sync_time: The time that the DAG should be marked as sync’ed :type sync_time: datetime :return: None
-
topological_sort
()[source]¶ Sorts tasks in topographical order, such that a task comes after any of its upstream dependencies.
Heavily inspired by: http://blog.jupo.org/2012/04/06/topological-sorting-acyclic-directed-graphs/ :returns: list of tasks in topological order
-
class
airflow.models.
DagBag
(dag_folder=None, executor=<airflow.executors.sequential_executor.SequentialExecutor object>, include_examples=True)[source]¶ Bases:
airflow.dag.base_dag.BaseDagBag
,airflow.utils.logging.LoggingMixin
A dagbag is a collection of dags, parsed out of a folder tree and has high level configuration settings, like what database to use as a backend and what executor to use to fire off tasks. This makes it easier to run distinct environments for say production and development, tests, or for different teams or security profiles. What would have been system level settings are now dagbag level so that one system can run multiple, independent settings sets.
Parameters: - dag_folder (unicode) – the folder to scan to find DAGs
- executor – the executor to use when executing task instances in this DagBag
- include_examples (bool) – whether to include the examples that ship with airflow or not
- sync_to_db (bool) – whether to sync the properties of the DAGs to the metadata DB while finding them, typically should be done by the scheduler job only
-
collect_dags
(dag_folder=None, only_if_updated=True)[source]¶ Given a file path or a folder, this method looks for python modules, imports them and adds them to the dagbag collection.
Note that if a .airflowignore file is found while processing, the directory, it will behaves much like a .gitignore does, ignoring files that match any of the regex patterns specified in the file.
-
class
airflow.models.
DagPickle
(dag)[source]¶ Bases:
sqlalchemy.ext.declarative.api.Base
Dags can originate from different places (user repos, master repo, …) and also get executed in different places (different executors). This object represents a version of a DAG and becomes a source of truth for a BackfillJob execution. A pickle is a native python serialized object, and in this case gets stored in the database for the duration of the job.
The executors pick up the DagPickle id and read the dag definition from the database.
-
class
airflow.models.
DagRun
(**kwargs)[source]¶ Bases:
sqlalchemy.ext.declarative.api.Base
DagRun describes an instance of a Dag. It can be created by the scheduler (for regular runs) or by an external trigger
-
static
find
(dag_id=None, run_id=None, execution_date=None, state=None, external_trigger=None, session=None)[source]¶ Returns a set of dag runs for the given search criteria. :param dag_id: the dag_id to find dag runs for :type dag_id: integer, list :param run_id: defines the the run id for this dag run :type run_id: string :param execution_date: the execution date :type execution_date: datetime :param state: the state of the dag run :type state: State :param external_trigger: whether this dag run is externally triggered :type external_trigger: bool :param session: database session :type session: Session
-
get_previous_scheduled_dagrun
(session=None)[source]¶ The previous, SCHEDULED DagRun, if there is one
-
static
get_run
(session, dag_id, execution_date)[source]¶ Parameters: - dag_id (unicode) – DAG ID
- execution_date (datetime) – execution date
Returns: DagRun corresponding to the given dag_id and execution date
if one exists. None otherwise. :rtype: DagRun
-
static
get_running_tasks
(session, dag_id, task_ids)[source]¶ Returns the number of tasks running in the given DAG.
Parameters: - session – ORM session
- dag_id (unicode) – ID of the DAG to get the task concurrency of
- task_ids (list[unicode]) – A list of valid task IDs for the given DAG
Returns: The number of running tasks
Return type: int
-
get_task_instance
(task_id, session=None)[source]¶ Returns the task instance specified by task_id for this dag run :param task_id: the task id
-
refresh_from_db
(session=None)[source]¶ Reloads the current dagrun from the database :param session: database session
-
static
-
class
airflow.models.
Log
(event, task_instance, owner=None, extra=None, **kwargs)[source]¶ Bases:
sqlalchemy.ext.declarative.api.Base
Used to actively log events to the database
-
class
airflow.models.
SlaMiss
(**kwargs)[source]¶ Bases:
sqlalchemy.ext.declarative.api.Base
Model that stores a history of the SLA that have been missed. It is used to keep track of SLA failures over time and to avoid double triggering alert emails.
-
class
airflow.models.
TaskFail
(task, execution_date, start_date, end_date)[source]¶ Bases:
sqlalchemy.ext.declarative.api.Base
TaskFail tracks the failed run durations of each task instance.
-
class
airflow.models.
TaskInstance
(task, execution_date, state=None)[source]¶ Bases:
sqlalchemy.ext.declarative.api.Base
Task instances store the state of a task instance. This table is the authority and single source of truth around what tasks have run and the state they are in.
The SqlAchemy model doesn’t have a SqlAlchemy foreign key to the task or dag model deliberately to have more control over transactions.
Database transactions on this table should insure double triggers and any confusion around what task instances are or aren’t ready to run even while multiple schedulers may be firing task instances.
-
are_dependencies_met
(dep_context=None, session=None, verbose=False)[source]¶ Returns whether or not all the conditions are met for this task instance to be run given the context for the dependencies (e.g. a task instance being force run from the UI will ignore some dependencies).
Parameters: - dep_context (DepContext) – The execution context that determines the dependencies that should be evaluated.
- session (Session) – database session
- verbose (boolean) – whether or not to print details on failed dependencies
-
are_dependents_done
(session=None)[source]¶ Checks whether the dependents of this task instance have all succeeded. This is meant to be used by wait_for_downstream.
This is useful when you do not want to start processing the next schedule of a task until the dependents are done. For instance, if the task DROPs and recreates a table.
-
command
(mark_success=False, ignore_all_deps=False, ignore_depends_on_past=False, ignore_task_deps=False, ignore_ti_state=False, local=False, pickle_id=None, raw=False, job_id=None, pool=None, cfg_path=None)[source]¶ Returns a command that can be executed anywhere where airflow is installed. This command is part of the message sent to executors by the orchestrator.
-
command_as_list
(mark_success=False, ignore_all_deps=False, ignore_task_deps=False, ignore_depends_on_past=False, ignore_ti_state=False, local=False, pickle_id=None, raw=False, job_id=None, pool=None, cfg_path=None)[source]¶ Returns a command that can be executed anywhere where airflow is installed. This command is part of the message sent to executors by the orchestrator.
-
current_state
(session=None)[source]¶ Get the very latest state from the database, if a session is passed, we use and looking up the state becomes part of the session, otherwise a new session is used.
-
static
generate_command
(dag_id, task_id, execution_date, mark_success=False, ignore_all_deps=False, ignore_depends_on_past=False, ignore_task_deps=False, ignore_ti_state=False, local=False, pickle_id=None, file_path=None, raw=False, job_id=None, pool=None, cfg_path=None)[source]¶ Generates the shell command required to execute this task instance.
Parameters: - dag_id (unicode) – DAG ID
- task_id (unicode) – Task ID
- execution_date (datetime) – Execution date for the task
- mark_success (bool) – Whether to mark the task as successful
- ignore_all_deps (boolean) – Ignore all ignoreable dependencies. Overrides the other ignore_* parameters.
- ignore_depends_on_past (boolean) – Ignore depends_on_past parameter of DAGs (e.g. for Backfills)
- ignore_task_deps (boolean) – Ignore task-specific dependencies such as depends_on_past and trigger rule
- ignore_ti_state (boolean) – Ignore the task instance’s previous failure/success
- local (bool) – Whether to run the task locally
- pickle_id – If the DAG was serialized to the DB, the ID
associated with the pickled DAG :type pickle_id: unicode :param file_path: path to the file containing the DAG definition :param raw: raw mode (needs more details) :param job_id: job ID (needs more details) :param pool: the Airflow pool that the task should run in :type pool: unicode :return: shell command that can be used to run the task instance
-
get_dagrun
(session)[source]¶ Returns the DagRun for this TaskInstance :param session: :return: DagRun
-
is_premature
¶ Returns whether a task is in UP_FOR_RETRY state and its retry interval has elapsed.
-
key
¶ Returns a tuple that identifies the task instance uniquely
-
next_retry_datetime
()[source]¶ Get datetime of the next retry if the task instance fails. For exponential backoff, retry_delay is used as base and will be converted to seconds.
-
pool_full
(session)[source]¶ Returns a boolean as to whether the slot pool has room for this task to run
-
previous_ti
¶ The task instance for the task that ran before this task instance
-
ready_for_retry
()[source]¶ Checks on whether the task instance is in the right state and timeframe to be retried.
-
refresh_from_db
(session=None, lock_for_update=False)[source]¶ Refreshes the task instance from the database based on the primary key
Parameters: lock_for_update – if True, indicates that the database should lock the TaskInstance (issuing a FOR UPDATE clause) until the session is committed.
-
run
(verbose=True, ignore_all_deps=False, ignore_depends_on_past=False, ignore_task_deps=False, ignore_ti_state=False, mark_success=False, test_mode=False, job_id=None, pool=None, session=None)[source]¶ Runs the task instance.
Parameters: - verbose (boolean) – whether to turn on more verbose loggin
- ignore_all_deps (boolean) – Ignore all of the non-critical dependencies, just runs
- ignore_depends_on_past (boolean) – Ignore depends_on_past DAG attribute
- ignore_task_deps (boolean) – Don’t check the dependencies of this TI’s task
- ignore_ti_state (boolean) – Disregards previous task instance state
- mark_success (boolean) – Don’t run the task, mark its state as success
- test_mode (boolean) – Doesn’t record success or failure in the DB
- pool (str) – specifies the pool to use to run the task instance
-
xcom_pull
(task_ids, dag_id=None, key=’return_value’, include_prior_dates=False)[source]¶ Pull XComs that optionally meet certain criteria.
The default value for key limits the search to XComs that were returned by other tasks (as opposed to those that were pushed manually). To remove this filter, pass key=None (or any desired value).
If a single task_id string is provided, the result is the value of the most recent matching XCom from that task_id. If multiple task_ids are provided, a tuple of matching values is returned. None is returned whenever no matches are found.
Parameters: - key (string) – A key for the XCom. If provided, only XComs with matching keys will be returned. The default key is ‘return_value’, also available as a constant XCOM_RETURN_KEY. This key is automatically given to XComs returned by tasks (as opposed to being pushed manually). To remove the filter, pass key=None.
- task_ids (string or iterable of strings (representing task_ids)) – Only XComs from tasks with matching ids will be pulled. Can pass None to remove the filter.
- dag_id (string) – If provided, only pulls XComs from this DAG. If None (default), the DAG of the calling task is used.
- include_prior_dates (bool) – If False, only XComs from the current execution_date are returned. If True, XComs from previous dates are returned as well.
-
xcom_push
(key, value, execution_date=None)[source]¶ Make an XCom available for tasks to pull.
Parameters: - key (string) – A key for the XCom
- value (any pickleable object) – A value for the XCom. The value is pickled and stored in the database.
- execution_date (datetime) – if provided, the XCom will not be visible until this date. This can be used, for example, to send a message to a task on a future date without it being immediately visible.
-
-
class
airflow.models.
XCom
(**kwargs)[source]¶ Bases:
sqlalchemy.ext.declarative.api.Base
Base class for XCom objects.
-
classmethod
get_many
(execution_date, key=None, task_ids=None, dag_ids=None, include_prior_dates=False, limit=100, session=None)[source]¶ Retrieve an XCom value, optionally meeting certain criteria
-
classmethod
Hooks¶
Hooks are interfaces to external platforms and databases, implementing a common interface when possible and acting as building blocks for operators.
-
class
airflow.hooks.dbapi_hook.
DbApiHook
(*args, **kwargs)[source]¶ Bases:
airflow.hooks.base_hook.BaseHook
Abstract base class for sql hooks.
-
bulk_dump
(table, tmp_file)[source]¶ Dumps a database table into a tab-delimited file
Parameters: - table (str) – The name of the source table
- tmp_file (str) – The path of the target file
-
bulk_load
(table, tmp_file)[source]¶ Loads a tab-delimited file into a database table
Parameters: - table (str) – The name of the target table
- tmp_file (str) – The path of the file to load into the table
-
get_first
(sql, parameters=None)[source]¶ Executes the sql and returns the first resulting row.
Parameters: - sql (str or list) – the sql statement to be executed (str) or a list of sql statements to execute
- parameters (mapping or iterable) – The parameters to render the SQL query with.
-
get_pandas_df
(sql, parameters=None)[source]¶ Executes the sql and returns a pandas dataframe
Parameters: - sql (str or list) – the sql statement to be executed (str) or a list of sql statements to execute
- parameters (mapping or iterable) – The parameters to render the SQL query with.
-
get_records
(sql, parameters=None)[source]¶ Executes the sql and returns a set of records.
Parameters: - sql (str or list) – the sql statement to be executed (str) or a list of sql statements to execute
- parameters (mapping or iterable) – The parameters to render the SQL query with.
-
insert_rows
(table, rows, target_fields=None, commit_every=1000)[source]¶ A generic way to insert a set of tuples into a table, the whole set of inserts is treated as one transaction
Parameters: - table (str) – Name of the target table
- rows (iterable of tuples) – The rows to insert into the table
- target_fields (iterable of strings) – The names of the columns to fill in the table
- commit_every (int) – The maximum number of rows to insert in one transaction. Set to 0 to insert all rows in one transaction.
-
run
(sql, autocommit=False, parameters=None)[source]¶ Runs a command or a list of commands. Pass a list of sql statements to the sql parameter to get them to execute sequentially
Parameters: - sql (str or list) – the sql statement to be executed (str) or a list of sql statements to execute
- autocommit (bool) – What to set the connection’s autocommit setting to before executing the query.
- parameters (mapping or iterable) – The parameters to render the SQL query with.
-
-
class
airflow.hooks.http_hook.
HttpHook
(method=’POST’, http_conn_id=’http_default’)[source]¶ Bases:
airflow.hooks.base_hook.BaseHook
Interact with HTTP servers.
-
class
airflow.hooks.mssql_hook.
MsSqlHook
(*args, **kwargs)[source]¶ Bases:
airflow.hooks.dbapi_hook.DbApiHook
Interact with Microsoft SQL Server.
-
class
airflow.hooks.sqlite_hook.
SqliteHook
(*args, **kwargs)[source]¶ Bases:
airflow.hooks.dbapi_hook.DbApiHook
Interact with SQLite.
Community contributed hooks¶
-
class
airflow.contrib.hooks.vertica_hook.
VerticaHook
(*args, **kwargs)[source]¶ Bases:
airflow.hooks.dbapi_hook.DbApiHook
Interact with Vertica.
-
class
airflow.contrib.hooks.ftp_hook.
FTPHook
(ftp_conn_id=’ftp_default’)[source]¶ Bases:
airflow.hooks.base_hook.BaseHook
Interact with FTP.
Errors that may occur throughout but should be handled downstream.
-
close_conn
()[source]¶ Closes the connection. An error will occur if the connection wasn’t ever opened.
-
create_directory
(path)[source]¶ Creates a directory on the remote system.
Parameters: path (str) – full path to the remote directory to create
-
delete_directory
(path)[source]¶ Deletes a directory on the remote system.
Parameters: path (str) – full path to the remote directory to delete
-
delete_file
(path)[source]¶ Removes a file on the FTP Server.
Parameters: path (str) – full path to the remote file
-
describe_directory
(path)[source]¶ Returns a dictionary of {filename: {attributes}} for all files on the remote system (where the MLSD command is supported).
Parameters: path (str) – full path to the remote directory
-
list_directory
(path, nlst=False)[source]¶ Returns a list of files on the remote system.
Parameters: path (str) – full path to the remote directory to list
-
rename
(from_name, to_name)[source]¶ Rename a file.
Parameters: - from_name – rename file from name
- to_name – rename file to name
-
retrieve_file
(remote_full_path, local_full_path_or_buffer)[source]¶ Transfers the remote file to a local location.
If local_full_path_or_buffer is a string path, the file will be put at that location; if it is a file-like buffer, the file will be written to the buffer but not closed.
Parameters: - remote_full_path (str) – full path to the remote file
- local_full_path_or_buffer – full path to the local file or a file-like buffer
-
store_file
(remote_full_path, local_full_path_or_buffer)[source]¶ Transfers a local file to the remote location.
If local_full_path_or_buffer is a string path, the file will be read from that location; if it is a file-like buffer, the file will be read from the buffer but not closed.
Parameters: - remote_full_path (str) – full path to the remote file
- local_full_path_or_buffer (str or file-like buffer) – full path to the local file or a file-like buffer
-
-
class
airflow.contrib.hooks.ssh_hook.
SSHHook
(conn_id=’ssh_default’)[source]¶ Bases:
airflow.hooks.base_hook.BaseHook
Light-weight remote execution library and utilities.
Using this hook (which is just a convenience wrapper for subprocess), is created to let you stream data from a remotely stored file.
As a bonus,
SSHHook
also provides a really cool feature that let’s you set up ssh tunnels super easily using a python context manager (there is an example in the integration part of unittests).Parameters: - key_file (str) – Typically the SSHHook uses the keys that are used by the user airflow is running under. This sets the behavior to use another file instead.
- connect_timeout (int) – sets the connection timeout for this connection.
- no_host_key_check (bool) – whether to check to host key. If True host keys will not be checked, but are also not stored in the current users’s known_hosts file.
- tty (bool) – allocate a tty.
- sshpass (bool) – Use to non-interactively perform password authentication by using sshpass.
-
Popen
(cmd, **kwargs)[source]¶ Remote Popen
Parameters: - cmd – command to remotely execute
- kwargs – extra arguments to Popen (see subprocess.Popen)
Returns: handle to subprocess
-
check_output
(cmd)[source]¶ Executes a remote command and returns the stdout a remote process. Simplified version of Popen when you only want the output as a string and detect any errors.
Parameters: cmd – command to remotely execute Returns: stdout
-
tunnel
(local_port, remote_port=None, remote_host=’localhost’)[source]¶ Creates a tunnel between two hosts. Like ssh -L <LOCAL_PORT>:host:<REMOTE_PORT>. Remember to close() the returned “tunnel” object in order to clean up after yourself when you are done with the tunnel.
Parameters: - local_port (int) –
- remote_port (int) –
- remote_host (str) –
Returns:
Executors¶
Executors are the mechanism by which task instances get run.
-
class
airflow.executors.local_executor.
LocalExecutor
(parallelism=32)[source]¶ Bases:
airflow.executors.base_executor.BaseExecutor
LocalExecutor executes tasks locally in parallel. It uses the multiprocessing Python library and queues to parallelize the execution of tasks.
-
class
airflow.executors.sequential_executor.
SequentialExecutor
[source]¶ Bases:
airflow.executors.base_executor.BaseExecutor
This executor will only run one task instance at a time, can be used for debugging. It is also the only executor that can be used with sqlite since sqlite doesn’t support multiple connections.
Since we want airflow to work out of the box, it defaults to this SequentialExecutor alongside sqlite as you first install it.
Community-contributed executors¶
-
class
airflow.contrib.executors.mesos_executor.
MesosExecutor
(parallelism=32)[source]¶ Bases:
airflow.executors.base_executor.BaseExecutor
MesosExecutor allows distributing the execution of task instances to multiple mesos workers.
Apache Mesos is a distributed systems kernel which abstracts CPU, memory, storage, and other compute resources away from machines (physical or virtual), enabling fault-tolerant and elastic distributed systems to easily be built and run effectively. See http://mesos.apache.org/