diff --git a/.gitignore b/.gitignore
index 089d7b1e2f7e3..0ccc81842735e 100644
--- a/.gitignore
+++ b/.gitignore
@@ -83,6 +83,7 @@ instance/
# Sphinx documentation
docs/_build/
+docs/_api/
# PyBuilder
target/
diff --git a/airflow/contrib/hooks/gcp_transfer_hook.py b/airflow/contrib/hooks/gcp_transfer_hook.py
index c6fd4d5e2f6a6..d51d7747a9644 100644
--- a/airflow/contrib/hooks/gcp_transfer_hook.py
+++ b/airflow/contrib/hooks/gcp_transfer_hook.py
@@ -267,7 +267,7 @@ def list_transfer_operations(self, filter):
* project_id is optional if you have a project id defined
in the connection
- See: :doc:`howto/connection/gcp`
+ See: :ref:`howto/connection:gcp`
:type filter: dict
:return: transfer operation
diff --git a/airflow/contrib/operators/kubernetes_pod_operator.py b/airflow/contrib/operators/kubernetes_pod_operator.py
index 83b57f6ca7268..f36bea8fc6b02 100644
--- a/airflow/contrib/operators/kubernetes_pod_operator.py
+++ b/airflow/contrib/operators/kubernetes_pod_operator.py
@@ -21,9 +21,6 @@
from airflow.contrib.kubernetes import kube_client, pod_generator, pod_launcher
from airflow.contrib.kubernetes.pod import Resources
from airflow.utils.state import State
-from airflow.contrib.kubernetes.volume_mount import VolumeMount # noqa
-from airflow.contrib.kubernetes.volume import Volume # noqa
-from airflow.contrib.kubernetes.secret import Secret # noqa
class KubernetesPodOperator(BaseOperator):
diff --git a/airflow/hooks/hdfs_hook.py b/airflow/hooks/hdfs_hook.py
index 597b7c4f7ec83..31b2f11501a01 100644
--- a/airflow/hooks/hdfs_hook.py
+++ b/airflow/hooks/hdfs_hook.py
@@ -16,18 +16,15 @@
# KIND, either express or implied. See the License for the
# specific language governing permissions and limitations
# under the License.
-
-from six import PY2
-
from airflow import configuration
from airflow.exceptions import AirflowException
from airflow.hooks.base_hook import BaseHook
-
-snakebite_imported = False
-if PY2:
+try:
from snakebite.client import Client, HAClient, Namenode, AutoConfigClient
- snakebite_imported = True
+ snakebite_loaded = True
+except ImportError:
+ snakebite_loaded = False
class HDFSHookException(AirflowException):
@@ -47,7 +44,7 @@ class HDFSHook(BaseHook):
"""
def __init__(self, hdfs_conn_id='hdfs_default', proxy_user=None,
autoconfig=False):
- if not snakebite_imported:
+ if not snakebite_loaded:
raise ImportError(
'This HDFSHook implementation requires snakebite, but '
'snakebite is not compatible with Python 3 '
diff --git a/airflow/operators/bash_operator.py b/airflow/operators/bash_operator.py
index bd755b544ba6c..00a4f3dc67ba0 100644
--- a/airflow/operators/bash_operator.py
+++ b/airflow/operators/bash_operator.py
@@ -37,7 +37,7 @@ class BashOperator(BaseOperator):
.. seealso::
For more information on how to use this operator, take a look at the guide:
- :doc:`howto/operator/bash`
+ :ref:`howto/operator:BashOperator`
If BaseOperator.do_xcom_push is True, the last line written to stdout
will also be pushed to an XCom when the bash command completes
diff --git a/airflow/operators/python_operator.py b/airflow/operators/python_operator.py
index a5289eb1b5100..c44d34b46a191 100644
--- a/airflow/operators/python_operator.py
+++ b/airflow/operators/python_operator.py
@@ -43,7 +43,7 @@ class PythonOperator(BaseOperator):
.. seealso::
For more information on how to use this operator, take a look at the guide:
- :doc:`howto/operator/python`
+ :ref:`howto/operator:PythonOperator`
:param python_callable: A reference to an object that is callable
:type python_callable: python callable
diff --git a/docs/autoapi_templates/index.rst b/docs/autoapi_templates/index.rst
new file mode 100644
index 0000000000000..ab32d3aa29c0c
--- /dev/null
+++ b/docs/autoapi_templates/index.rst
@@ -0,0 +1,129 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+ or more contributor license agreements. See the NOTICE file
+ distributed with this work for additional information
+ regarding copyright ownership. The ASF licenses this file
+ to you under the Apache License, Version 2.0 (the
+ "License"); you may not use this file except in compliance
+ with the License. You may obtain a copy of the License at
+
+.. http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+ software distributed under the License is distributed on an
+ "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ KIND, either express or implied. See the License for the
+ specific language governing permissions and limitations
+ under the License.
+
+API Reference
+=============
+
+Operators
+---------
+Operators allow for generation of certain types of tasks that become nodes in
+the DAG when instantiated. All operators derive from :class:`~airflow.models.BaseOperator` and
+inherit many attributes and methods that way.
+
+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 :class:`~airflow.sensors.base_sensor_operator.BaseSensorOperator` and run a poke
+ method at a specified :attr:`~airflow.sensors.base_sensor_operator.BaseSensorOperator.poke_interval` until it returns ``True``.
+
+BaseOperator
+''''''''''''
+All operators are derived from :class:`~airflow.models.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 :class:`~airflow.models.BaseOperator`
+to understand the primitive features that can be leveraged in your
+DAGs.
+
+BaseSensorOperator
+''''''''''''''''''
+All sensors are derived from :class:`~airflow.sensors.base_sensor_operator.BaseSensorOperator`. All sensors inherit
+the :attr:`~airflow.sensors.base_sensor_operator.BaseSensorOperator.timeout` and :attr:`~airflow.sensors.base_sensor_operator.BaseSensorOperator.poke_interval` on top of the :class:`~airflow.models.BaseOperator`
+attributes.
+
+Operators packages
+''''''''''''''''''
+All operators are in the following packages:
+
+.. toctree::
+ :includehidden:
+ :glob:
+ :maxdepth: 1
+
+ airflow/operators/index
+
+ airflow/sensors/index
+
+ airflow/contrib/operators/index
+
+ airflow/contrib/sensors/index
+
+
+Hooks
+-----
+Hooks are interfaces to external platforms and databases, implementing a common
+interface when possible and acting as building blocks for operators. All hooks
+are derived from :class:`~airflow.hooks.base_hook.BaseHook`.
+
+Hooks packages
+''''''''''''''
+All hooks are in the following packages:
+
+.. toctree::
+ :includehidden:
+ :glob:
+ :maxdepth: 1
+
+ airflow/hooks/index
+
+ airflow/contrib/hooks/index
+
+
+Executors
+---------
+Executors are the mechanism by which task instances get run. All executors are
+derived from :class:`~airflow.executors.base_executor.BaseExecutor`.
+
+Executors packages
+''''''''''''''''''
+All executors are in the following packages:
+
+.. toctree::
+ :includehidden:
+ :glob:
+ :maxdepth: 1
+
+ airflow/executors/index
+
+ airflow/contrib/executors/index
+
+
+Models
+------
+Models are built on top of the SQLAlchemy ORM Base class, and instances are
+persisted in the database.
+
+.. toctree::
+ :includehidden:
+ :glob:
+ :maxdepth: 1
+
+ airflow/models/index
+
+
+Core and community package
+--------------------------
+Formerly the core code was maintained by the original creators - Airbnb. The code
+that was in the contrib package was supported by the community. The project
+was passed to the Apache community and currently the entire code is
+maintained by the community, so now the division has no justification,
+and it is only due to historical reasons. Currently, all new classes are
+added only to the contrib package.
diff --git a/docs/build.sh b/docs/build.sh
index 7fcbfa2ac06ee..7d7a433f5ac02 100755
--- a/docs/build.sh
+++ b/docs/build.sh
@@ -23,7 +23,8 @@ set -e
FWDIR="$(cd "`dirname "$0"`"; pwd)"
cd "$FWDIR"
-[ -d _build ] && rm -r _build
+[[ -d "_build" ]] && rm -r _build
+[[ -d "_api" ]] && rm -r _api
SUCCEED_LINE=$(make html |\
tee /dev/tty |\
diff --git a/docs/code.rst b/docs/code.rst
deleted file mode 100644
index 8dfe187420842..0000000000000
--- a/docs/code.rst
+++ /dev/null
@@ -1,569 +0,0 @@
-.. Licensed to the Apache Software Foundation (ASF) under one
- or more contributor license agreements. See the NOTICE file
- distributed with this work for additional information
- regarding copyright ownership. The ASF licenses this file
- to you under the Apache License, Version 2.0 (the
- "License"); you may not use this file except in compliance
- with the License. You may obtain a copy of the License at
-
-.. http://www.apache.org/licenses/LICENSE-2.0
-
-.. Unless required by applicable law or agreed to in writing,
- software distributed under the License is distributed on an
- "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- KIND, either express or implied. See the License for the
- specific language governing permissions and limitations
- under the License.
-
-API Reference
-=============
-
-.. _api-reference-operators:
-
-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.
-
-
-.. autoclass:: airflow.models.BaseOperator
-
-
-BaseSensorOperator
-'''''''''''''''''''
-All sensors are derived from ``BaseSensorOperator``. All sensors inherit
-the ``timeout`` and ``poke_interval`` on top of the ``BaseOperator``
-attributes.
-
-.. autoclass:: airflow.sensors.base_sensor_operator.BaseSensorOperator
-
-
-Core Operators
-''''''''''''''
-
-Operators
-^^^^^^^^^
-
-.. autoclass:: airflow.operators.bash_operator.BashOperator
-.. autoclass:: airflow.operators.python_operator.BranchPythonOperator
-.. autoclass:: airflow.operators.check_operator.CheckOperator
-.. autoclass:: airflow.operators.docker_operator.DockerOperator
-.. autoclass:: airflow.operators.druid_check_operator.DruidCheckOperator
-.. autoclass:: airflow.operators.dummy_operator.DummyOperator
-.. autoclass:: airflow.operators.email_operator.EmailOperator
-.. autoclass:: airflow.operators.generic_transfer.GenericTransfer
-.. autoclass:: airflow.operators.hive_to_samba_operator.Hive2SambaOperator
-.. autoclass:: airflow.operators.hive_operator.HiveOperator
-.. autoclass:: airflow.operators.hive_stats_operator.HiveStatsCollectionOperator
-.. autoclass:: airflow.operators.hive_to_druid.HiveToDruidTransfer
-.. autoclass:: airflow.operators.hive_to_mysql.HiveToMySqlTransfer
-.. autoclass:: airflow.operators.check_operator.IntervalCheckOperator
-.. autoclass:: airflow.operators.jdbc_operator.JdbcOperator
-.. autoclass:: airflow.operators.latest_only_operator.LatestOnlyOperator
-.. autoclass:: airflow.operators.mssql_operator.MsSqlOperator
-.. autoclass:: airflow.operators.mssql_to_hive.MsSqlToHiveTransfer
-.. autoclass:: airflow.operators.mysql_operator.MySqlOperator
-.. autoclass:: airflow.operators.mysql_to_hive.MySqlToHiveTransfer
-.. autoclass:: airflow.operators.oracle_operator.OracleOperator
-.. autoclass:: airflow.operators.pig_operator.PigOperator
-.. autoclass:: airflow.operators.postgres_operator.PostgresOperator
-.. autoclass:: airflow.operators.presto_check_operator.PrestoCheckOperator
-.. autoclass:: airflow.operators.presto_check_operator.PrestoIntervalCheckOperator
-.. autoclass:: airflow.operators.presto_to_mysql.PrestoToMySqlTransfer
-.. autoclass:: airflow.operators.presto_check_operator.PrestoValueCheckOperator
-.. autoclass:: airflow.operators.python_operator.PythonOperator
-.. autoclass:: airflow.operators.python_operator.PythonVirtualenvOperator
-.. autoclass:: airflow.operators.redshift_to_s3_operator.RedshiftToS3Transfer
-.. autoclass:: airflow.operators.s3_file_transform_operator.S3FileTransformOperator
-.. autoclass:: airflow.operators.s3_to_hive_operator.S3ToHiveTransfer
-.. autoclass:: airflow.operators.s3_to_redshift_operator.S3ToRedshiftTransfer
-.. autoclass:: airflow.operators.python_operator.ShortCircuitOperator
-.. autoclass:: airflow.operators.http_operator.SimpleHttpOperator
-.. autoclass:: airflow.operators.slack_operator.SlackAPIOperator
-.. autoclass:: airflow.operators.slack_operator.SlackAPIPostOperator
-.. autoclass:: airflow.operators.sqlite_operator.SqliteOperator
-.. autoclass:: airflow.operators.subdag_operator.SubDagOperator
-.. autoclass:: airflow.operators.dagrun_operator.TriggerDagRunOperator
-.. autoclass:: airflow.operators.check_operator.ValueCheckOperator
-
-Sensors
-^^^^^^^
-.. autoclass:: airflow.sensors.external_task_sensor.ExternalTaskSensor
-.. autoclass:: airflow.sensors.hdfs_sensor.HdfsSensor
-.. autoclass:: airflow.sensors.hive_partition_sensor.HivePartitionSensor
-.. autoclass:: airflow.sensors.http_sensor.HttpSensor
-.. autoclass:: airflow.sensors.metastore_partition_sensor.MetastorePartitionSensor
-.. autoclass:: airflow.sensors.named_hive_partition_sensor.NamedHivePartitionSensor
-.. autoclass:: airflow.sensors.s3_key_sensor.S3KeySensor
-.. autoclass:: airflow.sensors.s3_prefix_sensor.S3PrefixSensor
-.. autoclass:: airflow.sensors.sql_sensor.SqlSensor
-.. autoclass:: airflow.sensors.time_delta_sensor.TimeDeltaSensor
-.. autoclass:: airflow.sensors.time_sensor.TimeSensor
-.. autoclass:: airflow.sensors.web_hdfs_sensor.WebHdfsSensor
-
-Community-contributed Operators
-'''''''''''''''''''''''''''''''
-
-Operators
-^^^^^^^^^
-.. Alphabetize this list
-
-.. autoclass:: airflow.contrib.operators.aws_athena_operator.AWSAthenaOperator
-.. autoclass:: airflow.contrib.operators.awsbatch_operator.AWSBatchOperator
-.. autoclass:: airflow.contrib.operators.adls_to_gcs.AdlsToGoogleCloudStorageOperator
-.. autoclass:: airflow.contrib.operators.azure_container_instances_operator.AzureContainerInstancesOperator
-.. autoclass:: airflow.contrib.operators.azure_cosmos_operator.AzureCosmosInsertDocumentOperator
-.. autoclass:: airflow.contrib.operators.adls_list_operator.AzureDataLakeStorageListOperator
-.. autoclass:: airflow.contrib.operators.bigquery_check_operator.BigQueryCheckOperator
-.. autoclass:: airflow.contrib.operators.bigquery_operator.BigQueryCreateEmptyDatasetOperator
-.. autoclass:: airflow.contrib.operators.bigquery_operator.BigQueryCreateEmptyTableOperator
-.. autoclass:: airflow.contrib.operators.bigquery_operator.BigQueryCreateExternalTableOperator
-.. autoclass:: airflow.contrib.operators.bigquery_operator.BigQueryDeleteDatasetOperator
-.. autoclass:: airflow.contrib.operators.bigquery_get_data.BigQueryGetDataOperator
-.. autoclass:: airflow.contrib.operators.bigquery_check_operator.BigQueryIntervalCheckOperator
-.. autoclass:: airflow.contrib.operators.bigquery_operator.BigQueryOperator
-.. autoclass:: airflow.contrib.operators.bigquery_table_delete_operator.BigQueryTableDeleteOperator
-.. autoclass:: airflow.contrib.operators.bigquery_to_bigquery.BigQueryToBigQueryOperator
-.. autoclass:: airflow.contrib.operators.bigquery_to_gcs.BigQueryToCloudStorageOperator
-.. autoclass:: airflow.contrib.operators.bigquery_check_operator.BigQueryValueCheckOperator
-.. autoclass:: airflow.contrib.operators.gcp_bigtable_operator.BigtableClusterUpdateOperator
-.. autoclass:: airflow.contrib.operators.gcp_bigtable_operator.BigtableInstanceCreateOperator
-.. autoclass:: airflow.contrib.operators.gcp_bigtable_operator.BigtableInstanceDeleteOperator
-.. autoclass:: airflow.contrib.operators.gcp_bigtable_operator.BigtableTableCreateOperator
-.. autoclass:: airflow.contrib.operators.gcp_bigtable_operator.BigtableTableDeleteOperator
-.. autoclass:: airflow.contrib.operators.gcp_bigtable_operator.BigtableTableWaitForReplicationSensor
-.. autoclass:: airflow.contrib.operators.cassandra_to_gcs.CassandraToGoogleCloudStorageOperator
-.. autoclass:: airflow.contrib.operators.gcp_spanner_operator.CloudSpannerInstanceDatabaseDeleteOperator
-.. autoclass:: airflow.contrib.operators.gcp_spanner_operator.CloudSpannerInstanceDatabaseDeployOperator
-.. autoclass:: airflow.contrib.operators.gcp_spanner_operator.CloudSpannerInstanceDatabaseQueryOperator
-.. autoclass:: airflow.contrib.operators.gcp_spanner_operator.CloudSpannerInstanceDatabaseUpdateOperator
-.. autoclass:: airflow.contrib.operators.gcp_spanner_operator.CloudSpannerInstanceDeleteOperator
-.. autoclass:: airflow.contrib.operators.gcp_spanner_operator.CloudSpannerInstanceDeployOperator
-.. autoclass:: airflow.contrib.operators.gcp_sql_operator.CloudSqlBaseOperator
-.. autoclass:: airflow.contrib.operators.gcp_sql_operator.CloudSqlInstanceCreateOperator
-.. autoclass:: airflow.contrib.operators.gcp_sql_operator.CloudSqlInstanceDatabaseCreateOperator
-.. autoclass:: airflow.contrib.operators.gcp_sql_operator.CloudSqlInstanceDatabaseDeleteOperator
-.. autoclass:: airflow.contrib.operators.gcp_sql_operator.CloudSqlInstanceDatabasePatchOperator
-.. autoclass:: airflow.contrib.operators.gcp_sql_operator.CloudSqlInstanceDeleteOperator
-.. autoclass:: airflow.contrib.operators.gcp_sql_operator.CloudSqlInstanceExportOperator
-.. autoclass:: airflow.contrib.operators.gcp_sql_operator.CloudSqlInstanceImportOperator
-.. autoclass:: airflow.contrib.operators.gcp_sql_operator.CloudSqlInstancePatchOperator
-.. autoclass:: airflow.contrib.operators.gcp_sql_operator.CloudSqlQueryOperator
-.. autoclass:: airflow.contrib.operators.gcp_translate_operator.CloudTranslateTextOperator
-.. autoclass:: airflow.contrib.operators.gcp_vision_operator.CloudVisionAddProductToProductSetOperator
-.. autoclass:: airflow.contrib.operators.gcp_vision_operator.CloudVisionAnnotateImageOperator
-.. autoclass:: airflow.contrib.operators.gcp_vision_operator.CloudVisionProductCreateOperator
-.. autoclass:: airflow.contrib.operators.gcp_vision_operator.CloudVisionProductDeleteOperator
-.. autoclass:: airflow.contrib.operators.gcp_vision_operator.CloudVisionProductGetOperator
-.. autoclass:: airflow.contrib.operators.gcp_vision_operator.CloudVisionProductSetCreateOperator
-.. autoclass:: airflow.contrib.operators.gcp_vision_operator.CloudVisionProductSetDeleteOperator
-.. autoclass:: airflow.contrib.operators.gcp_vision_operator.CloudVisionProductSetGetOperator
-.. autoclass:: airflow.contrib.operators.gcp_vision_operator.CloudVisionProductSetUpdateOperator
-.. autoclass:: airflow.contrib.operators.gcp_vision_operator.CloudVisionProductUpdateOperator
-.. autoclass:: airflow.contrib.operators.gcp_vision_operator.CloudVisionReferenceImageCreateOperator
-.. autoclass:: airflow.contrib.operators.gcp_vision_operator.CloudVisionRemoveProductFromProductSetOperator
-.. autoclass:: airflow.contrib.operators.dataflow_operator.DataFlowJavaOperator
-.. autoclass:: airflow.contrib.operators.dataflow_operator.DataFlowPythonOperator
-.. autoclass:: airflow.contrib.operators.dataproc_operator.DataProcHadoopOperator
-.. autoclass:: airflow.contrib.operators.dataproc_operator.DataProcHiveOperator
-.. autoclass:: airflow.contrib.operators.dataproc_operator.DataProcPigOperator
-.. autoclass:: airflow.contrib.operators.dataproc_operator.DataProcPySparkOperator
-.. autoclass:: airflow.contrib.operators.dataproc_operator.DataProcSparkOperator
-.. autoclass:: airflow.contrib.operators.dataproc_operator.DataProcSparkSqlOperator
-.. autoclass:: airflow.contrib.operators.databricks_operator.DatabricksRunNowOperator
-.. autoclass:: airflow.contrib.operators.databricks_operator.DatabricksSubmitRunOperator
-.. autoclass:: airflow.contrib.operators.dataflow_operator.DataflowTemplateOperator
-.. autoclass:: airflow.contrib.operators.dataproc_operator.DataprocClusterCreateOperator
-.. autoclass:: airflow.contrib.operators.dataproc_operator.DataprocClusterDeleteOperator
-.. autoclass:: airflow.contrib.operators.dataproc_operator.DataprocClusterScaleOperator
-.. autoclass:: airflow.contrib.operators.dataproc_operator.DataprocWorkflowTemplateBaseOperator
-.. autoclass:: airflow.contrib.operators.dataproc_operator.DataprocWorkflowTemplateInstantiateInlineOperator
-.. autoclass:: airflow.contrib.operators.dataproc_operator.DataprocWorkflowTemplateInstantiateOperator
-.. autoclass:: airflow.contrib.operators.datastore_export_operator.DatastoreExportOperator
-.. autoclass:: airflow.contrib.operators.datastore_import_operator.DatastoreImportOperator
-.. autoclass:: airflow.contrib.operators.discord_webhook_operator.DiscordWebhookOperator
-.. autoclass:: airflow.contrib.operators.druid_operator.DruidOperator
-.. autoclass:: airflow.contrib.operators.ecs_operator.ECSOperator
-.. autoclass:: airflow.contrib.operators.emr_add_steps_operator.EmrAddStepsOperator
-.. autoclass:: airflow.contrib.operators.emr_create_job_flow_operator.EmrCreateJobFlowOperator
-.. autoclass:: airflow.contrib.operators.emr_terminate_job_flow_operator.EmrTerminateJobFlowOperator
-.. autoclass:: airflow.contrib.operators.file_to_gcs.FileToGoogleCloudStorageOperator
-.. autoclass:: airflow.contrib.operators.file_to_wasb.FileToWasbOperator
-.. autoclass:: airflow.contrib.operators.gcp_container_operator.GKEClusterCreateOperator
-.. autoclass:: airflow.contrib.operators.gcp_container_operator.GKEClusterDeleteOperator
-.. autoclass:: airflow.contrib.operators.gcp_container_operator.GKEPodOperator
-.. autoclass:: airflow.contrib.operators.gcp_compute_operator.GceBaseOperator
-.. autoclass:: airflow.contrib.operators.gcp_compute_operator.GceInstanceGroupManagerUpdateTemplateOperator
-.. autoclass:: airflow.contrib.operators.gcp_compute_operator.GceInstanceStartOperator
-.. autoclass:: airflow.contrib.operators.gcp_compute_operator.GceInstanceStopOperator
-.. autoclass:: airflow.contrib.operators.gcp_compute_operator.GceInstanceTemplateCopyOperator
-.. autoclass:: airflow.contrib.operators.gcp_compute_operator.GceSetMachineTypeOperator
-.. autoclass:: airflow.contrib.operators.gcp_function_operator.GcfFunctionDeleteOperator
-.. autoclass:: airflow.contrib.operators.gcp_function_operator.GcfFunctionDeployOperator
-.. autoclass:: airflow.contrib.operators.gcp_transfer_operator.GcpTransferServiceJobCreateOperator
-.. autoclass:: airflow.contrib.operators.gcp_transfer_operator.GcpTransferServiceJobDeleteOperator
-.. autoclass:: airflow.contrib.operators.gcp_transfer_operator.GcpTransferServiceJobUpdateOperator
-.. autoclass:: airflow.contrib.operators.gcp_transfer_operator.GcpTransferServiceOperationCancelOperator
-.. autoclass:: airflow.contrib.operators.gcp_transfer_operator.GcpTransferServiceOperationGetOperator
-.. autoclass:: airflow.contrib.operators.gcp_transfer_operator.GcpTransferServiceOperationPauseOperator
-.. autoclass:: airflow.contrib.operators.gcp_transfer_operator.GcpTransferServiceOperationResumeOperator
-.. autoclass:: airflow.contrib.operators.gcp_transfer_operator.GcpTransferServiceOperationsListOperator
-.. autoclass:: airflow.contrib.operators.gcs_acl_operator.GoogleCloudStorageBucketCreateAclEntryOperator
-.. autoclass:: airflow.contrib.operators.gcs_operator.GoogleCloudStorageCreateBucketOperator
-.. autoclass:: airflow.contrib.operators.gcs_download_operator.GoogleCloudStorageDownloadOperator
-.. autoclass:: airflow.contrib.operators.gcs_list_operator.GoogleCloudStorageListOperator
-.. autoclass:: airflow.contrib.operators.gcs_acl_operator.GoogleCloudStorageObjectCreateAclEntryOperator
-.. autoclass:: airflow.contrib.operators.gcs_to_bq.GoogleCloudStorageToBigQueryOperator
-.. autoclass:: airflow.contrib.operators.gcs_to_gcs.GoogleCloudStorageToGoogleCloudStorageOperator
-.. autoclass:: airflow.contrib.operators.gcp_transfer_operator.GoogleCloudStorageToGoogleCloudStorageTransferOperator
-.. autoclass:: airflow.contrib.operators.gcs_to_s3.GoogleCloudStorageToS3Operator
-.. autoclass:: airflow.contrib.operators.hipchat_operator.HipChatAPIOperator
-.. autoclass:: airflow.contrib.operators.hipchat_operator.HipChatAPISendRoomNotificationOperator
-.. autoclass:: airflow.contrib.operators.hive_to_dynamodb.HiveToDynamoDBTransferOperator
-.. autoclass:: airflow.contrib.operators.imap_attachment_to_s3_operator.ImapAttachmentToS3Operator
-.. autoclass:: airflow.contrib.operators.jenkins_job_trigger_operator.JenkinsJobTriggerOperator
-.. autoclass:: airflow.contrib.operators.jira_operator.JiraOperator
-.. autoclass:: airflow.contrib.operators.kubernetes_pod_operator.KubernetesPodOperator
-.. autoclass:: airflow.contrib.operators.mlengine_operator.MLEngineBatchPredictionOperator
-.. autoclass:: airflow.contrib.operators.mlengine_operator.MLEngineModelOperator
-.. autoclass:: airflow.contrib.operators.mlengine_operator.MLEngineTrainingOperator
-.. autoclass:: airflow.contrib.operators.mlengine_operator.MLEngineVersionOperator
-.. autoclass:: airflow.contrib.operators.mongo_to_s3.MongoToS3Operator
-.. autoclass:: airflow.contrib.operators.mysql_to_gcs.MySqlToGoogleCloudStorageOperator
-.. autoclass:: airflow.contrib.operators.oracle_to_azure_data_lake_transfer.OracleToAzureDataLakeTransfer
-.. autoclass:: airflow.contrib.operators.oracle_to_oracle_transfer.OracleToOracleTransfer
-.. autoclass:: airflow.contrib.operators.postgres_to_gcs_operator.PostgresToGoogleCloudStorageOperator
-.. autoclass:: airflow.contrib.operators.pubsub_operator.PubSubPublishOperator
-.. autoclass:: airflow.contrib.operators.pubsub_operator.PubSubSubscriptionCreateOperator
-.. autoclass:: airflow.contrib.operators.pubsub_operator.PubSubSubscriptionDeleteOperator
-.. autoclass:: airflow.contrib.operators.pubsub_operator.PubSubTopicCreateOperator
-.. autoclass:: airflow.contrib.operators.pubsub_operator.PubSubTopicDeleteOperator
-.. autoclass:: airflow.contrib.operators.qubole_check_operator.QuboleCheckOperator
-.. autoclass:: airflow.contrib.operators.qubole_operator.QuboleOperator
-.. autoclass:: airflow.contrib.operators.qubole_check_operator.QuboleValueCheckOperator
-.. autoclass:: airflow.contrib.operators.s3_copy_object_operator.S3CopyObjectOperator
-.. autoclass:: airflow.contrib.operators.s3_delete_objects_operator.S3DeleteObjectsOperator
-.. autoclass:: airflow.contrib.operators.s3_list_operator.S3ListOperator
-.. autoclass:: airflow.contrib.operators.s3_to_gcs_operator.S3ToGoogleCloudStorageOperator
-.. autoclass:: airflow.contrib.operators.gcp_transfer_operator.S3ToGoogleCloudStorageTransferOperator
-.. autoclass:: airflow.contrib.operators.s3_to_sftp_operator.S3ToSFTPOperator
-.. autoclass:: airflow.contrib.operators.sftp_operator.SFTPOperator
-.. autoclass:: airflow.contrib.operators.sftp_to_s3_operator.SFTPToS3Operator
-.. autoclass:: airflow.contrib.operators.ssh_operator.SSHOperator
-.. autoclass:: airflow.contrib.operators.sagemaker_base_operator.SageMakerBaseOperator
-.. autoclass:: airflow.contrib.operators.sagemaker_endpoint_config_operator.SageMakerEndpointConfigOperator
-.. autoclass:: airflow.contrib.operators.sagemaker_endpoint_operator.SageMakerEndpointOperator
-.. autoclass:: airflow.contrib.operators.sagemaker_model_operator.SageMakerModelOperator
-.. autoclass:: airflow.contrib.operators.sagemaker_training_operator.SageMakerTrainingOperator
-.. autoclass:: airflow.contrib.operators.sagemaker_transform_operator.SageMakerTransformOperator
-.. autoclass:: airflow.contrib.operators.sagemaker_tuning_operator.SageMakerTuningOperator
-.. autoclass:: airflow.contrib.operators.segment_track_event_operator.SegmentTrackEventOperator
-.. autoclass:: airflow.contrib.operators.slack_webhook_operator.SlackWebhookOperator
-.. autoclass:: airflow.contrib.operators.snowflake_operator.SnowflakeOperator
-.. autoclass:: airflow.contrib.operators.sns_publish_operator.SnsPublishOperator
-.. autoclass:: airflow.contrib.operators.spark_jdbc_operator.SparkJDBCOperator
-.. autoclass:: airflow.contrib.operators.spark_sql_operator.SparkSqlOperator
-.. autoclass:: airflow.contrib.operators.spark_submit_operator.SparkSubmitOperator
-.. autoclass:: airflow.contrib.operators.sqoop_operator.SqoopOperator
-.. autoclass:: airflow.contrib.operators.vertica_operator.VerticaOperator
-.. autoclass:: airflow.contrib.operators.vertica_to_hive.VerticaToHiveTransfer
-.. autoclass:: airflow.contrib.operators.vertica_to_mysql.VerticaToMySqlTransfer
-.. autoclass:: airflow.contrib.operators.wasb_delete_blob_operator.WasbDeleteBlobOperator
-.. autoclass:: airflow.contrib.operators.winrm_operator.WinRMOperator
-
-Sensors
-^^^^^^^
-
-.. autoclass:: airflow.contrib.sensors.aws_athena_sensor.AthenaSensor
-.. autoclass:: airflow.contrib.sensors.aws_glue_catalog_partition_sensor.AwsGlueCatalogPartitionSensor
-.. autoclass:: airflow.contrib.sensors.aws_redshift_cluster_sensor.AwsRedshiftClusterSensor
-.. autoclass:: airflow.contrib.sensors.azure_cosmos_sensor.AzureCosmosDocumentSensor
-.. autoclass:: airflow.contrib.sensors.bash_sensor.BashSensor
-.. autoclass:: airflow.contrib.sensors.bigquery_sensor.BigQueryTableSensor
-.. autoclass:: airflow.contrib.sensors.cassandra_record_sensor.CassandraRecordSensor
-.. autoclass:: airflow.contrib.sensors.cassandra_table_sensor.CassandraTableSensor
-.. autoclass:: airflow.contrib.sensors.celery_queue_sensor.CeleryQueueSensor
-.. autoclass:: airflow.contrib.sensors.datadog_sensor.DatadogSensor
-.. autoclass:: airflow.contrib.sensors.weekday_sensor.DayOfWeekSensor
-.. autoclass:: airflow.contrib.sensors.emr_base_sensor.EmrBaseSensor
-.. autoclass:: airflow.contrib.sensors.emr_job_flow_sensor.EmrJobFlowSensor
-.. autoclass:: airflow.contrib.sensors.emr_step_sensor.EmrStepSensor
-.. autoclass:: airflow.contrib.sensors.ftp_sensor.FTPSSensor
-.. autoclass:: airflow.contrib.sensors.ftp_sensor.FTPSensor
-.. autoclass:: airflow.contrib.sensors.file_sensor.FileSensor
-.. autoclass:: airflow.contrib.sensors.gcp_transfer_sensor.GCPTransferServiceWaitForJobStatusSensor
-.. autoclass:: airflow.contrib.sensors.gcs_sensor.GoogleCloudStorageObjectSensor
-.. autoclass:: airflow.contrib.sensors.gcs_sensor.GoogleCloudStorageObjectUpdatedSensor
-.. autoclass:: airflow.contrib.sensors.gcs_sensor.GoogleCloudStoragePrefixSensor
-.. autoclass:: airflow.contrib.sensors.hdfs_sensor.HdfsSensorFolder
-.. autoclass:: airflow.contrib.sensors.hdfs_sensor.HdfsSensorRegex
-.. autoclass:: airflow.contrib.sensors.imap_attachment_sensor.ImapAttachmentSensor
-.. autoclass:: airflow.contrib.sensors.jira_sensor.JiraSensor
-.. autoclass:: airflow.contrib.sensors.jira_sensor.JiraTicketSensor
-.. autoclass:: airflow.contrib.sensors.mongo_sensor.MongoSensor
-.. autoclass:: airflow.contrib.sensors.pubsub_sensor.PubSubPullSensor
-.. autoclass:: airflow.contrib.sensors.python_sensor.PythonSensor
-.. autoclass:: airflow.contrib.sensors.qubole_sensor.QuboleFileSensor
-.. autoclass:: airflow.contrib.sensors.qubole_sensor.QubolePartitionSensor
-.. autoclass:: airflow.contrib.sensors.qubole_sensor.QuboleSensor
-.. autoclass:: airflow.contrib.sensors.redis_key_sensor.RedisKeySensor
-.. autoclass:: airflow.contrib.sensors.sftp_sensor.SFTPSensor
-.. autoclass:: airflow.contrib.sensors.sagemaker_base_sensor.SageMakerBaseSensor
-.. autoclass:: airflow.contrib.sensors.sagemaker_endpoint_sensor.SageMakerEndpointSensor
-.. autoclass:: airflow.contrib.sensors.sagemaker_training_sensor.SageMakerTrainingSensor
-.. autoclass:: airflow.contrib.sensors.sagemaker_transform_sensor.SageMakerTransformSensor
-.. autoclass:: airflow.contrib.sensors.sagemaker_tuning_sensor.SageMakerTuningSensor
-.. autoclass:: airflow.contrib.sensors.wasb_sensor.WasbBlobSensor
-.. autoclass:: airflow.contrib.sensors.wasb_sensor.WasbPrefixSensor
-
-.. _macros:
-
-Macros
-------
-Variables and macros can be used in templates (see the :ref:`jinja-templating` section)
-
-The following come for free out of the box with Airflow.
-Additional custom macros can be added globally through :ref:`plugins`, or at a DAG level through the ``DAG.user_defined_macros`` argument.
-
-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 ``2018-01-08`` and ``schedule_interval`` is ``@weekly``,
- ``{{ prev_ds }}`` will be ``2018-01-01``
-``{{ prev_ds_nodash }}`` the previous execution date as ``YYYYMMDD`` if exists, else ``None``
-``{{ next_ds }}`` the next execution date as ``YYYY-MM-DD``
- if ``{{ ds }}`` is ``2018-01-01`` and ``schedule_interval`` is ``@weekly``,
- ``{{ next_ds }}`` will be ``2018-01-08``
-``{{ next_ds_nodash }}`` the next execution date as ``YYYYMMDD`` if exists, else ``None``
-``{{ yesterday_ds }}`` the day before the execution date as ``YYYY-MM-DD``
-``{{ yesterday_ds_nodash }}`` the day before the execution date as ``YYYYMMDD``
-``{{ tomorrow_ds }}`` the day after the execution date as ``YYYY-MM-DD``
-``{{ tomorrow_ds_nodash }}`` the day after the execution date as ``YYYYMMDD``
-``{{ ts }}`` same as ``execution_date.isoformat()``. Example: ``2018-01-01T00:00:00+00:00``
-``{{ ts_nodash }}`` same as ``ts`` without ``-``, ``:`` and TimeZone info. Example: ``20180101T000000``
-``{{ ts_nodash_with_tz }}`` same as ``ts`` without ``-`` and ``:``. Example: ``20180101T000000+0000``
-``{{ execution_date }}`` the execution_date (pendulum.Pendulum)
-``{{ prev_execution_date }}`` the previous execution date (if available) (pendulum.Pendulum)
-``{{ next_execution_date }}`` the next execution date (pendulum.Pendulum)
-``{{ 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:
-
-.. automodule:: airflow.macros
- :show-inheritance:
- :members:
-
-.. autofunction:: airflow.macros.hive.closest_ds_partition
-.. autofunction:: airflow.macros.hive.max_partition
-
-.. _models_ref:
-
-Models
-------
-
-Models are built on top of the SQLAlchemy ORM Base class, and instances are
-persisted in the database.
-
-
-.. automodule:: airflow.models
- :show-inheritance:
- :members: DAG, TaskInstance, DagBag, Connection, Variable, Pool, KubeResourceVersion, KubeWorkerIdentifier
-
-Hooks
------
-
-Hooks are interfaces to external platforms and databases, implementing a common
-interface when possible and acting as building blocks for operators.
-
-.. autoclass:: airflow.hooks.base_hook.BaseHook
-.. autoclass:: airflow.hooks.dbapi_hook.DbApiHook
-.. autoclass:: airflow.hooks.docker_hook.DockerHook
-.. autoclass:: airflow.hooks.druid_hook.DruidDbApiHook
-.. autoclass:: airflow.hooks.druid_hook.DruidHook
-.. autoclass:: airflow.hooks.hdfs_hook.HDFSHook
-.. autoclass:: airflow.hooks.hive_hooks.HiveCliHook
-.. autoclass:: airflow.hooks.hive_hooks.HiveMetastoreHook
-.. autoclass:: airflow.hooks.hive_hooks.HiveServer2Hook
-.. autoclass:: airflow.hooks.http_hook.HttpHook
-.. autoclass:: airflow.hooks.jdbc_hook.JdbcHook
-.. autoclass:: airflow.hooks.mssql_hook.MsSqlHook
-.. autoclass:: airflow.hooks.mysql_hook.MySqlHook
-.. autoclass:: airflow.hooks.oracle_hook.OracleHook
-.. autoclass:: airflow.hooks.pig_hook.PigCliHook
-.. autoclass:: airflow.hooks.postgres_hook.PostgresHook
-.. autoclass:: airflow.hooks.presto_hook.PrestoHook
-.. autoclass:: airflow.hooks.S3_hook.S3Hook
-.. autoclass:: airflow.hooks.samba_hook.SambaHook
-.. autoclass:: airflow.hooks.slack_hook.SlackHook
-.. autoclass:: airflow.hooks.sqlite_hook.SqliteHook
-.. autoclass:: airflow.hooks.webhdfs_hook.WebHDFSHook
-.. autoclass:: airflow.hooks.zendesk_hook.ZendeskHook
-
-Community contributed hooks
-'''''''''''''''''''''''''''
-.. Alphabetize this list
-.. autoclass:: airflow.contrib.hooks.aws_athena_hook.AWSAthenaHook
-.. autoclass:: airflow.contrib.hooks.aws_dynamodb_hook.AwsDynamoDBHook
-.. autoclass:: airflow.contrib.hooks.aws_firehose_hook.AwsFirehoseHook
-.. autoclass:: airflow.contrib.hooks.aws_glue_catalog_hook.AwsGlueCatalogHook
-.. autoclass:: airflow.contrib.hooks.aws_hook.AwsHook
-.. autoclass:: airflow.contrib.hooks.aws_lambda_hook.AwsLambdaHook
-.. autoclass:: airflow.contrib.hooks.aws_sns_hook.AwsSnsHook
-.. autoclass:: airflow.contrib.hooks.azure_container_instance_hook.AzureContainerInstanceHook
-.. autoclass:: airflow.contrib.hooks.azure_container_registry_hook.AzureContainerRegistryHook
-.. autoclass:: airflow.contrib.hooks.azure_container_volume_hook.AzureContainerVolumeHook
-.. autoclass:: airflow.contrib.hooks.azure_cosmos_hook.AzureCosmosDBHook
-.. autoclass:: airflow.contrib.hooks.azure_data_lake_hook.AzureDataLakeHook
-.. autoclass:: airflow.contrib.hooks.azure_fileshare_hook.AzureFileShareHook
-.. autoclass:: airflow.contrib.hooks.bigquery_hook.BigQueryHook
-.. autoclass:: airflow.contrib.hooks.gcp_bigtable_hook.BigtableHook
-.. autoclass:: airflow.contrib.hooks.cassandra_hook.CassandraHook
-.. autoclass:: airflow.contrib.hooks.gcp_spanner_hook.CloudSpannerHook
-.. autoclass:: airflow.contrib.hooks.gcp_sql_hook.CloudSqlDatabaseHook
-.. autoclass:: airflow.contrib.hooks.gcp_sql_hook.CloudSqlHook
-.. autoclass:: airflow.contrib.hooks.gcp_translate_hook.CloudTranslateHook
-.. autoclass:: airflow.contrib.hooks.gcp_vision_hook.CloudVisionHook
-.. autoclass:: airflow.contrib.hooks.cloudant_hook.CloudantHook
-.. autoclass:: airflow.contrib.hooks.gcp_dataflow_hook.DataFlowHook
-.. autoclass:: airflow.contrib.hooks.gcp_dataproc_hook.DataProcHook
-.. autoclass:: airflow.contrib.hooks.databricks_hook.DatabricksHook
-.. autoclass:: airflow.contrib.hooks.datadog_hook.DatadogHook
-.. autoclass:: airflow.contrib.hooks.datastore_hook.DatastoreHook
-.. autoclass:: airflow.contrib.hooks.discord_webhook_hook.DiscordWebhookHook
-.. autoclass:: airflow.contrib.hooks.emr_hook.EmrHook
-.. autoclass:: airflow.contrib.hooks.fs_hook.FSHook
-.. autoclass:: airflow.contrib.hooks.ftp_hook.FTPHook
-.. autoclass:: airflow.contrib.hooks.ftp_hook.FTPSHook
-.. autoclass:: airflow.contrib.hooks.gcp_transfer_hook.GCPTransferServiceHook
-.. autoclass:: airflow.contrib.hooks.gcp_container_hook.GKEClusterHook
-.. autoclass:: airflow.contrib.hooks.gcp_compute_hook.GceHook
-.. autoclass:: airflow.contrib.hooks.gcp_function_hook.GcfHook
-.. autoclass:: airflow.contrib.hooks.gcp_api_base_hook.GoogleCloudBaseHook
-.. autoclass:: airflow.contrib.hooks.gcp_kms_hook.GoogleCloudKMSHook
-.. autoclass:: airflow.contrib.hooks.gcs_hook.GoogleCloudStorageHook
-.. autoclass:: airflow.contrib.hooks.grpc_hook.GrpcHook
-.. autoclass:: airflow.contrib.hooks.imap_hook.ImapHook
-.. autoclass:: airflow.contrib.hooks.jenkins_hook.JenkinsHook
-.. autoclass:: airflow.contrib.hooks.jira_hook.JiraHook
-.. autoclass:: airflow.contrib.hooks.gcp_mlengine_hook.MLEngineHook
-.. autoclass:: airflow.contrib.hooks.mongo_hook.MongoHook
-.. autoclass:: airflow.contrib.hooks.openfaas_hook.OpenFaasHook
-.. autoclass:: airflow.contrib.hooks.pinot_hook.PinotDbApiHook
-.. autoclass:: airflow.contrib.hooks.gcp_pubsub_hook.PubSubHook
-.. autoclass:: airflow.contrib.hooks.qubole_check_hook.QuboleCheckHook
-.. autoclass:: airflow.contrib.hooks.qubole_hook.QuboleHook
-.. autoclass:: airflow.contrib.hooks.redis_hook.RedisHook
-.. autoclass:: airflow.contrib.hooks.redshift_hook.RedshiftHook
-.. autoclass:: airflow.contrib.hooks.sftp_hook.SFTPHook
-.. autoclass:: airflow.contrib.hooks.ssh_hook.SSHHook
-.. autoclass:: airflow.contrib.hooks.sagemaker_hook.SageMakerHook
-.. autoclass:: airflow.contrib.hooks.segment_hook.SegmentHook
-.. autoclass:: airflow.contrib.hooks.slack_webhook_hook.SlackWebhookHook
-.. autoclass:: airflow.contrib.hooks.snowflake_hook.SnowflakeHook
-.. autoclass:: airflow.contrib.hooks.spark_jdbc_hook.SparkJDBCHook
-.. autoclass:: airflow.contrib.hooks.spark_sql_hook.SparkSqlHook
-.. autoclass:: airflow.contrib.hooks.spark_submit_hook.SparkSubmitHook
-.. autoclass:: airflow.contrib.hooks.sqoop_hook.SqoopHook
-.. autoclass:: airflow.contrib.hooks.vertica_hook.VerticaHook
-.. autoclass:: airflow.contrib.hooks.wasb_hook.WasbHook
-.. autoclass:: airflow.contrib.hooks.winrm_hook.WinRMHook
-
-Executors
----------
-Executors are the mechanism by which task instances get run.
-
-.. autoclass:: airflow.executors.celery_executor.CeleryExecutor
-.. autoclass:: airflow.executors.local_executor.LocalExecutor
-.. autoclass:: airflow.executors.sequential_executor.SequentialExecutor
-.. autoclass:: airflow.executors.dask_executor.DaskExecutor
-
-
-Community-contributed executors
-'''''''''''''''''''''''''''''''
-
-.. autoclass:: airflow.contrib.executors.mesos_executor.MesosExecutor
-.. autoclass:: airflow.contrib.executors.kubernetes_executor.KubernetesExecutor
diff --git a/docs/concepts.rst b/docs/concepts.rst
index a9296164aadbf..94a3fec4f63e1 100644
--- a/docs/concepts.rst
+++ b/docs/concepts.rst
@@ -964,7 +964,7 @@ Jinja Templating
Airflow leverages the power of
`Jinja Templating `_ and this can be a
-powerful tool to use in combination with macros (see the :ref:`macros` section).
+powerful tool to use in combination with macros (see the :doc:`macros` section).
For example, say you want to pass the execution date as an environment variable
to a Bash script using the ``BashOperator``.
diff --git a/docs/conf.py b/docs/conf.py
index 3d4a80e6e5d95..fdab02f66142a 100644
--- a/docs/conf.py
+++ b/docs/conf.py
@@ -115,6 +115,7 @@
'sphinxarg.ext',
'sphinxcontrib.httpdomain',
'sphinx.ext.intersphinx',
+ 'autoapi.extension',
]
autodoc_default_flags = ['show-inheritance', 'members']
@@ -122,7 +123,7 @@
viewcode_import = True
# Add any paths that contain templates here, relative to this directory.
-templates_path = ['_templates']
+templates_path = ['templates']
# The suffix of source filenames.
source_suffix = '.rst'
@@ -160,7 +161,41 @@
# List of patterns, relative to source directory, that match files and
# directories to ignore when looking for source files.
-exclude_patterns = ['_build']
+exclude_patterns = [
+ '_api/airflow/_vendor',
+ '_api/airflow/api',
+ '_api/airflow/bin',
+ '_api/airflow/config_templates',
+ '_api/airflow/configuration',
+ '_api/airflow/contrib/auth',
+ '_api/airflow/contrib/example_dags',
+ '_api/airflow/contrib/index.rst',
+ '_api/airflow/contrib/kubernetes',
+ '_api/airflow/contrib/task_runner',
+ '_api/airflow/contrib/utils',
+ '_api/airflow/dag',
+ '_api/airflow/default_login',
+ '_api/airflow/example_dags',
+ '_api/airflow/exceptions',
+ '_api/airflow/index.rst',
+ '_api/airflow/jobs',
+ '_api/airflow/lineage',
+ '_api/airflow/logging_config',
+ '_api/airflow/macros',
+ '_api/airflow/migrations',
+ '_api/airflow/plugins_manager',
+ '_api/airflow/security',
+ '_api/airflow/settings',
+ '_api/airflow/stats',
+ '_api/airflow/task',
+ '_api/airflow/ti_deps',
+ '_api/airflow/utils',
+ '_api/airflow/version',
+ '_api/airflow/www',
+ '_api/main',
+ '_build',
+ 'autoapi_templates',
+]
# The reST default role (used for this markup: `text`) to use for all
# documents.
@@ -184,7 +219,7 @@
# modindex_common_prefix = []
# If true, keep warnings as "system message" paragraphs in the built documents.
-# keep_warnings = False
+keep_warnings = True
intersphinx_mapping = {
@@ -363,3 +398,30 @@
# If true, do not generate a @detailmenu in the "Top" node's menu.
# texinfo_no_detailmenu = False
+
+# sphinx-autoapi configuration
+# See:
+# https://sphinx-autoapi.readthedocs.io/en/latest/config.html
+
+# Paths (relative or absolute) to the source code that you wish to generate
+# your API documentation from.
+autoapi_dirs = [
+ os.path.abspath('../airflow'),
+]
+
+# A directory that has user-defined templates to override our default templates.
+autoapi_template_dir = 'autoapi_templates'
+
+# A list of patterns to ignore when finding files
+autoapi_ignore = [
+ os.path.abspath('../airflow/contrib/operators/s3_to_gcs_transfer_operator.py'),
+ os.path.abspath('../airflow/contrib/operators/gcs_to_gcs_transfer_operator.py'),
+ os.path.abspath('../airflow/contrib/operators/gcs_to_gcs_transfer_operator.py'),
+]
+# Keep the AutoAPI generated files on the filesystem after the run.
+# Useful for debugging.
+autoapi_keep_files = False
+
+# Relative path to output the AutoAPI files into. This can also be used to place the generated documentation
+# anywhere in your documentation hierarchy.
+autoapi_root = '_api'
diff --git a/docs/howto/connection/gcp.rst b/docs/howto/connection/gcp.rst
index 5b96864ecdfef..9d1966c6e98db 100644
--- a/docs/howto/connection/gcp.rst
+++ b/docs/howto/connection/gcp.rst
@@ -15,6 +15,8 @@
specific language governing permissions and limitations
under the License.
+.. _howto/connection:gcp:
+
Google Cloud Platform Connection
================================
diff --git a/docs/howto/operator/bash.rst b/docs/howto/operator/bash.rst
index 463056196ee49..9795268b58240 100644
--- a/docs/howto/operator/bash.rst
+++ b/docs/howto/operator/bash.rst
@@ -15,6 +15,8 @@
specific language governing permissions and limitations
under the License.
+.. _howto/operator:BashOperator:
+
BashOperator
============
diff --git a/docs/howto/operator/index.rst b/docs/howto/operator/index.rst
index 647ffcc354fc6..74197a2993c10 100644
--- a/docs/howto/operator/index.rst
+++ b/docs/howto/operator/index.rst
@@ -22,7 +22,7 @@ An operator represents a single, ideally idempotent, task. Operators
determine what actually executes when your DAG runs.
See the :ref:`Operators Concepts ` documentation and the
-:ref:`Operators API Reference ` for more
+:doc:`Operators API Reference <../../_api/index>` for more
information.
.. toctree::
diff --git a/docs/howto/operator/python.rst b/docs/howto/operator/python.rst
index f6aa48f0b8970..c3ef068dbaf0b 100644
--- a/docs/howto/operator/python.rst
+++ b/docs/howto/operator/python.rst
@@ -15,6 +15,8 @@
specific language governing permissions and limitations
under the License.
+.. _howto/operator:PythonOperator:
+
PythonOperator
==============
@@ -41,8 +43,8 @@ Templating
^^^^^^^^^^
When you set the ``provide_context`` argument to ``True``, Airflow passes in
-an additional set of keyword arguments: one for each of the :ref:`Jinja
-template variables ` and a ``templates_dict`` argument.
+an additional set of keyword arguments: one for each of the :doc:`Jinja
+template variables <../../macros>` and a ``templates_dict`` argument.
The ``templates_dict`` argument is templated, so each value in the dictionary
is evaluated as a :ref:`Jinja template `.
diff --git a/docs/index.rst b/docs/index.rst
index e39f16cdf9126..dc230d86fea2e 100644
--- a/docs/index.rst
+++ b/docs/index.rst
@@ -92,4 +92,5 @@ Content
lineage
changelog
faq
- code
+ macros
+ _api/index
diff --git a/docs/kubernetes.rst b/docs/kubernetes.rst
index 9e5162401e78d..debc4579ca730 100644
--- a/docs/kubernetes.rst
+++ b/docs/kubernetes.rst
@@ -140,11 +140,4 @@ Kubernetes Operator
)
-.. autoclass:: airflow.contrib.operators.kubernetes_pod_operator.KubernetesPodOperator
- :noindex:
-
-
-.. autoclass:: airflow.contrib.kubernetes.secret.Secret
- :noindex:
-
-
+See :class:`airflow.contrib.operators.kubernetes_pod_operator.KubernetesPodOperator`
diff --git a/docs/macros.rst b/docs/macros.rst
new file mode 100644
index 0000000000000..43807b0ee37a7
--- /dev/null
+++ b/docs/macros.rst
@@ -0,0 +1,118 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+ or more contributor license agreements. See the NOTICE file
+ distributed with this work for additional information
+ regarding copyright ownership. The ASF licenses this file
+ to you under the Apache License, Version 2.0 (the
+ "License"); you may not use this file except in compliance
+ with the License. You may obtain a copy of the License at
+
+.. http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+ software distributed under the License is distributed on an
+ "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ KIND, either express or implied. See the License for the
+ specific language governing permissions and limitations
+ under the License.
+
+Macros reference
+================
+
+Variables and macros can be used in templates (see the :ref:`jinja-templating` section)
+
+The following come for free out of the box with Airflow.
+Additional custom macros can be added globally through :ref:`plugins`, or at a DAG level through the ``DAG.user_defined_macros`` argument.
+
+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 ``2018-01-08`` and ``schedule_interval`` is ``@weekly``,
+ ``{{ prev_ds }}`` will be ``2018-01-01``
+``{{ prev_ds_nodash }}`` the previous execution date as ``YYYYMMDD`` if exists, else ``None``
+``{{ next_ds }}`` the next execution date as ``YYYY-MM-DD``
+ if ``{{ ds }}`` is ``2018-01-01`` and ``schedule_interval`` is ``@weekly``,
+ ``{{ next_ds }}`` will be ``2018-01-08``
+``{{ next_ds_nodash }}`` the next execution date as ``YYYYMMDD`` if exists, else ``None``
+``{{ yesterday_ds }}`` the day before the execution date as ``YYYY-MM-DD``
+``{{ yesterday_ds_nodash }}`` the day before the execution date as ``YYYYMMDD``
+``{{ tomorrow_ds }}`` the day after the execution date as ``YYYY-MM-DD``
+``{{ tomorrow_ds_nodash }}`` the day after the execution date as ``YYYYMMDD``
+``{{ ts }}`` same as ``execution_date.isoformat()``. Example: ``2018-01-01T00:00:00+00:00``
+``{{ ts_nodash }}`` same as ``ts`` without ``-``, ``:`` and TimeZone info. Example: ``20180101T000000``
+``{{ ts_nodash_with_tz }}`` same as ``ts`` without ``-`` and ``:``. Example: ``20180101T000000+0000``
+``{{ execution_date }}`` the execution_date (pendulum.Pendulum)
+``{{ prev_execution_date }}`` the previous execution date (if available) (pendulum.Pendulum)
+``{{ next_execution_date }}`` the next execution date (pendulum.Pendulum)
+``{{ 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 :class:`datetime.datetime`
+``macros.timedelta`` The standard lib's :class:`datetime.datetime`
+``macros.dateutil`` A reference to the ``dateutil`` package
+``macros.time`` The standard lib's :class:`datetime.time`
+``macros.uuid`` The standard lib's :mod:`uuid`
+``macros.random`` The standard lib's :mod:`random`
+================================= ==============================================
+
+
+Some airflow specific macros are also defined:
+
+.. automodule:: airflow.macros
+ :show-inheritance:
+ :members:
+
+.. autofunction:: airflow.macros.hive.closest_ds_partition
+.. autofunction:: airflow.macros.hive.max_partition
diff --git a/docs/_templates/layout.html b/docs/templates/layout.html
similarity index 100%
rename from docs/_templates/layout.html
rename to docs/templates/layout.html
diff --git a/docs/tutorial.rst b/docs/tutorial.rst
index f8ed11c3d7952..5789e243962c4 100644
--- a/docs/tutorial.rst
+++ b/docs/tutorial.rst
@@ -260,7 +260,7 @@ regarding custom filters have a look at the
`Jinja Documentation `_
For more information on the variables and macros that can be referenced
-in templates, make sure to read through the :ref:`macros` section
+in templates, make sure to read through the :doc:`macros`
Setting up Dependencies
-----------------------
diff --git a/setup.py b/setup.py
index b7ddcda3cc85a..e948d7bf83a7e 100644
--- a/setup.py
+++ b/setup.py
@@ -160,11 +160,12 @@ def write_version(filename=os.path.join(*['airflow',
databricks = ['requests>=2.20.0, <3']
datadog = ['datadog>=0.14.0']
doc = [
- 'sphinx>=1.2.3',
'sphinx-argparse>=0.1.13',
+ 'sphinx-autoapi>=0.7.1',
+ 'Sphinx-PyPI-upload>=0.2.1',
'sphinx-rtd-theme>=0.1.6',
+ 'sphinx>=1.2.3',
'sphinxcontrib-httpdomain>=1.7.0',
- 'Sphinx-PyPI-upload>=0.2.1'
]
docker = ['docker~=3.0']
druid = ['pydruid>=0.4.1']