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 specified poke_interval until it returns True.

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, weight_rule='downstream', 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, task_concurrency=None, executor_config=None, inlets=None, outlets=None, *args, **kwargs)[source]

Bases: airflow.utils.log.logging_mixin.LoggingMixin

Abstract base class for all operators. Since operators create objects that become nodes 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 class should perform or trigger certain tasks synchronously (wait for completion). Example of operators could be an operator that 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.

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 the execution_date for the first task instance. The best practice is to have the start_date rounded to your DAG’s schedule_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 latest execution_date and adds the schedule_interval to determine the next execution_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 the TimeSensor and TimeDeltaSensor. We advise against using dynamic start_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.
  • weight_rule (str) – weighting method used for the effective total priority weight of the task. Options are: { downstream | upstream | absolute } default is downstream When set to downstream the effective weight of the task is the aggregate sum of all downstream descendants. As a result, upstream tasks will have higher weight and will be scheduled more aggressively when using positive weight values. This is useful when you have multiple dag run instances and desire to have all upstream tasks to complete for all runs before each dag can continue processing downstream tasks. When set to upstream the effective weight is the aggregate sum of all upstream ancestors. This is the opposite where downtream tasks have higher weight and will be scheduled more aggressively when using positive weight values. This is useful when you have multiple dag run instances and prefer to have each dag complete before starting upstream tasks of other dags. When set to absolute, the effective weight is the exact priority_weight specified without additional weighting. You may want to do this when you know exactly what priority weight each task should have. Additionally, when set to absolute, there is bonus effect of significantly speeding up the task creation process as for very large DAGS. Options can be set as string or using the constants defined in the static class airflow.utils.WeightRule
  • 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 an email soon after 1:00AM on the 2016-01-02 if the 2016-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 (callable) – much like the on_failure_callback except that it is executed when retries occur.
  • on_success_callback (callable) – much like the on_failure_callback except 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 is all_success. Options can be set as string or using the constants defined in the static class airflow.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
  • task_concurrency (int) – When set, a task will be able to limit the concurrent runs across execution_dates
  • executor_config (dict) –

    Additional task-level configuration parameters that are interpreted by a specific executor. Parameters are namespaced by the name of executor.

    Example: to run this task in a specific docker container through the KubernetesExecutor

    MyOperator(...,
        executor_config={
        "KubernetesExecutor":
            {"image": "myCustomDockerImage"}
            }
    )
    
clear(start_date=None, end_date=None, upstream=False, downstream=False, session=None)[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/overridden by subclasses.

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_relative_ids(upstream=False)[source]

Get the direct relative ids to the current task, upstream or downstream.

get_direct_relatives(upstream=False)[source]

Get the direct relatives to the current task, upstream or downstream.

get_flat_relative_ids(upstream=False, found_descendants=None)[source]

Get a flat list of relatives’ ids, either upstream or downstream.

get_flat_relatives(upstream=False)[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, result=None)[source]

This hook is triggered right after self.execute() is called. It is passed the execution context and any results returned by the operator.

pre_execute(context)[source]

This hook is triggered right before self.execute() is called.

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 list to be directly downstream from the current task.

set_upstream(task_or_task_list)[source]

Set a task or a task list to be directly upstream from the current task.

upstream_list

@property: list of tasks directly upstream

xcom_pull(context, task_ids=None, 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()

BaseSensorOperator

All sensors are derived from BaseSensorOperator. All sensors inherit the timeout and poke_interval on top of the BaseOperator attributes.

class airflow.sensors.base_sensor_operator.BaseSensorOperator(poke_interval=60, timeout=604800, soft_fail=False, *args, **kwargs)[source]

Bases: airflow.models.BaseOperator, airflow.models.SkipMixin

Sensor operators are derived from this class an inherit these attributes.

Sensor operators keep executing at a time interval and succeed when
a criteria is met and fail if and when they time out.
Parameters:
  • soft_fail (bool) – Set to true to mark the task as SKIPPED on failure
  • poke_interval (int) – Time in seconds that the job should wait in between each tries
  • timeout (int) – Time, in seconds before the task times out and fails.
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.

poke(context)[source]

Function that the sensors defined while deriving this class should override.

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. (templated)
  • 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)
execute(context)[source]

Execute the bash command in a temporary directory which will be cleaned afterwards

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.

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, airflow.models.SkipMixin

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. The skipped 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 from BranchPythonOperator is logically unsound as skipped status will invariably lead to block tasks that depend on their past successes. skipped states propagates where all directly upstream tasks are skipped.

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.

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 python bool casting. If any of the values return False 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. (templated)
execute(context=None)[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.

class airflow.operators.docker_operator.DockerOperator(image, api_version=None, command=None, cpus=1.0, docker_url='unix://var/run/docker.sock', environment=None, force_pull=False, mem_limit=None, network_mode=None, tls_ca_cert=None, tls_client_cert=None, tls_client_key=None, tls_hostname=None, tls_ssl_version=None, tmp_dir='/tmp/airflow', user=None, volumes=None, working_dir=None, xcom_push=False, xcom_all=False, docker_conn_id=None, dns=None, dns_search=None, shm_size=None, *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

Execute a command inside a docker container.

A temporary directory is created on the host and mounted into a container to allow storing files that together exceed the default disk size of 10GB in a container. The path to the mounted directory can be accessed via the environment variable AIRFLOW_TMP_DIR.

If a login to a private registry is required prior to pulling the image, a Docker connection needs to be configured in Airflow and the connection ID be provided with the parameter docker_conn_id.

Parameters:
  • image (str) – Docker image from which to create the container. If image tag is omitted, “latest” will be used.
  • api_version (str) – Remote API version. Set to auto to automatically detect the server’s version.
  • command (str or list) – Command to be run in the container. (templated)
  • cpus (float) – Number of CPUs to assign to the container. This value gets multiplied with 1024. See https://docs.docker.com/engine/reference/run/#cpu-share-constraint
  • dns (list of strings) – Docker custom DNS servers
  • dns_search (list of strings) – Docker custom DNS search domain
  • docker_url (str) – URL of the host running the docker daemon. Default is unix://var/run/docker.sock
  • environment (dict) – Environment variables to set in the container. (templated)
  • force_pull (bool) – Pull the docker image on every run. Default is False.
  • mem_limit (float or str) – Maximum amount of memory the container can use. Either a float value, which represents the limit in bytes, or a string like 128m or 1g.
  • network_mode (str) – Network mode for the container.
  • tls_ca_cert (str) – Path to a PEM-encoded certificate authority to secure the docker connection.
  • tls_client_cert (str) – Path to the PEM-encoded certificate used to authenticate docker client.
  • tls_client_key (str) – Path to the PEM-encoded key used to authenticate docker client.
  • tls_hostname (str or bool) – Hostname to match against the docker server certificate or False to disable the check.
  • tls_ssl_version (str) – Version of SSL to use when communicating with docker daemon.
  • tmp_dir (str) – Mount point inside the container to a temporary directory created on the host by the operator. The path is also made available via the environment variable AIRFLOW_TMP_DIR inside the container.
  • user (int or str) – Default user inside the docker container.
  • volumes – List of volumes to mount into the container, e.g. ['/host/path:/container/path', '/host/path2:/container/path2:ro'].
  • working_dir (str) – Working directory to set on the container (equivalent to the -w switch the docker client)
  • xcom_push (bool) – Does the stdout will be pushed to the next step using XCom. The default is False.
  • xcom_all (bool) – Push all the stdout or just the last line. The default is False (last line).
  • docker_conn_id (str) – ID of the Airflow connection to use
  • shm_size (int) – Size of /dev/shm in bytes. The size must be greater than 0. If omitted uses system default.
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.

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.

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.

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.

class airflow.operators.druid_check_operator.DruidCheckOperator(sql, druid_broker_conn_id='druid_broker_default', *args, **kwargs)[source]

Bases: airflow.operators.check_operator.CheckOperator

Performs checks against Druid. The DruidCheckOperator expects a sql query that will return a single row. Each value on that first row is evaluated using python bool casting. If any of the values return False 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 alterts without stopping the progress of the DAG.

Parameters:
  • sql (string) – the sql to be executed
  • druid_broker_conn_id (string) – reference to the druid broker
execute(context=None)[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_db_hook()[source]

Return the druid db api hook.

get_first(sql)[source]

Executes the druid sql to druid broker and returns the first resulting row.

Parameters:sql (str) – the sql statement to be executed (str)
class airflow.operators.email_operator.EmailOperator(to, subject, html_content, files=None, cc=None, bcc=None, mime_subtype='mixed', mime_charset='us_ascii', *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. (templated)
  • subject (string) – subject line for the email. (templated)
  • html_content (string) – content of the email, html markup is allowed. (templated)
  • 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
  • mime_subtype (string) – MIME sub content type
  • mime_charset (string) – character set parameter added to the Content-Type header.
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.

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 meant to be used on small-ish datasets that fit in memory.

Parameters:
  • sql (str) – SQL query to execute against the source database. (templated)
  • destination_table (str) – target table. (templated)
  • 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. (templated)
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.

class airflow.operators.hive_to_druid.HiveToDruidTransfer(sql, druid_datasource, ts_dim, metric_spec=None, hive_cli_conn_id='hive_cli_default', druid_ingest_conn_id='druid_ingest_default', metastore_conn_id='metastore_default', hadoop_dependency_coordinates=None, intervals=None, num_shards=-1, target_partition_size=-1, query_granularity='NONE', segment_granularity='DAY', hive_tblproperties=None, *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

Moves data from Hive to Druid, [del]note that for now the data is loaded into memory before being pushed to Druid, so this operator should be used for smallish amount of data.[/del]

Parameters:
  • sql (str) – SQL query to execute against the Druid database. (templated)
  • druid_datasource (str) – the datasource you want to ingest into in druid
  • ts_dim (str) – the timestamp dimension
  • metric_spec (list) – the metrics you want to define for your data
  • hive_cli_conn_id (str) – the hive connection id
  • druid_ingest_conn_id (str) – the druid ingest connection id
  • metastore_conn_id (str) – the metastore connection id
  • hadoop_dependency_coordinates (list of str) – list of coordinates to squeeze int the ingest json
  • intervals (list) – list of time intervals that defines segments, this is passed as is to the json object. (templated)
  • hive_tblproperties (dict) – additional properties for tblproperties in hive for the staging table
construct_ingest_query(static_path, columns)[source]

Builds an ingest query for an HDFS TSV load.

Parameters:
  • static_path (str) – The path on hdfs where the data is
  • columns (list) – List of all the columns that are available
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.

class airflow.operators.hive_to_mysql.HiveToMySqlTransfer(sql, mysql_table, hiveserver2_conn_id='hiveserver2_default', mysql_conn_id='mysql_default', mysql_preoperator=None, mysql_postoperator=None, bulk_load=False, *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

Moves data from Hive to MySQL, note that for now the data is loaded into memory before being pushed to MySQL, so this operator should be used for smallish amount of data.

Parameters:
  • sql (str) – SQL query to execute against Hive server. (templated)
  • mysql_table (str) – target MySQL table, use dot notation to target a specific database. (templated)
  • mysql_conn_id (str) – source mysql connection
  • hiveserver2_conn_id (str) – destination hive connection
  • mysql_preoperator (str) – sql statement to run against mysql prior to import, typically use to truncate of delete in place of the data coming in, allowing the task to be idempotent (running the task twice won’t double load data). (templated)
  • mysql_postoperator (str) – sql statement to run against mysql after the import, typically used to move data from staging to production and issue cleanup commands. (templated)
  • bulk_load (bool) – flag to use bulk_load option. This loads mysql directly from a tab-delimited text file using the LOAD DATA LOCAL INFILE command. This option requires an extra connection parameter for the destination MySQL connection: {‘local_infile’: true}.
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.

class airflow.operators.hive_to_samba_operator.Hive2SambaOperator(hql, destination_filepath, samba_conn_id='samba_default', hiveserver2_conn_id='hiveserver2_default', *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

Executes hql code in a specific Hive database and loads the results of the query as a csv to a Samba location.

Parameters:
  • hql (string) – the hql to be exported. (templated)
  • hiveserver2_conn_id (string) – reference to the hiveserver2 service
  • samba_conn_id (string) – reference to the samba destination
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.

class airflow.operators.hive_operator.HiveOperator(hql, hive_cli_conn_id='hive_cli_default', schema='default', hiveconfs=None, hiveconf_jinja_translate=False, script_begin_tag=None, run_as_owner=False, mapred_queue=None, mapred_queue_priority=None, mapred_job_name=None, *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

Executes hql code or hive script in a specific Hive database.

Parameters:
  • hql (string) – the hql to be executed. Note that you may also use a relative path from the dag file of a (template) hive script. (templated)
  • hive_cli_conn_id (string) – reference to the Hive database. (templated)
  • hiveconfs (dict) – if defined, these key value pairs will be passed to hive as -hiveconf "key"="value"
  • hiveconf_jinja_translate (boolean) – when True, hiveconf-type templating ${var} gets translated into jinja-type templating {{ var }} and ${hiveconf: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.
  • script_begin_tag (str) – If defined, the operator will get rid of the part of the script before the first occurrence of script_begin_tag
  • mapred_queue (string) – queue used by the Hadoop CapacityScheduler. (templated)
  • mapred_queue_priority (string) – priority within CapacityScheduler queue. Possible settings include: VERY_HIGH, HIGH, NORMAL, LOW, VERY_LOW
  • mapred_job_name (string) – This name will appear in the jobtracker. This can make monitoring easier.
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.

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.

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.

class airflow.operators.hive_stats_operator.HiveStatsCollectionOperator(table, partition, extra_exprs=None, col_blacklist=None, assignment_func=None, metastore_conn_id='metastore_default', presto_conn_id='presto_default', mysql_conn_id='airflow_db', *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

Gathers partition statistics using a dynamically generated Presto query, inserts the stats into a MySql table with this format. Stats overwrite themselves if you rerun the same date/partition.

CREATE TABLE hive_stats (
    ds VARCHAR(16),
    table_name VARCHAR(500),
    metric VARCHAR(200),
    value BIGINT
);
Parameters:
  • table (str) – the source table, in the format database.table_name. (templated)
  • partition (dict of {col:value}) – the source partition. (templated)
  • extra_exprs (dict) – dict of expression to run against the table where keys are metric names and values are Presto compatible expressions
  • col_blacklist (list) – list of columns to blacklist, consider blacklisting blobs, large json columns, …
  • assignment_func (function) – a function that receives a column name and a type, and returns a dict of metric names and an Presto expressions. If None is returned, the global defaults are applied. If an empty dictionary is returned, no stats are computed for that column.
execute(context=None)[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.

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
execute(context=None)[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.

class airflow.operators.jdbc_operator.JdbcOperator(sql, jdbc_conn_id='jdbc_default', autocommit=False, parameters=None, *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

Executes sql code in a database using jdbc driver.

Requires jaydebeapi.

Parameters:
  • jdbc_conn_id (string) – reference to a predefined 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. (templated)
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.

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, weight_rule='downstream', 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, task_concurrency=None, executor_config=None, inlets=None, outlets=None, *args, **kwargs)[source]

Bases: airflow.models.BaseOperator, airflow.models.SkipMixin

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.

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.

class airflow.operators.mssql_operator.MsSqlOperator(sql, mssql_conn_id='mssql_default', parameters=None, autocommit=False, database=None, *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

Executes sql code in a specific Microsoft SQL database

Parameters:
  • mssql_conn_id (string) – reference to a specific mssql database
  • sql (string or string pointing to a template file with .sql extension. (templated)) – the sql code to be executed
  • database (string) – name of database which overwrite defined one in connection
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.

class airflow.operators.mssql_to_hive.MsSqlToHiveTransfer(sql, hive_table, create=True, recreate=False, partition=None, delimiter='x01', mssql_conn_id='mssql_default', hive_cli_conn_id='hive_cli_default', tblproperties=None, *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

Moves data from Microsoft SQL Server to Hive. The operator runs your query against Microsoft SQL Server, stores the file locally before loading it into a Hive table. If the create or recreate arguments are set to True, a CREATE TABLE and DROP TABLE statements are generated. Hive data types are inferred from the cursor’s metadata. Note that the table generated in Hive uses STORED AS textfile which isn’t the most efficient serialization format. If a large amount of data is loaded and/or if the table gets queried considerably, you may want to use this operator only to stage the data into a temporary table before loading it into its final destination using a HiveOperator.

Parameters:
  • sql (str) – SQL query to execute against the Microsoft SQL Server database. (templated)
  • hive_table (str) – target Hive table, use dot notation to target a specific database. (templated)
  • create (bool) – whether to create the table if it doesn’t exist
  • recreate (bool) – whether to drop and recreate the table at every execution
  • partition (dict) – target partition as a dict of partition columns and values. (templated)
  • delimiter (str) – field delimiter in the file
  • mssql_conn_id (str) – source Microsoft SQL Server connection
  • hive_conn_id (str) – destination hive connection
  • tblproperties (dict) – TBLPROPERTIES of the hive table being created
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.

class airflow.operators.mysql_operator.MySqlOperator(sql, mysql_conn_id='mysql_default', parameters=None, autocommit=False, database=None, *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

Executes sql code in a specific MySQL database

Parameters:
  • mysql_conn_id (string) – reference to a specific mysql 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. (templated)
  • database (string) – name of database which overwrite defined one in connection
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.

class airflow.operators.mysql_to_hive.MySqlToHiveTransfer(sql, hive_table, create=True, recreate=False, partition=None, delimiter='x01', mysql_conn_id='mysql_default', hive_cli_conn_id='hive_cli_default', tblproperties=None, *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

Moves data from MySql to Hive. The operator runs your query against MySQL, stores the file locally before loading it into a Hive table. If the create or recreate arguments are set to True, a CREATE TABLE and DROP TABLE statements are generated. Hive data types are inferred from the cursor’s metadata. Note that the table generated in Hive uses STORED AS textfile which isn’t the most efficient serialization format. If a large amount of data is loaded and/or if the table gets queried considerably, you may want to use this operator only to stage the data into a temporary table before loading it into its final destination using a HiveOperator.

Parameters:
  • sql (str) – SQL query to execute against the MySQL database. (templated)
  • hive_table (str) – target Hive table, use dot notation to target a specific database. (templated)
  • create (bool) – whether to create the table if it doesn’t exist
  • recreate (bool) – whether to drop and recreate the table at every execution
  • partition (dict) – target partition as a dict of partition columns and values. (templated)
  • delimiter (str) – field delimiter in the file
  • mysql_conn_id (str) – source mysql connection
  • hive_conn_id (str) – destination hive connection
  • tblproperties (dict) – TBLPROPERTIES of the hive table being created
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.

class airflow.operators.oracle_operator.OracleOperator(sql, oracle_conn_id='oracle_default', parameters=None, autocommit=False, *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

Executes sql code in a specific Oracle database :param oracle_conn_id: reference to a specific Oracle database :type oracle_conn_id: string :param sql: the sql code to be executed. (templated) :type 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’
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.

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. (templated)
  • 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.
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.

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.

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.

class airflow.operators.postgres_operator.PostgresOperator(sql, postgres_conn_id='postgres_default', autocommit=False, parameters=None, database=None, *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

Executes sql code in a specific Postgres database

Parameters:
  • postgres_conn_id (string) – reference to a specific postgres 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. (templated)
  • database (string) – name of database which overwrite defined one in connection
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.

class airflow.operators.presto_check_operator.PrestoCheckOperator(sql, presto_conn_id='presto_default', *args, **kwargs)[source]

Bases: airflow.operators.check_operator.CheckOperator

Performs checks against Presto. The PrestoCheckOperator expects a sql query that will return a single row. Each value on that first row is evaluated using python bool casting. If any of the values return False 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 alterts without stopping the progress of the DAG.

Parameters:
  • sql (string) – the sql to be executed
  • presto_conn_id (string) – reference to the Presto database
class airflow.operators.presto_check_operator.PrestoIntervalCheckOperator(table, metrics_thresholds, date_filter_column='ds', days_back=-7, presto_conn_id='presto_default', *args, **kwargs)[source]

Bases: airflow.operators.check_operator.IntervalCheckOperator

Checks that the values of metrics given as SQL expressions are within a certain tolerance of the ones from days_back before.

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
  • presto_conn_id (string) – reference to the Presto database
class airflow.operators.presto_to_mysql.PrestoToMySqlTransfer(sql, mysql_table, presto_conn_id='presto_default', mysql_conn_id='mysql_default', mysql_preoperator=None, *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

Moves data from Presto to MySQL, note that for now the data is loaded into memory before being pushed to MySQL, so this operator should be used for smallish amount of data.

Parameters:
  • sql (str) – SQL query to execute against Presto. (templated)
  • mysql_table (str) – target MySQL table, use dot notation to target a specific database. (templated)
  • mysql_conn_id (str) – source mysql connection
  • presto_conn_id (str) – source presto connection
  • mysql_preoperator (str) – sql statement to run against mysql prior to import, typically use to truncate of delete in place of the data coming in, allowing the task to be idempotent (running the task twice won’t double load data). (templated)
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.

class airflow.operators.presto_check_operator.PrestoValueCheckOperator(sql, pass_value, tolerance=None, presto_conn_id='presto_default', *args, **kwargs)[source]

Bases: airflow.operators.check_operator.ValueCheckOperator

Performs a simple value check using sql code.

Parameters:
  • sql (string) – the sql to be executed
  • presto_conn_id (string) – reference to the Presto database
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__ and execute takes place and are made available in your callable’s context after the template has been applied. (templated)
  • templates_exts (list(str)) – a list of file extensions to resolve while processing templated fields, for examples ['.sql', '.hql']
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.

class airflow.operators.python_operator.PythonVirtualenvOperator(python_callable, requirements=None, python_version=None, use_dill=False, system_site_packages=True, op_args=None, op_kwargs=None, string_args=None, templates_dict=None, templates_exts=None, *args, **kwargs)[source]

Bases: airflow.operators.python_operator.PythonOperator

Allows one to run a function in a virtualenv that is created and destroyed automatically (with certain caveats).

The function must be defined using def, and not be part of a class. All imports must happen inside the function and no variables outside of the scope may be referenced. A global scope variable named virtualenv_string_args will be available (populated by string_args). In addition, one can pass stuff through op_args and op_kwargs, and one can use a return value.

Note that if your virtualenv runs in a different Python major version than Airflow, you cannot use return values, op_args, or op_kwargs. You can use string_args though.

Parameters:
  • python_callable (function) – A python function with no references to outside variables, defined with def, which will be run in a virtualenv
  • requirements (list(str)) – A list of requirements as specified in a pip install command
  • python_version (str) – The Python version to run the virtualenv with. Note that both 2 and 2.7 are acceptable forms.
  • use_dill (bool) – Whether to use dill to serialize the args and result (pickle is default). This allow more complex types but requires you to include dill in your requirements.
  • system_site_packages (bool) – Whether to include system_site_packages in your virtualenv. See virtualenv documentation for more information.
  • op_args – A list of positional arguments to pass to python_callable.
  • op_kwargs (dict) – A dict of keyword arguments to pass to python_callable.
  • string_args (list(str)) – Strings that are present in the global var virtualenv_string_args, available to python_callable at runtime as a list(str). Note that args are split by newline.
  • templates_dict (dict of str) – a dictionary where the values are templates that will get templated by the Airflow engine sometime between __init__ and execute takes place and are made available in your callable’s context after the template has been applied
  • templates_exts (list(str)) – a list of file extensions to resolve while processing templated fields, for examples ['.sql', '.hql']
class airflow.operators.s3_file_transform_operator.S3FileTransformOperator(source_s3_key, dest_s3_key, transform_script=None, select_expression=None, source_aws_conn_id='aws_default', dest_aws_conn_id='aws_default', replace=False, *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

Copies data from a source S3 location to a temporary location on the local filesystem. Runs a transformation on this file as specified by the transformation script and uploads the output to a destination S3 location.

The locations of the source and the destination files in the local filesystem is provided as an first and second arguments to the transformation script. The transformation script is expected to read the data from source, transform it and write the output to the local destination file. The operator then takes over control and uploads the local destination file to S3.

S3 Select is also available to filter the source contents. Users can omit the transformation script if S3 Select expression is specified.

Parameters:
  • source_s3_key (str) – The key to be retrieved from S3. (templated)
  • source_aws_conn_id (str) – source s3 connection
  • dest_s3_key (str) – The key to be written from S3. (templated)
  • dest_aws_conn_id (str) – destination s3 connection
  • replace (bool) – Replace dest S3 key if it already exists
  • transform_script (str) – location of the executable transformation script
  • select_expression (str) – S3 Select expression
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.

class airflow.operators.s3_to_hive_operator.S3ToHiveTransfer(s3_key, field_dict, hive_table, delimiter=', ', create=True, recreate=False, partition=None, headers=False, check_headers=False, wildcard_match=False, aws_conn_id='aws_default', hive_cli_conn_id='hive_cli_default', input_compressed=False, tblproperties=None, select_expression=None, *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

Moves data from S3 to Hive. The operator downloads a file from S3, stores the file locally before loading it into a Hive table. If the create or recreate arguments are set to True, a CREATE TABLE and DROP TABLE statements are generated. Hive data types are inferred from the cursor’s metadata from.

Note that the table generated in Hive uses STORED AS textfile which isn’t the most efficient serialization format. If a large amount of data is loaded and/or if the tables gets queried considerably, you may want to use this operator only to stage the data into a temporary table before loading it into its final destination using a HiveOperator.

Parameters:
  • s3_key (str) – The key to be retrieved from S3. (templated)
  • field_dict (dict) – A dictionary of the fields name in the file as keys and their Hive types as values
  • hive_table (str) – target Hive table, use dot notation to target a specific database. (templated)
  • create (bool) – whether to create the table if it doesn’t exist
  • recreate (bool) – whether to drop and recreate the table at every execution
  • partition (dict) – target partition as a dict of partition columns and values. (templated)
  • headers (bool) – whether the file contains column names on the first line
  • check_headers (bool) – whether the column names on the first line should be checked against the keys of field_dict
  • wildcard_match (bool) – whether the s3_key should be interpreted as a Unix wildcard pattern
  • delimiter (str) – field delimiter in the file
  • aws_conn_id (str) – source s3 connection
  • hive_cli_conn_id (str) – destination hive connection
  • input_compressed (bool) – Boolean to determine if file decompression is required to process headers
  • tblproperties (dict) – TBLPROPERTIES of the hive table being created
  • select_expression (str) – S3 Select expression
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.

class airflow.operators.s3_to_redshift_operator.S3ToRedshiftTransfer(schema, table, s3_bucket, s3_key, redshift_conn_id='redshift_default', aws_conn_id='aws_default', copy_options=(), autocommit=False, parameters=None, *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

Executes an COPY command to load files from s3 to Redshift

Parameters:
  • schema (string) – reference to a specific schema in redshift database
  • table (string) – reference to a specific table in redshift database
  • s3_bucket (string) – reference to a specific S3 bucket
  • s3_key (string) – reference to a specific S3 key
  • redshift_conn_id (string) – reference to a specific redshift database
  • aws_conn_id (string) – reference to a specific S3 connection
  • copy_options (list) – reference to a list of COPY options
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.

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, airflow.models.SkipMixin

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.

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.

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', log_response=False, *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. (templated)
  • 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. (templated)
  • 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.)
  • xcom_push (bool) – Push the response to Xcom (default: False)
  • log_response (bool) – Log the response (default: False)
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.

class airflow.operators.slack_operator.SlackAPIOperator(slack_conn_id=None, token=None, method=None, api_params=None, *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

Base Slack Operator The SlackAPIPostOperator is derived from this operator. In the future additional Slack API Operators will be derived from this class as well

Parameters:
construct_api_call_params()[source]

Used by the execute function. Allows templating on the source fields of the api_call_params dict before construction

Override in child classes. Each SlackAPIOperator child class is responsible for having a construct_api_call_params function which sets self.api_call_params with a dict of API call parameters (https://api.slack.com/methods)

execute(**kwargs)[source]

SlackAPIOperator calls will not fail even if the call is not unsuccessful. It should not prevent a DAG from completing in success

class airflow.operators.slack_operator.SlackAPIPostOperator(channel='#general', username='Airflow', text='No message has been set.nHere is a cat video insteadnhttps://www.youtube.com/watch?v=J---aiyznGQ', icon_url='https://raw.githubusercontent.com/airbnb/airflow/master/airflow/www/static/pin_100.png', attachments=None, *args, **kwargs)[source]

Bases: airflow.operators.slack_operator.SlackAPIOperator

Posts messages to a slack channel

Parameters:
  • channel (string) – channel in which to post message on slack name (#general) or ID (C12318391). (templated)
  • username (string) – Username that airflow will be posting to Slack as. (templated)
  • text (string) – message to send to slack. (templated)
  • icon_url (string) – url to icon used for this message
  • attachments (array of hashes) – extra formatting details. (templated) - see https://api.slack.com/docs/attachments.
construct_api_call_params()[source]

Used by the execute function. Allows templating on the source fields of the api_call_params dict before construction

Override in child classes. Each SlackAPIOperator child class is responsible for having a construct_api_call_params function which sets self.api_call_params with a dict of API call parameters (https://api.slack.com/methods)

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. (templated)
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.

class airflow.operators.subdag_operator.SubDagOperator(subdag, executor=<airflow.executors.sequential_executor.SequentialExecutor object>, *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

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.

class airflow.operators.dagrun_operator.TriggerDagRunOperator(trigger_dag_id, python_callable=None, execution_date=None, *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

Triggers a DAG run for a specified dag_id

Parameters:
  • 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 object obj for your callable to fill and return if you want a DagRun created. This obj object contains a run_id and payload attribute that you can modify in your function. The run_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 like def foo(context, dag_run_obj):
  • execution_date (datetime.datetime) – Execution date for the dag
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.

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. (templated)
execute(context=None)[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.

class airflow.operators.redshift_to_s3_operator.RedshiftToS3Transfer(schema, table, s3_bucket, s3_key, redshift_conn_id='redshift_default', aws_conn_id='aws_default', unload_options=(), autocommit=False, parameters=None, include_header=False, *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

Executes an UNLOAD command to s3 as a CSV with headers

Parameters:
  • schema (string) – reference to a specific schema in redshift database
  • table (string) – reference to a specific table in redshift database
  • s3_bucket (string) – reference to a specific S3 bucket
  • s3_key (string) – reference to a specific S3 key
  • redshift_conn_id (string) – reference to a specific redshift database
  • aws_conn_id (string) – reference to a specific S3 connection
  • unload_options (list) – reference to a list of UNLOAD options
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.

Sensors

class airflow.sensors.external_task_sensor.ExternalTaskSensor(external_dag_id, external_task_id, allowed_states=None, execution_delta=None, execution_date_fn=None, *args, **kwargs)[source]

Bases: airflow.sensors.base_sensor_operator.BaseSensorOperator

Waits for a task to complete in a different DAG

Parameters:
  • external_dag_id (string) – The dag_id that contains the task you want to wait for
  • external_task_id (string) – The task_id that contains the task you want to wait for
  • allowed_states (list) – list of allowed states, default is ['success']
  • execution_delta (datetime.timedelta) – time difference with the previous execution to look at, the default is the same execution_date as the current task. For yesterday, use [positive!] datetime.timedelta(days=1). Either execution_delta or execution_date_fn can be passed to ExternalTaskSensor, but not both.
  • execution_date_fn (callable) – function that receives the current execution date and returns the desired execution dates to query. Either execution_delta or execution_date_fn can be passed to ExternalTaskSensor, but not both.
poke(context, session=None)[source]

Function that the sensors defined while deriving this class should override.

class airflow.sensors.hdfs_sensor.HdfsSensor(filepath, hdfs_conn_id='hdfs_default', ignored_ext=['_COPYING_'], ignore_copying=True, file_size=None, hook=<class 'airflow.hooks.hdfs_hook.HDFSHook'>, *args, **kwargs)[source]

Bases: airflow.sensors.base_sensor_operator.BaseSensorOperator

Waits for a file or folder to land in HDFS

static filter_for_filesize(result, size=None)[source]

Will test the filepath result and test if its size is at least self.filesize

Parameters:
  • result – a list of dicts returned by Snakebite ls
  • size – the file size in MB a file should be at least to trigger True
Returns:

(bool) depending on the matching criteria

static filter_for_ignored_ext(result, ignored_ext, ignore_copying)[source]

Will filter if instructed to do so the result to remove matching criteria

Parameters:
  • result – (list) of dicts returned by Snakebite ls
  • ignored_ext – (list) of ignored extensions
  • ignore_copying – (bool) shall we ignore ?
Returns:

(list) of dicts which were not removed

poke(context)[source]

Function that the sensors defined while deriving this class should override.

class airflow.sensors.hive_partition_sensor.HivePartitionSensor(table, partition="ds='{{ ds }}'", metastore_conn_id='metastore_default', schema='default', poke_interval=180, *args, **kwargs)[source]

Bases: airflow.sensors.base_sensor_operator.BaseSensorOperator

Waits for a partition to show up in Hive.

Note: Because partition supports general logical operators, it can be inefficient. Consider using NamedHivePartitionSensor instead if you don’t need the full flexibility of HivePartitionSensor.

Parameters:
  • table (string) – The name of the table to wait for, supports the dot notation (my_database.my_table)
  • partition (string) – The partition clause to wait for. This is passed as is to the metastore Thrift client get_partitions_by_filter method, and apparently supports SQL like notation as in ds='2015-01-01' AND type='value' and comparison operators as in "ds>=2015-01-01"
  • metastore_conn_id (str) – reference to the metastore thrift service connection id
poke(context)[source]

Function that the sensors defined while deriving this class should override.

class airflow.sensors.http_sensor.HttpSensor(endpoint, http_conn_id='http_default', method='GET', request_params=None, headers=None, response_check=None, extra_options=None, *args, **kwargs)[source]

Bases: airflow.sensors.base_sensor_operator.BaseSensorOperator

Executes a HTTP get statement and returns False on failure:
404 not found or response_check function returned False
Parameters:
  • http_conn_id (string) – The connection to run the sensor against
  • method (string) – The HTTP request method to use
  • endpoint (string) – The relative part of the full url
  • request_params (a dictionary of string key/value pairs) – The parameters to be added to the GET url
  • 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.)
poke(context)[source]

Function that the sensors defined while deriving this class should override.

class airflow.sensors.metastore_partition_sensor.MetastorePartitionSensor(table, partition_name, schema='default', mysql_conn_id='metastore_mysql', *args, **kwargs)[source]

Bases: airflow.sensors.sql_sensor.SqlSensor

An alternative to the HivePartitionSensor that talk directly to the MySQL db. This was created as a result of observing sub optimal queries generated by the Metastore thrift service when hitting subpartitioned tables. The Thrift service’s queries were written in a way that wouldn’t leverage the indexes.

Parameters:
  • schema (str) – the schema
  • table (str) – the table
  • partition_name (str) – the partition name, as defined in the PARTITIONS table of the Metastore. Order of the fields does matter. Examples: ds=2016-01-01 or ds=2016-01-01/sub=foo for a sub partitioned table
  • mysql_conn_id (str) – a reference to the MySQL conn_id for the metastore
poke(context)[source]

Function that the sensors defined while deriving this class should override.

class airflow.sensors.named_hive_partition_sensor.NamedHivePartitionSensor(partition_names, metastore_conn_id='metastore_default', poke_interval=180, hook=None, *args, **kwargs)[source]

Bases: airflow.sensors.base_sensor_operator.BaseSensorOperator

Waits for a set of partitions to show up in Hive.

Parameters:
  • partition_names (list of strings) – List of fully qualified names of the partitions to wait for. A fully qualified name is of the form schema.table/pk1=pv1/pk2=pv2, for example, default.users/ds=2016-01-01. This is passed as is to the metastore Thrift client get_partitions_by_name method. Note that you cannot use logical or comparison operators as in HivePartitionSensor.
  • metastore_conn_id (str) – reference to the metastore thrift service connection id
poke(context)[source]

Function that the sensors defined while deriving this class should override.

class airflow.sensors.s3_key_sensor.S3KeySensor(bucket_key, bucket_name=None, wildcard_match=False, aws_conn_id='aws_default', *args, **kwargs)[source]

Bases: airflow.sensors.base_sensor_operator.BaseSensorOperator

Waits for a key (a file-like instance on S3) to be present in a S3 bucket. S3 being a key/value it does not support folders. The path is just a key a resource.

Parameters:
  • bucket_key (str) – The key being waited on. Supports full s3:// style url or relative path from root level.
  • bucket_name (str) – Name of the S3 bucket
  • wildcard_match (bool) – whether the bucket_key should be interpreted as a Unix wildcard pattern
  • aws_conn_id (str) – a reference to the s3 connection
poke(context)[source]

Function that the sensors defined while deriving this class should override.

class airflow.sensors.s3_prefix_sensor.S3PrefixSensor(bucket_name, prefix, delimiter='/', aws_conn_id='aws_default', *args, **kwargs)[source]

Bases: airflow.sensors.base_sensor_operator.BaseSensorOperator

Waits for a prefix to exist. A prefix is the first part of a key, thus enabling checking of constructs similar to glob airfl* or SQL LIKE ‘airfl%’. There is the possibility to precise a delimiter to indicate the hierarchy or keys, meaning that the match will stop at that delimiter. Current code accepts sane delimiters, i.e. characters that are NOT special characters in the Python regex engine.

Parameters:
  • bucket_name (str) – Name of the S3 bucket
  • prefix (str) – The prefix being waited on. Relative path from bucket root level.
  • delimiter (str) – The delimiter intended to show hierarchy. Defaults to ‘/’.
  • aws_conn_id (str) – a reference to the s3 connection
poke(context)[source]

Function that the sensors defined while deriving this class should override.

class airflow.sensors.sql_sensor.SqlSensor(conn_id, sql, *args, **kwargs)[source]

Bases: airflow.sensors.base_sensor_operator.BaseSensorOperator

Runs a sql statement until a criteria is met. It will keep trying while sql returns no row, or if the first cell in (0, ‘0’, ‘’).

Parameters:
  • conn_id (string) – The connection to run the sensor against
  • sql – The sql to run. To pass, it needs to return at least one cell that contains a non-zero / empty string value.
poke(context)[source]

Function that the sensors defined while deriving this class should override.

class airflow.sensors.time_sensor.TimeSensor(target_time, *args, **kwargs)[source]

Bases: airflow.sensors.base_sensor_operator.BaseSensorOperator

Waits until the specified time of the day.

Parameters:target_time (datetime.time) – time after which the job succeeds
poke(context)[source]

Function that the sensors defined while deriving this class should override.

class airflow.sensors.time_delta_sensor.TimeDeltaSensor(delta, *args, **kwargs)[source]

Bases: airflow.sensors.base_sensor_operator.BaseSensorOperator

Waits for a timedelta after the task’s execution_date + schedule_interval. In Airflow, the daily task stamped with execution_date 2016-01-01 can only start running on 2016-01-02. The timedelta here represents the time after the execution period has closed.

Parameters:delta (datetime.timedelta) – time length to wait after execution_date before succeeding
poke(context)[source]

Function that the sensors defined while deriving this class should override.

class airflow.sensors.web_hdfs_sensor.WebHdfsSensor(filepath, webhdfs_conn_id='webhdfs_default', *args, **kwargs)[source]

Bases: airflow.sensors.base_sensor_operator.BaseSensorOperator

Waits for a file or folder to land in HDFS

poke(context)[source]

Function that the sensors defined while deriving this class should override.

Community-contributed Operators

Operators

class airflow.contrib.operators.awsbatch_operator.AWSBatchOperator(job_name, job_definition, job_queue, overrides, max_retries=4200, aws_conn_id=None, region_name=None, **kwargs)[source]

Bases: airflow.models.BaseOperator

Execute a job on AWS Batch Service

Parameters:
  • job_name (str) – the name for the job that will run on AWS Batch
  • job_definition (str) – the job definition name on AWS Batch
  • job_queue (str) – the queue name on AWS Batch
  • overrides (dict) – the same parameter that boto3 will receive on containerOverrides (templated). http://boto3.readthedocs.io/en/latest/reference/services/batch.html#submit_job
  • max_retries (int) – exponential backoff retries while waiter is not merged, 4200 = 48 hours
  • aws_conn_id (str) – connection id of AWS credentials / region name. If None, credential boto3 strategy will be used (http://boto3.readthedocs.io/en/latest/guide/configuration.html).
  • region_name (str) – region name to use in AWS Hook. Override the region_name in connection (if provided)
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.

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.

class airflow.contrib.operators.bigquery_check_operator.BigQueryCheckOperator(sql, bigquery_conn_id='bigquery_default', use_legacy_sql=True, *args, **kwargs)[source]

Bases: airflow.operators.check_operator.CheckOperator

Performs checks against BigQuery. The BigQueryCheckOperator expects a sql query that will return a single row. Each value on that first row is evaluated using python bool casting. If any of the values return False 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 alterts without stopping the progress of the DAG.

Parameters:
  • sql (string) – the sql to be executed
  • bigquery_conn_id (string) – reference to the BigQuery database
  • use_legacy_sql (boolean) – Whether to use legacy SQL (true) or standard SQL (false).
class airflow.contrib.operators.bigquery_check_operator.BigQueryValueCheckOperator(sql, pass_value, tolerance=None, bigquery_conn_id='bigquery_default', use_legacy_sql=True, *args, **kwargs)[source]

Bases: airflow.operators.check_operator.ValueCheckOperator

Performs a simple value check using sql code.

Parameters:
  • sql (string) – the sql to be executed
  • use_legacy_sql (boolean) – Whether to use legacy SQL (true) or standard SQL (false).
class airflow.contrib.operators.bigquery_check_operator.BigQueryIntervalCheckOperator(table, metrics_thresholds, date_filter_column='ds', days_back=-7, bigquery_conn_id='bigquery_default', use_legacy_sql=True, *args, **kwargs)[source]

Bases: airflow.operators.check_operator.IntervalCheckOperator

Checks that the values of metrics given as SQL expressions are within a certain tolerance of the ones from days_back before.

This method constructs a query like so

SELECT {metrics_threshold_dict_key} FROM {table}
    WHERE {date_filter_column}=<date>
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, for example ‘COUNT(*)’: 1.5 would require a 50 percent or less difference between the current day, and the prior days_back.
  • use_legacy_sql (boolean) – Whether to use legacy SQL (true) or standard SQL (false).
class airflow.contrib.operators.bigquery_get_data.BigQueryGetDataOperator(dataset_id, table_id, max_results='100', selected_fields=None, bigquery_conn_id='bigquery_default', delegate_to=None, *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

Fetches the data from a BigQuery table (alternatively fetch data for selected columns) and returns data in a python list. The number of elements in the returned list will be equal to the number of rows fetched. Each element in the list will again be a list where element would represent the columns values for that row.

Example Result: [['Tony', '10'], ['Mike', '20'], ['Steve', '15']]

Note

If you pass fields to selected_fields which are in different order than the order of columns already in BQ table, the data will still be in the order of BQ table. For example if the BQ table has 3 columns as [A,B,C] and you pass ‘B,A’ in the selected_fields the data would still be of the form 'A,B'.

Example:

get_data = BigQueryGetDataOperator(
    task_id='get_data_from_bq',
    dataset_id='test_dataset',
    table_id='Transaction_partitions',
    max_results='100',
    selected_fields='DATE',
    bigquery_conn_id='airflow-service-account'
)
Parameters:
  • dataset_id – The dataset ID of the requested table. (templated)
  • table_id (string) – The table ID of the requested table. (templated)
  • max_results (string) – The maximum number of records (rows) to be fetched from the table. (templated)
  • selected_fields (string) – List of fields to return (comma-separated). If unspecified, all fields are returned.
  • bigquery_conn_id (string) – reference to a specific BigQuery hook.
  • delegate_to (string) – The account to impersonate, if any. For this to work, the service account making the request must have domain-wide delegation enabled.
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.

class airflow.contrib.operators.bigquery_operator.BigQueryCreateEmptyTableOperator(dataset_id, table_id, project_id=None, schema_fields=None, gcs_schema_object=None, time_partitioning={}, bigquery_conn_id='bigquery_default', google_cloud_storage_conn_id='google_cloud_default', delegate_to=None, labels=None, *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

Creates a new, empty table in the specified BigQuery dataset, optionally with schema.

The schema to be used for the BigQuery table may be specified in one of two ways. You may either directly pass the schema fields in, or you may point the operator to a Google cloud storage object name. The object in Google cloud storage must be a JSON file with the schema fields in it. You can also create a table without schema.

Parameters:
  • project_id (string) – The project to create the table into. (templated)
  • dataset_id (string) – The dataset to create the table into. (templated)
  • table_id (string) – The Name of the table to be created. (templated)
  • schema_fields (list) –

    If set, the schema field list as defined here: https://cloud.google.com/bigquery/docs/reference/rest/v2/jobs#configuration.load.schema

    Example:

    schema_fields=[{"name": "emp_name", "type": "STRING", "mode": "REQUIRED"},
                   {"name": "salary", "type": "INTEGER", "mode": "NULLABLE"}]
    
  • gcs_schema_object (string) – Full path to the JSON file containing schema (templated). For example: gs://test-bucket/dir1/dir2/employee_schema.json
  • time_partitioning (dict) –

    configure optional time partitioning fields i.e. partition by field, type and expiration as per API specifications.

  • bigquery_conn_id (string) – Reference to a specific BigQuery hook.
  • google_cloud_storage_conn_id (string) – Reference to a specific Google cloud storage hook.
  • delegate_to (string) – The account to impersonate, if any. For this to work, the service account making the request must have domain-wide delegation enabled.
  • labels (dict) –

    a dictionary containing labels for the table, passed to BigQuery

    Example (with schema JSON in GCS):

    CreateTable = BigQueryCreateEmptyTableOperator(
        task_id='BigQueryCreateEmptyTableOperator_task',
        dataset_id='ODS',
        table_id='Employees',
        project_id='internal-gcp-project',
        gcs_schema_object='gs://schema-bucket/employee_schema.json',
        bigquery_conn_id='airflow-service-account',
        google_cloud_storage_conn_id='airflow-service-account'
    )
    

    Corresponding Schema file (employee_schema.json):

    [
      {
        "mode": "NULLABLE",
        "name": "emp_name",
        "type": "STRING"
      },
      {
        "mode": "REQUIRED",
        "name": "salary",
        "type": "INTEGER"
      }
    ]
    

    Example (with schema in the DAG):

    CreateTable = BigQueryCreateEmptyTableOperator(
        task_id='BigQueryCreateEmptyTableOperator_task',
        dataset_id='ODS',
        table_id='Employees',
        project_id='internal-gcp-project',
        schema_fields=[{"name": "emp_name", "type": "STRING", "mode": "REQUIRED"},
                       {"name": "salary", "type": "INTEGER", "mode": "NULLABLE"}],
        bigquery_conn_id='airflow-service-account',
        google_cloud_storage_conn_id='airflow-service-account'
    )
    
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.

class airflow.contrib.operators.bigquery_operator.BigQueryCreateExternalTableOperator(bucket, source_objects, destination_project_dataset_table, schema_fields=None, schema_object=None, source_format='CSV', compression='NONE', skip_leading_rows=0, field_delimiter=', ', max_bad_records=0, quote_character=None, allow_quoted_newlines=False, allow_jagged_rows=False, bigquery_conn_id='bigquery_default', google_cloud_storage_conn_id='google_cloud_default', delegate_to=None, src_fmt_configs={}, labels=None, *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

Creates a new external table in the dataset with the data in Google Cloud Storage.

The schema to be used for the BigQuery table may be specified in one of two ways. You may either directly pass the schema fields in, or you may point the operator to a Google cloud storage object name. The object in Google cloud storage must be a JSON file with the schema fields in it.

Parameters:
  • bucket (string) – The bucket to point the external table to. (templated)
  • source_objects – List of Google cloud storage URIs to point table to. (templated) If source_format is ‘DATASTORE_BACKUP’, the list must only contain a single URI.
  • destination_project_dataset_table (string) – The dotted (<project>.)<dataset>.<table> BigQuery table to load data into (templated). If <project> is not included, project will be the project defined in the connection json.
  • schema_fields (list) –

    If set, the schema field list as defined here: https://cloud.google.com/bigquery/docs/reference/rest/v2/jobs#configuration.load.schema

    Example:

    schema_fields=[{"name": "emp_name", "type": "STRING", "mode": "REQUIRED"},
                   {"name": "salary", "type": "INTEGER", "mode": "NULLABLE"}]
    

    Should not be set when source_format is ‘DATASTORE_BACKUP’.

  • schema_object – If set, a GCS object path pointing to a .json file that contains the schema for the table. (templated)
  • schema_object – string
  • source_format (string) – File format of the data.
  • compression (string) – [Optional] The compression type of the data source. Possible values include GZIP and NONE. The default value is NONE. This setting is ignored for Google Cloud Bigtable, Google Cloud Datastore backups and Avro formats.
  • skip_leading_rows (int) – Number of rows to skip when loading from a CSV.
  • field_delimiter (string) – The delimiter to use for the CSV.
  • max_bad_records (int) – The maximum number of bad records that BigQuery can ignore when running the job.
  • quote_character (string) – The value that is used to quote data sections in a CSV file.
  • allow_quoted_newlines (boolean) – Whether to allow quoted newlines (true) or not (false).
  • allow_jagged_rows (bool) – Accept rows that are missing trailing optional columns. The missing values are treated as nulls. If false, records with missing trailing columns are treated as bad records, and if there are too many bad records, an invalid error is returned in the job result. Only applicable to CSV, ignored for other formats.
  • bigquery_conn_id (string) – Reference to a specific BigQuery hook.
  • google_cloud_storage_conn_id (string) – Reference to a specific Google cloud storage hook.
  • delegate_to (string) – The account to impersonate, if any. For this to work, the service account making the request must have domain-wide delegation enabled.
  • src_fmt_configs (dict) – configure optional fields specific to the source format

:param labels a dictionary containing labels for the table, passed to BigQuery :type labels: dict

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.

class airflow.contrib.operators.bigquery_operator.BigQueryDeleteDatasetOperator(dataset_id, project_id=None, bigquery_conn_id='bigquery_default', delegate_to=None, *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

” This operator deletes an existing dataset from your Project in Big query. https://cloud.google.com/bigquery/docs/reference/rest/v2/datasets/delete :param project_id: The project id of the dataset. :type project_id: string :param dataset_id: The dataset to be deleted. :type dataset_id: string

Example:

delete_temp_data = BigQueryDeleteDatasetOperator(
                                dataset_id = 'temp-dataset',
                                project_id = 'temp-project',
                                bigquery_conn_id='_my_gcp_conn_',
                                task_id='Deletetemp',
                                dag=dag)
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.

class airflow.contrib.operators.bigquery_operator.BigQueryOperator(bql=None, sql=None, destination_dataset_table=False, write_disposition='WRITE_EMPTY', allow_large_results=False, flatten_results=None, bigquery_conn_id='bigquery_default', delegate_to=None, udf_config=False, use_legacy_sql=True, maximum_billing_tier=None, maximum_bytes_billed=None, create_disposition='CREATE_IF_NEEDED', schema_update_options=(), query_params=None, labels=None, priority='INTERACTIVE', time_partitioning={}, *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

Executes BigQuery SQL queries in a specific BigQuery database

Parameters:
  • bql (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'.) – (Deprecated. Use sql parameter instead) the sql code to be executed (templated)
  • 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 (templated)
  • destination_dataset_table (string) – A dotted (<project>.|<project>:)<dataset>.<table> that, if set, will store the results of the query. (templated)
  • write_disposition (string) – Specifies the action that occurs if the destination table already exists. (default: ‘WRITE_EMPTY’)
  • create_disposition (string) – Specifies whether the job is allowed to create new tables. (default: ‘CREATE_IF_NEEDED’)
  • allow_large_results (boolean) – Whether to allow large results.
  • flatten_results (boolean) – If true and query uses legacy SQL dialect, flattens all nested and repeated fields in the query results. allow_large_results must be true if this is set to false. For standard SQL queries, this flag is ignored and results are never flattened.
  • bigquery_conn_id (string) – reference to a specific BigQuery hook.
  • delegate_to (string) – The account to impersonate, if any. For this to work, the service account making the request must have domain-wide delegation enabled.
  • udf_config (list) – The User Defined Function configuration for the query. See https://cloud.google.com/bigquery/user-defined-functions for details.
  • use_legacy_sql (boolean) – Whether to use legacy SQL (true) or standard SQL (false).
  • maximum_billing_tier (integer) – Positive integer that serves as a multiplier of the basic price. Defaults to None, in which case it uses the value set in the project.
  • maximum_bytes_billed (float) – Limits the bytes billed for this job. Queries that will have bytes billed beyond this limit will fail (without incurring a charge). If unspecified, this will be set to your project default.
  • schema_update_options (tuple) – Allows the schema of the destination table to be updated as a side effect of the load job.
  • query_params (dict) – a dictionary containing query parameter types and values, passed to BigQuery.
  • labels (dict) – a dictionary containing labels for the job/query, passed to BigQuery
  • priority (string) – Specifies a priority for the query. Possible values include INTERACTIVE and BATCH. The default value is INTERACTIVE.
  • time_partitioning (dict) – configure optional time partitioning fields i.e. partition by field, type and expiration as per API specifications.
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.

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.

class airflow.contrib.operators.bigquery_table_delete_operator.BigQueryTableDeleteOperator(deletion_dataset_table, bigquery_conn_id='bigquery_default', delegate_to=None, ignore_if_missing=False, *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

Deletes BigQuery tables

Parameters:
  • deletion_dataset_table (string) – A dotted (<project>.|<project>:)<dataset>.<table> that indicates which table will be deleted. (templated)
  • bigquery_conn_id (string) – reference to a specific BigQuery hook.
  • delegate_to (string) – The account to impersonate, if any. For this to work, the service account making the request must have domain-wide delegation enabled.
  • ignore_if_missing (boolean) – if True, then return success even if the requested table does not exist.
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.

class airflow.contrib.operators.bigquery_to_bigquery.BigQueryToBigQueryOperator(source_project_dataset_tables, destination_project_dataset_table, write_disposition='WRITE_EMPTY', create_disposition='CREATE_IF_NEEDED', bigquery_conn_id='bigquery_default', delegate_to=None, labels=None, *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

Copies data from one BigQuery table to another.

See also

For more details about these parameters: https://cloud.google.com/bigquery/docs/reference/v2/jobs#configuration.copy

Parameters:
  • source_project_dataset_tables (list|string) – One or more dotted (project:|project.)<dataset>.<table> BigQuery tables to use as the source data. If <project> is not included, project will be the project defined in the connection json. Use a list if there are multiple source tables. (templated)
  • destination_project_dataset_table (string) – The destination BigQuery table. Format is: (project:|project.)<dataset>.<table> (templated)
  • write_disposition (string) – The write disposition if the table already exists.
  • create_disposition (string) – The create disposition if the table doesn’t exist.
  • bigquery_conn_id (string) – reference to a specific BigQuery hook.
  • delegate_to (string) – The account to impersonate, if any. For this to work, the service account making the request must have domain-wide delegation enabled.
  • labels (dict) – a dictionary containing labels for the job/query, passed to BigQuery
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.

class airflow.contrib.operators.bigquery_to_gcs.BigQueryToCloudStorageOperator(source_project_dataset_table, destination_cloud_storage_uris, compression='NONE', export_format='CSV', field_delimiter=', ', print_header=True, bigquery_conn_id='bigquery_default', delegate_to=None, labels=None, *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

Transfers a BigQuery table to a Google Cloud Storage bucket.

See also

For more details about these parameters: https://cloud.google.com/bigquery/docs/reference/v2/jobs

Parameters:
  • source_project_dataset_table (string) – The dotted (<project>.|<project>:)<dataset>.<table> BigQuery table to use as the source data. If <project> is not included, project will be the project defined in the connection json. (templated)
  • destination_cloud_storage_uris (list) – The destination Google Cloud Storage URI (e.g. gs://some-bucket/some-file.txt). (templated) Follows convention defined here: https://cloud.google.com/bigquery/exporting-data-from-bigquery#exportingmultiple
  • compression (string) – Type of compression to use.
  • export_format – File format to export.
  • field_delimiter (string) – The delimiter to use when extracting to a CSV.
  • print_header (boolean) – Whether to print a header for a CSV file extract.
  • bigquery_conn_id (string) – reference to a specific BigQuery hook.
  • delegate_to (string) – The account to impersonate, if any. For this to work, the service account making the request must have domain-wide delegation enabled.
  • labels (dict) – a dictionary containing labels for the job/query, passed to BigQuery
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.

class airflow.contrib.operators.cassandra_to_gcs.CassandraToGoogleCloudStorageOperator(cql, bucket, filename, schema_filename=None, approx_max_file_size_bytes=1900000000, cassandra_conn_id='cassandra_default', google_cloud_storage_conn_id='google_cloud_default', delegate_to=None, *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

Copy data from Cassandra to Google cloud storage in JSON format

Note: Arrays of arrays are not supported.

classmethod convert_map_type(name, value)[source]

Converts a map to a repeated RECORD that contains two fields: ‘key’ and ‘value’, each will be converted to its corresopnding data type in BQ.

classmethod convert_tuple_type(name, value)[source]

Converts a tuple to RECORD that contains n fields, each will be converted to its corresponding data type in bq and will be named ‘field_<index>’, where index is determined by the order of the tuple elments defined in cassandra.

classmethod convert_user_type(name, value)[source]

Converts a user type to RECORD that contains n fields, where n is the number of attributes. Each element in the user type class will be converted to its corresponding data type in BQ.

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.

class airflow.contrib.operators.databricks_operator.DatabricksSubmitRunOperator(json=None, spark_jar_task=None, notebook_task=None, new_cluster=None, existing_cluster_id=None, libraries=None, run_name=None, timeout_seconds=None, databricks_conn_id='databricks_default', polling_period_seconds=30, databricks_retry_limit=3, databricks_retry_delay=1, do_xcom_push=False, **kwargs)[source]

Bases: airflow.models.BaseOperator

Submits an Spark job run to Databricks using the api/2.0/jobs/runs/submit API endpoint.

There are two ways to instantiate this operator.

In the first way, you can take the JSON payload that you typically use to call the api/2.0/jobs/runs/submit endpoint and pass it directly to our DatabricksSubmitRunOperator through the json parameter. For example

json = {
  'new_cluster': {
    'spark_version': '2.1.0-db3-scala2.11',
    'num_workers': 2
  },
  'notebook_task': {
    'notebook_path': '/Users/airflow@example.com/PrepareData',
  },
}
notebook_run = DatabricksSubmitRunOperator(task_id='notebook_run', json=json)

Another way to accomplish the same thing is to use the named parameters of the DatabricksSubmitRunOperator directly. Note that there is exactly one named parameter for each top level parameter in the runs/submit endpoint. In this method, your code would look like this:

new_cluster = {
  'spark_version': '2.1.0-db3-scala2.11',
  'num_workers': 2
}
notebook_task = {
  'notebook_path': '/Users/airflow@example.com/PrepareData',
}
notebook_run = DatabricksSubmitRunOperator(
    task_id='notebook_run',
    new_cluster=new_cluster,
    notebook_task=notebook_task)

In the case where both the json parameter AND the named parameters are provided, they will be merged together. If there are conflicts during the merge, the named parameters will take precedence and override the top level json keys.

Currently the named parameters that DatabricksSubmitRunOperator supports are
  • spark_jar_task
  • notebook_task
  • new_cluster
  • existing_cluster_id
  • libraries
  • run_name
  • timeout_seconds
Parameters:
  • json (dict) –

    A JSON object containing API parameters which will be passed directly to the api/2.0/jobs/runs/submit endpoint. The other named parameters (i.e. spark_jar_task, notebook_task..) to this operator will be merged with this json dictionary if they are provided. If there are conflicts during the merge, the named parameters will take precedence and override the top level json keys. (templated)

    See also

    For more information about templating see Jinja Templating. https://docs.databricks.com/api/latest/jobs.html#runs-submit

  • spark_jar_task (dict) –

    The main class and parameters for the JAR task. Note that the actual JAR is specified in the libraries. EITHER spark_jar_task OR notebook_task should be specified. This field will be templated.

  • notebook_task (dict) –

    The notebook path and parameters for the notebook task. EITHER spark_jar_task OR notebook_task should be specified. This field will be templated.

  • new_cluster (dict) –

    Specs for a new cluster on which this task will be run. EITHER new_cluster OR existing_cluster_id should be specified. This field will be templated.

  • existing_cluster_id (string) – ID for existing cluster on which to run this task. EITHER new_cluster OR existing_cluster_id should be specified. This field will be templated.
  • libraries (list of dicts) –

    Libraries which this run will use. This field will be templated.

  • run_name (string) – The run name used for this task. By default this will be set to the Airflow task_id. This task_id is a required parameter of the superclass BaseOperator. This field will be templated.
  • timeout_seconds (int32) – The timeout for this run. By default a value of 0 is used which means to have no timeout. This field will be templated.
  • databricks_conn_id (string) – The name of the Airflow connection to use. By default and in the common case this will be databricks_default. To use token based authentication, provide the key token in the extra field for the connection.
  • polling_period_seconds (int) – Controls the rate which we poll for the result of this run. By default the operator will poll every 30 seconds.
  • databricks_retry_limit (int) – Amount of times retry if the Databricks backend is unreachable. Its value must be greater than or equal to 1.
  • databricks_retry_delay (float) – Number of seconds to wait between retries (it might be a floating point number).
  • do_xcom_push (boolean) – Whether we should push run_id and run_page_url to xcom.
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.

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.

class airflow.contrib.operators.dataflow_operator.DataFlowJavaOperator(jar, dataflow_default_options=None, options=None, gcp_conn_id='google_cloud_default', delegate_to=None, poll_sleep=10, job_class=None, *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

Start a Java Cloud DataFlow batch job. The parameters of the operation will be passed to the job.

It’s a good practice to define dataflow_* parameters in the default_args of the dag like the project, zone and staging location.

default_args = {
    'dataflow_default_options': {
        'project': 'my-gcp-project',
        'zone': 'europe-west1-d',
        'stagingLocation': 'gs://my-staging-bucket/staging/'
    }
}

You need to pass the path to your dataflow as a file reference with the jar parameter, the jar needs to be a self executing jar (see documentation here: https://beam.apache.org/documentation/runners/dataflow/#self-executing-jar). Use options to pass on options to your job.

t1 = DataFlowOperation(
    task_id='datapflow_example',
    jar='{{var.value.gcp_dataflow_base}}pipeline/build/libs/pipeline-example-1.0.jar',
    options={
        'autoscalingAlgorithm': 'BASIC',
        'maxNumWorkers': '50',
        'start': '{{ds}}',
        'partitionType': 'DAY',
        'labels': {'foo' : 'bar'}
    },
    gcp_conn_id='gcp-airflow-service-account',
    dag=my-dag)

Both jar and options are templated so you can use variables in them.

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.

class airflow.contrib.operators.dataflow_operator.DataflowTemplateOperator(template, dataflow_default_options=None, parameters=None, gcp_conn_id='google_cloud_default', delegate_to=None, poll_sleep=10, *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

Start a Templated Cloud DataFlow batch job. The parameters of the operation will be passed to the job. It’s a good practice to define dataflow_* parameters in the default_args of the dag like the project, zone and staging location.

default_args = {
    'dataflow_default_options': {
        'project': 'my-gcp-project'
        'zone': 'europe-west1-d',
        'tempLocation': 'gs://my-staging-bucket/staging/'
        }
    }
}

You need to pass the path to your dataflow template as a file reference with the template parameter. Use parameters to pass on parameters to your job. Use environment to pass on runtime environment variables to your job.

t1 = DataflowTemplateOperator(
    task_id='datapflow_example',
    template='{{var.value.gcp_dataflow_base}}',
    parameters={
        'inputFile': "gs://bucket/input/my_input.txt",
        'outputFile': "gs://bucket/output/my_output.txt"
    },
    gcp_conn_id='gcp-airflow-service-account',
    dag=my-dag)

template, dataflow_default_options and parameters are templated so you can use variables in them.

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.

class airflow.contrib.operators.dataflow_operator.DataFlowPythonOperator(py_file, py_options=None, dataflow_default_options=None, options=None, gcp_conn_id='google_cloud_default', delegate_to=None, poll_sleep=10, *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

Create a new DataFlowPythonOperator. Note that both dataflow_default_options and options will be merged to specify pipeline execution parameter, and dataflow_default_options is expected to save high-level options, for instances, project and zone information, which apply to all dataflow operators in the DAG.

See also

For more detail on job submission have a look at the reference: https://cloud.google.com/dataflow/pipelines/specifying-exec-params

Parameters:
  • py_file (string) – Reference to the python dataflow pipleline file.py, e.g., /some/local/file/path/to/your/python/pipeline/file.
  • py_options – Additional python options.
  • dataflow_default_options (dict) – Map of default job options.
  • options (dict) – Map of job specific options.
  • gcp_conn_id (string) – The connection ID to use connecting to Google Cloud Platform.
  • delegate_to (string) – The account to impersonate, if any. For this to work, the service account making the request must have domain-wide delegation enabled.
  • poll_sleep (int) – The time in seconds to sleep between polling Google Cloud Platform for the dataflow job status while the job is in the JOB_STATE_RUNNING state.
execute(context)[source]

Execute the python dataflow job.

class airflow.contrib.operators.dataproc_operator.DataprocClusterCreateOperator(cluster_name, project_id, num_workers, zone, network_uri=None, subnetwork_uri=None, internal_ip_only=None, tags=None, storage_bucket=None, init_actions_uris=None, init_action_timeout='10m', metadata=None, custom_image=None, image_version=None, properties=None, master_machine_type='n1-standard-4', master_disk_type='pd-standard', master_disk_size=500, worker_machine_type='n1-standard-4', worker_disk_type='pd-standard', worker_disk_size=500, num_preemptible_workers=0, labels=None, region='global', gcp_conn_id='google_cloud_default', delegate_to=None, service_account=None, service_account_scopes=None, idle_delete_ttl=None, auto_delete_time=None, auto_delete_ttl=None, *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

Create a new cluster on Google Cloud Dataproc. The operator will wait until the creation is successful or an error occurs in the creation process.

The parameters allow to configure the cluster. Please refer to

https://cloud.google.com/dataproc/docs/reference/rest/v1/projects.regions.clusters

for a detailed explanation on the different parameters. Most of the configuration parameters detailed in the link are available as a parameter to this operator.

Parameters:
  • cluster_name (string) – The name of the DataProc cluster to create. (templated)
  • project_id (str) – The ID of the google cloud project in which to create the cluster. (templated)
  • num_workers (int) – The # of workers to spin up. If set to zero will spin up cluster in a single node mode
  • storage_bucket (string) – The storage bucket to use, setting to None lets dataproc generate a custom one for you
  • init_actions_uris (list[string]) – List of GCS uri’s containing dataproc initialization scripts
  • init_action_timeout (string) – Amount of time executable scripts in init_actions_uris has to complete
  • metadata (dict) – dict of key-value google compute engine metadata entries to add to all instances
  • image_version (string) – the version of software inside the Dataproc cluster
  • custom_image – custom Dataproc image for more info see https://cloud.google.com/dataproc/docs/guides/dataproc-images
  • properties (dict) – dict of properties to set on config files (e.g. spark-defaults.conf), see https://cloud.google.com/dataproc/docs/reference/rest/v1/ projects.regions.clusters#SoftwareConfig
  • master_machine_type (string) – Compute engine machine type to use for the master node
  • master_disk_type (string) – Type of the boot disk for the master node (default is pd-standard). Valid values: pd-ssd (Persistent Disk Solid State Drive) or pd-standard (Persistent Disk Hard Disk Drive).
  • master_disk_size (int) – Disk size for the master node
  • worker_machine_type (string) – Compute engine machine type to use for the worker nodes
  • worker_disk_type (string) – Type of the boot disk for the worker node (default is pd-standard). Valid values: pd-ssd (Persistent Disk Solid State Drive) or pd-standard (Persistent Disk Hard Disk Drive).
  • worker_disk_size (int) – Disk size for the worker nodes
  • num_preemptible_workers (int) – The # of preemptible worker nodes to spin up
  • labels (dict) – dict of labels to add to the cluster
  • zone (string) – The zone where the cluster will be located. (templated)
  • network_uri (string) – The network uri to be used for machine communication, cannot be specified with subnetwork_uri
  • subnetwork_uri (string) – The subnetwork uri to be used for machine communication, cannot be specified with network_uri
  • internal_ip_only (bool) – If true, all instances in the cluster will only have internal IP addresses. This can only be enabled for subnetwork enabled networks
  • tags (list[string]) – The GCE tags to add to all instances
  • region – leave as ‘global’, might become relevant in the future. (templated)
  • gcp_conn_id (string) – The connection ID to use connecting to Google Cloud Platform.
  • delegate_to (string) – The account to impersonate, if any. For this to work, the service account making the request must have domain-wide delegation enabled.
  • service_account (string) – The service account of the dataproc instances.
  • service_account_scopes (list[string]) – The URIs of service account scopes to be included.
  • idle_delete_ttl (int) – The longest duration that cluster would keep alive while staying idle. Passing this threshold will cause cluster to be auto-deleted. A duration in seconds.
  • auto_delete_time (datetime.datetime) – The time when cluster will be auto-deleted.
  • auto_delete_ttl (int) – The life duration of cluster, the cluster will be auto-deleted at the end of this duration. A duration in seconds. (If auto_delete_time is set this parameter will be ignored)
Type:

custom_image: string

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.

class airflow.contrib.operators.dataproc_operator.DataprocClusterScaleOperator(cluster_name, project_id, region='global', gcp_conn_id='google_cloud_default', delegate_to=None, num_workers=2, num_preemptible_workers=0, graceful_decommission_timeout=None, *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

Scale, up or down, a cluster on Google Cloud Dataproc. The operator will wait until the cluster is re-scaled.

Example:

t1 = DataprocClusterScaleOperator(
        task_id='dataproc_scale',
        project_id='my-project',
        cluster_name='cluster-1',
        num_workers=10,
        num_preemptible_workers=10,
        graceful_decommission_timeout='1h',
        dag=dag)

See also

For more detail on about scaling clusters have a look at the reference: https://cloud.google.com/dataproc/docs/concepts/configuring-clusters/scaling-clusters

Parameters:
  • cluster_name (string) – The name of the cluster to scale. (templated)
  • project_id (string) – The ID of the google cloud project in which the cluster runs. (templated)
  • region (string) – The region for the dataproc cluster. (templated)
  • gcp_conn_id (string) – The connection ID to use connecting to Google Cloud Platform.
  • num_workers (int) – The new number of workers
  • num_preemptible_workers (int) – The new number of preemptible workers
  • graceful_decommission_timeout (string) – Timeout for graceful YARN decomissioning. Maximum value is 1d
  • delegate_to (string) – The account to impersonate, if any. For this to work, the service account making the request must have domain-wide delegation enabled.
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.

class airflow.contrib.operators.dataproc_operator.DataprocClusterDeleteOperator(cluster_name, project_id, region='global', gcp_conn_id='google_cloud_default', delegate_to=None, *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

Delete a cluster on Google Cloud Dataproc. The operator will wait until the cluster is destroyed.

Parameters:
  • cluster_name (string) – The name of the cluster to create. (templated)
  • project_id (string) – The ID of the google cloud project in which the cluster runs. (templated)
  • region (string) – leave as ‘global’, might become relevant in the future. (templated)
  • gcp_conn_id (string) – The connection ID to use connecting to Google Cloud Platform.
  • delegate_to (string) – The account to impersonate, if any. For this to work, the service account making the request must have domain-wide delegation enabled.
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.

class airflow.contrib.operators.dataproc_operator.DataProcPigOperator(query=None, query_uri=None, variables=None, job_name='{{task.task_id}}_{{ds_nodash}}', cluster_name='cluster-1', dataproc_pig_properties=None, dataproc_pig_jars=None, gcp_conn_id='google_cloud_default', delegate_to=None, region='global', *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

Start a Pig query Job on a Cloud DataProc cluster. The parameters of the operation will be passed to the cluster.

It’s a good practice to define dataproc_* parameters in the default_args of the dag like the cluster name and UDFs.

default_args = {
    'cluster_name': 'cluster-1',
    'dataproc_pig_jars': [
        'gs://example/udf/jar/datafu/1.2.0/datafu.jar',
        'gs://example/udf/jar/gpig/1.2/gpig.jar'
    ]
}

You can pass a pig script as string or file reference. Use variables to pass on variables for the pig script to be resolved on the cluster or use the parameters to be resolved in the script as template parameters.

Example:

t1 = DataProcPigOperator(
        task_id='dataproc_pig',
        query='a_pig_script.pig',
        variables={'out': 'gs://example/output/{{ds}}'},
        dag=dag)

See also

For more detail on about job submission have a look at the reference: https://cloud.google.com/dataproc/reference/rest/v1/projects.regions.jobs

Parameters:
  • query (string) – The query or reference to the query file (pg or pig extension). (templated)
  • query_uri (string) – The uri of a pig script on Cloud Storage.
  • variables (dict) – Map of named parameters for the query. (templated)
  • job_name (string) – The job name used in the DataProc cluster. This name by default is the task_id appended with the execution data, but can be templated. The name will always be appended with a random number to avoid name clashes. (templated)
  • cluster_name (string) – The name of the DataProc cluster. (templated)
  • dataproc_pig_properties (dict) – Map for the Pig properties. Ideal to put in default arguments
  • dataproc_pig_jars (list) – URIs to jars provisioned in Cloud Storage (example: for UDFs and libs) and are ideal to put in default arguments.
  • gcp_conn_id (string) – The connection ID to use connecting to Google Cloud Platform.
  • delegate_to (string) – The account to impersonate, if any. For this to work, the service account making the request must have domain-wide delegation enabled.
  • region (string) – The specified region where the dataproc cluster is created.
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.

class airflow.contrib.operators.dataproc_operator.DataProcHiveOperator(query=None, query_uri=None, variables=None, job_name='{{task.task_id}}_{{ds_nodash}}', cluster_name='cluster-1', dataproc_hive_properties=None, dataproc_hive_jars=None, gcp_conn_id='google_cloud_default', delegate_to=None, region='global', *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

Start a Hive query Job on a Cloud DataProc cluster.

Parameters:
  • query (string) – The query or reference to the query file (q extension).
  • query_uri (string) – The uri of a hive script on Cloud Storage.
  • variables (dict) – Map of named parameters for the query.
  • job_name (string) – The job name used in the DataProc cluster. This name by default is the task_id appended with the execution data, but can be templated. The name will always be appended with a random number to avoid name clashes.
  • cluster_name (string) – The name of the DataProc cluster.
  • dataproc_hive_properties (dict) – Map for the Pig properties. Ideal to put in default arguments
  • dataproc_hive_jars (list) – URIs to jars provisioned in Cloud Storage (example: for UDFs and libs) and are ideal to put in default arguments.
  • gcp_conn_id (string) – The connection ID to use connecting to Google Cloud Platform.
  • delegate_to (string) – The account to impersonate, if any. For this to work, the service account making the request must have domain-wide delegation enabled.
  • region (string) – The specified region where the dataproc cluster is created.
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.

class airflow.contrib.operators.dataproc_operator.DataProcSparkSqlOperator(query=None, query_uri=None, variables=None, job_name='{{task.task_id}}_{{ds_nodash}}', cluster_name='cluster-1', dataproc_spark_properties=None, dataproc_spark_jars=None, gcp_conn_id='google_cloud_default', delegate_to=None, region='global', *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

Start a Spark SQL query Job on a Cloud DataProc cluster.

Parameters:
  • query (string) – The query or reference to the query file (q extension). (templated)
  • query_uri (string) – The uri of a spark sql script on Cloud Storage.
  • variables (dict) – Map of named parameters for the query. (templated)
  • job_name (string) – The job name used in the DataProc cluster. This name by default is the task_id appended with the execution data, but can be templated. The name will always be appended with a random number to avoid name clashes. (templated)
  • cluster_name (string) – The name of the DataProc cluster. (templated)
  • dataproc_spark_properties (dict) – Map for the Pig properties. Ideal to put in default arguments
  • dataproc_spark_jars (list) – URIs to jars provisioned in Cloud Storage (example: for UDFs and libs) and are ideal to put in default arguments.
  • gcp_conn_id (string) – The connection ID to use connecting to Google Cloud Platform.
  • delegate_to (string) – The account to impersonate, if any. For this to work, the service account making the request must have domain-wide delegation enabled.
  • region (string) – The specified region where the dataproc cluster is created.
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.

class airflow.contrib.operators.dataproc_operator.DataProcSparkOperator(main_jar=None, main_class=None, arguments=None, archives=None, files=None, job_name='{{task.task_id}}_{{ds_nodash}}', cluster_name='cluster-1', dataproc_spark_properties=None, dataproc_spark_jars=None, gcp_conn_id='google_cloud_default', delegate_to=None, region='global', *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

Start a Spark Job on a Cloud DataProc cluster.

Parameters:
  • main_jar (string) – URI of the job jar provisioned on Cloud Storage. (use this or the main_class, not both together).
  • main_class (string) – Name of the job class. (use this or the main_jar, not both together).
  • arguments (list) – Arguments for the job. (templated)
  • archives (list) – List of archived files that will be unpacked in the work directory. Should be stored in Cloud Storage.
  • files (list) – List of files to be copied to the working directory
  • job_name (string) – The job name used in the DataProc cluster. This name by default is the task_id appended with the execution data, but can be templated. The name will always be appended with a random number to avoid name clashes. (templated)
  • cluster_name (string) – The name of the DataProc cluster. (templated)
  • dataproc_spark_properties (dict) – Map for the Pig properties. Ideal to put in default arguments
  • dataproc_spark_jars (list) – URIs to jars provisioned in Cloud Storage (example: for UDFs and libs) and are ideal to put in default arguments.
  • gcp_conn_id (string) – The connection ID to use connecting to Google Cloud Platform.
  • delegate_to (string) – The account to impersonate, if any. For this to work, the service account making the request must have domain-wide delegation enabled.
  • region (string) – The specified region where the dataproc cluster is created.
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.

class airflow.contrib.operators.dataproc_operator.DataProcHadoopOperator(main_jar=None, main_class=None, arguments=None, archives=None, files=None, job_name='{{task.task_id}}_{{ds_nodash}}', cluster_name='cluster-1', dataproc_hadoop_properties=None, dataproc_hadoop_jars=None, gcp_conn_id='google_cloud_default', delegate_to=None, region='global', *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

Start a Hadoop Job on a Cloud DataProc cluster.

Parameters:
  • main_jar (string) – URI of the job jar provisioned on Cloud Storage. (use this or the main_class, not both together).
  • main_class (string) – Name of the job class. (use this or the main_jar, not both together).
  • arguments (list) – Arguments for the job. (templated)
  • archives (list) – List of archived files that will be unpacked in the work directory. Should be stored in Cloud Storage.
  • files (list) – List of files to be copied to the working directory
  • job_name (string) – The job name used in the DataProc cluster. This name by default is the task_id appended with the execution data, but can be templated. The name will always be appended with a random number to avoid name clashes. (templated)
  • cluster_name (string) – The name of the DataProc cluster. (templated)
  • dataproc_hadoop_properties (dict) – Map for the Pig properties. Ideal to put in default arguments
  • dataproc_hadoop_jars (list) – URIs to jars provisioned in Cloud Storage (example: for UDFs and libs) and are ideal to put in default arguments.
  • gcp_conn_id (string) – The connection ID to use connecting to Google Cloud Platform.
  • delegate_to (string) – The account to impersonate, if any. For this to work, the service account making the request must have domain-wide delegation enabled.
  • region (string) – The specified region where the dataproc cluster is created.
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.

class airflow.contrib.operators.dataproc_operator.DataProcPySparkOperator(main, arguments=None, archives=None, pyfiles=None, files=None, job_name='{{task.task_id}}_{{ds_nodash}}', cluster_name='cluster-1', dataproc_pyspark_properties=None, dataproc_pyspark_jars=None, gcp_conn_id='google_cloud_default', delegate_to=None, region='global', *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

Start a PySpark Job on a Cloud DataProc cluster.

Parameters:
  • main (string) – [Required] The Hadoop Compatible Filesystem (HCFS) URI of the main Python file to use as the driver. Must be a .py file.
  • arguments (list) – Arguments for the job. (templated)
  • archives (list) – List of archived files that will be unpacked in the work directory. Should be stored in Cloud Storage.
  • files (list) – List of files to be copied to the working directory
  • pyfiles (list) – List of Python files to pass to the PySpark framework. Supported file types: .py, .egg, and .zip
  • job_name (string) – The job name used in the DataProc cluster. This name by default is the task_id appended with the execution data, but can be templated. The name will always be appended with a random number to avoid name clashes. (templated)
  • cluster_name (string) – The name of the DataProc cluster.
  • dataproc_pyspark_properties (dict) – Map for the Pig properties. Ideal to put in default arguments
  • dataproc_pyspark_jars (list) – URIs to jars provisioned in Cloud Storage (example: for UDFs and libs) and are ideal to put in default arguments.
  • gcp_conn_id (string) – The connection ID to use connecting to Google Cloud Platform.
  • delegate_to (string) – The account to impersonate, if any. For this to work, the service account making the request must have domain-wide delegation enabled.
  • region (string) – The specified region where the dataproc cluster is created.
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.

class airflow.contrib.operators.dataproc_operator.DataprocWorkflowTemplateBaseOperator(project_id, region='global', gcp_conn_id='google_cloud_default', delegate_to=None, *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

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.

class airflow.contrib.operators.dataproc_operator.DataprocWorkflowTemplateInstantiateOperator(template_id, *args, **kwargs)[source]

Bases: airflow.contrib.operators.dataproc_operator.DataprocWorkflowTemplateBaseOperator

Instantiate a WorkflowTemplate on Google Cloud Dataproc. The operator will wait until the WorkflowTemplate is finished executing.

Parameters:
  • template_id (string) – The id of the template. (templated)
  • project_id (string) – The ID of the google cloud project in which the template runs
  • region (string) – leave as ‘global’, might become relevant in the future
  • gcp_conn_id (string) – The connection ID to use connecting to Google Cloud Platform.
  • delegate_to (string) – The account to impersonate, if any. For this to work, the service account making the request must have domain-wide delegation enabled.
class airflow.contrib.operators.dataproc_operator.DataprocWorkflowTemplateInstantiateInlineOperator(template, *args, **kwargs)[source]

Bases: airflow.contrib.operators.dataproc_operator.DataprocWorkflowTemplateBaseOperator

Instantiate a WorkflowTemplate Inline on Google Cloud Dataproc. The operator will wait until the WorkflowTemplate is finished executing.

Parameters:
  • template (map) – The template contents. (templated)
  • project_id (string) – The ID of the google cloud project in which the template runs
  • region (string) – leave as ‘global’, might become relevant in the future
  • gcp_conn_id (string) – The connection ID to use connecting to Google Cloud Platform.
  • delegate_to (string) – The account to impersonate, if any. For this to work, the service account making the request must have domain-wide delegation enabled.
class airflow.contrib.operators.datastore_export_operator.DatastoreExportOperator(bucket, namespace=None, datastore_conn_id='google_cloud_default', cloud_storage_conn_id='google_cloud_default', delegate_to=None, entity_filter=None, labels=None, polling_interval_in_seconds=10, overwrite_existing=False, xcom_push=False, *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

Export entities from Google Cloud Datastore to Cloud Storage

Parameters:
  • bucket (string) – name of the cloud storage bucket to backup data
  • namespace (str) – optional namespace path in the specified Cloud Storage bucket to backup data. If this namespace does not exist in GCS, it will be created.
  • datastore_conn_id (string) – the name of the Datastore connection id to use
  • cloud_storage_conn_id (string) – the name of the cloud storage connection id to force-write backup
  • delegate_to (string) – The account to impersonate, if any. For this to work, the service account making the request must have domain-wide delegation enabled.
  • entity_filter (dict) – description of what data from the project is included in the export, refer to https://cloud.google.com/datastore/docs/reference/rest/Shared.Types/EntityFilter
  • labels (dict) – client-assigned labels for cloud storage
  • polling_interval_in_seconds (int) – number of seconds to wait before polling for execution status again
  • overwrite_existing (bool) – if the storage bucket + namespace is not empty, it will be emptied prior to exports. This enables overwriting existing backups.
  • xcom_push (bool) – push operation name to xcom for reference
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.

class airflow.contrib.operators.datastore_import_operator.DatastoreImportOperator(bucket, file, namespace=None, entity_filter=None, labels=None, datastore_conn_id='google_cloud_default', delegate_to=None, polling_interval_in_seconds=10, xcom_push=False, *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

Import entities from Cloud Storage to Google Cloud Datastore

Parameters:
  • bucket (string) – container in Cloud Storage to store data
  • file (string) – path of the backup metadata file in the specified Cloud Storage bucket. It should have the extension .overall_export_metadata
  • namespace (str) – optional namespace of the backup metadata file in the specified Cloud Storage bucket.
  • entity_filter (dict) – description of what data from the project is included in the export, refer to https://cloud.google.com/datastore/docs/reference/rest/Shared.Types/EntityFilter
  • labels (dict) – client-assigned labels for cloud storage
  • datastore_conn_id (string) – the name of the connection id to use
  • delegate_to (string) – The account to impersonate, if any. For this to work, the service account making the request must have domain-wide delegation enabled.
  • polling_interval_in_seconds (int) – number of seconds to wait before polling for execution status again
  • xcom_push (bool) – push operation name to xcom for reference
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.

class airflow.contrib.operators.discord_webhook_operator.DiscordWebhookOperator(http_conn_id=None, webhook_endpoint=None, message='', username=None, avatar_url=None, tts=False, proxy=None, *args, **kwargs)[source]

Bases: airflow.operators.http_operator.SimpleHttpOperator

This operator allows you to post messages to Discord using incoming webhooks. Takes a Discord connection ID with a default relative webhook endpoint. The default endpoint can be overridden using the webhook_endpoint parameter (https://discordapp.com/developers/docs/resources/webhook).

Each Discord webhook can be pre-configured to use a specific username and avatar_url. You can override these defaults in this operator.

Parameters:
  • http_conn_id (str) – Http connection ID with host as “https://discord.com/api/” and default webhook endpoint in the extra field in the form of {“webhook_endpoint”: “webhooks/{webhook.id}/{webhook.token}”}
  • webhook_endpoint (str) – Discord webhook endpoint in the form of “webhooks/{webhook.id}/{webhook.token}”
  • message (str) – The message you want to send to your Discord channel (max 2000 characters). (templated)
  • username (str) – Override the default username of the webhook. (templated)
  • avatar_url (str) – Override the default avatar of the webhook
  • tts (bool) – Is a text-to-speech message
  • proxy (str) – Proxy to use to make the Discord webhook call
execute(context)[source]

Call the DiscordWebhookHook to post message

class airflow.contrib.operators.druid_operator.DruidOperator(json_index_file, druid_ingest_conn_id='druid_ingest_default', max_ingestion_time=None, *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

Allows to submit a task directly to druid

Parameters:
  • json_index_file (str) – The filepath to the druid index specification
  • druid_ingest_conn_id (str) – The connection id of the Druid overlord which accepts index jobs
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.

class airflow.contrib.operators.ecs_operator.ECSOperator(task_definition, cluster, overrides, aws_conn_id=None, region_name=None, launch_type='EC2', **kwargs)[source]

Bases: airflow.models.BaseOperator

Execute a task on AWS EC2 Container Service

Parameters:
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.

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.

class airflow.contrib.operators.emr_add_steps_operator.EmrAddStepsOperator(job_flow_id, aws_conn_id='s3_default', steps=None, *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

An operator that adds steps to an existing EMR job_flow.

Parameters:
  • job_flow_id – id of the JobFlow to add steps to. (templated)
  • aws_conn_id (str) – aws connection to uses
  • steps (list) – boto3 style steps to be added to the jobflow. (templated)
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.

class airflow.contrib.operators.emr_create_job_flow_operator.EmrCreateJobFlowOperator(aws_conn_id='s3_default', emr_conn_id='emr_default', job_flow_overrides=None, *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

Creates an EMR JobFlow, reading the config from the EMR connection. A dictionary of JobFlow overrides can be passed that override the config from the connection.

Parameters:
  • aws_conn_id (str) – aws connection to uses
  • emr_conn_id (str) – emr connection to use
  • job_flow_overrides – boto3 style arguments to override emr_connection extra. (templated)
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.

class airflow.contrib.operators.emr_terminate_job_flow_operator.EmrTerminateJobFlowOperator(job_flow_id, aws_conn_id='s3_default', *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

Operator to terminate EMR JobFlows.

Parameters:
  • job_flow_id – id of the JobFlow to terminate. (templated)
  • aws_conn_id (str) – aws connection to uses
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.

class airflow.contrib.operators.file_to_gcs.FileToGoogleCloudStorageOperator(src, dst, bucket, google_cloud_storage_conn_id='google_cloud_default', mime_type='application/octet-stream', delegate_to=None, gzip=False, *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

Uploads a file to Google Cloud Storage. Optionally can compress the file for upload.

Parameters:
  • src (string) – Path to the local file. (templated)
  • dst (string) – Destination path within the specified bucket. (templated)
  • bucket (string) – The bucket to upload to. (templated)
  • google_cloud_storage_conn_id (string) – The Airflow connection ID to upload with
  • mime_type (string) – The mime-type string
  • delegate_to (str) – The account to impersonate, if any
  • gzip (bool) – Allows for file to be compressed and uploaded as gzip
execute(context)[source]

Uploads the file to Google cloud storage

class airflow.contrib.operators.file_to_wasb.FileToWasbOperator(file_path, container_name, blob_name, wasb_conn_id='wasb_default', load_options=None, *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

Uploads a file to Azure Blob Storage.

Parameters:
  • file_path (str) – Path to the file to load. (templated)
  • container_name (str) – Name of the container. (templated)
  • blob_name (str) – Name of the blob. (templated)
  • wasb_conn_id (str) – Reference to the wasb connection.
  • load_options (dict) – Optional keyword arguments that WasbHook.load_file() takes.
execute(context)[source]

Upload a file to Azure Blob Storage.

class airflow.contrib.operators.gcp_container_operator.GKEClusterCreateOperator(project_id, location, body={}, gcp_conn_id='google_cloud_default', api_version='v2', *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

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.

class airflow.contrib.operators.gcp_container_operator.GKEClusterDeleteOperator(project_id, name, location, gcp_conn_id='google_cloud_default', api_version='v2', *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

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.

class airflow.contrib.operators.gcs_download_operator.GoogleCloudStorageDownloadOperator(bucket, object, filename=None, store_to_xcom_key=None, google_cloud_storage_conn_id='google_cloud_default', delegate_to=None, *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

Downloads a file from Google Cloud Storage.

Parameters:
  • bucket (string) – The Google cloud storage bucket where the object is. (templated)
  • object (string) – The name of the object to download in the Google cloud storage bucket. (templated)
  • filename (string) – The file path on the local file system (where the operator is being executed) that the file should be downloaded to. (templated) If no filename passed, the downloaded data will not be stored on the local file system.
  • store_to_xcom_key (string) – If this param is set, the operator will push the contents of the downloaded file to XCom with the key set in this parameter. If not set, the downloaded data will not be pushed to XCom. (templated)
  • google_cloud_storage_conn_id (string) – The connection ID to use when connecting to Google cloud storage.
  • delegate_to (string) – The account to impersonate, if any. For this to work, the service account making the request must have domain-wide delegation enabled.
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.

class airflow.contrib.operators.gcs_list_operator.GoogleCloudStorageListOperator(bucket, prefix=None, delimiter=None, google_cloud_storage_conn_id='google_cloud_default', delegate_to=None, *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

List all objects from the bucket with the give string prefix and delimiter in name.

This operator returns a python list with the name of objects which can be used by
xcom in the downstream task.
Parameters:
  • bucket (string) – The Google cloud storage bucket to find the objects. (templated)
  • prefix (string) – Prefix string which filters objects whose name begin with this prefix. (templated)
  • delimiter (string) – The delimiter by which you want to filter the objects. (templated) For e.g to lists the CSV files from in a directory in GCS you would use delimiter=’.csv’.
  • google_cloud_storage_conn_id (string) – The connection ID to use when connecting to Google cloud storage.
  • delegate_to (string) – The account to impersonate, if any. For this to work, the service account making the request must have domain-wide delegation enabled.
Example:

The following Operator would list all the Avro files from sales/sales-2017 folder in data bucket.

GCS_Files = GoogleCloudStorageListOperator(
    task_id='GCS_Files',
    bucket='data',
    prefix='sales/sales-2017/',
    delimiter='.avro',
    google_cloud_storage_conn_id=google_cloud_conn_id
)
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.

class airflow.contrib.operators.gcs_operator.GoogleCloudStorageCreateBucketOperator(bucket_name, storage_class='MULTI_REGIONAL', location='US', project_id=None, labels=None, google_cloud_storage_conn_id='google_cloud_default', delegate_to=None, *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

Creates a new bucket. Google Cloud Storage uses a flat namespace, so you can’t create a bucket with a name that is already in use.

See also

For more information, see Bucket Naming Guidelines: https://cloud.google.com/storage/docs/bucketnaming.html#requirements

Parameters:
  • bucket_name (string) – The name of the bucket. (templated)
  • storage_class (string) –

    This defines how objects in the bucket are stored and determines the SLA and the cost of storage (templated). Values include

    • MULTI_REGIONAL
    • REGIONAL
    • STANDARD
    • NEARLINE
    • COLDLINE.

    If this value is not specified when the bucket is created, it will default to STANDARD.

  • location (string) –

    The location of the bucket. (templated) Object data for objects in the bucket resides in physical storage within this region. Defaults to US.

  • project_id (string) – The ID of the GCP Project. (templated)
  • labels (dict) – User-provided labels, in key/value pairs.
  • google_cloud_storage_conn_id (string) – The connection ID to use when connecting to Google cloud storage.
  • delegate_to (string) – The account to impersonate, if any. For this to work, the service account making the request must have domain-wide delegation enabled.
Example:

The following Operator would create a new bucket test-bucket with MULTI_REGIONAL storage class in EU region

CreateBucket = GoogleCloudStorageCreateBucketOperator(
    task_id='CreateNewBucket',
    bucket_name='test-bucket',
    storage_class='MULTI_REGIONAL',
    location='EU',
    labels={'env': 'dev', 'team': 'airflow'},
    google_cloud_storage_conn_id='airflow-service-account'
)
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.

class airflow.contrib.operators.gcs_to_bq.GoogleCloudStorageToBigQueryOperator(bucket, source_objects, destination_project_dataset_table, schema_fields=None, schema_object=None, source_format='CSV', compression='NONE', create_disposition='CREATE_IF_NEEDED', skip_leading_rows=0, write_disposition='WRITE_EMPTY', field_delimiter=', ', max_bad_records=0, quote_character=None, ignore_unknown_values=False, allow_quoted_newlines=False, allow_jagged_rows=False, max_id_key=None, bigquery_conn_id='bigquery_default', google_cloud_storage_conn_id='google_cloud_default', delegate_to=None, schema_update_options=(), src_fmt_configs={}, external_table=False, time_partitioning={}, *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

Loads files from Google cloud storage into BigQuery.

The schema to be used for the BigQuery table may be specified in one of two ways. You may either directly pass the schema fields in, or you may point the operator to a Google cloud storage object name. The object in Google cloud storage must be a JSON file with the schema fields in it.

Parameters:
  • bucket (string) – The bucket to load from. (templated)
  • source_objects – List of Google cloud storage URIs to load from. (templated) If source_format is ‘DATASTORE_BACKUP’, the list must only contain a single URI.
  • destination_project_dataset_table (string) – The dotted (<project>.)<dataset>.<table> BigQuery table to load data into. If <project> is not included, project will be the project defined in the connection json. (templated)
  • schema_fields (list) – If set, the schema field list as defined here: https://cloud.google.com/bigquery/docs/reference/v2/jobs#configuration.load Should not be set when source_format is ‘DATASTORE_BACKUP’.
  • schema_object – If set, a GCS object path pointing to a .json file that contains the schema for the table. (templated)
  • schema_object – string
  • source_format (string) – File format to export.
  • compression (string) – [Optional] The compression type of the data source. Possible values include GZIP and NONE. The default value is NONE. This setting is ignored for Google Cloud Bigtable, Google Cloud Datastore backups and Avro formats.
  • create_disposition (string) – The create disposition if the table doesn’t exist.
  • skip_leading_rows (int) – Number of rows to skip when loading from a CSV.
  • write_disposition (string) – The write disposition if the table already exists.
  • field_delimiter (string) – The delimiter to use when loading from a CSV.
  • max_bad_records (int) – The maximum number of bad records that BigQuery can ignore when running the job.
  • quote_character (string) – The value that is used to quote data sections in a CSV file.
  • ignore_unknown_values (bool) – [Optional] Indicates if BigQuery should allow extra values that are not represented in the table schema. If true, the extra values are ignored. If false, records with extra columns are treated as bad records, and if there are too many bad records, an invalid error is returned in the job result.
  • allow_quoted_newlines (boolean) – Whether to allow quoted newlines (true) or not (false).
  • allow_jagged_rows (bool) – Accept rows that are missing trailing optional columns. The missing values are treated as nulls. If false, records with missing trailing columns are treated as bad records, and if there are too many bad records, an invalid error is returned in the job result. Only applicable to CSV, ignored for other formats.
  • max_id_key (string) – If set, the name of a column in the BigQuery table that’s to be loaded. Thsi will be used to select the MAX value from BigQuery after the load occurs. The results will be returned by the execute() command, which in turn gets stored in XCom for future operators to use. This can be helpful with incremental loads–during future executions, you can pick up from the max ID.
  • bigquery_conn_id (string) – Reference to a specific BigQuery hook.
  • google_cloud_storage_conn_id (string) – Reference to a specific Google cloud storage hook.
  • delegate_to (string) – The account to impersonate, if any. For this to work, the service account making the request must have domain-wide delegation enabled.
  • schema_update_options (list) – Allows the schema of the destination table to be updated as a side effect of the load job.
  • src_fmt_configs (dict) – configure optional fields specific to the source format
  • external_table (bool) – Flag to specify if the destination table should be a BigQuery external table. Default Value is False.
  • time_partitioning (dict) – configure optional time partitioning fields i.e. partition by field, type and expiration as per API specifications. Note that ‘field’ is not available in concurrency with dataset.table$partition.
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.

class airflow.contrib.operators.gcs_to_gcs.GoogleCloudStorageToGoogleCloudStorageOperator(source_bucket, source_object, destination_bucket=None, destination_object=None, move_object=False, google_cloud_storage_conn_id='google_cloud_default', delegate_to=None, *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

Copies objects from a bucket to another, with renaming if requested.

Parameters:
  • source_bucket (string) – The source Google cloud storage bucket where the object is. (templated)
  • source_object (string) –

    The source name of the object to copy in the Google cloud storage bucket. (templated) If wildcards are used in this argument:

    You can use only one wildcard for objects (filenames) within your bucket. The wildcard can appear inside the object name or at the end of the object name. Appending a wildcard to the bucket name is unsupported.
  • destination_bucket – The destination Google cloud storage bucket

where the object should be. (templated) :type destination_bucket: string :param destination_object: The destination name of the object in the

destination Google cloud storage bucket. (templated) If a wildcard is supplied in the source_object argument, this is the prefix that will be prepended to the final destination objects’ paths. Note that the source path’s part before the wildcard will be removed; if it needs to be retained it should be appended to destination_object. For example, with prefix foo/* and destination_object ‘blah/`, the file foo/baz will be copied to blah/baz; to retain the prefix write the destination_object as e.g. blah/foo, in which case the copied file will be named blah/foo/baz.
Parameters:move_object – When move object is True, the object is moved instead
of copied to the new location.
This is the equivalent of a mv command as opposed to a cp command.
Parameters:
  • google_cloud_storage_conn_id (string) – The connection ID to use when connecting to Google cloud storage.
  • delegate_to (string) – The account to impersonate, if any. For this to work, the service account making the request must have domain-wide delegation enabled.
Examples:

The following Operator would copy a single file named sales/sales-2017/january.avro in the data bucket to the file named copied_sales/2017/january-backup.avro` in the ``data_backup bucket

copy_single_file = GoogleCloudStorageToGoogleCloudStorageOperator(
    task_id='copy_single_file',
    source_bucket='data',
    source_object='sales/sales-2017/january.avro',
    destination_bucket='data_backup',
    destination_object='copied_sales/2017/january-backup.avro',
    google_cloud_storage_conn_id=google_cloud_conn_id
)

The following Operator would copy all the Avro files from sales/sales-2017 folder (i.e. with names starting with that prefix) in data bucket to the copied_sales/2017 folder in the data_backup bucket.

copy_files = GoogleCloudStorageToGoogleCloudStorageOperator(
    task_id='copy_files',
    source_bucket='data',
    source_object='sales/sales-2017/*.avro',
    destination_bucket='data_backup',
    destination_object='copied_sales/2017/',
    google_cloud_storage_conn_id=google_cloud_conn_id
)

The following Operator would move all the Avro files from sales/sales-2017 folder (i.e. with names starting with that prefix) in data bucket to the same folder in the data_backup bucket, deleting the original files in the process.

move_files = GoogleCloudStorageToGoogleCloudStorageOperator(
    task_id='move_files',
    source_bucket='data',
    source_object='sales/sales-2017/*.avro',
    destination_bucket='data_backup',
    move_object=True,
    google_cloud_storage_conn_id=google_cloud_conn_id
)
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.

class airflow.contrib.operators.gcs_to_s3.GoogleCloudStorageToS3Operator(bucket, prefix=None, delimiter=None, google_cloud_storage_conn_id='google_cloud_storage_default', delegate_to=None, dest_aws_conn_id=None, dest_s3_key=None, replace=False, *args, **kwargs)[source]

Bases: airflow.contrib.operators.gcs_list_operator.GoogleCloudStorageListOperator

Synchronizes a Google Cloud Storage bucket with an S3 bucket.

Parameters:
  • bucket (string) – The Google Cloud Storage bucket to find the objects. (templated)
  • prefix (string) – Prefix string which filters objects whose name begin with this prefix. (templated)
  • delimiter (string) – The delimiter by which you want to filter the objects. (templated) For e.g to lists the CSV files from in a directory in GCS you would use delimiter=’.csv’.
  • google_cloud_storage_conn_id (string) – The connection ID to use when connecting to Google Cloud Storage.
  • delegate_to (string) – The account to impersonate, if any. For this to work, the service account making the request must have domain-wide delegation enabled.
  • dest_aws_conn_id (str) – The destination S3 connection
  • dest_s3_key (str) – The base S3 key to be used to store the files. (templated)
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.

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.

Parameters:
  • token (str) – HipChat REST API authentication token
  • base_url (str) – HipChat REST API base url.
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.

prepare_request()[source]

Used by the execute function. Set the request method, url, and body of HipChat’s REST API call. Override in child class. Each HipChatAPI child operator is responsible for having a prepare_request method call which sets self.method, self.url, and self.body.

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

Parameters:
  • room_id (str) – Room in which to send notification on HipChat. (templated)
  • message (str) – The message body. (templated)
  • frm (str) – Label to be shown in addition to sender’s name
  • message_format (str) – How the notification is rendered: html or text
  • color (str) – Background color of the msg: yellow, green, red, purple, gray, or random
  • attach_to (str) – The message id to attach this notification to
  • notify (bool) – Whether this message should trigger a user notification
  • card (dict) – HipChat-defined card object
prepare_request()[source]

Used by the execute function. Set the request method, url, and body of HipChat’s REST API call. Override in child class. Each HipChatAPI child operator is responsible for having a prepare_request method call which sets self.method, self.url, and self.body.

class airflow.contrib.operators.hive_to_dynamodb.HiveToDynamoDBTransferOperator(sql, table_name, table_keys, pre_process=None, pre_process_args=None, pre_process_kwargs=None, region_name=None, schema='default', hiveserver2_conn_id='hiveserver2_default', aws_conn_id='aws_default', *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

Moves data from Hive to DynamoDB, note that for now the data is loaded into memory before being pushed to DynamoDB, so this operator should be used for smallish amount of data.

Parameters:
  • sql (str) – SQL query to execute against the hive database. (templated)
  • table_name (str) – target DynamoDB table
  • table_keys (list) – partition key and sort key
  • pre_process (function) – implement pre-processing of source data
  • pre_process_args (list) – list of pre_process function arguments
  • pre_process_kwargs (dict) – dict of pre_process function arguments
  • region_name (str) – aws region name (example: us-east-1)
  • schema (str) – hive database schema
  • hiveserver2_conn_id (str) – source hive connection
  • aws_conn_id (str) – aws connection
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.

class airflow.contrib.operators.jenkins_job_trigger_operator.JenkinsJobTriggerOperator(jenkins_connection_id, job_name, parameters='', sleep_time=10, max_try_before_job_appears=10, *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

Trigger a Jenkins Job and monitor it’s execution. This operator depend on python-jenkins library, version >= 0.4.15 to communicate with jenkins server. You’ll also need to configure a Jenkins connection in the connections screen. :param jenkins_connection_id: The jenkins connection to use for this job :type jenkins_connection_id: string :param job_name: The name of the job to trigger :type job_name: string :param parameters: The parameters block to provide to jenkins. (templated) :type parameters: string :param sleep_time: How long will the operator sleep between each status request for the job (min 1, default 10) :type sleep_time: int :param max_try_before_job_appears: The maximum number of requests to make

while waiting for the job to appears on jenkins server (default 10)
build_job(jenkins_server)[source]

This function makes an API call to Jenkins to trigger a build for ‘job_name’ It returned a dict with 2 keys : body and headers. headers contains also a dict-like object which can be queried to get the location to poll in the queue. :param jenkins_server: The jenkins server where the job should be triggered :return: Dict containing the response body (key body) and the headers coming along (headers)

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.

poll_job_in_queue(location, jenkins_server)[source]

This method poll the jenkins queue until the job is executed. When we trigger a job through an API call, the job is first put in the queue without having a build number assigned. Thus we have to wait the job exit the queue to know its build number. To do so, we have to add /api/json (or /api/xml) to the location returned by the build_job call and poll this file. When a ‘executable’ block appears in the json, it means the job execution started and the field ‘number’ then contains the build number. :param location: Location to poll, returned in the header of the build_job call :param jenkins_server: The jenkins server to poll :return: The build_number corresponding to the triggered job

class airflow.contrib.operators.jira_operator.JiraOperator(jira_conn_id='jira_default', jira_method=None, jira_method_args=None, result_processor=None, get_jira_resource_method=None, *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

JiraOperator to interact and perform action on Jira issue tracking system. This operator is designed to use Jira Python SDK: http://jira.readthedocs.io

Parameters:
  • jira_conn_id (str) – reference to a pre-defined Jira Connection
  • jira_method (str) – method name from Jira Python SDK to be called
  • jira_method_args (dict) – required method parameters for the jira_method. (templated)
  • result_processor (function) – function to further process the response from Jira
  • get_jira_resource_method (function) – function or operator to get jira resource on which the provided jira_method will be executed
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.

class airflow.contrib.operators.kubernetes_pod_operator.KubernetesPodOperator(namespace, image, name, cmds=None, arguments=None, volume_mounts=None, volumes=None, env_vars=None, secrets=None, in_cluster=False, cluster_context=None, labels=None, startup_timeout_seconds=120, get_logs=True, image_pull_policy='IfNotPresent', annotations=None, resources=None, affinity=None, config_file=None, xcom_push=False, node_selectors=None, image_pull_secrets=None, service_account_name='default', is_delete_operator_pod=False, hostnetwork=False, tolerations=None, *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

Execute a task in a Kubernetes Pod

Parameters:
  • image (str) – Docker image you wish to launch. Defaults to dockerhub.io, but fully qualified URLS will point to custom repositories
  • cmds (list of str) – entrypoint of the container. (templated) The docker images’s entrypoint is used if this is not provide.
  • arguments (list of str) – arguments of to the entrypoint. (templated) The docker image’s CMD is used if this is not provided.
  • volume_mounts (list of VolumeMount) – volumeMounts for launched pod
  • volumes (list of Volume) – volumes for launched pod. Includes ConfigMaps and PersistentVolumes
  • labels (dict) – labels to apply to the Pod
  • startup_timeout_seconds (int) – timeout in seconds to startup the pod
  • name (str) – name of the task you want to run, will be used to generate a pod id
  • env_vars (dict) – Environment variables initialized in the container. (templated)
  • secrets (list of Secret) – Kubernetes secrets to inject in the container, They can be exposed as environment vars or files in a volume.
  • in_cluster (bool) – run kubernetes client with in_cluster configuration
  • cluster_context (string) – context that points to kubernetes cluster. Ignored when in_cluster is True. If None, current-context is used.
  • get_logs (bool) – get the stdout of the container as logs of the tasks
  • affinity (dict) – A dict containing a group of affinity scheduling rules
  • node_selectors (dict) – A dict containing a group of scheduling rules
  • config_file (str) – The path to the Kubernetes config file
  • xcom_push (bool) – If xcom_push is True, the content of the file /airflow/xcom/return.json in the container will also be pushed to an XCom when the container completes.
  • tolerations – Kubernetes tolerations
Param:

namespace: the namespace to run within kubernetes

Type:

namespace: str

:type list of tolerations

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.

class airflow.contrib.operators.mlengine_operator.MLEngineBatchPredictionOperator(project_id, job_id, region, data_format, input_paths, output_path, model_name=None, version_name=None, uri=None, max_worker_count=None, runtime_version=None, gcp_conn_id='google_cloud_default', delegate_to=None, *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

Start a Google Cloud ML Engine prediction job.

NOTE: For model origin, users should consider exactly one from the three options below: 1. Populate ‘uri’ field only, which should be a GCS location that points to a tensorflow savedModel directory. 2. Populate ‘model_name’ field only, which refers to an existing model, and the default version of the model will be used. 3. Populate both ‘model_name’ and ‘version_name’ fields, which refers to a specific version of a specific model.

In options 2 and 3, both model and version name should contain the minimal identifier. For instance, call

MLEngineBatchPredictionOperator(
    ...,
    model_name='my_model',
    version_name='my_version',
    ...)

if the desired model version is “projects/my_project/models/my_model/versions/my_version”.

See https://cloud.google.com/ml-engine/reference/rest/v1/projects.jobs for further documentation on the parameters.

Parameters:
  • project_id (string) – The Google Cloud project name where the prediction job is submitted. (templated)
  • job_id (string) – A unique id for the prediction job on Google Cloud ML Engine. (templated)
  • data_format (string) – The format of the input data. It will default to ‘DATA_FORMAT_UNSPECIFIED’ if is not provided or is not one of [“TEXT”, “TF_RECORD”, “TF_RECORD_GZIP”].
  • input_paths (list of string) – A list of GCS paths of input data for batch prediction. Accepting wildcard operator *, but only at the end. (templated)
  • output_path (string) – The GCS path where the prediction results are written to. (templated)
  • region (string) – The Google Compute Engine region to run the prediction job in. (templated)
  • model_name (string) – The Google Cloud ML Engine model to use for prediction. If version_name is not provided, the default version of this model will be used. Should not be None if version_name is provided. Should be None if uri is provided. (templated)
  • version_name (string) – The Google Cloud ML Engine model version to use for prediction. Should be None if uri is provided. (templated)
  • uri (string) – The GCS path of the saved model to use for prediction. Should be None if model_name is provided. It should be a GCS path pointing to a tensorflow SavedModel. (templated)
  • max_worker_count (int) – The maximum number of workers to be used for parallel processing. Defaults to 10 if not specified.
  • runtime_version (string) – The Google Cloud ML Engine runtime version to use for batch prediction.
  • gcp_conn_id (string) – The connection ID used for connection to Google Cloud Platform.
  • delegate_to (string) – The account to impersonate, if any. For this to work, the service account making the request must have doamin-wide delegation enabled.
Raises:
ValueError: if a unique model/version origin cannot be determined.
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.

class airflow.contrib.operators.mlengine_operator.MLEngineModelOperator(project_id, model, operation='create', gcp_conn_id='google_cloud_default', delegate_to=None, *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

Operator for managing a Google Cloud ML Engine model.

Parameters:
  • project_id (string) – The Google Cloud project name to which MLEngine model belongs. (templated)
  • model (dict) –

    A dictionary containing the information about the model. If the operation is create, then the model parameter should contain all the information about this model such as name.

    If the operation is get, the model parameter should contain the name of the model.

  • operation

    The operation to perform. Available operations are:

    • create: Creates a new model as provided by the model parameter.
    • get: Gets a particular model where the name is specified in model.
  • gcp_conn_id (string) – The connection ID to use when fetching connection info.
  • delegate_to (string) – The account to impersonate, if any. For this to work, the service account making the request must have domain-wide delegation enabled.
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.

class airflow.contrib.operators.mlengine_operator.MLEngineVersionOperator(project_id, model_name, version_name=None, version=None, operation='create', gcp_conn_id='google_cloud_default', delegate_to=None, *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

Operator for managing a Google Cloud ML Engine version.

Parameters:
  • project_id (string) – The Google Cloud project name to which MLEngine model belongs.
  • model_name (string) – The name of the Google Cloud ML Engine model that the version belongs to. (templated)
  • version_name (string) – A name to use for the version being operated upon. If not None and the version argument is None or does not have a value for the name key, then this will be populated in the payload for the name key. (templated)
  • version (dict) – A dictionary containing the information about the version. If the operation is create, version should contain all the information about this version such as name, and deploymentUrl. If the operation is get or delete, the version parameter should contain the name of the version. If it is None, the only operation possible would be list. (templated)
  • operation (string) –

    The operation to perform. Available operations are:

    • create: Creates a new version in the model specified by model_name, in which case the version parameter should contain all the information to create that version (e.g. name, deploymentUrl).
    • get: Gets full information of a particular version in the model specified by model_name. The name of the version should be specified in the version parameter.
    • list: Lists all available versions of the model specified by model_name.
    • delete: Deletes the version specified in version parameter from the model specified by model_name). The name of the version should be specified in the version parameter.
  • gcp_conn_id (string) – The connection ID to use when fetching connection info.
  • delegate_to (string) – The account to impersonate, if any. For this to work, the service account making the request must have domain-wide delegation enabled.
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.

class airflow.contrib.operators.mlengine_operator.MLEngineTrainingOperator(project_id, job_id, package_uris, training_python_module, training_args, region, scale_tier=None, runtime_version=None, python_version=None, job_dir=None, gcp_conn_id='google_cloud_default', delegate_to=None, mode='PRODUCTION', *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

Operator for launching a MLEngine training job.

Parameters:
  • project_id (string) – The Google Cloud project name within which MLEngine training job should run (templated).
  • job_id (string) – A unique templated id for the submitted Google MLEngine training job. (templated)
  • package_uris (string) – A list of package locations for MLEngine training job, which should include the main training program + any additional dependencies. (templated)
  • training_python_module (string) – The Python module name to run within MLEngine training job after installing ‘package_uris’ packages. (templated)
  • training_args (string) – A list of templated command line arguments to pass to the MLEngine training program. (templated)
  • region (string) – The Google Compute Engine region to run the MLEngine training job in (templated).
  • scale_tier (string) – Resource tier for MLEngine training job. (templated)
  • runtime_version (string) – The Google Cloud ML runtime version to use for training. (templated)
  • python_version (string) – The version of Python used in training. (templated)
  • job_dir (string) – A Google Cloud Storage path in which to store training outputs and other data needed for training. (templated)
  • gcp_conn_id (string) – The connection ID to use when fetching connection info.
  • delegate_to (string) – The account to impersonate, if any. For this to work, the service account making the request must have domain-wide delegation enabled.
  • mode (string) – Can be one of ‘DRY_RUN’/’CLOUD’. In ‘DRY_RUN’ mode, no real training job will be launched, but the MLEngine training job request will be printed out. In ‘CLOUD’ mode, a real MLEngine training job creation request will be issued.
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.

class airflow.contrib.operators.mongo_to_s3.MongoToS3Operator(mongo_conn_id, s3_conn_id, mongo_collection, mongo_query, s3_bucket, s3_key, mongo_db=None, *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

Mongo -> S3

A more specific baseOperator meant to move data from mongo via pymongo to s3 via boto

things to note
.execute() is written to depend on .transform() .transform() is meant to be extended by child classes to perform transformations unique to those operators needs
execute(context)[source]

Executed by task_instance at runtime

transform(docs)[source]
Processes pyMongo cursor and returns an iterable with each element being
a JSON serializable dictionary

Base transform() assumes no processing is needed ie. docs is a pyMongo cursor of documents and cursor just needs to be passed through

Override this method for custom transformations

class airflow.contrib.operators.mysql_to_gcs.MySqlToGoogleCloudStorageOperator(sql, bucket, filename, schema_filename=None, approx_max_file_size_bytes=1900000000, mysql_conn_id='mysql_default', google_cloud_storage_conn_id='google_cloud_default', schema=None, delegate_to=None, *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

Copy data from MySQL to Google cloud storage in JSON format.

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.

classmethod type_map(mysql_type)[source]

Helper function that maps from MySQL fields to BigQuery fields. Used when a schema_filename is set.

class airflow.contrib.operators.postgres_to_gcs_operator.PostgresToGoogleCloudStorageOperator(sql, bucket, filename, schema_filename=None, approx_max_file_size_bytes=1900000000, postgres_conn_id='postgres_default', google_cloud_storage_conn_id='google_cloud_default', delegate_to=None, parameters=None, *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

Copy data from Postgres to Google Cloud Storage in JSON format.

classmethod convert_types(value)[source]

Takes a value from Postgres, and converts it to a value that’s safe for JSON/Google Cloud Storage/BigQuery. Dates are converted to UTC seconds. Decimals are converted to floats. Times are converted to seconds.

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.

classmethod type_map(postgres_type)[source]

Helper function that maps from Postgres fields to BigQuery fields. Used when a schema_filename is set.

class airflow.contrib.operators.pubsub_operator.PubSubTopicCreateOperator(project, topic, fail_if_exists=False, gcp_conn_id='google_cloud_default', delegate_to=None, *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

Create a PubSub topic.

By default, if the topic already exists, this operator will not cause the DAG to fail.

with DAG('successful DAG') as dag:
    (
        dag
        >> PubSubTopicCreateOperator(project='my-project',
                                     topic='my_new_topic')
        >> PubSubTopicCreateOperator(project='my-project',
                                     topic='my_new_topic')
    )

The operator can be configured to fail if the topic already exists.

with DAG('failing DAG') as dag:
    (
        dag
        >> PubSubTopicCreateOperator(project='my-project',
                                     topic='my_new_topic')
        >> PubSubTopicCreateOperator(project='my-project',
                                     topic='my_new_topic',
                                     fail_if_exists=True)
    )

Both project and topic are templated so you can use variables in them.

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.

class airflow.contrib.operators.pubsub_operator.PubSubTopicDeleteOperator(project, topic, fail_if_not_exists=False, gcp_conn_id='google_cloud_default', delegate_to=None, *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

Delete a PubSub topic.

By default, if the topic does not exist, this operator will not cause the DAG to fail.

with DAG('successful DAG') as dag:
    (
        dag
        >> PubSubTopicDeleteOperator(project='my-project',
                                     topic='non_existing_topic')
    )

The operator can be configured to fail if the topic does not exist.

with DAG('failing DAG') as dag:
    (
        dag
        >> PubSubTopicCreateOperator(project='my-project',
                                     topic='non_existing_topic',
                                     fail_if_not_exists=True)
    )

Both project and topic are templated so you can use variables in them.

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.

class airflow.contrib.operators.pubsub_operator.PubSubSubscriptionCreateOperator(topic_project, topic, subscription=None, subscription_project=None, ack_deadline_secs=10, fail_if_exists=False, gcp_conn_id='google_cloud_default', delegate_to=None, *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

Create a PubSub subscription.

By default, the subscription will be created in topic_project. If subscription_project is specified and the GCP credentials allow, the Subscription can be created in a different project from its topic.

By default, if the subscription already exists, this operator will not cause the DAG to fail. However, the topic must exist in the project.

with DAG('successful DAG') as dag:
    (
        dag
        >> PubSubSubscriptionCreateOperator(
            topic_project='my-project', topic='my-topic',
            subscription='my-subscription')
        >> PubSubSubscriptionCreateOperator(
            topic_project='my-project', topic='my-topic',
            subscription='my-subscription')
    )

The operator can be configured to fail if the subscription already exists.

with DAG('failing DAG') as dag:
    (
        dag
        >> PubSubSubscriptionCreateOperator(
            topic_project='my-project', topic='my-topic',
            subscription='my-subscription')
        >> PubSubSubscriptionCreateOperator(
            topic_project='my-project', topic='my-topic',
            subscription='my-subscription', fail_if_exists=True)
    )

Finally, subscription is not required. If not passed, the operator will generated a universally unique identifier for the subscription’s name.

with DAG('DAG') as dag:
    (
        dag >> PubSubSubscriptionCreateOperator(
            topic_project='my-project', topic='my-topic')
    )

topic_project, topic, subscription, and subscription are templated so you can use variables in them.

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.

class airflow.contrib.operators.pubsub_operator.PubSubSubscriptionDeleteOperator(project, subscription, fail_if_not_exists=False, gcp_conn_id='google_cloud_default', delegate_to=None, *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

Delete a PubSub subscription.

By default, if the subscription does not exist, this operator will not cause the DAG to fail.

with DAG('successful DAG') as dag:
    (
        dag
        >> PubSubSubscriptionDeleteOperator(project='my-project',
                                            subscription='non-existing')
    )

The operator can be configured to fail if the subscription already exists.

with DAG('failing DAG') as dag:
    (
        dag
        >> PubSubSubscriptionDeleteOperator(
             project='my-project', subscription='non-existing',
             fail_if_not_exists=True)
    )

project, and subscription are templated so you can use variables in them.

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.

class airflow.contrib.operators.pubsub_operator.PubSubPublishOperator(project, topic, messages, gcp_conn_id='google_cloud_default', delegate_to=None, *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

Publish messages to a PubSub topic.

Each Task publishes all provided messages to the same topic in a single GCP project. If the topic does not exist, this task will fail.

   from base64 import b64encode as b64e

   m1 = {'data': b64e('Hello, World!'),
         'attributes': {'type': 'greeting'}
        }
   m2 = {'data': b64e('Knock, knock')}
   m3 = {'attributes': {'foo': ''}}

   t1 = PubSubPublishOperator(
       project='my-project',topic='my_topic',
       messages=[m1, m2, m3],
       create_topic=True,
       dag=dag)

``project`` , ``topic``, and ``messages`` are templated so you can use

variables in them.

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.

class airflow.contrib.operators.qubole_check_operator.QuboleCheckOperator(qubole_conn_id='qubole_default', *args, **kwargs)[source]

Bases: airflow.operators.check_operator.CheckOperator, airflow.contrib.operators.qubole_operator.QuboleOperator

Performs checks against Qubole Commands. QuboleCheckOperator expects a command that will be executed on QDS. By default, each value on first row of the result of this Qubole Commmand is evaluated using python bool casting. If any of the values return False, 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.

Parameters:qubole_conn_id (str) – Connection id which consists of qds auth_token

kwargs:

Arguments specific to Qubole command can be referred from QuboleOperator docs.

results_parser_callable:
 This is an optional parameter to extend the flexibility of parsing the results of Qubole command to the users. This is a python callable which can hold the logic to parse list of rows returned by Qubole command. By default, only the values on first row are used for performing checks. This callable should return a list of records on which the checks have to be performed.

Note

All fields in common with template fields of QuboleOperator and CheckOperator are template-supported.

execute(context=None)[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.

class airflow.contrib.operators.qubole_check_operator.QuboleValueCheckOperator(pass_value, tolerance=None, qubole_conn_id='qubole_default', *args, **kwargs)[source]

Bases: airflow.operators.check_operator.ValueCheckOperator, airflow.contrib.operators.qubole_operator.QuboleOperator

Performs a simple value check using Qubole command. By default, each value on the first row of this Qubole command is compared with a pre-defined value. The check fails and errors out if the output of the command is not within the permissible limit of expected value.

Parameters:
  • qubole_conn_id (str) – Connection id which consists of qds auth_token
  • pass_value (str/int/float) – Expected value of the query results.
  • tolerance (int/float) – Defines the permissible pass_value range, for example if tolerance is 2, the Qubole command output can be anything between -2*pass_value and 2*pass_value, without the operator erring out.

kwargs:

Arguments specific to Qubole command can be referred from QuboleOperator docs.

results_parser_callable:
 This is an optional parameter to extend the flexibility of parsing the results of Qubole command to the users. This is a python callable which can hold the logic to parse list of rows returned by Qubole command. By default, only the values on first row are used for performing checks. This callable should return a list of records on which the checks have to be performed.

Note

All fields in common with template fields of QuboleOperator and ValueCheckOperator are template-supported.

execute(context=None)[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.

class airflow.contrib.operators.qubole_operator.QuboleOperator(qubole_conn_id='qubole_default', *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

Execute tasks (commands) on QDS (https://qubole.com).

Parameters:qubole_conn_id (str) – Connection id which consists of qds auth_token
kwargs:
command_type:type of command to be executed, e.g. hivecmd, shellcmd, hadoopcmd
tags:array of tags to be assigned with the command
cluster_label:cluster label on which the command will be executed
name:name to be given to command
notify:whether to send email on command completion or not (default is False)

Arguments specific to command types

hivecmd:
query:inline query statement
script_location:
 s3 location containing query statement
sample_size:size of sample in bytes on which to run query
macros:macro values which were used in query
prestocmd:
query:inline query statement
script_location:
 s3 location containing query statement
macros:macro values which were used in query
hadoopcmd:
sub_commnad:must be one these [“jar”, “s3distcp”, “streaming”] followed by 1 or more args
shellcmd:
script:inline command with args
script_location:
 s3 location containing query statement
files:list of files in s3 bucket as file1,file2 format. These files will be copied into the working directory where the qubole command is being executed.
archives:list of archives in s3 bucket as archive1,archive2 format. These will be unarchived intothe working directory where the qubole command is being executed
parameters:any extra args which need to be passed to script (only when script_location is supplied)
pigcmd:
script:inline query statement (latin_statements)
script_location:
 s3 location containing pig query
parameters:any extra args which need to be passed to script (only when script_location is supplied
sparkcmd:
program:the complete Spark Program in Scala, SQL, Command, R, or Python
cmdline:spark-submit command line, all required information must be specify in cmdline itself.
sql:inline sql query
script_location:
 s3 location containing query statement
language:language of the program, Scala, SQL, Command, R, or Python
app_id:ID of an Spark job server app
arguments:spark-submit command line arguments
user_program_arguments:
 arguments that the user program takes in
macros:macro values which were used in query
dbtapquerycmd:
db_tap_id:data store ID of the target database, in Qubole.
query:inline query statement
macros:macro values which were used in query
dbexportcmd:
mode:1 (simple), 2 (advance)
hive_table:Name of the hive table
partition_spec:partition specification for Hive table.
dbtap_id:data store ID of the target database, in Qubole.
db_table:name of the db table
db_update_mode:allowinsert or updateonly
db_update_keys:columns used to determine the uniqueness of rows
export_dir:HDFS/S3 location from which data will be exported.
fields_terminated_by:
 hex of the char used as column separator in the dataset
dbimportcmd:
mode:1 (simple), 2 (advance)
hive_table:Name of the hive table
dbtap_id:data store ID of the target database, in Qubole.
db_table:name of the db table
where_clause:where clause, if any
parallelism:number of parallel db connections to use for extracting data
extract_query:SQL query to extract data from db. $CONDITIONS must be part of the where clause.
boundary_query:Query to be used get range of row IDs to be extracted
split_column:Column used as row ID to split data into ranges (mode 2)

Note

Following fields are template-supported : query, script_location, sub_command, script, files, archives, program, cmdline, sql, where_clause, extract_query, boundary_query, macros, tags, name, parameters, dbtap_id, hive_table, db_table, split_column, note_id, db_update_keys, export_dir, partition_spec, qubole_conn_id, arguments, user_program_arguments.

You can also use .txt files for template driven use cases.

Note

In QuboleOperator there is a default handler for task failures and retries, which generally kills the command running at QDS for the corresponding task instance. You can override this behavior by providing your own failure and retry handler in task definition.

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.

on_kill(ti=None)[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.

class airflow.contrib.operators.s3_list_operator.S3ListOperator(bucket, prefix='', delimiter='', aws_conn_id='aws_default', *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

List all objects from the bucket with the given string prefix in name.

This operator returns a python list with the name of objects which can be used by xcom in the downstream task.

Parameters:
  • bucket (string) – The S3 bucket where to find the objects. (templated)
  • prefix (string) – Prefix string to filters the objects whose name begin with such prefix. (templated)
  • delimiter (string) – the delimiter marks key hierarchy. (templated)
  • aws_conn_id (string) – The connection ID to use when connecting to S3 storage.
Example:

The following operator would list all the files (excluding subfolders) from the S3 customers/2018/04/ key in the data bucket.

s3_file = S3ListOperator(
    task_id='list_3s_files',
    bucket='data',
    prefix='customers/2018/04/',
    delimiter='/',
    aws_conn_id='aws_customers_conn'
)
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.

class airflow.contrib.operators.s3_to_gcs_operator.S3ToGoogleCloudStorageOperator(bucket, prefix='', delimiter='', aws_conn_id='aws_default', dest_gcs_conn_id=None, dest_gcs=None, delegate_to=None, replace=False, *args, **kwargs)[source]

Bases: airflow.contrib.operators.s3_list_operator.S3ListOperator

Synchronizes an S3 key, possibly a prefix, with a Google Cloud Storage destination path.

Parameters:
  • bucket (string) – The S3 bucket where to find the objects. (templated)
  • prefix (string) – Prefix string which filters objects whose name begin with such prefix. (templated)
  • delimiter (string) – the delimiter marks key hierarchy. (templated)
  • aws_conn_id (string) – The source S3 connection
  • dest_gcs_conn_id (string) – The destination connection ID to use when connecting to Google Cloud Storage.
  • dest_gcs (string) – The destination Google Cloud Storage bucket and prefix where you want to store the files. (templated)
  • delegate_to (string) – The account to impersonate, if any. For this to work, the service account making the request must have domain-wide delegation enabled.
  • replace (bool) – Whether you want to replace existing destination files or not.

Example: .. code-block:: python

s3_to_gcs_op = S3ToGoogleCloudStorageOperator(
task_id=’s3_to_gcs_example’, bucket=’my-s3-bucket’, prefix=’data/customers-201804’, dest_gcs_conn_id=’google_cloud_default’, dest_gcs=’gs://my.gcs.bucket/some/customers/’, replace=False, dag=my-dag)

Note that bucket, prefix, delimiter and dest_gcs are templated, so you can use variables in them if you wish.

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.

class airflow.contrib.operators.sagemaker_base_operator.SageMakerBaseOperator(config, aws_conn_id='aws_default', *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

This is the base operator for all SageMaker operators.

Parameters:
  • config (dict) – The configuration necessary to start a training job (templated)
  • aws_conn_id (str) – The AWS connection ID to use.
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.

class airflow.contrib.operators.sagemaker_endpoint_operator.SageMakerEndpointOperator(config, wait_for_completion=True, check_interval=30, max_ingestion_time=None, operation='create', *args, **kwargs)[source]

Bases: airflow.contrib.operators.sagemaker_base_operator.SageMakerBaseOperator

Create a SageMaker endpoint.

This operator returns The ARN of the endpoint created in Amazon SageMaker

Parameters:
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.

class airflow.contrib.operators.sagemaker_endpoint_config_operator.SageMakerEndpointConfigOperator(config, *args, **kwargs)[source]

Bases: airflow.contrib.operators.sagemaker_base_operator.SageMakerBaseOperator

Create a SageMaker endpoint config.

This operator returns The ARN of the endpoint config created in Amazon SageMaker

Parameters:
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.

class airflow.contrib.operators.sagemaker_model_operator.SageMakerModelOperator(config, *args, **kwargs)[source]

Bases: airflow.contrib.operators.sagemaker_base_operator.SageMakerBaseOperator

Create a SageMaker model.

This operator returns The ARN of the model created in Amazon SageMaker

Parameters:
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.

class airflow.contrib.operators.sagemaker_training_operator.SageMakerTrainingOperator(config, wait_for_completion=True, print_log=True, check_interval=30, max_ingestion_time=None, *args, **kwargs)[source]

Bases: airflow.contrib.operators.sagemaker_base_operator.SageMakerBaseOperator

Initiate a SageMaker training job.

This operator returns The ARN of the training job created in Amazon SageMaker.

Parameters:
  • config (dict) –

    The configuration necessary to start a training job (templated).

    For details of the configuration parameter, See: https://boto3.amazonaws.com/v1/documentation/api/latest/reference/services/sagemaker.html#SageMaker.Client.create_training_job

  • aws_conn_id (str) – The AWS connection ID to use.
  • wait_for_completion (bool) – If wait is set to True, the time interval, in seconds, that the operation waits to check the status of the training job.
  • print_log (bool) – if the operator should print the cloudwatch log during training
  • check_interval (int) – if wait is set to be true, this is the time interval in seconds which the operator will check the status of the training job
  • max_ingestion_time (int) – If wait is set to True, the operation fails if the training job doesn’t finish within max_ingestion_time seconds. If you set this parameter to None, the operation does not timeout.
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.

class airflow.contrib.operators.sagemaker_transform_operator.SageMakerTransformOperator(config, wait_for_completion=True, check_interval=30, max_ingestion_time=None, *args, **kwargs)[source]

Bases: airflow.contrib.operators.sagemaker_base_operator.SageMakerBaseOperator

Initiate a SageMaker transform job.

This operator returns The ARN of the model created in Amazon SageMaker.

Parameters:
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.

class airflow.contrib.operators.sagemaker_tuning_operator.SageMakerTuningOperator(config, wait_for_completion=True, check_interval=30, max_ingestion_time=None, *args, **kwargs)[source]

Bases: airflow.contrib.operators.sagemaker_base_operator.SageMakerBaseOperator

Initiate a SageMaker hyperparameter tuning job.

This operator returns The ARN of the tuning job created in Amazon SageMaker.

Parameters:
  • config (dict) –

    The configuration necessary to start a tuning job (templated).

    For details of the configuration parameter, See: https://boto3.amazonaws.com/v1/documentation/api/latest/reference/services/sagemaker.html#SageMaker.Client.create_hyper_parameter_tuning_job

  • aws_conn_id (str) – The AWS connection ID to use.
  • wait_for_completion (bool) – Set to True to wait until the tuning job finishes.
  • check_interval (int) – If wait is set to True, the time interval, in seconds, that this operation waits to check the status of the tuning job.
  • max_ingestion_time (int) – If wait is set to True, the operation fails if the tuning job doesn’t finish within max_ingestion_time seconds. If you set this parameter to None, the operation does not timeout.
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.

class airflow.contrib.operators.segment_track_event_operator.SegmentTrackEventOperator(user_id, event, properties=None, segment_conn_id='segment_default', segment_debug_mode=False, *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

Send Track Event to Segment for a specified user_id and event

Parameters:
  • user_id (string) – The ID for this user in your database. (templated)
  • event (string) – The name of the event you’re tracking. (templated)
  • properties (dict) – A dictionary of properties for the event. (templated)
  • segment_conn_id (string) – The connection ID to use when connecting to Segment.
  • segment_debug_mode (boolean) – Determines whether Segment should run in debug mode. Defaults to False
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.

class airflow.contrib.operators.sftp_operator.SFTPOperator(ssh_hook=None, ssh_conn_id=None, remote_host=None, local_filepath=None, remote_filepath=None, operation='put', confirm=True, *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

SFTPOperator for transferring files from remote host to local or vice a versa. This operator uses ssh_hook to open sftp trasport channel that serve as basis for file transfer.

Parameters:
  • ssh_hook (SSHHook) – predefined ssh_hook to use for remote execution
  • ssh_conn_id (str) – connection id from airflow Connections
  • remote_host (str) – remote host to connect
  • local_filepath (str) – local file path to get or put. (templated)
  • remote_filepath (str) – remote file path to get or put. (templated)
  • operation – specify operation ‘get’ or ‘put’, defaults to put
  • confirm (bool) – specify if the SFTP operation should be confirmed, defaults to True
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.

class airflow.contrib.operators.slack_webhook_operator.SlackWebhookOperator(http_conn_id=None, webhook_token=None, message='', channel=None, username=None, icon_emoji=None, link_names=False, proxy=None, *args, **kwargs)[source]

Bases: airflow.operators.http_operator.SimpleHttpOperator

This operator allows you to post messages to Slack using incoming webhooks. Takes both Slack webhook token directly and connection that has Slack webhook token. If both supplied, Slack webhook token will be used.

Each Slack webhook token can be pre-configured to use a specific channel, username and icon. You can override these defaults in this hook.

Parameters:
  • http_conn_id (str) – connection that has Slack webhook token in the extra field
  • webhook_token (str) – Slack webhook token
  • message (str) – The message you want to send on Slack
  • channel (str) – The channel the message should be posted to
  • username (str) – The username to post to slack with
  • icon_emoji (str) – The emoji to use as icon for the user posting to Slack
  • link_names (bool) – Whether or not to find and link channel and usernames in your message
  • proxy (str) – Proxy to use to make the Slack webhook call
execute(context)[source]

Call the SparkSqlHook to run the provided sql query

class airflow.contrib.operators.snowflake_operator.SnowflakeOperator(sql, snowflake_conn_id='snowflake_default', parameters=None, autocommit=True, warehouse=None, database=None, *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

Executes sql code in a Snowflake database

Parameters:
  • snowflake_conn_id (string) – reference to specific snowflake connection id
  • 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. (templated)
  • warehouse (string) – name of warehouse which overwrite defined one in connection
  • database (string) – name of database which overwrite defined one in connection
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.

class airflow.contrib.operators.spark_jdbc_operator.SparkJDBCOperator(spark_app_name='airflow-spark-jdbc', spark_conn_id='spark-default', spark_conf=None, spark_py_files=None, spark_files=None, spark_jars=None, num_executors=None, executor_cores=None, executor_memory=None, driver_memory=None, verbose=False, keytab=None, principal=None, cmd_type='spark_to_jdbc', jdbc_table=None, jdbc_conn_id='jdbc-default', jdbc_driver=None, metastore_table=None, jdbc_truncate=False, save_mode=None, save_format=None, batch_size=None, fetch_size=None, num_partitions=None, partition_column=None, lower_bound=None, upper_bound=None, create_table_column_types=None, *args, **kwargs)[source]

Bases: airflow.contrib.operators.spark_submit_operator.SparkSubmitOperator

This operator extends the SparkSubmitOperator specifically for performing data transfers to/from JDBC-based databases with Apache Spark. As with the SparkSubmitOperator, it assumes that the “spark-submit” binary is available on the PATH.

Parameters:
  • spark_app_name (str) – Name of the job (default airflow-spark-jdbc)
  • spark_conn_id (str) – Connection id as configured in Airflow administration
  • spark_conf (dict) – Any additional Spark configuration properties
  • spark_py_files (str) – Additional python files used (.zip, .egg, or .py)
  • spark_files (str) – Additional files to upload to the container running the job
  • spark_jars (str) – Additional jars to upload and add to the driver and executor classpath
  • num_executors (int) – number of executor to run. This should be set so as to manage the number of connections made with the JDBC database
  • executor_cores (int) – Number of cores per executor
  • executor_memory (str) – Memory per executor (e.g. 1000M, 2G)
  • driver_memory (str) – Memory allocated to the driver (e.g. 1000M, 2G)
  • verbose (bool) – Whether to pass the verbose flag to spark-submit for debugging
  • keytab (str) – Full path to the file that contains the keytab
  • principal (str) – The name of the kerberos principal used for keytab
  • cmd_type (str) – Which way the data should flow. 2 possible values: spark_to_jdbc: data written by spark from metastore to jdbc jdbc_to_spark: data written by spark from jdbc to metastore
  • jdbc_table (str) – The name of the JDBC table
  • jdbc_conn_id – Connection id used for connection to JDBC database
  • jdbc_driver (str) – Name of the JDBC driver to use for the JDBC connection. This driver (usually a jar) should be passed in the ‘jars’ parameter
  • metastore_table (str) – The name of the metastore table,
  • jdbc_truncate (bool) – (spark_to_jdbc only) Whether or not Spark should truncate or drop and recreate the JDBC table. This only takes effect if ‘save_mode’ is set to Overwrite. Also, if the schema is different, Spark cannot truncate, and will drop and recreate
  • save_mode (str) – The Spark save-mode to use (e.g. overwrite, append, etc.)
  • save_format (str) – (jdbc_to_spark-only) The Spark save-format to use (e.g. parquet)
  • batch_size (int) – (spark_to_jdbc only) The size of the batch to insert per round trip to the JDBC database. Defaults to 1000
  • fetch_size (int) – (jdbc_to_spark only) The size of the batch to fetch per round trip from the JDBC database. Default depends on the JDBC driver
  • num_partitions (int) – The maximum number of partitions that can be used by Spark simultaneously, both for spark_to_jdbc and jdbc_to_spark operations. This will also cap the number of JDBC connections that can be opened
  • partition_column (str) – (jdbc_to_spark-only) A numeric column to be used to partition the metastore table by. If specified, you must also specify: num_partitions, lower_bound, upper_bound
  • lower_bound (int) – (jdbc_to_spark-only) Lower bound of the range of the numeric partition column to fetch. If specified, you must also specify: num_partitions, partition_column, upper_bound
  • upper_bound (int) – (jdbc_to_spark-only) Upper bound of the range of the numeric partition column to fetch. If specified, you must also specify: num_partitions, partition_column, lower_bound
  • create_table_column_types – (spark_to_jdbc-only) The database column data types to use instead of the defaults, when creating the table. Data type information should be specified in the same format as CREATE TABLE columns syntax (e.g: “name CHAR(64), comments VARCHAR(1024)”). The specified types should be valid spark sql data types.
Type:

jdbc_conn_id: str

execute(context)[source]

Call the SparkSubmitHook to run the provided spark job

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.

class airflow.contrib.operators.spark_sql_operator.SparkSqlOperator(sql, conf=None, conn_id='spark_sql_default', total_executor_cores=None, executor_cores=None, executor_memory=None, keytab=None, principal=None, master='yarn', name='default-name', num_executors=None, yarn_queue='default', *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

Execute Spark SQL query

Parameters:
  • sql (str) – The SQL query to execute. (templated)
  • conf (str (format: PROP=VALUE)) – arbitrary Spark configuration property
  • conn_id (str) – connection_id string
  • total_executor_cores (int) – (Standalone & Mesos only) Total cores for all executors (Default: all the available cores on the worker)
  • executor_cores (int) – (Standalone & YARN only) 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
  • master (str) – spark://host:port, mesos://host:port, yarn, or local
  • name (str) – Name of the job
  • num_executors (int) – Number of executors to launch
  • verbose (bool) – Whether to pass the verbose flag to spark-sql
  • yarn_queue (str) – The YARN queue to submit to (Default: “default”)
execute(context)[source]

Call the SparkSqlHook to run the provided sql query

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.

class airflow.contrib.operators.spark_submit_operator.SparkSubmitOperator(application='', conf=None, conn_id='spark_default', files=None, py_files=None, driver_classpath=None, jars=None, java_class=None, packages=None, exclude_packages=None, repositories=None, total_executor_cores=None, executor_cores=None, executor_memory=None, driver_memory=None, keytab=None, principal=None, name='airflow-spark', num_executors=None, application_args=None, env_vars=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 or the spark-home is set in the extra on the connection.

Parameters:
  • application (str) – The application that submitted as a job, either jar or py file. (templated)
  • conf (dict) – Arbitrary Spark configuration properties
  • conn_id (str) – The connection id as configured in Airflow administration. When an invalid connection_id is supplied, it will default to yarn.
  • files (str) – Upload additional files to the executor running the job, separated by a comma. Files will be placed in the working directory of each executor. For example, serialized objects.
  • 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.
  • driver_classpath (str) – Additional, driver-specific, classpath settings.
  • java_class (str) – the main class of the Java application
  • packages (str) – Comma-separated list of maven coordinates of jars to include on the driver and executor classpaths. (templated)
  • exclude_packages (str) – Comma-separated list of maven coordinates of jars to exclude while resolving the dependencies provided in ‘packages’
  • repositories (str) – Comma-separated list of additional remote repositories to search for the maven coordinates given with ‘packages’
  • total_executor_cores (int) – (Standalone & Mesos only) Total cores for all executors (Default: all the available cores on the worker)
  • executor_cores (int) – (Standalone & YARN only) Number of cores per executor (Default: 2)
  • executor_memory (str) – Memory per executor (e.g. 1000M, 2G) (Default: 1G)
  • driver_memory (str) – Memory allocated to the driver (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). (templated)
  • num_executors (int) – Number of executors to launch
  • application_args (list) – Arguments for the application being submitted
  • env_vars (dict) – Environment variables for spark-submit. It supports yarn and k8s mode too.
  • 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

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.

class airflow.contrib.operators.sqoop_operator.SqoopOperator(conn_id='sqoop_default', cmd_type='import', table=None, query=None, target_dir=None, append=None, file_type='text', columns=None, num_mappers=None, split_by=None, where=None, export_dir=None, input_null_string=None, input_null_non_string=None, staging_table=None, clear_staging_table=False, enclosed_by=None, escaped_by=None, input_fields_terminated_by=None, input_lines_terminated_by=None, input_optionally_enclosed_by=None, batch=False, direct=False, driver=None, verbose=False, relaxed_isolation=False, properties=None, hcatalog_database=None, hcatalog_table=None, create_hcatalog_table=False, extra_import_options=None, extra_export_options=None, *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

Execute a Sqoop job. Documentation for Apache Sqoop can be found here:

execute(context)[source]

Execute sqoop job

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.

class airflow.contrib.operators.ssh_operator.SSHOperator(ssh_hook=None, ssh_conn_id=None, remote_host=None, command=None, timeout=10, do_xcom_push=False, *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

SSHOperator to execute commands on given remote host using the ssh_hook.

Parameters:
  • ssh_hook (SSHHook) – predefined ssh_hook to use for remote execution
  • ssh_conn_id (str) – connection id from airflow Connections
  • remote_host (str) – remote host to connect
  • command (str) – command to execute on remote host. (templated)
  • timeout (int) – timeout (in seconds) for executing the command.
  • do_xcom_push (bool) – return the stdout which also get set in xcom by airflow platform
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.

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. (templated)
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.

class airflow.contrib.operators.vertica_to_hive.VerticaToHiveTransfer(sql, hive_table, create=True, recreate=False, partition=None, delimiter='x01', vertica_conn_id='vertica_default', hive_cli_conn_id='hive_cli_default', *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

Moves data from Vertia to Hive. The operator runs your query against Vertia, stores the file locally before loading it into a Hive table. If the create or recreate arguments are set to True, a CREATE TABLE and DROP TABLE statements are generated. Hive data types are inferred from the cursor’s metadata. Note that the table generated in Hive uses STORED AS textfile which isn’t the most efficient serialization format. If a large amount of data is loaded and/or if the table gets queried considerably, you may want to use this operator only to stage the data into a temporary table before loading it into its final destination using a HiveOperator.

Parameters:
  • sql (str) – SQL query to execute against the Vertia database. (templated)
  • hive_table (str) – target Hive table, use dot notation to target a specific database. (templated)
  • create (bool) – whether to create the table if it doesn’t exist
  • recreate (bool) – whether to drop and recreate the table at every execution
  • partition (dict) – target partition as a dict of partition columns and values. (templated)
  • delimiter (str) – field delimiter in the file
  • vertica_conn_id (str) – source Vertica connection
  • hive_conn_id (str) – destination hive connection
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.

class airflow.contrib.operators.winrm_operator.WinRMOperator(winrm_hook=None, ssh_conn_id=None, remote_host=None, command=None, timeout=10, do_xcom_push=False, *args, **kwargs)[source]

Bases: airflow.models.BaseOperator

WinRMOperator to execute commands on given remote host using the winrm_hook.

Parameters:
  • winrm_hook (WinRMHook) – predefined ssh_hook to use for remote execution
  • ssh_conn_id (str) – connection id from airflow Connections
  • remote_host (str) – remote host to connect
  • command (str) – command to execute on remote host. (templated)
  • timeout (int) – timeout for executing the command.
  • do_xcom_push (bool) – return the stdout which also get set in xcom by airflow platform
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.

Sensors

class airflow.contrib.sensors.aws_redshift_cluster_sensor.AwsRedshiftClusterSensor(cluster_identifier, target_status='available', aws_conn_id='aws_default', *args, **kwargs)[source]

Bases: airflow.sensors.base_sensor_operator.BaseSensorOperator

Waits for a Redshift cluster to reach a specific status.

Parameters:
  • cluster_identifier (str) – The identifier for the cluster being pinged.
  • target_status (str) – The cluster status desired.
poke(context)[source]

Function that the sensors defined while deriving this class should override.

class airflow.contrib.sensors.bash_sensor.BashSensor(bash_command, env=None, output_encoding='utf-8', *args, **kwargs)[source]

Bases: airflow.sensors.base_sensor_operator.BaseSensorOperator

Executes a bash command/script and returns True if and only if the return code is 0.

Parameters:
  • bash_command (string) – The command, set of commands or reference to a bash script (must be ‘.sh’) to be executed.
  • 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)
  • output_encoding (string) – output encoding of bash command.
poke(context)[source]

Execute the bash command in a temporary directory which will be cleaned afterwards

class airflow.contrib.sensors.bigquery_sensor.BigQueryTableSensor(project_id, dataset_id, table_id, bigquery_conn_id='bigquery_default_conn', delegate_to=None, *args, **kwargs)[source]

Bases: airflow.sensors.base_sensor_operator.BaseSensorOperator

Checks for the existence of a table in Google Bigquery.

param project_id:
 The Google cloud project in which to look for the table. The connection supplied to the hook must provide access to the specified project.
type project_id:
 string
param dataset_id:
 The name of the dataset in which to look for the table. storage bucket.
type dataset_id:
 string
param table_id:The name of the table to check the existence of.
type table_id:string
param bigquery_conn_id:
 The connection ID to use when connecting to Google BigQuery.
type bigquery_conn_id:
 string
param delegate_to:
 The account to impersonate, if any. For this to work, the service account making the request must have domain-wide delegation enabled.
type delegate_to:
 string
poke(context)[source]

Function that the sensors defined while deriving this class should override.

class airflow.contrib.sensors.cassandra_sensor.CassandraRecordSensor(table, keys, cassandra_conn_id, *args, **kwargs)[source]

Bases: airflow.sensors.base_sensor_operator.BaseSensorOperator

Checks for the existence of a record in a Cassandra cluster.

For example, if you want to wait for a record that has values ‘v1’ and ‘v2’ for each primary keys ‘p1’ and ‘p2’ to be populated in keyspace ‘k’ and table ‘t’, instantiate it as follows:

>>> CassandraRecordSensor(table="k.t", keys={"p1": "v1", "p2": "v2"},
...     cassandra_conn_id="cassandra_default", task_id="cassandra_sensor")
<Task(CassandraRecordSensor): cassandra_sensor>
poke(context)[source]

Function that the sensors defined while deriving this class should override.

class airflow.contrib.sensors.datadog_sensor.DatadogSensor(datadog_conn_id='datadog_default', from_seconds_ago=3600, up_to_seconds_from_now=0, priority=None, sources=None, tags=None, response_check=None, *args, **kwargs)[source]

Bases: airflow.sensors.base_sensor_operator.BaseSensorOperator

A sensor to listen, with a filter, to datadog event streams and determine if some event was emitted.

Depends on the datadog API, which has to be deployed on the same server where Airflow runs.

Parameters:
  • datadog_conn_id – The connection to datadog, containing metadata for api keys.
  • datadog_conn_id – string
poke(context)[source]

Function that the sensors defined while deriving this class should override.

class airflow.contrib.sensors.emr_base_sensor.EmrBaseSensor(aws_conn_id='aws_default', *args, **kwargs)[source]

Bases: airflow.sensors.base_sensor_operator.BaseSensorOperator

Contains general sensor behavior for EMR. Subclasses should implement get_emr_response() and state_from_response() methods. Subclasses should also implement NON_TERMINAL_STATES and FAILED_STATE constants.

poke(context)[source]

Function that the sensors defined while deriving this class should override.

class airflow.contrib.sensors.emr_job_flow_sensor.EmrJobFlowSensor(job_flow_id, *args, **kwargs)[source]

Bases: airflow.contrib.sensors.emr_base_sensor.EmrBaseSensor

Asks for the state of the JobFlow until it reaches a terminal state. If it fails the sensor errors, failing the task.

Parameters:job_flow_id (string) – job_flow_id to check the state of
class airflow.contrib.sensors.emr_step_sensor.EmrStepSensor(job_flow_id, step_id, *args, **kwargs)[source]

Bases: airflow.contrib.sensors.emr_base_sensor.EmrBaseSensor

Asks for the state of the step until it reaches a terminal state. If it fails the sensor errors, failing the task.

Parameters:
  • job_flow_id (string) – job_flow_id which contains the step check the state of
  • step_id (string) – step to check the state of
class airflow.contrib.sensors.file_sensor.FileSensor(filepath, fs_conn_id='fs_default2', *args, **kwargs)[source]

Bases: airflow.sensors.base_sensor_operator.BaseSensorOperator

Waits for a file or folder to land in a filesystem.

If the path given is a directory then this sensor will only return true if any files exist inside it (either directly, or within a subdirectory)

Parameters:
  • fs_conn_id (string) – reference to the File (path) connection id
  • filepath – File or folder name (relative to the base path set within the connection)
poke(context)[source]

Function that the sensors defined while deriving this class should override.

class airflow.contrib.sensors.ftp_sensor.FTPSensor(path, ftp_conn_id='ftp_default', *args, **kwargs)[source]

Bases: airflow.sensors.base_sensor_operator.BaseSensorOperator

Waits for a file or directory to be present on FTP.

Parameters:
  • path (str) – Remote file or directory path
  • ftp_conn_id (str) – The connection to run the sensor against
poke(context)[source]

Function that the sensors defined while deriving this class should override.

class airflow.contrib.sensors.ftp_sensor.FTPSSensor(path, ftp_conn_id='ftp_default', *args, **kwargs)[source]

Bases: airflow.contrib.sensors.ftp_sensor.FTPSensor

Waits for a file or directory to be present on FTP over SSL.

class airflow.contrib.sensors.gcs_sensor.GoogleCloudStorageObjectSensor(bucket, object, google_cloud_conn_id='google_cloud_default', delegate_to=None, *args, **kwargs)[source]

Bases: airflow.sensors.base_sensor_operator.BaseSensorOperator

Checks for the existence of a file in Google Cloud Storage. Create a new GoogleCloudStorageObjectSensor.

param bucket:The Google cloud storage bucket where the object is.
type bucket:string
param object:The name of the object to check in the Google cloud storage bucket.
type object:string
param google_cloud_storage_conn_id:
 The connection ID to use when connecting to Google cloud storage.
type google_cloud_storage_conn_id:
 string
param delegate_to:
 The account to impersonate, if any. For this to work, the service account making the request must have domain-wide delegation enabled.
type delegate_to:
 string
poke(context)[source]

Function that the sensors defined while deriving this class should override.

class airflow.contrib.sensors.gcs_sensor.GoogleCloudStorageObjectUpdatedSensor(bucket, object, ts_func=<function ts_function>, google_cloud_conn_id='google_cloud_default', delegate_to=None, *args, **kwargs)[source]

Bases: airflow.sensors.base_sensor_operator.BaseSensorOperator

Checks if an object is updated in Google Cloud Storage. Create a new GoogleCloudStorageObjectUpdatedSensor.

param bucket:The Google cloud storage bucket where the object is.
type bucket:string
param object:The name of the object to download in the Google cloud storage bucket.
type object:string
param ts_func:Callback for defining the update condition. The default callback returns execution_date + schedule_interval. The callback takes the context as parameter.
type ts_func:function
param google_cloud_storage_conn_id:
 The connection ID to use when connecting to Google cloud storage.
type google_cloud_storage_conn_id:
 string
param delegate_to:
 The account to impersonate, if any. For this to work, the service account making the request must have domain-wide delegation enabled.
type delegate_to:
 string
poke(context)[source]

Function that the sensors defined while deriving this class should override.

class airflow.contrib.sensors.gcs_sensor.GoogleCloudStoragePrefixSensor(bucket, prefix, google_cloud_conn_id='google_cloud_default', delegate_to=None, *args, **kwargs)[source]

Bases: airflow.sensors.base_sensor_operator.BaseSensorOperator

Checks for the existence of a files at prefix in Google Cloud Storage bucket. Create a new GoogleCloudStorageObjectSensor.

param bucket:The Google cloud storage bucket where the object is.
type bucket:string
param prefix:The name of the prefix to check in the Google cloud storage bucket.
type prefix:string
param google_cloud_storage_conn_id:
 The connection ID to use when connecting to Google cloud storage.
type google_cloud_storage_conn_id:
 string
param delegate_to:
 The account to impersonate, if any. For this to work, the service account making the request must have domain-wide delegation enabled.
type delegate_to:
 string
poke(context)[source]

Function that the sensors defined while deriving this class should override.

class airflow.contrib.sensors.hdfs_sensor.HdfsSensorFolder(be_empty=False, *args, **kwargs)[source]

Bases: airflow.sensors.hdfs_sensor.HdfsSensor

poke(context)[source]

poke for a non empty directory

Returns:Bool depending on the search criteria
class airflow.contrib.sensors.hdfs_sensor.HdfsSensorRegex(regex, *args, **kwargs)[source]

Bases: airflow.sensors.hdfs_sensor.HdfsSensor

poke(context)[source]

poke matching files in a directory with self.regex

Returns:Bool depending on the search criteria
class airflow.contrib.sensors.jira_sensor.JiraSensor(jira_conn_id='jira_default', method_name=None, method_params=None, result_processor=None, *args, **kwargs)[source]

Bases: airflow.sensors.base_sensor_operator.BaseSensorOperator

Monitors a jira ticket for any change.

Parameters:
  • jira_conn_id (str) – reference to a pre-defined Jira Connection
  • method_name (str) – method name from jira-python-sdk to be execute
  • method_params (dict) – parameters for the method method_name
  • result_processor (function) – function that return boolean and act as a sensor response
poke(context)[source]

Function that the sensors defined while deriving this class should override.

class airflow.contrib.sensors.pubsub_sensor.PubSubPullSensor(project, subscription, max_messages=5, return_immediately=False, ack_messages=False, gcp_conn_id='google_cloud_default', delegate_to=None, *args, **kwargs)[source]

Bases: airflow.sensors.base_sensor_operator.BaseSensorOperator

Pulls messages from a PubSub subscription and passes them through XCom.

This sensor operator will pull up to max_messages messages from the specified PubSub subscription. When the subscription returns messages, the poke method’s criteria will be fulfilled and the messages will be returned from the operator and passed through XCom for downstream tasks.

If ack_messages is set to True, messages will be immediately acknowledged before being returned, otherwise, downstream tasks will be responsible for acknowledging them.

project and subscription are templated so you can use variables in them.

execute(context)[source]

Overridden to allow messages to be passed

poke(context)[source]

Function that the sensors defined while deriving this class should override.

class airflow.contrib.sensors.qubole_sensor.QuboleSensor(data, qubole_conn_id='qubole_default', *args, **kwargs)[source]

Bases: airflow.sensors.base_sensor_operator.BaseSensorOperator

Base class for all Qubole Sensors

Parameters:
  • qubole_conn_id (string) – The qubole connection to run the sensor against
  • data (a JSON object) – a JSON object containing payload, whose presence needs to be checked

Note

Both data and qubole_conn_id fields are template-supported. You can

also use .txt files for template driven use cases.

poke(context)[source]

Function that the sensors defined while deriving this class should override.

class airflow.contrib.sensors.redis_key_sensor.RedisKeySensor(key, redis_conn_id, *args, **kwargs)[source]

Bases: airflow.sensors.base_sensor_operator.BaseSensorOperator

Checks for the existence of a key in a Redis database

poke(context)[source]

Function that the sensors defined while deriving this class should override.

class airflow.contrib.sensors.sagemaker_base_sensor.SageMakerBaseSensor(aws_conn_id='aws_default', *args, **kwargs)[source]

Bases: airflow.sensors.base_sensor_operator.BaseSensorOperator

Contains general sensor behavior for SageMaker. Subclasses should implement get_sagemaker_response() and state_from_response() methods. Subclasses should also implement NON_TERMINAL_STATES and FAILED_STATE methods.

poke(context)[source]

Function that the sensors defined while deriving this class should override.

class airflow.contrib.sensors.sagemaker_endpoint_sensor.SageMakerEndpointSensor(endpoint_name, *args, **kwargs)[source]

Bases: airflow.contrib.sensors.sagemaker_base_sensor.SageMakerBaseSensor

Asks for the state of the endpoint state until it reaches a terminal state. If it fails the sensor errors, the task fails.

Parameters:job_name (str) – job_name of the endpoint instance to check the state of
class airflow.contrib.sensors.sagemaker_training_sensor.SageMakerTrainingSensor(job_name, print_log=True, *args, **kwargs)[source]

Bases: airflow.contrib.sensors.sagemaker_base_sensor.SageMakerBaseSensor

Asks for the state of the training state until it reaches a terminal state. If it fails the sensor errors, failing the task.

Parameters:
  • job_name (str) – name of the SageMaker training job to check the state of
  • print_log (bool) – if the operator should print the cloudwatch log
class airflow.contrib.sensors.sagemaker_transform_sensor.SageMakerTransformSensor(job_name, *args, **kwargs)[source]

Bases: airflow.contrib.sensors.sagemaker_base_sensor.SageMakerBaseSensor

Asks for the state of the transform state until it reaches a terminal state. The sensor will error if the job errors, throwing a AirflowException containing the failure reason.

Parameters:job_name (string) – job_name of the transform job instance to check the state of
class airflow.contrib.sensors.sagemaker_tuning_sensor.SageMakerTuningSensor(job_name, *args, **kwargs)[source]

Bases: airflow.contrib.sensors.sagemaker_base_sensor.SageMakerBaseSensor

Asks for the state of the tuning state until it reaches a terminal state. The sensor will error if the job errors, throwing a AirflowException containing the failure reason.

Parameters:job_name (str) – job_name of the tuning instance to check the state of
class airflow.contrib.sensors.sftp_sensor.SFTPSensor(path, sftp_conn_id='sftp_default', *args, **kwargs)[source]

Bases: airflow.operators.sensors.BaseSensorOperator

Waits for a file or directory to be present on SFTP. :param path: Remote file or directory path :type path: str :param sftp_conn_id: The connection to run the sensor against :type sftp_conn_id: str

poke(context)[source]

Function that the sensors defined while deriving this class should override.

class airflow.contrib.sensors.wasb_sensor.WasbBlobSensor(container_name, blob_name, wasb_conn_id='wasb_default', check_options=None, *args, **kwargs)[source]

Bases: airflow.sensors.base_sensor_operator.BaseSensorOperator

Waits for a blob to arrive on Azure Blob Storage.

Parameters:
  • container_name (str) – Name of the container.
  • blob_name (str) – Name of the blob.
  • wasb_conn_id (str) – Reference to the wasb connection.
  • check_options (dict) – Optional keyword arguments that WasbHook.check_for_blob() takes.
poke(context)[source]

Function that the sensors defined while deriving this class should override.

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
{{ prev_ds }} the previous execution date as YYYY-MM-DD. if {{ ds }} is 2016-01-08 and schedule_interval is @weekly, {{ prev_ds }} will be 2016-01-01.
{{ next_ds }} the next execution date as YYYY-MM-DD. if {{ ds }} is 2016-01-01 and schedule_interval is @weekly, {{ prev_ds }} will be 2016-01-08.
{{ 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 which can be overridden by the dictionary passed through trigger_dag -c if you enabled dag_run_conf_overrides_params` in ``airflow.cfg
{{ 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'
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_map=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
  • metastore_conn_id (string) – The hive connection you are interested in. If your default is set you don’t need to use this parameter.
  • filter_map (map) – partition_key:partition_value map used for partition filtering, e.g. {‘key1’: ‘value1’, ‘key2’: ‘value2’}. Only partitions matching all partition_key:partition_value pairs will be considered as candidates of max partition.
  • field (str) – 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, weight_rule='downstream', 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, task_concurrency=None, executor_config=None, inlets=None, outlets=None, *args, **kwargs)[source]

Bases: airflow.utils.log.logging_mixin.LoggingMixin

Abstract base class for all operators. Since operators create objects that become nodes 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 class should perform or trigger certain tasks synchronously (wait for completion). Example of operators could be an operator that 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.

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 the execution_date for the first task instance. The best practice is to have the start_date rounded to your DAG’s schedule_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 latest execution_date and adds the schedule_interval to determine the next execution_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 the TimeSensor and TimeDeltaSensor. We advise against using dynamic start_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.
  • weight_rule (str) – weighting method used for the effective total priority weight of the task. Options are: { downstream | upstream | absolute } default is downstream When set to downstream the effective weight of the task is the aggregate sum of all downstream descendants. As a result, upstream tasks will have higher weight and will be scheduled more aggressively when using positive weight values. This is useful when you have multiple dag run instances and desire to have all upstream tasks to complete for all runs before each dag can continue processing downstream tasks. When set to upstream the effective weight is the aggregate sum of all upstream ancestors. This is the opposite where downtream tasks have higher weight and will be scheduled more aggressively when using positive weight values. This is useful when you have multiple dag run instances and prefer to have each dag complete before starting upstream tasks of other dags. When set to absolute, the effective weight is the exact priority_weight specified without additional weighting. You may want to do this when you know exactly what priority weight each task should have. Additionally, when set to absolute, there is bonus effect of significantly speeding up the task creation process as for very large DAGS. Options can be set as string or using the constants defined in the static class airflow.utils.WeightRule
  • 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 an email soon after 1:00AM on the 2016-01-02 if the 2016-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 (callable) – much like the on_failure_callback except that it is executed when retries occur.
  • on_success_callback (callable) – much like the on_failure_callback except 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 is all_success. Options can be set as string or using the constants defined in the static class airflow.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
  • task_concurrency (int) – When set, a task will be able to limit the concurrent runs across execution_dates
  • executor_config (dict) –

    Additional task-level configuration parameters that are interpreted by a specific executor. Parameters are namespaced by the name of executor.

    Example: to run this task in a specific docker container through the KubernetesExecutor

    MyOperator(...,
        executor_config={
        "KubernetesExecutor":
            {"image": "myCustomDockerImage"}
            }
    )
    
clear(start_date=None, end_date=None, upstream=False, downstream=False, session=None)[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/overridden by subclasses.

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_relative_ids(upstream=False)[source]

Get the direct relative ids to the current task, upstream or downstream.

get_direct_relatives(upstream=False)[source]

Get the direct relatives to the current task, upstream or downstream.

get_flat_relative_ids(upstream=False, found_descendants=None)[source]

Get a flat list of relatives’ ids, either upstream or downstream.

get_flat_relatives(upstream=False)[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, result=None)[source]

This hook is triggered right after self.execute() is called. It is passed the execution context and any results returned by the operator.

pre_execute(context)[source]

This hook is triggered right before self.execute() is called.

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 list to be directly downstream from the current task.

set_upstream(task_or_task_list)[source]

Set a task or a task list to be directly upstream from the current task.

upstream_list

@property: list of tasks directly upstream

xcom_pull(context, task_ids=None, 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.Chart(**kwargs)[source]

Bases: sqlalchemy.ext.declarative.api.Base

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, airflow.utils.log.logging_mixin.LoggingMixin

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, user_defined_filters=None, default_args=None, concurrency=16, max_active_runs=16, dagrun_timeout=None, sla_miss_callback=None, default_view='tree', orientation='LR', catchup=True, on_success_callback=None, on_failure_callback=None, params=None)[source]

Bases: airflow.dag.base_dag.BaseDag, airflow.utils.log.logging_mixin.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.
  • user_defined_filters (dict) – a dictionary of filters that will be exposed in your jinja templates. For example, passing dict(hello=lambda name: 'Hello %s' % name) to this argument allows you to {{ 'world' | hello }} in all jinja templates related to this DAG.
  • 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.
  • default_view (string) – Specify DAG default view (tree, graph, duration, gantt, landing_times)
  • orientation (string) – Specify DAG orientation in graph view (LR, TB, RL, BT)
  • catchup (bool) – Perform scheduler catchup (or only run latest)? Defaults to True
  • on_failure_callback (callable) – A function to be called when a DagRun of this dag fails. A context dictionary is passed as a single parameter to this function.
  • on_success_callback (callable) – Much like the on_failure_callback except that it is executed when the dag succeeds.
add_task(task)[source]

Add a task to the DAG

Parameters:task (task) – the task you want to add
add_tasks(tasks)[source]

Add a list of tasks to the DAG

Parameters:tasks (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, include_parentdag=True, reset_dag_runs=True, dry_run=False, session=None, get_tis=False)[source]

Clears a set of task instances associated with the current dag for a specified date range.

cli()[source]

Exposes a CLI specific to this DAG

concurrency
Returns:maximum number of tasks that can run simultaneously from this DAG
Return type:int
concurrency_reached

Returns a boolean indicating whether the concurrency limit for this DAG has been reached

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
dag_id
Returns:the DAG ID
Return type:unicode
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 (datetime) – set inactive DAGs that were touched before this time
Returns: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

following_schedule(dttm)[source]

Calculates the following schedule for this dag in UTC.

Parameters:dttm – utc datetime
Returns:utc datetime
full_filepath
Returns:The absolute path to the file that contains this DAG’s definition
Return type:unicode
get_active_runs(session=None)[source]

Returns a list of dag run execution dates currently running

Parameters:session
Returns: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.

Parameters:
  • execution_date – The execution date of the DagRun to find.
  • session
Returns:

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. Overridden DagRuns are ignored

get_num_active_runs(external_trigger=None, session=None)[source]

Returns the number of active “running” dag runs

Parameters:
  • external_trigger (bool) – True for externally triggered active dag runs
  • session
Returns:

number greater than 0 for active dag runs

static get_num_task_instances(dag_id, task_ids, states=None, session=None)[source]

Returns the number of task instances 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
  • states (list[state]) – A list of states to filter by if supplied
Returns:

The number of running tasks

Return type:

int

get_run_dates(start_date, end_date=None)[source]

Returns a list of dates between the interval received as parameter using this dag’s schedule interval. Returned dates can be used for execution dates.

Parameters:
  • start_date (datetime) – the start date of the interval
  • end_date (datetime) – the end date of the interval, defaults to timezone.utcnow()
Returns:

a list of dates within the interval following the dag’s schedule

Return type:

list

get_template_env()[source]

Returns a jinja2 Environment while taking into account the DAGs template_searchpath, user_defined_macros and user_defined_filters

handle_callback(dagrun, success=True, reason=None, session=None)[source]

Triggers the appropriate callback depending on the value of success, namely the on_failure_callback or on_success_callback. This method gets the context of a single TaskInstance part of this DagRun and passes that to the callable along with a ‘reason’, primarily to differentiate DagRun failures. .. note:

The logs end up in $AIRFLOW_HOME/logs/scheduler/latest/PROJECT/DAG_FILE.py.log
Parameters:
  • dagrun – DagRun object
  • success – Flag to specify if failure or success callback should be called
  • reason – Completion reason
  • session – Database session
is_fixed_time_schedule()[source]

Figures out if the DAG schedule has a fixed time (e.g. 3 AM).

Returns:True if the schedule has a fixed time, False if not.
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

pickle_id
Returns:The pickle ID for this DAG, if it has one. Otherwise None.
Return type:unicode
previous_schedule(dttm)[source]

Calculates the previous schedule for this dag in UTC

Parameters:dttm – utc datetime
Returns:utc datetime
run(start_date=None, end_date=None, mark_success=False, local=False, executor=None, donot_pickle=False, ignore_task_deps=False, ignore_first_depends_on_past=False, pool=None, delay_on_limit_secs=1.0, verbose=False, conf=None, rerun_failed_tasks=False)[source]

Runs the DAG.

Parameters:
  • start_date (datetime) – the start date of the range to run
  • end_date (datetime) – the end date of the range to run
  • mark_success (bool) – True to mark jobs as succeeded without running them
  • local (bool) – True to run the tasks using the LocalExecutor
  • executor (BaseExecutor) – The executor instance to run the tasks
  • donot_pickle (bool) – True to avoid pickling DAG object and send to workers
  • ignore_task_deps (bool) – True to skip upstream tasks
  • ignore_first_depends_on_past (bool) – True to ignore depends_on_past dependencies for the first set of tasks only
  • pool (string) – Resource pool to use
  • delay_on_limit_secs (float) – Time in seconds to wait before next attempt to run dag run when max_active_runs limit has been reached
  • verbose (boolean) – Make logging output more verbose
  • conf (dict) – user defined dictionary passed from CLI
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

sync_to_db(owner=None, sync_time=None, 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
  • sync_time (datetime) – The time that the DAG should be marked as sync’ed
Returns:

None

task_ids
Returns:A list of task IDs that are in this DAG
Return type:List[unicode]
test_cycle()[source]

Check to see if there are any cycles in the DAG. Returns False if no cycle found, otherwise raises exception.

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
tree_view()[source]

Shows an ascii tree representation of the DAG

class airflow.models.DagBag(dag_folder=None, executor=None, include_examples=True)[source]

Bases: airflow.dag.base_dag.BaseDagBag, airflow.utils.log.logging_mixin.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
  • has_logged – an instance boolean that gets flipped from False to True after a file has been skipped. This is to prevent overloading the user with logging messages about skipped files. Therefore only once per DagBag is a file logged being skipped.
bag_dag(dag, parent_dag, root_dag)[source]

Adds the DAG into the bag, recurses into sub dags. Throws AirflowDagCycleException if a cycle is detected in this dag or its subdags

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 behave much like a .gitignore, ignoring files that match any of the regex patterns specified in the file.

Note: The patterns in .airflowignore are treated as un-anchored regexes, not shell-like glob patterns.

dagbag_report()[source]

Prints a report around DagBag loading stats

get_dag(dag_id)[source]

Gets the DAG out of the dictionary, and refreshes it if expired

kill_zombies(session=None)[source]

Fails tasks that haven’t had a heartbeat in too long

process_file(filepath, only_if_updated=True, safe_mode=True)[source]

Given a path to a python module or zip file, this method imports the module and look for dag objects within it.

size()[source]
Returns:the amount of dags contained in this dagbag
class airflow.models.DagModel(**kwargs)[source]

Bases: sqlalchemy.ext.declarative.api.Base

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, airflow.utils.log.logging_mixin.LoggingMixin

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, no_backfills=False, session=None)[source]

Returns a set of dag runs for the given search criteria.

Parameters:
  • dag_id (integer, list) – the dag_id to find dag runs for
  • run_id (string) – defines the the run id for this dag run
  • execution_date (datetime) – the execution date
  • state (State) – the state of the dag run
  • external_trigger (bool) – whether this dag run is externally triggered
  • no_backfills – return no backfills (True), return all (False).

Defaults to False :type no_backfills: bool :param session: database session :type session: Session

get_dag()[source]

Returns the Dag associated with this DagRun.

Returns:DAG
classmethod get_latest_runs(session)[source]

Returns the latest DagRun for each DAG.

get_previous_dagrun(session=None)[source]

The previous DagRun, if there is one

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

get_task_instance(task_id, session=None)[source]

Returns the task instance specified by task_id for this dag run

Parameters:task_id – the task id
get_task_instances(state=None, session=None)[source]

Returns the task instances for this dag run

refresh_from_db(session=None)[source]

Reloads the current dagrun from the database :param session: database session

update_state(session=None)[source]

Determines the overall state of the DagRun based on the state of its TaskInstances.

Returns:State
verify_integrity(session=None)[source]

Verifies the DagRun by checking for removed tasks or tasks that are not in the database yet. It will set state to removed or add the task if required.

class airflow.models.DagStat(dag_id, state, count=0, dirty=False)[source]

Bases: sqlalchemy.ext.declarative.api.Base

static create(dag_id, session=None)[source]

Creates the missing states the stats table for the dag specified

Parameters:
  • dag_id – dag id of the dag to create stats for
  • session – database session
Returns:

static set_dirty(dag_id, session=None)[source]
Parameters:
  • dag_id – the dag_id to mark dirty
  • session – database session
Returns:

static update(dag_ids=None, dirty_only=True, session=None)[source]

Updates the stats for dirty/out-of-sync dags

Parameters:
  • dag_ids (list) – dag_ids to be updated
  • dirty_only (bool) – only updated for marked dirty, defaults to True
  • session (Session) – db session to use
class airflow.models.ImportError(**kwargs)[source]

Bases: sqlalchemy.ext.declarative.api.Base

exception airflow.models.InvalidFernetToken[source]

Bases: Exception

class airflow.models.KnownEvent(**kwargs)[source]

Bases: sqlalchemy.ext.declarative.api.Base

class airflow.models.KnownEventType(**kwargs)[source]

Bases: sqlalchemy.ext.declarative.api.Base

class airflow.models.KubeResourceVersion(**kwargs)[source]

Bases: sqlalchemy.ext.declarative.api.Base

class airflow.models.KubeWorkerIdentifier(**kwargs)[source]

Bases: sqlalchemy.ext.declarative.api.Base

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.NullFernet[source]

Bases: object

A “Null” encryptor class that doesn’t encrypt or decrypt but that presents a similar interface to Fernet.

The purpose of this is to make the rest of the code not have to know the difference, and to only display the message once, not 20 times when airflow initdb is ran.

class airflow.models.Pool(**kwargs)[source]

Bases: sqlalchemy.ext.declarative.api.Base

open_slots(session)[source]

Returns the number of slots open at the moment

queued_slots(session)[source]

Returns the number of slots used at the moment

used_slots(session)[source]

Returns the number of slots used at the moment

class airflow.models.SkipMixin(context=None)[source]

Bases: airflow.utils.log.logging_mixin.LoggingMixin

skip(dag_run, execution_date, tasks, session=None)[source]

Sets tasks instances to skipped from the same dag run.

Parameters:
  • dag_run – the DagRun for which to set the tasks to skipped
  • execution_date – execution_date
  • tasks – tasks to skip (not task_ids)
  • session – db session to use
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, airflow.utils.log.logging_mixin.LoggingMixin

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 SqlAlchemy 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 log details on failed dependencies on info or debug log level
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.

clear_xcom_data(session=None)[source]

Clears all XCom data from the database for the task instance

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.

error(session=None)[source]

Forces the task instance’s state to FAILED in the database.

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 ignorable 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 (unicode) – If the DAG was serialized to the DB, the ID associated with the pickled DAG
  • file_path – path to the file containing the DAG definition
  • raw – raw mode (needs more details)
  • job_id – job ID (needs more details)
  • pool (unicode) – the Airflow pool that the task should run in
  • cfg_path (basestring) – the Path to the configuration file
Returns:

shell command that can be used to run the task instance

get_dagrun(session)[source]

Returns the DagRun for this TaskInstance

Parameters:session
Returns:DagRun
init_on_load()[source]

Initialize the attributes that aren’t stored in the DB.

init_run_context(raw=False)[source]

Sets the log context.

is_eligible_to_retry()[source]

Is task instance is eligible for retry

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.
try_number

Return the try number that this task number will be when it is acutally run.

If the TI is currently running, this will match the column in the databse, in all othercases this will be incremenetd

xcom_pull(task_ids=None, 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.User(**kwargs)[source]

Bases: sqlalchemy.ext.declarative.api.Base

class airflow.models.Variable(**kwargs)[source]

Bases: sqlalchemy.ext.declarative.api.Base, airflow.utils.log.logging_mixin.LoggingMixin

classmethod setdefault(key, default, deserialize_json=False)[source]

Like a Python builtin dict object, setdefault returns the current value for a key, and if it isn’t there, stores the default value and returns it.

Parameters:
  • key (String) – Dict key for this Variable
  • default – Default value to set and return if the variable

isn’t already in the DB :type default: Mixed :param deserialize_json: Store this as a JSON encoded value in the DB

and un-encode it when retrieving a value
Returns:Mixed
class airflow.models.XCom(**kwargs)[source]

Bases: sqlalchemy.ext.declarative.api.Base, airflow.utils.log.logging_mixin.LoggingMixin

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 TODO: “pickling” has been deprecated and JSON is preferred.

“pickling” will be removed in Airflow 2.0.
classmethod get_one(execution_date, key=None, task_id=None, dag_id=None, include_prior_dates=False, session=None)[source]

Retrieve an XCom value, optionally meeting certain criteria. TODO: “pickling” has been deprecated and JSON is preferred.

“pickling” will be removed in Airflow 2.0.
Returns:XCom value
classmethod set(key, value, execution_date, task_id, dag_id, session=None)[source]

Store an XCom value. TODO: “pickling” has been deprecated and JSON is preferred.

“pickling” will be removed in Airflow 2.0.
Returns:None
airflow.models.clear_task_instances(tis, session, activate_dag_runs=True, dag=None)[source]

Clears a set of task instances, but makes sure the running ones get killed.

Parameters:
  • tis – a list of task instances
  • session – current session
  • activate_dag_runs – flag to check for active dag run
  • dag – DAG object
airflow.models.get_fernet()[source]

Deferred load of Fernet key.

This function could fail either because Cryptography is not installed or because the Fernet key is invalid.

Returns:Fernet object
Raises:AirflowException if there’s a problem trying to load Fernet

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_autocommit(conn)[source]

Get autocommit setting for the provided connection. Return True if conn.autocommit is set to True. Return False if conn.autocommit is not set or set to False or conn does not support autocommit. :param conn: Connection to get autocommit setting from. :type conn: connection object. :return: connection autocommit setting. :rtype bool.

get_conn()[source]

Returns a connection object

get_cursor()[source]

Returns a cursor

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, replace=False)[source]

A generic way to insert a set of tuples into a table, a new transaction is created every commit_every rows

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.
  • replace (bool) – Whether to replace instead of insert
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.
set_autocommit(conn, autocommit)[source]

Sets the autocommit flag on the connection

class airflow.hooks.docker_hook.DockerHook(docker_conn_id='docker_default', base_url=None, version=None, tls=None)[source]

Bases: airflow.hooks.base_hook.BaseHook, airflow.utils.log.logging_mixin.LoggingMixin

Interact with a private Docker registry.

Parameters:docker_conn_id (str) – ID of the Airflow connection where credentials and extra configuration are stored
class airflow.hooks.hive_hooks.HiveCliHook(hive_cli_conn_id='hive_cli_default', run_as=None, mapred_queue=None, mapred_queue_priority=None, mapred_job_name=None)[source]

Bases: airflow.hooks.base_hook.BaseHook

Simple wrapper around the hive CLI.

It also supports the beeline a lighter CLI that runs JDBC and is replacing the heavier traditional CLI. To enable beeline, set the use_beeline param in the extra field of your connection as in { "use_beeline": true }

Note that you can also set default hive CLI parameters using the hive_cli_params to be used in your connection as in {"hive_cli_params": "-hiveconf mapred.job.tracker=some.jobtracker:444"} Parameters passed here can be overridden by run_cli’s hive_conf param

The extra connection parameter auth gets passed as in the jdbc connection string as is.

Parameters:
  • mapred_queue (string) – queue used by the Hadoop Scheduler (Capacity or Fair)
  • mapred_queue_priority (string) – priority within the job queue. Possible settings include: VERY_HIGH, HIGH, NORMAL, LOW, VERY_LOW
  • mapred_job_name (string) – This name will appear in the jobtracker. This can make monitoring easier.
load_df(df, table, field_dict=None, delimiter=', ', encoding='utf8', pandas_kwargs=None, **kwargs)[source]

Loads a pandas DataFrame into hive.

Hive data types will be inferred if not passed but column names will not be sanitized.

Parameters:
  • df (DataFrame) – DataFrame to load into a Hive table
  • table (str) – target Hive table, use dot notation to target a specific database
  • field_dict (OrderedDict) – mapping from column name to hive data type. Note that it must be OrderedDict so as to keep columns’ order.
  • delimiter (str) – field delimiter in the file
  • encoding (str) – string encoding to use when writing DataFrame to file
  • pandas_kwargs (dict) – passed to DataFrame.to_csv
  • kwargs – passed to self.load_file
load_file(filepath, table, delimiter=', ', field_dict=None, create=True, overwrite=True, partition=None, recreate=False, tblproperties=None)[source]

Loads a local file into Hive

Note that the table generated in Hive uses STORED AS textfile which isn’t the most efficient serialization format. If a large amount of data is loaded and/or if the tables gets queried considerably, you may want to use this operator only to stage the data into a temporary table before loading it into its final destination using a HiveOperator.

Parameters:
  • filepath (str) – local filepath of the file to load
  • table (str) – target Hive table, use dot notation to target a specific database
  • delimiter (str) – field delimiter in the file
  • field_dict (OrderedDict) – A dictionary of the fields name in the file as keys and their Hive types as values. Note that it must be OrderedDict so as to keep columns’ order.
  • create (bool) – whether to create the table if it doesn’t exist
  • overwrite (bool) – whether to overwrite the data in table or partition
  • partition (dict) – target partition as a dict of partition columns and values
  • recreate (bool) – whether to drop and recreate the table at every execution
  • tblproperties (dict) – TBLPROPERTIES of the hive table being created
run_cli(hql, schema=None, verbose=True, hive_conf=None)[source]

Run an hql statement using the hive cli. If hive_conf is specified it should be a dict and the entries will be set as key/value pairs in HiveConf

Parameters:hive_conf (dict) – if specified these key value pairs will be passed to hive as -hiveconf "key"="value". Note that they will be passed after the hive_cli_params and thus will override whatever values are specified in the database.
>>> hh = HiveCliHook()
>>> result = hh.run_cli("USE airflow;")
>>> ("OK" in result)
True
test_hql(hql)[source]

Test an hql statement using the hive cli and EXPLAIN

class airflow.hooks.hive_hooks.HiveMetastoreHook(metastore_conn_id='metastore_default')[source]

Bases: airflow.hooks.base_hook.BaseHook

Wrapper to interact with the Hive Metastore

check_for_named_partition(schema, table, partition_name)[source]

Checks whether a partition with a given name exists

Parameters:
  • schema (string) – Name of hive schema (database) @table belongs to
  • table – Name of hive table @partition belongs to
Partition:

Name of the partitions to check for (eg a=b/c=d)

Return type:

boolean

>>> hh = HiveMetastoreHook()
>>> t = 'static_babynames_partitioned'
>>> hh.check_for_named_partition('airflow', t, "ds=2015-01-01")
True
>>> hh.check_for_named_partition('airflow', t, "ds=xxx")
False
check_for_partition(schema, table, partition)[source]

Checks whether a partition exists

Parameters:
  • schema (string) – Name of hive schema (database) @table belongs to
  • table – Name of hive table @partition belongs to
Partition:

Expression that matches the partitions to check for (eg a = ‘b’ AND c = ‘d’)

Return type:

boolean

>>> hh = HiveMetastoreHook()
>>> t = 'static_babynames_partitioned'
>>> hh.check_for_partition('airflow', t, "ds='2015-01-01'")
True
get_databases(pattern='*')[source]

Get a metastore table object

get_metastore_client()[source]

Returns a Hive thrift client.

get_partitions(schema, table_name, filter=None)[source]

Returns a list of all partitions in a table. Works only for tables with less than 32767 (java short max val). For subpartitioned table, the number might easily exceed this.

>>> hh = HiveMetastoreHook()
>>> t = 'static_babynames_partitioned'
>>> parts = hh.get_partitions(schema='airflow', table_name=t)
>>> len(parts)
1
>>> parts
[{'ds': '2015-01-01'}]
get_table(table_name, db='default')[source]

Get a metastore table object

>>> hh = HiveMetastoreHook()
>>> t = hh.get_table(db='airflow', table_name='static_babynames')
>>> t.tableName
'static_babynames'
>>> [col.name for col in t.sd.cols]
['state', 'year', 'name', 'gender', 'num']
get_tables(db, pattern='*')[source]

Get a metastore table object

max_partition(schema, table_name, field=None, filter_map=None)[source]

Returns the maximum value for all partitions with given field in a table. If only one partition key exist in the table, the key will be used as field. filter_map should be a partition_key:partition_value map and will be used to filter out partitions.

Parameters:
  • schema (string) – schema name.
  • table_name (string) – table name.
  • field (string) – partition key to get max partition from.
  • filter_map (map) – partition_key:partition_value map used for partition filtering.
>>> hh = HiveMetastoreHook()
>>> filter_map = {'ds': '2015-01-01', 'ds': '2014-01-01'}
>>> t = 'static_babynames_partitioned'
>>> hh.max_partition(schema='airflow',        ... table_name=t, field='ds', filter_map=filter_map)
'2015-01-01'
table_exists(table_name, db='default')[source]

Check if table exists

>>> hh = HiveMetastoreHook()
>>> hh.table_exists(db='airflow', table_name='static_babynames')
True
>>> hh.table_exists(db='airflow', table_name='does_not_exist')
False
class airflow.hooks.hive_hooks.HiveServer2Hook(hiveserver2_conn_id='hiveserver2_default')[source]

Bases: airflow.hooks.base_hook.BaseHook

Wrapper around the pyhive library

Note that the default authMechanism is PLAIN, to override it you can specify it in the extra of your connection in the UI as in

get_pandas_df(hql, schema='default')[source]

Get a pandas dataframe from a Hive query

>>> hh = HiveServer2Hook()
>>> sql = "SELECT * FROM airflow.static_babynames LIMIT 100"
>>> df = hh.get_pandas_df(sql)
>>> len(df.index)
100
get_records(hql, schema='default')[source]

Get a set of records from a Hive query.

>>> hh = HiveServer2Hook()
>>> sql = "SELECT * FROM airflow.static_babynames LIMIT 100"
>>> len(hh.get_records(sql))
100
class airflow.hooks.http_hook.HttpHook(method='POST', http_conn_id='http_default')[source]

Bases: airflow.hooks.base_hook.BaseHook

Interact with HTTP servers. :param http_conn_id: connection that has the base API url i.e https://www.google.com/

and optional authentication credentials. Default headers can also be specified in the Extra field in json format.
Parameters:method (str) – the API method to be called
check_response(response)[source]

Checks the status code and raise an AirflowException exception on non 2XX or 3XX status codes :param response: A requests response object :type response: requests.response

get_conn(headers=None)[source]

Returns http session for use with requests :param headers: additional headers to be passed through as a dictionary :type headers: dict

run(endpoint, data=None, headers=None, extra_options=None)[source]

Performs the request :param endpoint: the endpoint to be called i.e. resource/v1/query? :type endpoint: str :param data: payload to be uploaded or request parameters :type data: dict :param headers: additional headers to be passed through as a dictionary :type headers: dict :param extra_options: additional options to be used when executing the request

i.e. {‘check_response’: False} to avoid checking raising exceptions on non 2XX or 3XX status codes
run_and_check(session, prepped_request, extra_options)[source]

Grabs extra options like timeout and actually runs the request, checking for the result :param session: the session to be used to execute the request :type session: requests.Session :param prepped_request: the prepared request generated in run() :type prepped_request: session.prepare_request :param extra_options: additional options to be used when executing the request

i.e. {‘check_response’: False} to avoid checking raising exceptions on non 2XX or 3XX status codes
run_with_advanced_retry(_retry_args, *args, **kwargs)[source]

Runs Hook.run() with a Tenacity decorator attached to it. This is useful for connectors which might be disturbed by intermittent issues and should not instantly fail. :param _retry_args: Arguments which define the retry behaviour.

See Tenacity documentation at https://github.com/jd/tenacity
Example: ::

hook = HttpHook(http_conn_id=’my_conn’,method=’GET’) retry_args = dict(

wait=tenacity.wait_exponential(), stop=tenacity.stop_after_attempt(10), retry=requests.exceptions.ConnectionError

) hook.run_with_advanced_retry(

endpoint=’v1/test’, _retry_args=retry_args

)

class airflow.hooks.druid_hook.DruidDbApiHook(*args, **kwargs)[source]

Bases: airflow.hooks.dbapi_hook.DbApiHook

Interact with Druid broker

This hook is purely for users to query druid broker. For ingestion, please use druidHook.

get_conn()[source]

Establish a connection to druid broker.

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_uri()[source]

Get the connection uri for druid broker.

e.g: druid://localhost:8082/druid/v2/sql/

insert_rows(table, rows, target_fields=None, commit_every=1000)[source]

A generic way to insert a set of tuples into a table, a new transaction is created every commit_every rows

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.
  • replace (bool) – Whether to replace instead of insert
set_autocommit(conn, autocommit)[source]

Sets the autocommit flag on the connection

class airflow.hooks.druid_hook.DruidHook(druid_ingest_conn_id='druid_ingest_default', timeout=1, max_ingestion_time=None)[source]

Bases: airflow.hooks.base_hook.BaseHook

Connection to Druid overlord for ingestion

Parameters:
  • druid_ingest_conn_id (string) – The connection id to the Druid overlord machine which accepts index jobs
  • timeout (int) – The interval between polling the Druid job for the status of the ingestion job. Must be greater than or equal to 1
  • max_ingestion_time (int) – The maximum ingestion time before assuming the job failed
class airflow.hooks.hdfs_hook.HDFSHook(hdfs_conn_id='hdfs_default', proxy_user=None, autoconfig=False)[source]

Bases: airflow.hooks.base_hook.BaseHook

Interact with HDFS. This class is a wrapper around the snakebite library.

Parameters:
  • hdfs_conn_id – Connection id to fetch connection info
  • proxy_user (string) – effective user for HDFS operations
  • autoconfig (bool) – use snakebite’s automatically configured client
get_conn()[source]

Returns a snakebite HDFSClient object.

class airflow.hooks.jdbc_hook.JdbcHook(*args, **kwargs)[source]

Bases: airflow.hooks.dbapi_hook.DbApiHook

General hook for jdbc db access.

JDBC URL, username and password will be taken from the predefined connection. Note that the whole JDBC URL must be specified in the “host” field in the DB. Raises an airflow error if the given connection id doesn’t exist.

get_conn()[source]

Returns a connection object

set_autocommit(conn, autocommit)[source]

Enable or disable autocommit for the given connection.

Parameters:conn – The connection
Returns:
class airflow.hooks.mssql_hook.MsSqlHook(*args, **kwargs)[source]

Bases: airflow.hooks.dbapi_hook.DbApiHook

Interact with Microsoft SQL Server.

get_conn()[source]

Returns a mssql connection object

set_autocommit(conn, autocommit)[source]

Sets the autocommit flag on the connection

class airflow.hooks.mysql_hook.MySqlHook(*args, **kwargs)[source]

Bases: airflow.hooks.dbapi_hook.DbApiHook

Interact with MySQL.

You can specify charset in the extra field of your connection as {"charset": "utf8"}. Also you can choose cursor as {"cursor": "SSCursor"}. Refer to the MySQLdb.cursors for more details.

bulk_dump(table, tmp_file)[source]

Dumps a database table into a tab-delimited file

bulk_load(table, tmp_file)[source]

Loads a tab-delimited file into a database table

get_autocommit(conn)[source]

MySql connection gets autocommit in a different way. :param conn: connection to get autocommit setting from. :type conn: connection object. :return: connection autocommit setting :rtype bool

get_conn()[source]

Returns a mysql connection object

set_autocommit(conn, autocommit)[source]

MySql connection sets autocommit in a different way.

class airflow.hooks.oracle_hook.OracleHook(*args, **kwargs)[source]

Bases: airflow.hooks.dbapi_hook.DbApiHook

Interact with Oracle SQL.

bulk_insert_rows(table, rows, target_fields=None, commit_every=5000)[source]

A performant bulk insert for cx_Oracle that uses prepared statements via executemany(). For best performance, pass in rows as an iterator.

get_conn()[source]

Returns a oracle connection object Optional parameters for using a custom DSN connection (instead of using a server alias from tnsnames.ora) The dsn (data source name) is the TNS entry (from the Oracle names server or tnsnames.ora file) or is a string like the one returned from makedsn().

Parameters:
  • dsn – the host address for the Oracle server
  • service_name – the db_unique_name of the database that you are connecting to (CONNECT_DATA part of TNS)

You can set these parameters in the extra fields of your connection as in { "dsn":"some.host.address" , "service_name":"some.service.name" }

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 Changes from standard DbApiHook implementation: - Oracle SQL queries in cx_Oracle can not be terminated with a semicolon (‘;’) - Replace NaN values with NULL using numpy.nan_to_num (not using is_nan()

because of input types error for strings)
  • Coerce datetime cells to Oracle DATETIME format during insert
class airflow.hooks.pig_hook.PigCliHook(pig_cli_conn_id='pig_cli_default')[source]

Bases: airflow.hooks.base_hook.BaseHook

Simple wrapper around the pig CLI.

Note that you can also set default pig CLI properties using the pig_properties to be used in your connection as in {"pig_properties": "-Dpig.tmpfilecompression=true"}

run_cli(pig, verbose=True)[source]

Run an pig script using the pig cli

>>> ph = PigCliHook()
>>> result = ph.run_cli("ls /;")
>>> ("hdfs://" in result)
True
class airflow.hooks.postgres_hook.PostgresHook(*args, **kwargs)[source]

Bases: airflow.hooks.dbapi_hook.DbApiHook

Interact with Postgres. You can specify ssl parameters in the extra field of your connection as {"sslmode": "require", "sslcert": "/path/to/cert.pem", etc}.

Note: For Redshift, use keepalives_idle in the extra connection parameters and set it to less than 300 seconds.

bulk_dump(table, tmp_file)[source]

Dumps a database table into a tab-delimited file

bulk_load(table, tmp_file)[source]

Loads a tab-delimited file into a database table

copy_expert(sql, filename, open=<built-in function open>)[source]

Executes SQL using psycopg2 copy_expert method. Necessary to execute COPY command without access to a superuser.

Note: if this method is called with a “COPY FROM” statement and the specified input file does not exist, it creates an empty file and no data is loaded, but the operation succeeds. So if users want to be aware when the input file does not exist, they have to check its existence by themselves.

get_conn()[source]

Returns a connection object

class airflow.hooks.presto_hook.PrestoHook(*args, **kwargs)[source]

Bases: airflow.hooks.dbapi_hook.DbApiHook

Interact with Presto through PyHive!

>>> ph = PrestoHook()
>>> sql = "SELECT count(1) AS num FROM airflow.static_babynames"
>>> ph.get_records(sql)
[[340698]]
get_conn()[source]

Returns a connection object

get_first(hql, parameters=None)[source]

Returns only the first row, regardless of how many rows the query returns.

get_pandas_df(hql, parameters=None)[source]

Get a pandas dataframe from a sql query.

get_records(hql, parameters=None)[source]

Get a set of records from Presto

insert_rows(table, rows, target_fields=None)[source]

A generic way to insert a set of tuples into a table.

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
run(hql, parameters=None)[source]

Execute the statement against Presto. Can be used to create views.

class airflow.hooks.S3_hook.S3Hook(aws_conn_id='aws_default')[source]

Bases: airflow.contrib.hooks.aws_hook.AwsHook

Interact with AWS S3, using the boto3 library.

check_for_bucket(bucket_name)[source]

Check if bucket_name exists.

Parameters:bucket_name (str) – the name of the bucket
check_for_key(key, bucket_name=None)[source]

Checks if a key exists in a bucket

Parameters:
  • key (str) – S3 key that will point to the file
  • bucket_name (str) – Name of the bucket in which the file is stored
check_for_prefix(bucket_name, prefix, delimiter)[source]

Checks that a prefix exists in a bucket

check_for_wildcard_key(wildcard_key, bucket_name=None, delimiter='')[source]

Checks that a key matching a wildcard expression exists in a bucket

copy_object(source_bucket_key, dest_bucket_key, source_bucket_name=None, dest_bucket_name=None, source_version_id=None)[source]

Creates a copy of an object that is already stored in S3.

Note: the S3 connection used here needs to have access to both source and destination bucket/key.

Parameters:
  • source_bucket_key (str) –

    The key of the source object.

    It can be either full s3:// style url or relative path from root level.

    When it’s specified as a full s3:// url, please omit source_bucket_name.

  • dest_bucket_key (str) –

    The key of the object to copy to.

    The convention to specify dest_bucket_key is the same as source_bucket_key.

  • source_bucket_name (str) –

    Name of the S3 bucket where the source object is in.

    It should be omitted when source_bucket_key is provided as a full s3:// url.

  • dest_bucket_name (str) –

    Name of the S3 bucket to where the object is copied.

    It should be omitted when dest_bucket_key is provided as a full s3:// url.

  • source_version_id (str) – Version ID of the source object (OPTIONAL)
create_bucket(bucket_name, region_name=None)[source]

Creates an Amazon S3 bucket.

Parameters:
  • bucket_name (str) – The name of the bucket
  • region_name (str) – The name of the aws region in which to create the bucket.
delete_objects(bucket, keys)[source]
Parameters:
  • bucket (str) – Name of the bucket in which you are going to delete object(s)
  • keys (str or list) –

    The key(s) to delete from S3 bucket.

    When keys is a string, it’s supposed to be the key name of the single object to delete.

    When keys is a list, it’s supposed to be the list of the keys to delete.

get_bucket(bucket_name)[source]

Returns a boto3.S3.Bucket object

Parameters:bucket_name (str) – the name of the bucket
get_key(key, bucket_name=None)[source]

Returns a boto3.s3.Object

Parameters:
  • key (str) – the path to the key
  • bucket_name (str) – the name of the bucket
get_wildcard_key(wildcard_key, bucket_name=None, delimiter='')[source]

Returns a boto3.s3.Object object matching the wildcard expression

Parameters:
  • wildcard_key (str) – the path to the key
  • bucket_name (str) – the name of the bucket
list_keys(bucket_name, prefix='', delimiter='', page_size=None, max_items=None)[source]

Lists keys in a bucket under prefix and not containing delimiter

Parameters:
  • bucket_name (str) – the name of the bucket
  • prefix (str) – a key prefix
  • delimiter (str) – the delimiter marks key hierarchy.
  • page_size (int) – pagination size
  • max_items (int) – maximum items to return
list_prefixes(bucket_name, prefix='', delimiter='', page_size=None, max_items=None)[source]

Lists prefixes in a bucket under prefix

Parameters:
  • bucket_name (str) – the name of the bucket
  • prefix (str) – a key prefix
  • delimiter (str) – the delimiter marks key hierarchy.
  • page_size (int) – pagination size
  • max_items (int) – maximum items to return
load_bytes(bytes_data, key, bucket_name=None, replace=False, encrypt=False)[source]

Loads bytes to S3

This is provided as a convenience to drop a string in S3. It uses the boto infrastructure to ship a file to s3.

Parameters:
  • bytes_data (bytes) – bytes to set as content for the key.
  • key (str) – S3 key that will point to the file
  • bucket_name (str) – Name of the bucket in which to store the file
  • replace (bool) – A flag to decide whether or not to overwrite the key if it already exists
  • encrypt (bool) – If True, the file will be encrypted on the server-side by S3 and will be stored in an encrypted form while at rest in S3.
load_file(filename, key, bucket_name=None, replace=False, encrypt=False)[source]

Loads a local file to S3

Parameters:
  • filename (str) – name of the file to load.
  • key (str) – S3 key that will point to the file
  • bucket_name (str) – Name of the bucket in which to store the file
  • replace (bool) – A flag to decide whether or not to overwrite the key if it already exists. If replace is False and the key exists, an error will be raised.
  • encrypt (bool) – If True, the file will be encrypted on the server-side by S3 and will be stored in an encrypted form while at rest in S3.
load_file_obj(file_obj, key, bucket_name=None, replace=False, encrypt=False)[source]

Loads a file object to S3

Parameters:
  • file_obj (file-like object) – The file-like object to set as the content for the S3 key.
  • key (str) – S3 key that will point to the file
  • bucket_name (str) – Name of the bucket in which to store the file
  • replace (bool) – A flag that indicates whether to overwrite the key if it already exists.
  • encrypt (bool) – If True, S3 encrypts the file on the server, and the file is stored in encrypted form at rest in S3.
load_string(string_data, key, bucket_name=None, replace=False, encrypt=False, encoding='utf-8')[source]

Loads a string to S3

This is provided as a convenience to drop a string in S3. It uses the boto infrastructure to ship a file to s3.

Parameters:
  • string_data (str) – string to set as content for the key.
  • key (str) – S3 key that will point to the file
  • bucket_name (str) – Name of the bucket in which to store the file
  • replace (bool) – A flag to decide whether or not to overwrite the key if it already exists
  • encrypt (bool) – If True, the file will be encrypted on the server-side by S3 and will be stored in an encrypted form while at rest in S3.
read_key(key, bucket_name=None)[source]

Reads a key from S3

Parameters:
  • key (str) – S3 key that will point to the file
  • bucket_name (str) – Name of the bucket in which the file is stored
select_key(key, bucket_name=None, expression='SELECT * FROM S3Object', expression_type='SQL', input_serialization={'CSV': {}}, output_serialization={'CSV': {}})[source]

Reads a key with S3 Select.

Parameters:
  • key (str) – S3 key that will point to the file
  • bucket_name (str) – Name of the bucket in which the file is stored
  • expression (str) – S3 Select expression
  • expression_type (str) – S3 Select expression type
  • input_serialization (dict) – S3 Select input data serialization format
  • output_serialization (dict) – S3 Select output data serialization format
Returns:

retrieved subset of original data by S3 Select

Return type:

str

class airflow.hooks.samba_hook.SambaHook(samba_conn_id)[source]

Bases: airflow.hooks.base_hook.BaseHook

Allows for interaction with an samba server.

class airflow.hooks.slack_hook.SlackHook(token=None, slack_conn_id=None)[source]

Bases: airflow.hooks.base_hook.BaseHook

Interact with Slack, using slackclient library.

class airflow.hooks.sqlite_hook.SqliteHook(*args, **kwargs)[source]

Bases: airflow.hooks.dbapi_hook.DbApiHook

Interact with SQLite.

get_conn()[source]

Returns a sqlite connection object

class airflow.hooks.webhdfs_hook.WebHDFSHook(webhdfs_conn_id='webhdfs_default', proxy_user=None)[source]

Bases: airflow.hooks.base_hook.BaseHook

Interact with HDFS. This class is a wrapper around the hdfscli library.

check_for_path(hdfs_path)[source]

Check for the existence of a path in HDFS by querying FileStatus.

get_conn()[source]

Returns a hdfscli InsecureClient object.

load_file(source, destination, overwrite=True, parallelism=1, **kwargs)[source]

Uploads a file to HDFS

Parameters:
  • source (str) – Local path to file or folder. If a folder, all the files inside of it will be uploaded (note that this implies that folders empty of files will not be created remotely).
  • destination (str) – PTarget HDFS path. If it already exists and is a directory, files will be uploaded inside.
  • overwrite (bool) – Overwrite any existing file or directory.
  • parallelism (int) – Number of threads to use for parallelization. A value of 0 (or negative) uses as many threads as there are files.
  • **kwargs – Keyword arguments forwarded to upload().
class airflow.hooks.zendesk_hook.ZendeskHook(zendesk_conn_id)[source]

Bases: airflow.hooks.base_hook.BaseHook

A hook to talk to Zendesk

call(path, query=None, get_all_pages=True, side_loading=False)[source]

Call Zendesk API and return results

Parameters:
  • path – The Zendesk API to call
  • query – Query parameters
  • get_all_pages – Accumulate results over all pages before returning. Due to strict rate limiting, this can often timeout. Waits for recommended period between tries after a timeout.
  • side_loading – Retrieve related records as part of a single request. In order to enable side-loading, add an ‘include’ query parameter containing a comma-separated list of resources to load. For more information on side-loading see https://developer.zendesk.com/rest_api/docs/core/side_loading

Community contributed hooks

class airflow.contrib.hooks.aws_dynamodb_hook.AwsDynamoDBHook(table_keys=None, table_name=None, region_name=None, *args, **kwargs)[source]

Bases: airflow.contrib.hooks.aws_hook.AwsHook

Interact with AWS DynamoDB.

Parameters:
  • table_keys (list) – partition key and sort key
  • table_name (str) – target DynamoDB table
  • region_name (str) – aws region name (example: us-east-1)
write_batch_data(items)[source]

Write batch items to dynamodb table with provisioned throughout capacity.

class airflow.contrib.hooks.aws_firehose_hook.AwsFirehoseHook(delivery_stream, region_name=None, *args, **kwargs)[source]

Bases: airflow.contrib.hooks.aws_hook.AwsHook

Interact with AWS Kinesis Firehose. :param delivery_stream: Name of the delivery stream :type delivery_stream: str :param region_name: AWS region name (example: us-east-1) :type region_name: str

get_conn()[source]

Returns AwsHook connection object.

put_records(records)[source]

Write batch records to Kinesis Firehose

class airflow.contrib.hooks.aws_hook.AwsHook(aws_conn_id='aws_default')[source]

Bases: airflow.hooks.base_hook.BaseHook

Interact with AWS. This class is a thin wrapper around the boto3 python library.

expand_role(role)[source]

If the IAM role is a role name, get the Amazon Resource Name (ARN) for the role. If IAM role is already an IAM role ARN, no change is made.

Parameters:role – IAM role name or ARN
Returns:IAM role ARN
get_credentials(region_name=None)[source]

Get the underlying botocore.Credentials object.

This contains the following authentication attributes: access_key, secret_key and token.

get_session(region_name=None)[source]

Get the underlying boto3.session.

class airflow.contrib.hooks.aws_lambda_hook.AwsLambdaHook(function_name, region_name=None, log_type='None', qualifier='$LATEST', invocation_type='RequestResponse', *args, **kwargs)[source]

Bases: airflow.contrib.hooks.aws_hook.AwsHook

Interact with AWS Lambda

Parameters:
  • function_name (str) – AWS Lambda Function Name
  • region_name (str) – AWS Region Name (example: us-west-2)
  • log_type (str) – Tail Invocation Request
  • qualifier (str) – AWS Lambda Function Version or Alias Name
  • invocation_type (str) – AWS Lambda Invocation Type (RequestResponse, Event etc)
invoke_lambda(payload)[source]

Invoke Lambda Function

class airflow.contrib.hooks.azure_data_lake_hook.AzureDataLakeHook(azure_data_lake_conn_id='azure_data_lake_default')[source]

Bases: airflow.hooks.base_hook.BaseHook

Interacts with Azure Data Lake.

Client ID and client secret should be in user and password parameters. Tenant and account name should be extra field as {“tenant”: “<TENANT>”, “account_name”: “ACCOUNT_NAME”}.

Parameters:azure_data_lake_conn_id (str) – Reference to the Azure Data Lake connection.
check_for_file(file_path)[source]

Check if a file exists on Azure Data Lake.

Parameters:file_path (str) – Path and name of the file.
Returns:True if the file exists, False otherwise.

:rtype bool

download_file(local_path, remote_path, nthreads=64, overwrite=True, buffersize=4194304, blocksize=4194304)[source]

Download a file from Azure Blob Storage.

Parameters:
  • local_path (str) – local path. If downloading a single file, will write to this specific file, unless it is an existing directory, in which case a file is created within it. If downloading multiple files, this is the root directory to write within. Will create directories as required.
  • remote_path (str) – remote path/globstring to use to find remote files. Recursive glob patterns using ** are not supported.
  • nthreads (int) – Number of threads to use. If None, uses the number of cores.
  • overwrite (bool) – Whether to forcibly overwrite existing files/directories. If False and remote path is a directory, will quit regardless if any files would be overwritten or not. If True, only matching filenames are actually overwritten.
  • buffersize (int) – int [2**22] Number of bytes for internal buffer. This block cannot be bigger than a chunk and cannot be smaller than a block.
  • blocksize (int) – int [2**22] Number of bytes for a block. Within each chunk, we write a smaller block for each API call. This block cannot be bigger than a chunk.
get_conn()[source]

Return a AzureDLFileSystem object.

upload_file(local_path, remote_path, nthreads=64, overwrite=True, buffersize=4194304, blocksize=4194304)[source]

Upload a file to Azure Data Lake.

Parameters:
  • local_path (str) – local path. Can be single file, directory (in which case, upload recursively) or glob pattern. Recursive glob patterns using ** are not supported.
  • remote_path (str) – Remote path to upload to; if multiple files, this is the dircetory root to write within.
  • nthreads (int) – Number of threads to use. If None, uses the number of cores.
  • overwrite (bool) – Whether to forcibly overwrite existing files/directories. If False and remote path is a directory, will quit regardless if any files would be overwritten or not. If True, only matching filenames are actually overwritten.
  • buffersize (int) – int [2**22] Number of bytes for internal buffer. This block cannot be bigger than a chunk and cannot be smaller than a block.
  • blocksize (int) – int [2**22] Number of bytes for a block. Within each chunk, we write a smaller block for each API call. This block cannot be bigger than a chunk.
class airflow.contrib.hooks.azure_fileshare_hook.AzureFileShareHook(wasb_conn_id='wasb_default')[source]

Bases: airflow.hooks.base_hook.BaseHook

Interacts with Azure FileShare Storage.

Additional options passed in the ‘extra’ field of the connection will be passed to the FileService() constructor.

Parameters:wasb_conn_id (str) – Reference to the wasb connection.
check_for_directory(share_name, directory_name, **kwargs)[source]

Check if a directory exists on Azure File Share.

Parameters:
  • share_name (str) – Name of the share.
  • directory_name (str) – Name of the directory.
  • kwargs (object) – Optional keyword arguments that FileService.exists() takes.
Returns:

True if the file exists, False otherwise.

:rtype bool

check_for_file(share_name, directory_name, file_name, **kwargs)[source]

Check if a file exists on Azure File Share.

Parameters:
  • share_name (str) – Name of the share.
  • directory_name (str) – Name of the directory.
  • file_name (str) – Name of the file.
  • kwargs (object) – Optional keyword arguments that FileService.exists() takes.
Returns:

True if the file exists, False otherwise.

:rtype bool

create_directory(share_name, directory_name, **kwargs)[source]

Create a new direcotry on a Azure File Share.

Parameters:
  • share_name (str) – Name of the share.
  • directory_name (str) – Name of the directory.
  • kwargs (object) – Optional keyword arguments that FileService.create_directory() takes.
Returns:

A list of files and directories

:rtype list

get_conn()[source]

Return the FileService object.

get_file(file_path, share_name, directory_name, file_name, **kwargs)[source]

Download a file from Azure File Share.

Parameters:
  • file_path (str) – Where to store the file.
  • share_name (str) – Name of the share.
  • directory_name (str) – Name of the directory.
  • file_name (str) – Name of the file.
  • kwargs (object) – Optional keyword arguments that FileService.get_file_to_path() takes.
get_file_to_stream(stream, share_name, directory_name, file_name, **kwargs)[source]

Download a file from Azure File Share.

Parameters:
  • stream (file-like object) – A filehandle to store the file to.
  • share_name (str) – Name of the share.
  • directory_name (str) – Name of the directory.
  • file_name (str) – Name of the file.
  • kwargs (object) – Optional keyword arguments that FileService.get_file_to_stream() takes.
list_directories_and_files(share_name, directory_name=None, **kwargs)[source]

Return the list of directories and files stored on a Azure File Share.

Parameters:
  • share_name (str) – Name of the share.
  • directory_name (str) – Name of the directory.
  • kwargs (object) – Optional keyword arguments that FileService.list_directories_and_files() takes.
Returns:

A list of files and directories

:rtype list

load_file(file_path, share_name, directory_name, file_name, **kwargs)[source]

Upload a file to Azure File Share.

Parameters:
  • file_path (str) – Path to the file to load.
  • share_name (str) – Name of the share.
  • directory_name (str) – Name of the directory.
  • file_name (str) – Name of the file.
  • kwargs (object) – Optional keyword arguments that FileService.create_file_from_path() takes.
load_stream(stream, share_name, directory_name, file_name, count, **kwargs)[source]

Upload a stream to Azure File Share.

Parameters:
  • stream (file-like) – Opened file/stream to upload as the file content.
  • share_name (str) – Name of the share.
  • directory_name (str) – Name of the directory.
  • file_name (str) – Name of the file.
  • count (int) – Size of the stream in bytes
  • kwargs (object) – Optional keyword arguments that FileService.create_file_from_stream() takes.
load_string(string_data, share_name, directory_name, file_name, **kwargs)[source]

Upload a string to Azure File Share.

Parameters:
  • string_data (str) – String to load.
  • share_name (str) – Name of the share.
  • directory_name (str) – Name of the directory.
  • file_name (str) – Name of the file.
  • kwargs (object) – Optional keyword arguments that FileService.create_file_from_text() takes.
class airflow.contrib.hooks.bigquery_hook.BigQueryHook(bigquery_conn_id='bigquery_default', delegate_to=None, use_legacy_sql=True)[source]

Bases: airflow.contrib.hooks.gcp_api_base_hook.GoogleCloudBaseHook, airflow.hooks.dbapi_hook.DbApiHook, airflow.utils.log.logging_mixin.LoggingMixin

Interact with BigQuery. This hook uses the Google Cloud Platform connection.

get_conn()[source]

Returns a BigQuery PEP 249 connection object.

get_pandas_df(sql, parameters=None, dialect=None)[source]

Returns a Pandas DataFrame for the results produced by a BigQuery query. The DbApiHook method must be overridden because Pandas doesn’t support PEP 249 connections, except for SQLite. See:

https://github.com/pydata/pandas/blob/master/pandas/io/sql.py#L447 https://github.com/pydata/pandas/issues/6900

Parameters:
  • sql (string) – The BigQuery SQL to execute.
  • parameters (mapping or iterable) – The parameters to render the SQL query with (not used, leave to override superclass method)
  • dialect (string in {'legacy', 'standard'}) – Dialect of BigQuery SQL – legacy SQL or standard SQL defaults to use self.use_legacy_sql if not specified
get_service()[source]

Returns a BigQuery service object.

insert_rows(table, rows, target_fields=None, commit_every=1000)[source]

Insertion is currently unsupported. Theoretically, you could use BigQuery’s streaming API to insert rows into a table, but this hasn’t been implemented.

table_exists(project_id, dataset_id, table_id)[source]

Checks for the existence of a table in Google BigQuery.

Parameters:
  • project_id (string) – The Google cloud project in which to look for the table. The connection supplied to the hook must provide access to the specified project.
  • dataset_id (string) – The name of the dataset in which to look for the table.
  • table_id (string) – The name of the table to check the existence of.
class airflow.contrib.hooks.cassandra_hook.CassandraHook(cassandra_conn_id='cassandra_default')[source]

Bases: airflow.hooks.base_hook.BaseHook, airflow.utils.log.logging_mixin.LoggingMixin

Hook used to interact with Cassandra

Contact points can be specified as a comma-separated string in the ‘hosts’ field of the connection.

Port can be specified in the port field of the connection.

If SSL is enabled in Cassandra, pass in a dict in the extra field as kwargs for ssl.wrap_socket(). For example:

{
‘ssl_options’ : {
‘ca_certs’ : PATH_TO_CA_CERTS

}

}

Default load balancing policy is RoundRobinPolicy. To specify a different LB policy:
  • DCAwareRoundRobinPolicy
    {

    ‘load_balancing_policy’: ‘DCAwareRoundRobinPolicy’, ‘load_balancing_policy_args’: {

    ‘local_dc’: LOCAL_DC_NAME, // optional ‘used_hosts_per_remote_dc’: SOME_INT_VALUE, // optional

    }

    }

  • WhiteListRoundRobinPolicy
    {

    ‘load_balancing_policy’: ‘WhiteListRoundRobinPolicy’, ‘load_balancing_policy_args’: {

    ‘hosts’: [‘HOST1’, ‘HOST2’, ‘HOST3’]

    }

    }

  • TokenAwarePolicy
    {

    ‘load_balancing_policy’: ‘TokenAwarePolicy’, ‘load_balancing_policy_args’: {

    ‘child_load_balancing_policy’: CHILD_POLICY_NAME, // optional ‘child_load_balancing_policy_args’: { … } // optional

    }

    }

For details of the Cluster config, see cassandra.cluster.

get_conn()[source]

Returns a cassandra Session object

record_exists(table, keys)[source]

Checks if a record exists in Cassandra

Parameters:
  • table (string) – Target Cassandra table. Use dot notation to target a specific keyspace.
  • keys (dict) – The keys and their values to check the existence.
shutdown_cluster()[source]

Closes all sessions and connections associated with this Cluster.

class airflow.contrib.hooks.cloudant_hook.CloudantHook(cloudant_conn_id='cloudant_default')[source]

Bases: airflow.hooks.base_hook.BaseHook

Interact with Cloudant.

This class is a thin wrapper around the cloudant python library. See the documentation here.

db()[source]

Returns the Database object for this hook.

See the documentation for cloudant-python here https://github.com/cloudant-labs/cloudant-python.

class airflow.contrib.hooks.databricks_hook.DatabricksHook(databricks_conn_id='databricks_default', timeout_seconds=180, retry_limit=3, retry_delay=1.0)[source]

Bases: airflow.hooks.base_hook.BaseHook, airflow.utils.log.logging_mixin.LoggingMixin

Interact with Databricks.

submit_run(json)[source]

Utility function to call the api/2.0/jobs/runs/submit endpoint.

Parameters:json (dict) – The data used in the body of the request to the submit endpoint.
Returns:the run_id as a string
Return type:string
class airflow.contrib.hooks.datadog_hook.DatadogHook(datadog_conn_id='datadog_default')[source]

Bases: airflow.hooks.base_hook.BaseHook, airflow.utils.log.logging_mixin.LoggingMixin

Uses datadog API to send metrics of practically anything measurable, so it’s possible to track # of db records inserted/deleted, records read from file and many other useful metrics.

Depends on the datadog API, which has to be deployed on the same server where Airflow runs.

Parameters:
  • datadog_conn_id – The connection to datadog, containing metadata for api keys.
  • datadog_conn_id – string
post_event(title, text, tags=None, alert_type=None, aggregation_key=None)[source]

Posts an event to datadog (processing finished, potentially alerts, other issues) Think about this as a means to maintain persistence of alerts, rather than alerting itself.

Parameters:
  • title (string) – The title of the event
  • text (string) – The body of the event (more information)
  • tags (list) – List of string tags to apply to the event
  • alert_type (string) – The alert type for the event, one of [“error”, “warning”, “info”, “success”]
  • aggregation_key (string) – Key that can be used to aggregate this event in a stream
query_metric(query, from_seconds_ago, to_seconds_ago)[source]

Queries datadog for a specific metric, potentially with some function applied to it and returns the results.

Parameters:
  • query (string) – The datadog query to execute (see datadog docs)
  • from_seconds_ago (int) – How many seconds ago to start querying for.
  • to_seconds_ago (int) – Up to how many seconds ago to query for.
send_metric(metric_name, datapoint, tags=None)[source]

Sends a single datapoint metric to DataDog

Parameters:
  • metric_name (string) – The name of the metric
  • datapoint (integer or float) – A single integer or float related to the metric
  • tags (list) – A list of tags associated with the metric
class airflow.contrib.hooks.datastore_hook.DatastoreHook(datastore_conn_id='google_cloud_datastore_default', delegate_to=None)[source]

Bases: airflow.contrib.hooks.gcp_api_base_hook.GoogleCloudBaseHook

Interact with Google Cloud Datastore. This hook uses the Google Cloud Platform connection.

This object is not threads safe. If you want to make multiple requests simultaneously, you will need to create a hook per thread.

allocate_ids(partialKeys)[source]

Allocate IDs for incomplete keys. see https://cloud.google.com/datastore/docs/reference/rest/v1/projects/allocateIds

Parameters:partialKeys – a list of partial keys
Returns:a list of full keys.
begin_transaction()[source]

Get a new transaction handle

Returns:a transaction handle
commit(body)[source]

Commit a transaction, optionally creating, deleting or modifying some entities.

Parameters:body – the body of the commit request
Returns:the response body of the commit request
delete_operation(name)[source]

Deletes the long-running operation

Parameters:name – the name of the operation resource
export_to_storage_bucket(bucket, namespace=None, entity_filter=None, labels=None)[source]

Export entities from Cloud Datastore to Cloud Storage for backup

get_conn(version='v1')[source]

Returns a Google Cloud Storage service object.

get_operation(name)[source]

Gets the latest state of a long-running operation

Parameters:name – the name of the operation resource
import_from_storage_bucket(bucket, file, namespace=None, entity_filter=None, labels=None)[source]

Import a backup from Cloud Storage to Cloud Datastore

lookup(keys, read_consistency=None, transaction=None)[source]

Lookup some entities by key

Parameters:
  • keys – the keys to lookup
  • read_consistency – the read consistency to use. default, strong or eventual. Cannot be used with a transaction.
  • transaction – the transaction to use, if any.
Returns:

the response body of the lookup request.

poll_operation_until_done(name, polling_interval_in_seconds)[source]

Poll backup operation state until it’s completed

rollback(transaction)[source]

Roll back a transaction

Parameters:transaction – the transaction to roll back
run_query(body)[source]

Run a query for entities.

Parameters:body – the body of the query request
Returns:the batch of query results.
class airflow.contrib.hooks.discord_webhook_hook.DiscordWebhookHook(http_conn_id=None, webhook_endpoint=None, message='', username=None, avatar_url=None, tts=False, proxy=None, *args, **kwargs)[source]

Bases: airflow.hooks.http_hook.HttpHook

This hook allows you to post messages to Discord using incoming webhooks. Takes a Discord connection ID with a default relative webhook endpoint. The default endpoint can be overridden using the webhook_endpoint parameter (https://discordapp.com/developers/docs/resources/webhook).

Each Discord webhook can be pre-configured to use a specific username and avatar_url. You can override these defaults in this hook.

Parameters:
  • http_conn_id (str) – Http connection ID with host as “https://discord.com/api/” and default webhook endpoint in the extra field in the form of {“webhook_endpoint”: “webhooks/{webhook.id}/{webhook.token}”}
  • webhook_endpoint (str) – Discord webhook endpoint in the form of “webhooks/{webhook.id}/{webhook.token}”
  • message (str) – The message you want to send to your Discord channel (max 2000 characters)
  • username (str) – Override the default username of the webhook
  • avatar_url (str) – Override the default avatar of the webhook
  • tts (bool) – Is a text-to-speech message
  • proxy (str) – Proxy to use to make the Discord webhook call
execute()[source]

Execute the Discord webhook call

class airflow.contrib.hooks.emr_hook.EmrHook(emr_conn_id=None, *args, **kwargs)[source]

Bases: airflow.contrib.hooks.aws_hook.AwsHook

Interact with AWS EMR. emr_conn_id is only neccessary for using the create_job_flow method.

create_job_flow(job_flow_overrides)[source]

Creates a job flow using the config from the EMR connection. Keys of the json extra hash may have the arguments of the boto3 run_job_flow method. Overrides for this config may be passed as the job_flow_overrides.

class airflow.contrib.hooks.fs_hook.FSHook(conn_id='fs_default')[source]

Bases: airflow.hooks.base_hook.BaseHook

Allows for interaction with an file server.

Connection should have a name and a path specified under extra:

example: Conn Id: fs_test Conn Type: File (path) Host, Shchema, Login, Password, Port: empty Extra: {“path”: “/tmp”}

class airflow.contrib.hooks.ftp_hook.FTPHook(ftp_conn_id='ftp_default')[source]

Bases: airflow.hooks.base_hook.BaseHook, airflow.utils.log.logging_mixin.LoggingMixin

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
get_conn()[source]

Returns a FTP connection object

get_mod_time(path)[source]

Returns a datetime object representing the last time the file was modified

Parameters:path (string) – remote file path
get_size(path)[source]

Returns the size of a file (in bytes)

Parameters:path (string) – remote file path
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, callback=None)[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 (str or file-like buffer) – full path to the local file or a file-like buffer
  • callback (callable) – callback which is called each time a block of data is read. if you do not use a callback, these blocks will be written to the file or buffer passed in. if you do pass in a callback, note that writing to a file or buffer will need to be handled inside the callback. [default: output_handle.write()]
Example::

hook = FTPHook(ftp_conn_id=’my_conn’)

remote_path = ‘/path/to/remote/file’ local_path = ‘/path/to/local/file’

# with a custom callback (in this case displaying progress on each read) def print_progress(percent_progress):

self.log.info(‘Percent Downloaded: %s%%’ % percent_progress)

total_downloaded = 0 total_file_size = hook.get_size(remote_path) output_handle = open(local_path, ‘wb’) def write_to_file_with_progress(data):

total_downloaded += len(data) output_handle.write(data) percent_progress = (total_downloaded / total_file_size) * 100 print_progress(percent_progress)

hook.retrieve_file(remote_path, None, callback=write_to_file_with_progress)

# without a custom callback data is written to the local_path hook.retrieve_file(remote_path, local_path)

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.ftp_hook.FTPSHook(ftp_conn_id='ftp_default')[source]

Bases: airflow.contrib.hooks.ftp_hook.FTPHook

get_conn()[source]

Returns a FTPS connection object.

class airflow.contrib.hooks.gcp_api_base_hook.GoogleCloudBaseHook(gcp_conn_id='google_cloud_default', delegate_to=None)[source]

Bases: airflow.hooks.base_hook.BaseHook, airflow.utils.log.logging_mixin.LoggingMixin

A base hook for Google cloud-related hooks. Google cloud has a shared REST API client that is built in the same way no matter which service you use. This class helps construct and authorize the credentials needed to then call apiclient.discovery.build() to actually discover and build a client for a Google cloud service.

The class also contains some miscellaneous helper functions.

All hook derived from this base hook use the ‘Google Cloud Platform’ connection type. Two ways of authentication are supported:

Default credentials: Only the ‘Project Id’ is required. You’ll need to have set up default credentials, such as by the GOOGLE_APPLICATION_DEFAULT environment variable or from the metadata server on Google Compute Engine.

JSON key file: Specify ‘Project Id’, ‘Key Path’ and ‘Scope’.

Legacy P12 key files are not supported.

class airflow.contrib.hooks.gcp_container_hook.GKEClusterHook(project_id, location)[source]

Bases: airflow.hooks.base_hook.BaseHook

create_cluster(cluster, retry=<object object>, timeout=<object object>)[source]

Creates a cluster, consisting of the specified number and type of Google Compute Engine instances.

Parameters:
  • cluster (dict or google.cloud.container_v1.types.Cluster) – A Cluster protobuf or dict. If dict is provided, it must be of the same form as the protobuf message google.cloud.container_v1.types.Cluster
  • retry (google.api_core.retry.Retry) – A retry object (google.api_core.retry.Retry) used to retry requests. If None is specified, requests will not be retried.
  • timeout (float) – The amount of time, in seconds, to wait for the request to complete. Note that if retry is specified, the timeout applies to each individual attempt.
Returns:

The full url to the new, or existing, cluster

:raises
ParseError: On JSON parsing problems when trying to convert dict AirflowException: cluster is not dict type nor Cluster proto type
delete_cluster(name, retry=<object object>, timeout=<object object>)[source]

Deletes the cluster, including the Kubernetes endpoint and all worker nodes. Firewalls and routes that were configured during cluster creation are also deleted. Other Google Compute Engine resources that might be in use by the cluster (e.g. load balancer resources) will not be deleted if they weren’t present at the initial create time.

Parameters:
  • name (str) – The name of the cluster to delete
  • retry (google.api_core.retry.Retry) – Retry object used to determine when/if to retry requests. If None is specified, requests will not be retried.
  • timeout (float) – The amount of time, in seconds, to wait for the request to complete. Note that if retry is specified, the timeout applies to each individual attempt.
Returns:

The full url to the delete operation if successful, else None

get_cluster(name, retry=<object object>, timeout=<object object>)[source]

Gets details of specified cluster

Parameters:
  • name (str) – The name of the cluster to retrieve
  • retry (google.api_core.retry.Retry) – A retry object used to retry requests. If None is specified, requests will not be retried.
  • timeout (float) – The amount of time, in seconds, to wait for the request to complete. Note that if retry is specified, the timeout applies to each individual attempt.
Returns:

A google.cloud.container_v1.types.Cluster instance

get_operation(operation_name)[source]

Fetches the operation from Google Cloud

Parameters:operation_name (str) – Name of operation to fetch
Returns:The new, updated operation from Google Cloud
wait_for_operation(operation)[source]

Given an operation, continuously fetches the status from Google Cloud until either completion or an error occurring

Parameters:operation (A google.cloud.container_V1.gapic.enums.Operator) – The Operation to wait for
Returns:A new, updated operation fetched from Google Cloud
class airflow.contrib.hooks.gcp_dataflow_hook.DataFlowHook(gcp_conn_id='google_cloud_default', delegate_to=None, poll_sleep=10)[source]

Bases: airflow.contrib.hooks.gcp_api_base_hook.GoogleCloudBaseHook

get_conn()[source]

Returns a Google Cloud Dataflow service object.

class airflow.contrib.hooks.gcp_dataproc_hook.DataProcHook(gcp_conn_id='google_cloud_default', delegate_to=None, api_version='v1beta2')[source]

Bases: airflow.contrib.hooks.gcp_api_base_hook.GoogleCloudBaseHook

Hook for Google Cloud Dataproc APIs.

get_conn()[source]

Returns a Google Cloud Dataproc service object.

wait(operation)[source]

Awaits for Google Cloud Dataproc Operation to complete.

class airflow.contrib.hooks.gcp_mlengine_hook.MLEngineHook(gcp_conn_id='google_cloud_default', delegate_to=None)[source]

Bases: airflow.contrib.hooks.gcp_api_base_hook.GoogleCloudBaseHook

create_job(project_id, job, use_existing_job_fn=None)[source]

Launches a MLEngine job and wait for it to reach a terminal state.

Parameters:
  • project_id (string) – The Google Cloud project id within which MLEngine job will be launched.
  • job (dict) –

    MLEngine Job object that should be provided to the MLEngine API, such as:

    {
      'jobId': 'my_job_id',
      'trainingInput': {
        'scaleTier': 'STANDARD_1',
        ...
      }
    }
    
  • use_existing_job_fn (function) – In case that a MLEngine job with the same job_id already exist, this method (if provided) will decide whether we should use this existing job, continue waiting for it to finish and returning the job object. It should accepts a MLEngine job object, and returns a boolean value indicating whether it is OK to reuse the existing job. If ‘use_existing_job_fn’ is not provided, we by default reuse the existing MLEngine job.
Returns:

The MLEngine job object if the job successfully reach a terminal state (which might be FAILED or CANCELLED state).

Return type:

dict

create_model(project_id, model)[source]

Create a Model. Blocks until finished.

create_version(project_id, model_name, version_spec)[source]

Creates the Version on Google Cloud ML Engine.

Returns the operation if the version was created successfully and raises an error otherwise.

delete_version(project_id, model_name, version_name)[source]

Deletes the given version of a model. Blocks until finished.

get_conn()[source]

Returns a Google MLEngine service object.

get_model(project_id, model_name)[source]

Gets a Model. Blocks until finished.

list_versions(project_id, model_name)[source]

Lists all available versions of a model. Blocks until finished.

set_default_version(project_id, model_name, version_name)[source]

Sets a version to be the default. Blocks until finished.

class airflow.contrib.hooks.gcp_pubsub_hook.PubSubHook(gcp_conn_id='google_cloud_default', delegate_to=None)[source]

Bases: airflow.contrib.hooks.gcp_api_base_hook.GoogleCloudBaseHook

Hook for accessing Google Pub/Sub.

The GCP project against which actions are applied is determined by the project embedded in the Connection referenced by gcp_conn_id.

acknowledge(project, subscription, ack_ids)[source]

Pulls up to max_messages messages from Pub/Sub subscription.

Parameters:
  • project (string) – the GCP project name or ID in which to create the topic
  • subscription (string) – the Pub/Sub subscription name to delete; do not include the ‘projects/{project}/topics/’ prefix.
  • ack_ids (list) – List of ReceivedMessage ackIds from a previous pull response
create_subscription(topic_project, topic, subscription=None, subscription_project=None, ack_deadline_secs=10, fail_if_exists=False)[source]

Creates a Pub/Sub subscription, if it does not already exist.

Parameters:
  • topic_project (string) – the GCP project ID of the topic that the subscription will be bound to.
  • topic (string) – the Pub/Sub topic name that the subscription will be bound to create; do not include the projects/{project}/subscriptions/ prefix.
  • subscription (string) – the Pub/Sub subscription name. If empty, a random name will be generated using the uuid module
  • subscription_project (string) – the GCP project ID where the subscription will be created. If unspecified, topic_project will be used.
  • ack_deadline_secs (int) – Number of seconds that a subscriber has to acknowledge each message pulled from the subscription
  • fail_if_exists (bool) – if set, raise an exception if the topic already exists
Returns:

subscription name which will be the system-generated value if the subscription parameter is not supplied

Return type:

string

create_topic(project, topic, fail_if_exists=False)[source]

Creates a Pub/Sub topic, if it does not already exist.

Parameters:
  • project (string) – the GCP project ID in which to create the topic
  • topic (string) – the Pub/Sub topic name to create; do not include the projects/{project}/topics/ prefix.
  • fail_if_exists (bool) – if set, raise an exception if the topic already exists
delete_subscription(project, subscription, fail_if_not_exists=False)[source]

Deletes a Pub/Sub subscription, if it exists.

Parameters:
  • project (string) – the GCP project ID where the subscription exists
  • subscription (string) – the Pub/Sub subscription name to delete; do not include the projects/{project}/subscriptions/ prefix.
  • fail_if_not_exists (bool) – if set, raise an exception if the topic does not exist
delete_topic(project, topic, fail_if_not_exists=False)[source]

Deletes a Pub/Sub topic if it exists.

Parameters:
  • project (string) – the GCP project ID in which to delete the topic
  • topic (string) – the Pub/Sub topic name to delete; do not include the projects/{project}/topics/ prefix.
  • fail_if_not_exists (bool) – if set, raise an exception if the topic does not exist
get_conn()[source]

Returns a Pub/Sub service object.

Return type:apiclient.discovery.Resource
publish(project, topic, messages)[source]

Publishes messages to a Pub/Sub topic.

Parameters:
  • project (string) – the GCP project ID in which to publish
  • topic (string) – the Pub/Sub topic to which to publish; do not include the projects/{project}/topics/ prefix.
  • messages (list of PubSub messages; see http://cloud.google.com/pubsub/docs/reference/rest/v1/PubsubMessage) – messages to publish; if the data field in a message is set, it should already be base64 encoded.
pull(project, subscription, max_messages, return_immediately=False)[source]

Pulls up to max_messages messages from Pub/Sub subscription.

Parameters:
  • project (string) – the GCP project ID where the subscription exists
  • subscription (string) – the Pub/Sub subscription name to pull from; do not include the ‘projects/{project}/topics/’ prefix.
  • max_messages (int) – The maximum number of messages to return from the Pub/Sub API.
  • return_immediately (bool) – If set, the Pub/Sub API will immediately return if no messages are available. Otherwise, the request will block for an undisclosed, but bounded period of time
:return A list of Pub/Sub ReceivedMessage objects each containing
an ackId property and a message property, which includes the base64-encoded message content. See https://cloud.google.com/pubsub/docs/reference/rest/v1/ projects.subscriptions/pull#ReceivedMessage
class airflow.contrib.hooks.gcs_hook.GoogleCloudStorageHook(google_cloud_storage_conn_id='google_cloud_default', delegate_to=None)[source]

Bases: airflow.contrib.hooks.gcp_api_base_hook.GoogleCloudBaseHook

Interact with Google Cloud Storage. This hook uses the Google Cloud Platform connection.

copy(source_bucket, source_object, destination_bucket=None, destination_object=None)[source]

Copies an object from a bucket to another, with renaming if requested.

destination_bucket or destination_object can be omitted, in which case source bucket/object is used, but not both.

Parameters:
  • source_bucket (string) – The bucket of the object to copy from.
  • source_object (string) – The object to copy.
  • destination_bucket (string) – The destination of the object to copied to. Can be omitted; then the same bucket is used.
  • destination_object – The (renamed) path of the object if given. Can be omitted; then the same name is used.
create_bucket(bucket_name, storage_class='MULTI_REGIONAL', location='US', project_id=None, labels=None)[source]

Creates a new bucket. Google Cloud Storage uses a flat namespace, so you can’t create a bucket with a name that is already in use.

See also

For more information, see Bucket Naming Guidelines: https://cloud.google.com/storage/docs/bucketnaming.html#requirements

Parameters:
  • bucket_name (string) – The name of the bucket.
  • storage_class (string) –

    This defines how objects in the bucket are stored and determines the SLA and the cost of storage. Values include

    • MULTI_REGIONAL
    • REGIONAL
    • STANDARD
    • NEARLINE
    • COLDLINE.

    If this value is not specified when the bucket is created, it will default to STANDARD.

  • location (string) –

    The location of the bucket. Object data for objects in the bucket resides in physical storage within this region. Defaults to US.

  • project_id (string) – The ID of the GCP Project.
  • labels (dict) – User-provided labels, in key/value pairs.
Returns:

If successful, it returns the id of the bucket.

delete(bucket, object, generation=None)[source]

Delete an object if versioning is not enabled for the bucket, or if generation parameter is used.

Parameters:
  • bucket (string) – name of the bucket, where the object resides
  • object (string) – name of the object to delete
  • generation (string) – if present, permanently delete the object of this generation
Returns:

True if succeeded

download(bucket, object, filename=None)[source]

Get a file from Google Cloud Storage.

Parameters:
  • bucket (string) – The bucket to fetch from.
  • object (string) – The object to fetch.
  • filename (string) – If set, a local file path where the file should be written to.
exists(bucket, object)[source]

Checks for the existence of a file in Google Cloud Storage.

Parameters:
  • bucket (string) – The Google cloud storage bucket where the object is.
  • object (string) – The name of the object to check in the Google cloud storage bucket.
get_conn()[source]

Returns a Google Cloud Storage service object.

get_crc32c(bucket, object)[source]

Gets the CRC32c checksum of an object in Google Cloud Storage.

Parameters:
  • bucket (string) – The Google cloud storage bucket where the object is.
  • object (string) – The name of the object to check in the Google cloud storage bucket.
get_md5hash(bucket, object)[source]

Gets the MD5 hash of an object in Google Cloud Storage.

Parameters:
  • bucket (string) – The Google cloud storage bucket where the object is.
  • object (string) – The name of the object to check in the Google cloud storage bucket.
get_size(bucket, object)[source]

Gets the size of a file in Google Cloud Storage.

Parameters:
  • bucket (string) – The Google cloud storage bucket where the object is.
  • object (string) – The name of the object to check in the Google cloud storage bucket.
is_updated_after(bucket, object, ts)[source]

Checks if an object is updated in Google Cloud Storage.

Parameters:
  • bucket (string) – The Google cloud storage bucket where the object is.
  • object (string) – The name of the object to check in the Google cloud storage bucket.
  • ts (datetime) – The timestamp to check against.
list(bucket, versions=None, maxResults=None, prefix=None, delimiter=None)[source]

List all objects from the bucket with the give string prefix in name

Parameters:
  • bucket (string) – bucket name
  • versions (boolean) – if true, list all versions of the objects
  • maxResults (integer) – max count of items to return in a single page of responses
  • prefix (string) – prefix string which filters objects whose name begin with this prefix
  • delimiter (string) – filters objects based on the delimiter (for e.g ‘.csv’)
Returns:

a stream of object names matching the filtering criteria

rewrite(source_bucket, source_object, destination_bucket, destination_object=None)[source]

Has the same functionality as copy, except that will work on files over 5 TB, as well as when copying between locations and/or storage classes.

destination_object can be omitted, in which case source_object is used.

Parameters:
  • source_bucket (string) – The bucket of the object to copy from.
  • source_object (string) – The object to copy.
  • destination_bucket (string) – The destination of the object to copied to.
  • destination_object – The (renamed) path of the object if given. Can be omitted; then the same name is used.
upload(bucket, object, filename, mime_type='application/octet-stream', gzip=False)[source]

Uploads a local file to Google Cloud Storage.

Parameters:
  • bucket (string) – The bucket to upload to.
  • object (string) – The object name to set when uploading the local file.
  • filename (string) – The local file path to the file to be uploaded.
  • mime_type (str) – The MIME type to set when uploading the file.
  • gzip (bool) – Option to compress file for upload
class airflow.contrib.hooks.imap_hook.ImapHook(imap_conn_id='imap_default')[source]

Bases: airflow.hooks.base_hook.BaseHook

This hook connects to a mail server by using the imap protocol.

Parameters:imap_conn_id (str) – The connection id that contains the information used to authenticate the client. The default value is ‘imap_default’.
download_mail_attachments(name, local_output_directory, mail_folder='INBOX', check_regex=False, latest_only=False)[source]

Downloads mail’s attachments in the mail folder by its name to the local directory.

Parameters:
  • name (str) – The name of the attachment that will be downloaded.
  • local_output_directory (str) – The output directory on the local machine where the files will be downloaded to.
  • mail_folder (str) – The mail folder where to look at. The default value is ‘INBOX’.
  • check_regex (bool) – Checks the name for a regular expression. The default value is False.
  • latest_only (bool) – If set to True it will only download the first matched attachment. The default value is False.
has_mail_attachment(name, mail_folder='INBOX', check_regex=False)[source]

Checks the mail folder for mails containing attachments with the given name.

Parameters:
  • name (str) – The name of the attachment that will be searched for.
  • mail_folder (str) – The mail folder where to look at. The default value is ‘INBOX’.
  • check_regex (bool) – Checks the name for a regular expression. The default value is False.
Returns:

True if there is an attachment with the given name and False if not.

Return type:

bool

retrieve_mail_attachments(name, mail_folder='INBOX', check_regex=False, latest_only=False)[source]

Retrieves mail’s attachments in the mail folder by its name.

Parameters:
  • name (str) – The name of the attachment that will be downloaded.
  • mail_folder (str) – The mail folder where to look at. The default value is ‘INBOX’.
  • check_regex (bool) – Checks the name for a regular expression. The default value is False.
  • latest_only (bool) – If set to True it will only retrieve the first matched attachment. The default value is False.
Returns:

a list of tuple each containing the attachment filename and its payload.

Return type:

a list of tuple

class airflow.contrib.hooks.jenkins_hook.JenkinsHook(conn_id='jenkins_default')[source]

Bases: airflow.hooks.base_hook.BaseHook

Hook to manage connection to jenkins server

class airflow.contrib.hooks.jira_hook.JiraHook(jira_conn_id='jira_default', proxies=None)[source]

Bases: airflow.hooks.base_hook.BaseHook, airflow.utils.log.logging_mixin.LoggingMixin

Jira interaction hook, a Wrapper around JIRA Python SDK.

Parameters:jira_conn_id (string) – reference to a pre-defined Jira Connection
class airflow.contrib.hooks.mongo_hook.MongoHook(conn_id='mongo_default', *args, **kwargs)[source]

Bases: airflow.hooks.base_hook.BaseHook

PyMongo Wrapper to Interact With Mongo Database Mongo Connection Documentation https://docs.mongodb.com/manual/reference/connection-string/index.html You can specify connection string options in extra field of your connection https://docs.mongodb.com/manual/reference/connection-string/index.html#connection-string-options ex.

{replicaSet: test, ssl: True, connectTimeoutMS: 30000}
aggregate(mongo_collection, aggregate_query, mongo_db=None, **kwargs)[source]

Runs an aggregation pipeline and returns the results https://api.mongodb.com/python/current/api/pymongo/collection.html#pymongo.collection.Collection.aggregate https://api.mongodb.com/python/current/examples/aggregation.html

find(mongo_collection, query, find_one=False, mongo_db=None, **kwargs)[source]

Runs a mongo find query and returns the results https://api.mongodb.com/python/current/api/pymongo/collection.html#pymongo.collection.Collection.find

get_collection(mongo_collection, mongo_db=None)[source]

Fetches a mongo collection object for querying.

Uses connection schema as DB unless specified.

get_conn()[source]

Fetches PyMongo Client

insert_many(mongo_collection, docs, mongo_db=None, **kwargs)[source]

Inserts many docs into a mongo collection. https://api.mongodb.com/python/current/api/pymongo/collection.html#pymongo.collection.Collection.insert_many

insert_one(mongo_collection, doc, mongo_db=None, **kwargs)[source]

Inserts a single document into a mongo collection https://api.mongodb.com/python/current/api/pymongo/collection.html#pymongo.collection.Collection.insert_one

class airflow.contrib.hooks.pinot_hook.PinotDbApiHook(*args, **kwargs)[source]

Bases: airflow.hooks.dbapi_hook.DbApiHook

Connect to pinot db(https://github.com/linkedin/pinot) to issue pql

get_conn()[source]

Establish a connection to pinot broker through pinot dbqpi.

get_first(sql)[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
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)[source]

Executes the sql and returns a set of records.

Parameters:sql (str) – the sql statement to be executed (str) or a list of sql statements to execute
get_uri()[source]

Get the connection uri for pinot broker.

e.g: http://localhost:9000/pql

insert_rows(table, rows, target_fields=None, commit_every=1000)[source]

A generic way to insert a set of tuples into a table, a new transaction is created every commit_every rows

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.
  • replace (bool) – Whether to replace instead of insert
set_autocommit(conn, autocommit)[source]

Sets the autocommit flag on the connection

class airflow.contrib.hooks.qubole_hook.QuboleHook(*args, **kwargs)[source]

Bases: airflow.hooks.base_hook.BaseHook, airflow.utils.log.logging_mixin.LoggingMixin

get_jobs_id(ti)[source]

Get jobs associated with a Qubole commands :param ti: Task Instance of the dag, used to determine the Quboles command id :return: Job informations assoiciated with command

get_log(ti)[source]

Get Logs of a command from Qubole :param ti: Task Instance of the dag, used to determine the Quboles command id :return: command log as text

get_results(ti=None, fp=None, inline=True, delim=None, fetch=True)[source]

Get results (or just s3 locations) of a command from Qubole and save into a file :param ti: Task Instance of the dag, used to determine the Quboles command id :param fp: Optional file pointer, will create one and return if None passed :param inline: True to download actual results, False to get s3 locations only :param delim: Replaces the CTL-A chars with the given delim, defaults to ‘,’ :param fetch: when inline is True, get results directly from s3 (if large) :return: file location containing actual results or s3 locations of results

kill(ti)[source]

Kill (cancel) a Qubole commmand :param ti: Task Instance of the dag, used to determine the Quboles command id :return: response from Qubole

class airflow.contrib.hooks.redis_hook.RedisHook(redis_conn_id='redis_default')[source]

Bases: airflow.hooks.base_hook.BaseHook, airflow.utils.log.logging_mixin.LoggingMixin

Hook to interact with Redis database

get_conn()[source]

Returns a Redis connection.

key_exists(key)[source]

Checks if a key exists in Redis database

Parameters:key (string) – The key to check the existence.
class airflow.contrib.hooks.redshift_hook.RedshiftHook(aws_conn_id='aws_default')[source]

Bases: airflow.contrib.hooks.aws_hook.AwsHook

Interact with AWS Redshift, using the boto3 library

cluster_status(cluster_identifier)[source]

Return status of a cluster

Parameters:cluster_identifier (str) – unique identifier of a cluster
create_cluster_snapshot(snapshot_identifier, cluster_identifier)[source]

Creates a snapshot of a cluster

Parameters:
  • snapshot_identifier (str) – unique identifier for a snapshot of a cluster
  • cluster_identifier (str) – unique identifier of a cluster
delete_cluster(cluster_identifier, skip_final_cluster_snapshot=True, final_cluster_snapshot_identifier='')[source]

Delete a cluster and optionally create a snapshot

Parameters:
  • cluster_identifier (str) – unique identifier of a cluster
  • skip_final_cluster_snapshot (bool) – determines cluster snapshot creation
  • final_cluster_snapshot_identifier (str) – name of final cluster snapshot
describe_cluster_snapshots(cluster_identifier)[source]

Gets a list of snapshots for a cluster

Parameters:cluster_identifier (str) – unique identifier of a cluster
restore_from_cluster_snapshot(cluster_identifier, snapshot_identifier)[source]

Restores a cluster from its snapshot

Parameters:
  • cluster_identifier (str) – unique identifier of a cluster
  • snapshot_identifier (str) – unique identifier for a snapshot of a cluster
class airflow.contrib.hooks.sagemaker_hook.SageMakerHook(*args, **kwargs)[source]

Bases: airflow.contrib.hooks.aws_hook.AwsHook

Interact with Amazon SageMaker.

check_s3_url(s3url)[source]

Check if an S3 URL exists

Parameters:s3url – S3 url

:type s3url:str :return: bool

check_status(job_name, key, describe_function, check_interval, max_ingestion_time, non_terminal_states=None)[source]

Check status of a SageMaker job

Parameters:
  • job_name (str) – name of the job to check status
  • key (str) – the key of the response dict that points to the state
  • describe_function (python callable) – the function used to retrieve the status
  • args – the arguments for the function
  • check_interval (int) – the time interval in seconds which the operator will check the status of any SageMaker job
  • max_ingestion_time (int) – the maximum ingestion time in seconds. Any SageMaker jobs that run longer than this will fail. Setting this to None implies no timeout for any SageMaker job.
  • non_terminal_states (set) – the set of nonterminal states
Returns:

response of describe call after job is done

check_training_config(training_config)[source]

Check if a training configuration is valid

Parameters:training_config (dict) – training_config
Returns:None
check_training_status_with_log(job_name, non_terminal_states, failed_states, wait_for_completion, check_interval, max_ingestion_time)[source]

Display the logs for a given training job, optionally tailing them until the job is complete.

Parameters:
  • job_name (str) – name of the training job to check status and display logs for
  • non_terminal_states (set) – the set of non_terminal states
  • failed_states (set) – the set of failed states
  • wait_for_completion (bool) – Whether to keep looking for new log entries until the job completes
  • check_interval (int) – The interval in seconds between polling for new log entries and job completion
  • max_ingestion_time (int) – the maximum ingestion time in seconds. Any SageMaker jobs that run longer than this will fail. Setting this to None implies no timeout for any SageMaker job.
Returns:

None

check_tuning_config(tuning_config)[source]

Check if a tuning configuration is valid

Parameters:tuning_config (dict) – tuning_config
Returns:None
configure_s3_resources(config)[source]

Extract the S3 operations from the configuration and execute them.

Parameters:config (dict) – config of SageMaker operation
Returns:dict
create_endpoint(config, wait_for_completion=True, check_interval=30, max_ingestion_time=None)[source]

Create an endpoint

Parameters:
  • config (dict) – the config for endpoint
  • wait_for_completion (bool) – if the program should keep running until job finishes
  • check_interval (int) – the time interval in seconds which the operator will check the status of any SageMaker job
  • max_ingestion_time (int) – the maximum ingestion time in seconds. Any SageMaker jobs that run longer than this will fail. Setting this to None implies no timeout for any SageMaker job.
Returns:

A response to endpoint creation

create_endpoint_config(config)[source]

Create an endpoint config

Parameters:config (dict) – the config for endpoint-config
Returns:A response to endpoint config creation
create_model(config)[source]

Create a model job

Parameters:config (dict) – the config for model
Returns:A response to model creation
create_training_job(config, wait_for_completion=True, print_log=True, check_interval=30, max_ingestion_time=None)[source]

Create a training job

Parameters:
  • config (dict) – the config for training
  • wait_for_completion (bool) – if the program should keep running until job finishes
  • check_interval (int) – the time interval in seconds which the operator will check the status of any SageMaker job
  • max_ingestion_time (int) – the maximum ingestion time in seconds. Any SageMaker jobs that run longer than this will fail. Setting this to None implies no timeout for any SageMaker job.
Returns:

A response to training job creation

create_transform_job(config, wait_for_completion=True, check_interval=30, max_ingestion_time=None)[source]

Create a transform job

Parameters:
  • config (dict) – the config for transform job
  • wait_for_completion (bool) – if the program should keep running until job finishes
  • check_interval (int) – the time interval in seconds which the operator will check the status of any SageMaker job
  • max_ingestion_time (int) – the maximum ingestion time in seconds. Any SageMaker jobs that run longer than this will fail. Setting this to None implies no timeout for any SageMaker job.
Returns:

A response to transform job creation

create_tuning_job(config, wait_for_completion=True, check_interval=30, max_ingestion_time=None)[source]

Create a tuning job

Parameters:
  • config (dict) – the config for tuning
  • wait_for_completion – if the program should keep running until job finishes
  • wait_for_completion – bool
  • check_interval (int) – the time interval in seconds which the operator will check the status of any SageMaker job
  • max_ingestion_time (int) – the maximum ingestion time in seconds. Any SageMaker jobs that run longer than this will fail. Setting this to None implies no timeout for any SageMaker job.
Returns:

A response to tuning job creation

describe_endpoint(name)[source]
Parameters:name (string) – the name of the endpoint
Returns:A dict contains all the endpoint info
describe_endpoint_config(name)[source]

Return the endpoint config info associated with the name

Parameters:name (string) – the name of the endpoint config
Returns:A dict contains all the endpoint config info
describe_model(name)[source]

Return the SageMaker model info associated with the name

Parameters:name (string) – the name of the SageMaker model
Returns:A dict contains all the model info
describe_training_job(name)[source]

Return the training job info associated with the name

Parameters:name (str) – the name of the training job
Returns:A dict contains all the training job info
describe_training_job_with_log(job_name, positions, stream_names, instance_count, state, last_description, last_describe_job_call)[source]

Return the training job info associated with job_name and print CloudWatch logs

describe_transform_job(name)[source]

Return the transform job info associated with the name

Parameters:name (string) – the name of the transform job
Returns:A dict contains all the transform job info
describe_tuning_job(name)[source]

Return the tuning job info associated with the name

Parameters:name (string) – the name of the tuning job
Returns:A dict contains all the tuning job info
get_conn()[source]

Establish an AWS connection for SageMaker

Returns:a boto3 SageMaker client
get_log_conn()[source]

Establish an AWS connection for retrieving logs during training

Returns:a boto3 CloudWatchLog client
log_stream(log_group, stream_name, start_time=0, skip=0)[source]

A generator for log items in a single stream. This will yield all the items that are available at the current moment.

Parameters:
  • log_group (str) – The name of the log group.
  • stream_name (str) – The name of the specific stream.
  • start_time (int) – The time stamp value to start reading the logs from (default: 0).
  • skip (int) – The number of log entries to skip at the start (default: 0). This is for when there are multiple entries at the same timestamp.
:return:A CloudWatch log event with the following key-value pairs:
‘timestamp’ (int): The time in milliseconds of the event. ‘message’ (str): The log event data. ‘ingestionTime’ (int): The time in milliseconds the event was ingested.
multi_stream_iter(log_group, streams, positions=None)[source]

Iterate over the available events coming from a set of log streams in a single log group interleaving the events from each stream so they’re yielded in timestamp order.

Parameters:
  • log_group (str) – The name of the log group.
  • streams (list) – A list of the log stream names. The position of the stream in this list is the stream number.
  • positions (list) – A list of pairs of (timestamp, skip) which represents the last record read from each stream.
Returns:

A tuple of (stream number, cloudwatch log event).

tar_and_s3_upload(path, key, bucket)[source]

Tar the local file or directory and upload to s3

Parameters:
  • path (str) – local file or directory
  • key (str) – s3 key
  • bucket (str) – s3 bucket
Returns:

None

update_endpoint(config, wait_for_completion=True, check_interval=30, max_ingestion_time=None)[source]

Update an endpoint

Parameters:
  • config (dict) – the config for endpoint
  • wait_for_completion (bool) – if the program should keep running until job finishes
  • check_interval (int) – the time interval in seconds which the operator will check the status of any SageMaker job
  • max_ingestion_time (int) – the maximum ingestion time in seconds. Any SageMaker jobs that run longer than this will fail. Setting this to None implies no timeout for any SageMaker job.
Returns:

A response to endpoint update

class airflow.contrib.hooks.segment_hook.SegmentHook(segment_conn_id='segment_default', segment_debug_mode=False, *args, **kwargs)[source]

Bases: airflow.hooks.base_hook.BaseHook, airflow.utils.log.logging_mixin.LoggingMixin

on_error(error, items)[source]

Handles error callbacks when using Segment with segment_debug_mode set to True

class airflow.contrib.hooks.sftp_hook.SFTPHook(ftp_conn_id='sftp_default', *args, **kwargs)[source]

Bases: airflow.contrib.hooks.ssh_hook.SSHHook

This hook is inherited from SSH hook. Please refer to SSH hook for the input arguments.

Interact with SFTP. Aims to be interchangeable with FTPHook.

Pitfalls: - In contrast with FTPHook describe_directory only returns size, type and
modify. It doesn’t return unix.owner, unix.mode, perm, unix.group and unique.
  • retrieve_file and store_file only take a local full path and not a buffer.
  • If no mode is passed to create_directory it will be created with 777 permissions.

Errors that may occur throughout but should be handled downstream.

close_conn()[source]

Closes the connection. An error will occur if the connection wasnt ever opened.

create_directory(path, mode=777)[source]

Creates a directory on the remote system. :param path: full path to the remote directory to create :type path: str :param mode: int representation of octal mode for directory

delete_directory(path)[source]

Deletes a directory on the remote system. :param path: full path to the remote directory to delete :type path: str

delete_file(path)[source]

Removes a file on the FTP Server :param path: full path to the remote file :type path: str

describe_directory(path)[source]

Returns a dictionary of {filename: {attributes}} for all files on the remote system (where the MLSD command is supported). :param path: full path to the remote directory :type path: str

get_conn()[source]

Returns an SFTP connection object

list_directory(path)[source]

Returns a list of files on the remote system. :param path: full path to the remote directory to list :type path: str

retrieve_file(remote_full_path, local_full_path)[source]

Transfers the remote file to a local location. If local_full_path is a string path, the file will be put at that location :param remote_full_path: full path to the remote file :type remote_full_path: str :param local_full_path: full path to the local file :type local_full_path: str

store_file(remote_full_path, local_full_path)[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 :param remote_full_path: full path to the remote file :type remote_full_path: str :param local_full_path: full path to the local file :type local_full_path: str

class airflow.contrib.hooks.slack_webhook_hook.SlackWebhookHook(http_conn_id=None, webhook_token=None, message='', channel=None, username=None, icon_emoji=None, link_names=False, proxy=None, *args, **kwargs)[source]

Bases: airflow.hooks.http_hook.HttpHook

This hook allows you to post messages to Slack using incoming webhooks. Takes both Slack webhook token directly and connection that has Slack webhook token. If both supplied, Slack webhook token will be used.

Each Slack webhook token can be pre-configured to use a specific channel, username and icon. You can override these defaults in this hook.

Parameters:
  • http_conn_id (str) – connection that has Slack webhook token in the extra field
  • webhook_token (str) – Slack webhook token
  • message (str) – The message you want to send on Slack
  • channel (str) – The channel the message should be posted to
  • username (str) – The username to post to slack with
  • icon_emoji (str) – The emoji to use as icon for the user posting to Slack
  • link_names (bool) – Whether or not to find and link channel and usernames in your message
  • proxy (str) – Proxy to use to make the Slack webhook call
execute()[source]

Remote Popen (actually execute the slack webhook call)

Parameters:
  • cmd – command to remotely execute
  • kwargs – extra arguments to Popen (see subprocess.Popen)
class airflow.contrib.hooks.snowflake_hook.SnowflakeHook(*args, **kwargs)[source]

Bases: airflow.hooks.dbapi_hook.DbApiHook

Interact with Snowflake.

get_sqlalchemy_engine() depends on snowflake-sqlalchemy

get_conn()[source]

Returns a snowflake.connection object

get_uri()[source]

override DbApiHook get_uri method for get_sqlalchemy_engine()

set_autocommit(conn, autocommit)[source]

Sets the autocommit flag on the connection

class airflow.contrib.hooks.spark_jdbc_hook.SparkJDBCHook(spark_app_name='airflow-spark-jdbc', spark_conn_id='spark-default', spark_conf=None, spark_py_files=None, spark_files=None, spark_jars=None, num_executors=None, executor_cores=None, executor_memory=None, driver_memory=None, verbose=False, principal=None, keytab=None, cmd_type='spark_to_jdbc', jdbc_table=None, jdbc_conn_id='jdbc-default', jdbc_driver=None, metastore_table=None, jdbc_truncate=False, save_mode=None, save_format=None, batch_size=None, fetch_size=None, num_partitions=None, partition_column=None, lower_bound=None, upper_bound=None, create_table_column_types=None, *args, **kwargs)[source]

Bases: airflow.contrib.hooks.spark_submit_hook.SparkSubmitHook

This hook extends the SparkSubmitHook specifically for performing data transfers to/from JDBC-based databases with Apache Spark.

Parameters:
  • spark_app_name (str) – Name of the job (default airflow-spark-jdbc)
  • spark_conn_id (str) – Connection id as configured in Airflow administration
  • spark_conf (dict) – Any additional Spark configuration properties
  • spark_py_files (str) – Additional python files used (.zip, .egg, or .py)
  • spark_files (str) – Additional files to upload to the container running the job
  • spark_jars (str) – Additional jars to upload and add to the driver and executor classpath
  • num_executors (int) – number of executor to run. This should be set so as to manage the number of connections made with the JDBC database
  • executor_cores (int) – Number of cores per executor
  • executor_memory (str) – Memory per executor (e.g. 1000M, 2G)
  • driver_memory (str) – Memory allocated to the driver (e.g. 1000M, 2G)
  • verbose (bool) – Whether to pass the verbose flag to spark-submit for debugging
  • keytab (str) – Full path to the file that contains the keytab
  • principal (str) – The name of the kerberos principal used for keytab
  • cmd_type (str) – Which way the data should flow. 2 possible values: spark_to_jdbc: data written by spark from metastore to jdbc jdbc_to_spark: data written by spark from jdbc to metastore
  • jdbc_table (str) – The name of the JDBC table
  • jdbc_conn_id – Connection id used for connection to JDBC database
  • jdbc_driver (str) – Name of the JDBC driver to use for the JDBC connection. This driver (usually a jar) should be passed in the ‘jars’ parameter
  • metastore_table (str) – The name of the metastore table,
  • jdbc_truncate (bool) – (spark_to_jdbc only) Whether or not Spark should truncate or drop and recreate the JDBC table. This only takes effect if ‘save_mode’ is set to Overwrite. Also, if the schema is different, Spark cannot truncate, and will drop and recreate
  • save_mode (str) – The Spark save-mode to use (e.g. overwrite, append, etc.)
  • save_format (str) – (jdbc_to_spark-only) The Spark save-format to use (e.g. parquet)
  • batch_size (int) – (spark_to_jdbc only) The size of the batch to insert per round trip to the JDBC database. Defaults to 1000
  • fetch_size (int) – (jdbc_to_spark only) The size of the batch to fetch per round trip from the JDBC database. Default depends on the JDBC driver
  • num_partitions (int) – The maximum number of partitions that can be used by Spark simultaneously, both for spark_to_jdbc and jdbc_to_spark operations. This will also cap the number of JDBC connections that can be opened
  • partition_column (str) – (jdbc_to_spark-only) A numeric column to be used to partition the metastore table by. If specified, you must also specify: num_partitions, lower_bound, upper_bound
  • lower_bound (int) – (jdbc_to_spark-only) Lower bound of the range of the numeric partition column to fetch. If specified, you must also specify: num_partitions, partition_column, upper_bound
  • upper_bound (int) – (jdbc_to_spark-only) Upper bound of the range of the numeric partition column to fetch. If specified, you must also specify: num_partitions, partition_column, lower_bound
  • create_table_column_types – (spark_to_jdbc-only) The database column data types to use instead of the defaults, when creating the table. Data type information should be specified in the same format as CREATE TABLE columns syntax (e.g: “name CHAR(64), comments VARCHAR(1024)”). The specified types should be valid spark sql data types.
Type:

jdbc_conn_id: str

class airflow.contrib.hooks.spark_sql_hook.SparkSqlHook(sql, conf=None, conn_id='spark_sql_default', total_executor_cores=None, executor_cores=None, executor_memory=None, keytab=None, principal=None, master='yarn', name='default-name', num_executors=None, verbose=True, yarn_queue='default')[source]

Bases: airflow.hooks.base_hook.BaseHook

This hook is a wrapper around the spark-sql binary. It requires that the “spark-sql” binary is in the PATH. :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 total_executor_cores: (Standalone & Mesos only) Total cores for all executors

(Default: all the available cores on the worker)
Parameters:
  • executor_cores (int) – (Standalone & YARN only) 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
  • master (str) – spark://host:port, mesos://host:port, yarn, or local
  • name (str) – Name of the job.
  • num_executors (int) – Number of executors to launch
  • verbose (bool) – Whether to pass the verbose flag to spark-sql
  • yarn_queue (str) – The YARN queue to submit to (Default: “default”)
run_query(cmd='', **kwargs)[source]

Remote Popen (actually execute the Spark-sql query)

Parameters:
  • cmd – command to remotely execute
  • kwargs – extra arguments to Popen (see subprocess.Popen)
class airflow.contrib.hooks.spark_submit_hook.SparkSubmitHook(conf=None, conn_id='spark_default', files=None, py_files=None, driver_classpath=None, jars=None, java_class=None, packages=None, exclude_packages=None, repositories=None, total_executor_cores=None, executor_cores=None, executor_memory=None, driver_memory=None, keytab=None, principal=None, name='default-name', num_executors=None, application_args=None, env_vars=None, verbose=False)[source]

Bases: airflow.hooks.base_hook.BaseHook, airflow.utils.log.logging_mixin.LoggingMixin

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 or the spark_home to be supplied. :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 executor running the job, separated by a comma. Files will be placed in the working directory of each executor. For example, serialized objects.
  • py_files (str) – Additional python files used by the job, can be .zip, .egg or .py.
  • driver_classpath (str) – Additional, driver-specific, classpath settings.
  • jars (str) – Submit additional jars to upload and place them in executor classpath.
  • java_class (str) – the main class of the Java application
  • packages – Comma-separated list of maven coordinates of jars to include on the

driver and executor classpaths :type packages: str :param exclude_packages: Comma-separated list of maven coordinates of jars to exclude while resolving the dependencies provided in ‘packages’ :type exclude_packages: str :param repositories: Comma-separated list of additional remote repositories to search for the maven coordinates given with ‘packages’ :type repositories: str :param total_executor_cores: (Standalone & Mesos only) Total cores for all executors (Default: all the available cores on the worker) :type total_executor_cores: int :param executor_cores: (Standalone, YARN and Kubernetes only) Number of cores per executor (Default: 2) :type executor_cores: int :param executor_memory: Memory per executor (e.g. 1000M, 2G) (Default: 1G) :type executor_memory: str :param driver_memory: Memory allocated to the driver (e.g. 1000M, 2G) (Default: 1G) :type driver_memory: str :param keytab: Full path to the file that contains the keytab :type keytab: str :param principal: The name of the kerberos principal used for keytab :type principal: str :param name: Name of the job (default airflow-spark) :type name: str :param num_executors: Number of executors to launch :type num_executors: int :param application_args: Arguments for the application being submitted :type application_args: list :param env_vars: Environment variables for spark-submit. It

supports yarn and k8s mode too.
Parameters:verbose (bool) – Whether to pass the verbose flag to spark-submit process for debugging
submit(application='', **kwargs)[source]

Remote Popen to execute the spark-submit job

Parameters:
  • application (str) – Submitted application, jar or py file
  • kwargs – extra arguments to Popen (see subprocess.Popen)
class airflow.contrib.hooks.sqoop_hook.SqoopHook(conn_id='sqoop_default', verbose=False, num_mappers=None, hcatalog_database=None, hcatalog_table=None, properties=None)[source]

Bases: airflow.hooks.base_hook.BaseHook, airflow.utils.log.logging_mixin.LoggingMixin

This hook is a wrapper around the sqoop 1 binary. To be able to use the hook it is required that “sqoop” is in the PATH.

Additional arguments that can be passed via the ‘extra’ JSON field of the sqoop connection: * job_tracker: Job tracker local|jobtracker:port. * namenode: Namenode. * lib_jars: Comma separated jar files to include in the classpath. * files: Comma separated files to be copied to the map reduce cluster. * archives: Comma separated archives to be unarchived on the compute

machines.
  • password_file: Path to file containing the password.
Parameters:
  • conn_id (str) – Reference to the sqoop connection.
  • verbose (bool) – Set sqoop to verbose.
  • num_mappers (int) – Number of map tasks to import in parallel.
  • properties (dict) – Properties to set via the -D argument
Popen(cmd, **kwargs)[source]

Remote Popen

Parameters:
  • cmd – command to remotely execute
  • kwargs – extra arguments to Popen (see subprocess.Popen)
Returns:

handle to subprocess

export_table(table, export_dir, input_null_string, input_null_non_string, staging_table, clear_staging_table, enclosed_by, escaped_by, input_fields_terminated_by, input_lines_terminated_by, input_optionally_enclosed_by, batch, relaxed_isolation, extra_export_options=None)[source]

Exports Hive table to remote location. Arguments are copies of direct sqoop command line Arguments :param table: Table remote destination :param export_dir: Hive table to export :param input_null_string: The string to be interpreted as null for

string columns
Parameters:
  • input_null_non_string – The string to be interpreted as null for non-string columns
  • staging_table – The table in which data will be staged before being inserted into the destination table
  • clear_staging_table – Indicate that any data present in the staging table can be deleted
  • enclosed_by – Sets a required field enclosing character
  • escaped_by – Sets the escape character
  • input_fields_terminated_by – Sets the field separator character
  • input_lines_terminated_by – Sets the end-of-line character
  • input_optionally_enclosed_by – Sets a field enclosing character
  • batch – Use batch mode for underlying statement execution
  • relaxed_isolation – Transaction isolation to read uncommitted for the mappers
  • extra_export_options – Extra export options to pass as dict. If a key doesn’t have a value, just pass an empty string to it. Don’t include prefix of – for sqoop options.
import_query(query, target_dir, append=False, file_type='text', split_by=None, direct=None, driver=None, extra_import_options=None)[source]

Imports a specific query from the rdbms to hdfs :param query: Free format query to run :param target_dir: HDFS destination dir :param append: Append data to an existing dataset in HDFS :param file_type: “avro”, “sequence”, “text” or “parquet”

Imports data to hdfs into the specified format. Defaults to text.
Parameters:
  • split_by – Column of the table used to split work units
  • direct – Use direct import fast path
  • driver – Manually specify JDBC driver class to use
  • extra_import_options – Extra import options to pass as dict. If a key doesn’t have a value, just pass an empty string to it. Don’t include prefix of – for sqoop options.
import_table(table, target_dir=None, append=False, file_type='text', columns=None, split_by=None, where=None, direct=False, driver=None, extra_import_options=None)[source]

Imports table from remote location to target dir. Arguments are copies of direct sqoop command line arguments :param table: Table to read :param target_dir: HDFS destination dir :param append: Append data to an existing dataset in HDFS :param file_type: “avro”, “sequence”, “text” or “parquet”.

Imports data to into the specified format. Defaults to text.
Parameters:
  • columns – <col,col,col…> Columns to import from table
  • split_by – Column of the table used to split work units
  • where – WHERE clause to use during import
  • direct – Use direct connector if exists for the database
  • driver – Manually specify JDBC driver class to use
  • extra_import_options – Extra import options to pass as dict. If a key doesn’t have a value, just pass an empty string to it. Don’t include prefix of – for sqoop options.
class airflow.contrib.hooks.ssh_hook.SSHHook(ssh_conn_id=None, remote_host=None, username=None, password=None, key_file=None, port=None, timeout=10, keepalive_interval=30)[source]

Bases: airflow.hooks.base_hook.BaseHook, airflow.utils.log.logging_mixin.LoggingMixin

Hook for ssh remote execution using Paramiko. ref: https://github.com/paramiko/paramiko This hook also lets you create ssh tunnel and serve as basis for SFTP file transfer

Parameters:
  • ssh_conn_id (str) – connection id from airflow Connections from where all the required parameters can be fetched like username, password or key_file. Thought the priority is given to the param passed during init
  • remote_host (str) – remote host to connect
  • username (str) – username to connect to the remote_host
  • password (str) – password of the username to connect to the remote_host
  • key_file (str) – key file to use to connect to the remote_host.
  • port (int) – port of remote host to connect (Default is paramiko SSH_PORT)
  • timeout (int) – timeout for the attempt to connect to the remote_host.
  • keepalive_interval (int) – send a keepalive packet to remote host every keepalive_interval seconds
get_conn()[source]

Opens a ssh connection to the remote host.

:return paramiko.SSHClient object

get_tunnel(remote_port, remote_host='localhost', local_port=None)[source]

Creates a tunnel between two hosts. Like ssh -L <LOCAL_PORT>:host:<REMOTE_PORT>.

Parameters:
  • remote_port (int) – The remote port to create a tunnel to
  • remote_host (str) – The remote host to create a tunnel to (default localhost)
  • local_port (int) – The local port to attach the tunnel to
Returns:

sshtunnel.SSHTunnelForwarder object

class airflow.contrib.hooks.vertica_hook.VerticaHook(*args, **kwargs)[source]

Bases: airflow.hooks.dbapi_hook.DbApiHook

Interact with Vertica.

get_conn()[source]

Returns verticaql connection object

class airflow.contrib.hooks.wasb_hook.WasbHook(wasb_conn_id='wasb_default')[source]

Bases: airflow.hooks.base_hook.BaseHook

Interacts with Azure Blob Storage through the wasb:// protocol.

Additional options passed in the ‘extra’ field of the connection will be passed to the BlockBlockService() constructor. For example, authenticate using a SAS token by adding {“sas_token”: “YOUR_TOKEN”}.

Parameters:wasb_conn_id (str) – Reference to the wasb connection.
check_for_blob(container_name, blob_name, **kwargs)[source]

Check if a blob exists on Azure Blob Storage.

Parameters:
  • container_name (str) – Name of the container.
  • blob_name (str) – Name of the blob.
  • kwargs (object) – Optional keyword arguments that BlockBlobService.exists() takes.
Returns:

True if the blob exists, False otherwise.

:rtype bool

check_for_prefix(container_name, prefix, **kwargs)[source]

Check if a prefix exists on Azure Blob storage.

Parameters:
  • container_name (str) – Name of the container.
  • prefix (str) – Prefix of the blob.
  • kwargs (object) – Optional keyword arguments that BlockBlobService.list_blobs() takes.
Returns:

True if blobs matching the prefix exist, False otherwise.

:rtype bool

delete_file(container_name, blob_name, is_prefix=False, ignore_if_missing=False, **kwargs)[source]

Delete a file from Azure Blob Storage.

Parameters:
  • container_name (str) – Name of the container.
  • blob_name (str) – Name of the blob.
  • is_prefix (bool) – If blob_name is a prefix, delete all matching files
  • ignore_if_missing – if True, then return success even if the

blob does not exist. :type ignore_if_missing: bool :param kwargs: Optional keyword arguments that

BlockBlobService.create_blob_from_path() takes.
get_conn()[source]

Return the BlockBlobService object.

get_file(file_path, container_name, blob_name, **kwargs)[source]

Download a file from Azure Blob Storage.

Parameters:
  • file_path (str) – Path to the file to download.
  • container_name (str) – Name of the container.
  • blob_name (str) – Name of the blob.
  • kwargs (object) – Optional keyword arguments that BlockBlobService.create_blob_from_path() takes.
load_file(file_path, container_name, blob_name, **kwargs)[source]

Upload a file to Azure Blob Storage.

Parameters:
  • file_path (str) – Path to the file to load.
  • container_name (str) – Name of the container.
  • blob_name (str) – Name of the blob.
  • kwargs (object) – Optional keyword arguments that BlockBlobService.create_blob_from_path() takes.
load_string(string_data, container_name, blob_name, **kwargs)[source]

Upload a string to Azure Blob Storage.

Parameters:
  • string_data (str) – String to load.
  • container_name (str) – Name of the container.
  • blob_name (str) – Name of the blob.
  • kwargs (object) – Optional keyword arguments that BlockBlobService.create_blob_from_text() takes.
read_file(container_name, blob_name, **kwargs)[source]

Read a file from Azure Blob Storage and return as a string.

Parameters:
  • container_name (str) – Name of the container.
  • blob_name (str) – Name of the blob.
  • kwargs (object) – Optional keyword arguments that BlockBlobService.create_blob_from_path() takes.
class airflow.contrib.hooks.winrm_hook.WinRMHook(ssh_conn_id=None, remote_host=None, username=None, password=None, key_file=None, timeout=10, keepalive_interval=30)[source]

Bases: airflow.hooks.base_hook.BaseHook, airflow.utils.log.logging_mixin.LoggingMixin

Hook for winrm remote execution using pywinrm.

Parameters:
  • ssh_conn_id (str) – connection id from airflow Connections from where all the required parameters can be fetched like username, password or key_file. Thought the priority is given to the param passed during init
  • remote_host (str) – remote host to connect
  • username (str) – username to connect to the remote_host
  • password (str) – password of the username to connect to the remote_host
  • key_file (str) – key file to use to connect to the remote_host.
  • timeout (int) – timeout for the attempt to connect to the remote_host.
  • keepalive_interval (int) – send a keepalive packet to remote host every keepalive_interval seconds

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.

end()[source]

This method is called when the caller is done submitting job and is wants to wait synchronously for the job submitted previously to be all done.

execute_async(key, command, queue=None, executor_config=None)[source]

This method will execute the command asynchronously.

start()[source]

Executors may need to get things started. For example LocalExecutor starts N workers.

sync()[source]

Sync will get called periodically by the heartbeat method. Executors should override this to perform gather statuses.

class airflow.executors.celery_executor.CeleryExecutor(parallelism=32)[source]

Bases: airflow.executors.base_executor.BaseExecutor

CeleryExecutor is recommended for production use of Airflow. It allows distributing the execution of task instances to multiple worker nodes.

Celery is a simple, flexible and reliable distributed system to process vast amounts of messages, while providing operations with the tools required to maintain such a system.

end(synchronous=False)[source]

This method is called when the caller is done submitting job and is wants to wait synchronously for the job submitted previously to be all done.

execute_async(key, command, queue='default', executor_config=None)[source]

This method will execute the command asynchronously.

start()[source]

Executors may need to get things started. For example LocalExecutor starts N workers.

sync()[source]

Sync will get called periodically by the heartbeat method. Executors should override this to perform gather statuses.

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.

end()[source]

This method is called when the caller is done submitting job and is wants to wait synchronously for the job submitted previously to be all done.

execute_async(key, command, queue=None, executor_config=None)[source]

This method will execute the command asynchronously.

sync()[source]

Sync will get called periodically by the heartbeat method. Executors should override this to perform gather statuses.

Community-contributed executors