You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by ma...@apache.org on 2016/06/05 05:24:07 UTC

[17/34] incubator-airflow-site git commit: Initial commit

http://git-wip-us.apache.org/repos/asf/incubator-airflow-site/blob/9e19165c/_sources/configuration.txt
----------------------------------------------------------------------
diff --git a/_sources/configuration.txt b/_sources/configuration.txt
new file mode 100644
index 0000000..3eed553
--- /dev/null
+++ b/_sources/configuration.txt
@@ -0,0 +1,230 @@
+Configuration
+-------------
+
+Setting up the sandbox in the :doc:`start` section was easy;
+building a production-grade environment requires a bit more work!
+
+Setting Configuration Options
+'''''''''''''''''''''''''''''
+
+The first time you run Airflow, it will create a file called ``airflow.cfg`` in
+your ``$AIRFLOW_HOME`` directory (``~/airflow`` by default). This file contains Airflow's configuration and you
+can edit it to change any of the settings. You can also set options with environment variables by using this format:
+``$AIRFLOW__{SECTION}__{KEY}`` (note the double underscores).
+
+For example, the
+metadata database connection string can either be set in ``airflow.cfg`` like this:
+
+.. code-block:: bash
+
+    [core]
+    sql_alchemy_conn = my_conn_string
+
+or by creating a corresponding environment variable:
+
+.. code-block:: bash
+
+    AIRFLOW__CORE__SQL_ALCHEMY_CONN=my_conn_string
+
+You can also derive the connection string at run time by appending ``_cmd`` to the key like this:
+
+.. code-block:: bash
+
+    [core]
+    sql_alchemy_conn_cmd = bash_command_to_run
+
+But only three such configuration elements namely sql_alchemy_conn, broker_url and celery_result_backend can be fetched as a command. The idea behind this is to not store passwords on boxes in plain text files. The order of precedence is as follows -
+
+1. environment variable
+2. configuration in airflow.cfg
+3. command in airflow.cfg
+4. default
+
+Setting up a Backend
+''''''''''''''''''''
+If you want to take a real test drive of Airflow, you should consider
+setting up a real database backend and switching to the LocalExecutor.
+
+As Airflow was built to interact with its metadata using the great SqlAlchemy
+library, you should be able to use any database backend supported as a
+SqlAlchemy backend. We recommend using **MySQL** or **Postgres**.
+
+.. note:: If you decide to use **Postgres**, we recommend using the ``psycopg2``
+   driver and specifying it in your SqlAlchemy connection string.
+   Also note that since SqlAlchemy does not expose a way to target a
+   specific schema in the Postgres connection URI, you may
+   want to set a default schema for your role with a
+   command similar to ``ALTER ROLE username SET search_path = airflow, foobar;``
+
+Once you've setup your database to host Airflow, you'll need to alter the
+SqlAlchemy connection string located in your configuration file
+``$AIRFLOW_HOME/airflow.cfg``. You should then also change the "executor"
+setting to use "LocalExecutor", an executor that can parallelize task
+instances locally.
+
+.. code-block:: bash
+
+    # initialize the database
+    airflow initdb
+
+Connections
+'''''''''''
+Airflow needs to know how to connect to your environment. Information
+such as hostname, port, login and passwords to other systems and services is
+handled in the ``Admin->Connection`` section of the UI. The pipeline code you
+will author will reference the 'conn_id' of the Connection objects.
+
+.. image:: img/connections.png
+
+By default, Airflow will save the passwords for the connection in plain text
+within the metadata database. The ``crypto`` package is highly recommended
+during installation. The ``crypto`` package does require that your operating
+system have libffi-dev installed.
+
+Connections in Airflow pipelines can be created using environment variables.
+The environment variable needs to have a prefix of ``AIRFLOW_CONN_`` for
+Airflow with the value in a URI format to use the connection properly. Please
+see the :doc:`concepts` documentation for more information on environment
+variables and connections.
+
+Scaling Out with Celery
+'''''''''''''''''''''''
+``CeleryExecutor`` is one of the ways you can scale out the number of workers. For this
+to work, you need to setup a Celery backend (**RabbitMQ**, **Redis**, ...) and
+change your ``airflow.cfg`` to point the executor parameter to
+``CeleryExecutor`` and provide the related Celery settings.
+
+For more information about setting up a Celery broker, refer to the
+exhaustive `Celery documentation on the topic <http://docs.celeryproject.org/en/latest/getting-started/brokers/index.html>`_.
+
+Here are a few imperative requirements for your workers:
+
+- ``airflow`` needs to be installed, and the CLI needs to be in the path
+- Airflow configuration settings should be homogeneous across the cluster
+- Operators that are executed on the worker need to have their dependencies
+  met in that context. For example, if you use the ``HiveOperator``,
+  the hive CLI needs to be installed on that box, or if you use the
+  ``MySqlOperator``, the required Python library needs to be available in
+  the ``PYTHONPATH`` somehow
+- The worker needs to have access to its ``DAGS_FOLDER``, and you need to
+  synchronize the filesystems by your own means. A common setup would be to
+  store your DAGS_FOLDER in a Git repository and sync it across machines using
+  Chef, Puppet, Ansible, or whatever you use to configure machines in your
+  environment. If all your boxes have a common mount point, having your
+  pipelines files shared there should work as well
+
+
+To kick off a worker, you need to setup Airflow and kick off the worker
+subcommand
+
+.. code-block:: bash
+
+    airflow worker
+
+Your worker should start picking up tasks as soon as they get fired in
+its direction.
+
+Note that you can also run "Celery Flower", a web UI built on top of Celery,
+to monitor your workers. You can use the shortcut command ``airflow flower``
+to start a Flower web server.
+
+
+Logs
+''''
+Users can specify a logs folder in ``airflow.cfg``. By default, it is in
+the ``AIRFLOW_HOME`` directory.
+
+In addition, users can supply a remote location for storing logs and log backups
+in cloud storage. At this time, Amazon S3 and Google Cloud Storage are supported.
+To enable this feature, ``airflow.cfg`` must be configured as in this example:
+
+.. code-block:: bash
+
+    [core]
+    # Airflow can store logs remotely in AWS S3 or Google Cloud Storage. Users
+    # must supply a remote location URL (starting with either 's3://...' or
+    # 'gs://...') and an Airflow connection id that provides access to the storage
+    # location.
+    remote_base_log_folder = s3://my-bucket/path/to/logs
+    remote_log_conn_id = MyS3Conn
+    # Use server-side encryption for logs stored in S3
+    encrypt_s3_logs = False
+
+Remote logging uses an existing Airflow connection to read/write logs. If you don't
+have a connection properly setup, this will fail. In the above example, Airflow will
+try to use ``S3Hook('MyS3Conn')``.
+
+In the Airflow Web UI, local logs take precedance over remote logs. If local logs
+can not be found or accessed, the remote logs will be displayed. Note that logs
+are only sent to remote storage once a task completes (including failure). In other
+words, remote logs for running tasks are unavailable.
+
+Scaling Out on Mesos (community contributed)
+''''''''''''''''''''''''''''''''''''''''''''
+``MesosExecutor`` allows you to schedule airflow tasks on a Mesos cluster.
+For this to work, you need a running mesos cluster and you must perform the following
+steps -
+
+1. Install airflow on a machine where web server and scheduler will run,
+   let's refer to this as the "Airflow server".
+2. On the Airflow server, install mesos python eggs from `mesos downloads <http://open.mesosphere.com/downloads/mesos/>`_.
+3. On the Airflow server, use a database (such as mysql) which can be accessed from mesos
+   slave machines and add configuration in ``airflow.cfg``.
+4. Change your ``airflow.cfg`` to point executor parameter to
+   `MesosExecutor` and provide related Mesos settings.
+5. On all mesos slaves, install airflow. Copy the ``airflow.cfg`` from
+   Airflow server (so that it uses same sql alchemy connection).
+6. On all mesos slaves, run the following for serving logs:
+
+.. code-block:: bash
+
+    airflow serve_logs
+
+7. On Airflow server, to start processing/scheduling DAGs on mesos, run:
+
+.. code-block:: bash
+
+    airflow scheduler -p
+
+Note: We need -p parameter to pickle the DAGs.
+
+You can now see the airflow framework and corresponding tasks in mesos UI.
+The logs for airflow tasks can be seen in airflow UI as usual.
+
+For more information about mesos, refer to `mesos documentation <http://mesos.apache.org/documentation/latest/>`_.
+For any queries/bugs on `MesosExecutor`, please contact `@kapil-malik <https://github.com/kapil-malik>`_.
+
+Integration with systemd
+''''''''''''''''''''''''
+Airflow can integrate with systemd based systems. This makes watching your
+daemons easy as systemd can take care of restarting a daemon on failure.
+In the ``scripts/systemd`` directory you can find unit files that
+have been tested on Redhat based systems. You can copy those to
+``/usr/lib/systemd/system``. It is assumed that Airflow will run under
+``airflow:airflow``. If not (or if you are running on a non Redhat
+based system) you probably need to adjust the unit files.
+
+Environment configuration is picked up from ``/etc/sysconfig/airflow``.
+An example file is supplied. Make sure to specify the ``SCHEDULER_RUNS``
+variable in this file when you run the scheduler. You
+can also define here, for example, ``AIRFLOW_HOME`` or ``AIRFLOW_CONFIG``.
+
+Integration with upstart
+''''''''''''''''''''''''
+Airflow can integrate with upstart based systems. Upstart automatically starts all airflow services for which you
+have a corresponding ``*.conf`` file in ``/etc/init`` upon system boot. On failure, upstart automatically restarts
+the process (until it reaches re-spawn limit set in a ``*.conf`` file).
+
+You can find sample upstart job files in the ``scripts/upstart`` directory. These files have been tested on
+Ubuntu 14.04 LTS. You may have to adjust ``start on`` and ``stop on`` stanzas to make it work on other upstart
+systems. Some of the possible options are listed in ``scripts/upstart/README``.
+
+Modify ``*.conf`` files as needed and copy to ``/etc/init`` directory. It is assumed that airflow will run
+under ``airflow:airflow``. Change ``setuid`` and ``setgid`` in ``*.conf`` files if you use other user/group
+
+You can use ``initctl`` to manually start, stop, view status of the airflow process that has been
+integrated with upstart
+
+.. code-block:: bash
+
+    initctl airflow-webserver status

http://git-wip-us.apache.org/repos/asf/incubator-airflow-site/blob/9e19165c/_sources/faq.txt
----------------------------------------------------------------------
diff --git a/_sources/faq.txt b/_sources/faq.txt
new file mode 100644
index 0000000..21623fc
--- /dev/null
+++ b/_sources/faq.txt
@@ -0,0 +1,100 @@
+FAQ
+========
+
+**Why isn't my task getting scheduled?**
+
+There are very many reasons why your task might not be getting scheduled.
+Here are some of the common causes:
+
+- Does your script "compile", can the Airflow engine parse it and find your
+  DAG object. To test this, you can run ``airflow list_dags`` and
+  confirm that your DAG shows up in the list. You can also run
+  ``airflow list_tasks foo_dag_id --tree`` and confirm that your task
+  shows up in the list as expected. If you use the CeleryExecutor, you
+  may way to confirm that this works both where the scheduler runs as well
+  as where the worker runs.
+
+- Is your ``start_date`` set properly? The Airflow scheduler triggers the
+  task soon after the ``start_date + scheduler_interval`` is passed.
+
+- Is your ``start_date`` beyond where you can see it in the UI? If you
+  set your it to some time say 3 months ago, you won't be able to see
+  it in the main view in the UI, but you should be able to see it in the
+  ``Menu -> Browse ->Task Instances``.
+
+- Are the dependencies for the task met. The task instances directly
+  upstream from the task need to be in a ``success`` state. Also,
+  if you have set ``depends_on_past=True``, the previous task instance
+  needs to have succeeded (except if it is the first run for that task).
+  Also, if ``wait_for_downstream=True``, make sure you understand
+  what it means.
+  You can view how these properties are set from the ``Task Details``
+  page for your task.
+
+- Are the DagRuns you need created and active? A DagRun represents a specific
+  execution of an entire DAG and has a state (running, success, failed, ...).
+  The scheduler creates new DagRun as it moves forward, but never goes back
+  in time to create new ones. The scheduler only evaluates ``running`` DagRuns
+  to see what task instances it can trigger. Note that clearing tasks
+  instances (from the UI or CLI) does set the state of a DagRun back to
+  running. You can bulk view the list of DagRuns and alter states by clicking
+  on the schedule tag for a DAG.
+
+- Is the ``concurrency`` parameter of your DAG reached? ``concurency`` defines
+  how many ``running`` task instances a DAG is allowed to have, beyond which
+  point things get queued.
+
+- Is the ``max_active_runs`` parameter of your DAG reached? ``max_active_runs`` defines
+  how many ``running`` concurrent instances of a DAG there are allowed to be.
+
+You may also want to read the Scheduler section of the docs and make
+sure you fully understand how it proceeds.
+
+
+**How do I trigger tasks based on another task's failure?**
+
+Check out the ``Trigger Rule`` section in the Concepts section of the
+documentation
+
+**Why are connection passwords still not encrypted in the metadata db after I installed airflow[crypto]**?
+
+- Verify that the ``fernet_key`` defined in ``$AIRFLOW_HOME/airflow.cfg`` is a valid Fernet key. It must be a base64-encoded 32-byte key. You need to restart the webserver after you update the key
+- For existing connections (the ones that you had defined before installing ``airflow[crypto]`` and creating a Fernet key), you need to open each connection in the connection admin UI, re-type the password, and save it
+
+**What's the deal with ``start_date``?**
+
+``start_date`` is partly legacy from the pre-DagRun era, but it is still
+relevant in many ways. When creating a new DAG, you probably want to set
+a global ``start_date`` for your tasks using ``default_args``. The first
+DagRun to be created will be based on the ``min(start_date)`` for all your
+task. From that point on, the scheduler creates new DagRuns based on
+your ``schedule_interval`` and the corresponding task instances run as your
+dependencies are met. When introducing new tasks to your DAG, you need to
+pay special attention to ``start_date``, and may want to reactivate
+inactive DagRuns to get the new task to get onboarded properly.
+
+We recommend against using dynamic values as ``start_date``, especially
+``datetime.now()`` as it can be quite confusing. The task is triggered
+once the period closes, and in theory an ``@hourly`` DAG would never get to
+an hour after now as ``now()`` moves along.
+
+We also recommend using rounded ``start_date`` in relation to your
+``schedule_interval``. This means an ``@hourly`` would be at ``00:00``
+minutes:seconds, a ``@daily`` job at midnight, a ``@monthly`` job on the
+first of the month. You can use any sensor or a ``TimeDeltaSensor`` to delay
+the execution of tasks within that period. While ``schedule_interval``
+does allow specifying a ``datetime.timedelta``
+object, we recommend using the macros or cron expressions instead, as
+it enforces this idea of rounded schedules.
+
+When using ``depends_on_past=True`` it's important to pay special attention
+to ``start_date`` as the past dependency is not enforced only on the specific
+schedule of the ``start_date`` specified for the task. It' also
+important to watch DagRun activity status in time when introducing
+new ``depends_on_past=True``, unless you are planning on running a backfill
+for the new task(s).
+
+Also important to note is that the tasks ``start_date``, in the context of a
+backfill CLI command, get overridden by the backfill's command ``start_date``.
+This allows for a backfill on tasks that have ``depends_on_past=True`` to
+actually start, if it wasn't the case, the backfill just wouldn't start.

http://git-wip-us.apache.org/repos/asf/incubator-airflow-site/blob/9e19165c/_sources/index.txt
----------------------------------------------------------------------
diff --git a/_sources/index.txt b/_sources/index.txt
new file mode 100644
index 0000000..70f9355
--- /dev/null
+++ b/_sources/index.txt
@@ -0,0 +1,75 @@
+
+.. image:: img/pin_large.png
+    :width: 100
+.. image:: img/incubator.jpg
+    :width: 150
+
+Apache Airflow (incubating) Documentation
+================================
+
+Airflow is a platform to programmatically author, schedule and monitor
+workflows.
+
+Use airflow to author workflows as directed acyclic graphs (DAGs) of tasks.
+The airflow scheduler executes your tasks on an array of workers while
+following the specified dependencies. Rich command line utilities make
+performing complex surgeries on DAGs a snap. The rich user interface
+makes it easy to visualize pipelines running in production,
+monitor progress, and troubleshoot issues when needed.
+
+When workflows are defined as code, they become more maintainable,
+versionable, testable, and collaborative.
+
+------------
+
+
+.. image:: img/airflow.gif
+
+------------
+
+Principles
+----------
+
+- **Dynamic**:  Airflow pipelines are configuration as code (Python), allowing for dynamic pipeline generation. This allows for writing code that instantiates pipelines dynamically.
+- **Extensible**:  Easily define your own operators, executors and extend the library so that it fits the level of abstraction that suits your environment.
+- **Elegant**:  Airflow pipelines are lean and explicit. Parameterizing your scripts is built into the core of Airflow using the powerful **Jinja** templating engine.
+- **Scalable**:  Airflow has a modular architecture and uses a message queue to orchestrate an arbitrary number of workers. Airflow is ready to scale to infinity.
+
+
+Beyond the Horizon
+------------------
+
+Airflow **is not** a data streaming solution. Tasks do not move data from
+one to the other (though tasks can exchange metadata!). Airflow is not
+in the `Spark Streaming <http://spark.apache.org/streaming/>`_
+or `Storm <https://storm.apache.org/>`_ space, it is more comparable to
+`Oozie <http://oozie.apache.org/>`_ or
+`Azkaban <http://data.linkedin.com/opensource/azkaban>`_.
+
+Workflows are expected to be mostly static or slowly changing. You can think
+of the structure of the tasks in your workflow as slightly more dynamic
+than a database structure would be. Airflow workflows are expected to look
+similar from a run to the next, this allows for clarity around
+unit of work and continuity.
+
+
+Content
+-------
+.. toctree::
+    :maxdepth: 4
+
+    project
+    license
+    start
+    installation
+    tutorial
+    configuration
+    ui
+    concepts
+    profiling
+    cli
+    scheduler
+    plugins
+    security
+    faq
+    code

http://git-wip-us.apache.org/repos/asf/incubator-airflow-site/blob/9e19165c/_sources/installation.txt
----------------------------------------------------------------------
diff --git a/_sources/installation.txt b/_sources/installation.txt
new file mode 100644
index 0000000..289f64f
--- /dev/null
+++ b/_sources/installation.txt
@@ -0,0 +1,90 @@
+Installation
+------------
+
+Getting Airflow
+'''''''''''''''
+
+The easiest way to install the latest stable version of Airflow is with ``pip``:
+
+.. code-block:: bash
+
+    pip install airflow
+
+You can also install Airflow with support for extra features like ``s3`` or ``postgres``:
+
+.. code-block:: bash
+
+    pip install "airflow[s3, postgres]"
+
+Extra Packages
+''''''''''''''
+
+The ``airflow`` PyPI basic package only installs what's needed to get started.
+Subpackages can be installed depending on what will be useful in your
+environment. For instance, if you don't need connectivity with Postgres,
+you won't have to go through the trouble of installing the ``postgres-devel``
+yum package, or whatever equivalent applies on the distribution you are using.
+
+Behind the scenes, Airflow does conditional imports of operators that require
+these extra dependencies.
+
+Here's the list of the subpackages and what they enable:
+
++---------------+-------------------------------------+-------------------------------------------------+
+| subpackage    |     install command                 | enables                                         |
++===============+=====================================+=================================================+
+|  all          | ``pip install airflow[all]``        | All Airflow features known to man               |
++---------------+-------------------------------------+-------------------------------------------------+
+|  all_dbs      | ``pip install airflow[all_dbs]``    | All databases integrations                      |
++---------------+-------------------------------------+-------------------------------------------------+
+|  async        | ``pip install airflow[async]``      | Async worker classes for gunicorn               |
++---------------+-------------------------------------+-------------------------------------------------+
+|  devel        | ``pip install airflow[devel]``      | Minimum dev tools requirements                  |
++---------------+-------------------------------------+-------------------------------------------------+
+|  devel_hadoop |``pip install airflow[devel_hadoop]``| Airflow + dependencies on the Hadoop stack      |
++---------------+-------------------------------------+-------------------------------------------------+
+|  celery       | ``pip install airflow[celery]``     | CeleryExecutor                                  |
++---------------+-------------------------------------+-------------------------------------------------+
+|  crypto       | ``pip install airflow[crypto]``     | Encrypt connection passwords in metadata db     |
++---------------+-------------------------------------+-------------------------------------------------+
+|  druid        | ``pip install airflow[druid]``      | Druid.io related operators & hooks              |
++---------------+-------------------------------------+-------------------------------------------------+
+|  gcp_api      | ``pip install airflow[gcp_api]``    | Google Cloud Platform hooks and operators       |
+|               |                                     | (using ``google-api-python-client``)            |
++---------------+-------------------------------------+-------------------------------------------------+
+|  jdbc         | ``pip install airflow[jdbc]``       | JDBC hooks and operators                        |
++---------------+-------------------------------------+-------------------------------------------------+
+|  hdfs         | ``pip install airflow[hdfs]``       | HDFS hooks and operators                        |
++---------------+-------------------------------------+-------------------------------------------------+
+|  hive         | ``pip install airflow[hive]``       | All Hive related operators                      |
++---------------+-------------------------------------+-------------------------------------------------+
+|  kerberos     | ``pip install airflow[kerberos]``   | kerberos integration for kerberized hadoop      |
++---------------+-------------------------------------+-------------------------------------------------+
+|  ldap         | ``pip install airflow[ldap]``       | ldap authentication for users                   |
++---------------+-------------------------------------+-------------------------------------------------+
+|  mssql        | ``pip install airflow[mssql]``      | Microsoft SQL operators and hook,               |
+|               |                                     | support as an Airflow backend                   |
++---------------+-------------------------------------+-------------------------------------------------+
+|  mysql        | ``pip install airflow[mysql]``      | MySQL operators and hook, support as            |
+|               |                                     | an Airflow backend                              |
++---------------+-------------------------------------+-------------------------------------------------+
+|  password     | ``pip install airflow[password]``   | Password Authentication for users               |
++---------------+-------------------------------------+-------------------------------------------------+
+|  postgres     | ``pip install airflow[postgres]``   | Postgres operators and hook, support            |
+|               |                                     | as an Airflow backend                           |
++---------------+-------------------------------------+-------------------------------------------------+
+|  qds          | ``pip install airflow[qds]``        | Enable QDS (qubole data services) support       |
++---------------+-------------------------------------+-------------------------------------------------+
+|  rabbitmq     | ``pip install airflow[rabbitmq]``   | Rabbitmq support as a Celery backend            |
++---------------+-------------------------------------+-------------------------------------------------+
+|  s3           | ``pip install airflow[s3]``         | ``S3KeySensor``, ``S3PrefixSensor``             |
++---------------+-------------------------------------+-------------------------------------------------+
+|  samba        | ``pip install airflow[samba]``      | ``Hive2SambaOperator``                          |
++---------------+-------------------------------------+-------------------------------------------------+
+|  slack        | ``pip install airflow[slack]``      | ``SlackAPIPostOperator``                        |
++---------------+-------------------------------------+-------------------------------------------------+
+|  vertica      | ``pip install airflow[vertica]``    | Vertica hook                                    |
+|               |                                     | support as an Airflow backend                   |
++---------------+-------------------------------------+-------------------------------------------------+
+|  cloudant     | ``pip install airflow[cloudant]``   | Cloudant hook                                   |
++---------------+-------------------------------------+-------------------------------------------------+

http://git-wip-us.apache.org/repos/asf/incubator-airflow-site/blob/9e19165c/_sources/license.txt
----------------------------------------------------------------------
diff --git a/_sources/license.txt b/_sources/license.txt
new file mode 100644
index 0000000..9da26c0
--- /dev/null
+++ b/_sources/license.txt
@@ -0,0 +1,211 @@
+License
+=======
+
+.. image:: img/apache.jpg
+    :width: 150
+
+::
+
+    Apache License
+    Version 2.0, January 2004
+    http://www.apache.org/licenses/
+
+    TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+    1. Definitions.
+
+       "License" shall mean the terms and conditions for use, reproduction,
+       and distribution as defined by Sections 1 through 9 of this document.
+
+       "Licensor" shall mean the copyright owner or entity authorized by
+       the copyright owner that is granting the License.
+
+       "Legal Entity" shall mean the union of the acting entity and all
+       other entities that control, are controlled by, or are under common
+       control with that entity. For the purposes of this definition,
+       "control" means (i) the power, direct or indirect, to cause the
+       direction or management of such entity, whether by contract or
+       otherwise, or (ii) ownership of fifty percent (50%) or more of the
+       outstanding shares, or (iii) beneficial ownership of such entity.
+
+       "You" (or "Your") shall mean an individual or Legal Entity
+       exercising permissions granted by this License.
+
+       "Source" form shall mean the preferred form for making modifications,
+       including but not limited to software source code, documentation
+       source, and configuration files.
+
+       "Object" form shall mean any form resulting from mechanical
+       transformation or translation of a Source form, including but
+       not limited to compiled object code, generated documentation,
+       and conversions to other media types.
+
+       "Work" shall mean the work of authorship, whether in Source or
+       Object form, made available under the License, as indicated by a
+       copyright notice that is included in or attached to the work
+       (an example is provided in the Appendix below).
+
+       "Derivative Works" shall mean any work, whether in Source or Object
+       form, that is based on (or derived from) the Work and for which the
+       editorial revisions, annotations, elaborations, or other modifications
+       represent, as a whole, an original work of authorship. For the purposes
+       of this License, Derivative Works shall not include works that remain
+       separable from, or merely link (or bind by name) to the interfaces of,
+       the Work and Derivative Works thereof.
+
+       "Contribution" shall mean any work of authorship, including
+       the original version of the Work and any modifications or additions
+       to that Work or Derivative Works thereof, that is intentionally
+       submitted to Licensor for inclusion in the Work by the copyright owner
+       or by an individual or Legal Entity authorized to submit on behalf of
+       the copyright owner. For the purposes of this definition, "submitted"
+       means any form of electronic, verbal, or written communication sent
+       to the Licensor or its representatives, including but not limited to
+       communication on electronic mailing lists, source code control systems,
+       and issue tracking systems that are managed by, or on behalf of, the
+       Licensor for the purpose of discussing and improving the Work, but
+       excluding communication that is conspicuously marked or otherwise
+       designated in writing by the copyright owner as "Not a Contribution."
+
+       "Contributor" shall mean Licensor and any individual or Legal Entity
+       on behalf of whom a Contribution has been received by Licensor and
+       subsequently incorporated within the Work.
+
+    2. Grant of Copyright License. Subject to the terms and conditions of
+       this License, each Contributor hereby grants to You a perpetual,
+       worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+       copyright license to reproduce, prepare Derivative Works of,
+       publicly display, publicly perform, sublicense, and distribute the
+       Work and such Derivative Works in Source or Object form.
+
+    3. Grant of Patent License. Subject to the terms and conditions of
+       this License, each Contributor hereby grants to You a perpetual,
+       worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+       (except as stated in this section) patent license to make, have made,
+       use, offer to sell, sell, import, and otherwise transfer the Work,
+       where such license applies only to those patent claims licensable
+       by such Contributor that are necessarily infringed by their
+       Contribution(s) alone or by combination of their Contribution(s)
+       with the Work to which such Contribution(s) was submitted. If You
+       institute patent litigation against any entity (including a
+       cross-claim or counterclaim in a lawsuit) alleging that the Work
+       or a Contribution incorporated within the Work constitutes direct
+       or contributory patent infringement, then any patent licenses
+       granted to You under this License for that Work shall terminate
+       as of the date such litigation is filed.
+
+    4. Redistribution. You may reproduce and distribute copies of the
+       Work or Derivative Works thereof in any medium, with or without
+       modifications, and in Source or Object form, provided that You
+       meet the following conditions:
+
+       (a) You must give any other recipients of the Work or
+           Derivative Works a copy of this License; and
+
+       (b) You must cause any modified files to carry prominent notices
+           stating that You changed the files; and
+
+       (c) You must retain, in the Source form of any Derivative Works
+           that You distribute, all copyright, patent, trademark, and
+           attribution notices from the Source form of the Work,
+           excluding those notices that do not pertain to any part of
+           the Derivative Works; and
+
+       (d) If the Work includes a "NOTICE" text file as part of its
+           distribution, then any Derivative Works that You distribute must
+           include a readable copy of the attribution notices contained
+           within such NOTICE file, excluding those notices that do not
+           pertain to any part of the Derivative Works, in at least one
+           of the following places: within a NOTICE text file distributed
+           as part of the Derivative Works; within the Source form or
+           documentation, if provided along with the Derivative Works; or,
+           within a display generated by the Derivative Works, if and
+           wherever such third-party notices normally appear. The contents
+           of the NOTICE file are for informational purposes only and
+           do not modify the License. You may add Your own attribution
+           notices within Derivative Works that You distribute, alongside
+           or as an addendum to the NOTICE text from the Work, provided
+           that such additional attribution notices cannot be construed
+           as modifying the License.
+
+       You may add Your own copyright statement to Your modifications and
+       may provide additional or different license terms and conditions
+       for use, reproduction, or distribution of Your modifications, or
+       for any such Derivative Works as a whole, provided Your use,
+       reproduction, and distribution of the Work otherwise complies with
+       the conditions stated in this License.
+
+    5. Submission of Contributions. Unless You explicitly state otherwise,
+       any Contribution intentionally submitted for inclusion in the Work
+       by You to the Licensor shall be under the terms and conditions of
+       this License, without any additional terms or conditions.
+       Notwithstanding the above, nothing herein shall supersede or modify
+       the terms of any separate license agreement you may have executed
+       with Licensor regarding such Contributions.
+
+    6. Trademarks. This License does not grant permission to use the trade
+       names, trademarks, service marks, or product names of the Licensor,
+       except as required for reasonable and customary use in describing the
+       origin of the Work and reproducing the content of the NOTICE file.
+
+    7. Disclaimer of Warranty. Unless required by applicable law or
+       agreed to in writing, Licensor provides the Work (and each
+       Contributor provides its Contributions) on an "AS IS" BASIS,
+       WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+       implied, including, without limitation, any warranties or conditions
+       of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+       PARTICULAR PURPOSE. You are solely responsible for determining the
+       appropriateness of using or redistributing the Work and assume any
+       risks associated with Your exercise of permissions under this License.
+
+    8. Limitation of Liability. In no event and under no legal theory,
+       whether in tort (including negligence), contract, or otherwise,
+       unless required by applicable law (such as deliberate and grossly
+       negligent acts) or agreed to in writing, shall any Contributor be
+       liable to You for damages, including any direct, indirect, special,
+       incidental, or consequential damages of any character arising as a
+       result of this License or out of the use or inability to use the
+       Work (including but not limited to damages for loss of goodwill,
+       work stoppage, computer failure or malfunction, or any and all
+       other commercial damages or losses), even if such Contributor
+       has been advised of the possibility of such damages.
+
+    9. Accepting Warranty or Additional Liability. While redistributing
+       the Work or Derivative Works thereof, You may choose to offer,
+       and charge a fee for, acceptance of support, warranty, indemnity,
+       or other liability obligations and/or rights consistent with this
+       License. However, in accepting such obligations, You may act only
+       on Your own behalf and on Your sole responsibility, not on behalf
+       of any other Contributor, and only if You agree to indemnify,
+       defend, and hold each Contributor harmless for any liability
+       incurred by, or claims asserted against, such Contributor by reason
+       of your accepting any such warranty or additional liability.
+
+    END OF TERMS AND CONDITIONS
+
+    APPENDIX: How to apply the Apache License to your work.
+
+       To apply the Apache License to your work, attach the following
+       boilerplate notice, with the fields enclosed by brackets "[]"
+       replaced with your own identifying information. (Don't include
+       the brackets!)  The text should be enclosed in the appropriate
+       comment syntax for the file format. We also recommend that a
+       file or class name and description of purpose be included on the
+       same "printed page" as the copyright notice for easier
+       identification within third-party archives.
+
+    Copyright 2015 Apache Software Foundation
+
+    Licensed 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.
+    Status API Training Shop Blog About
+    � 2016 GitHub, Inc. Terms Privacy Security Contact Help

http://git-wip-us.apache.org/repos/asf/incubator-airflow-site/blob/9e19165c/_sources/plugins.txt
----------------------------------------------------------------------
diff --git a/_sources/plugins.txt b/_sources/plugins.txt
new file mode 100644
index 0000000..5dde383
--- /dev/null
+++ b/_sources/plugins.txt
@@ -0,0 +1,139 @@
+Plugins
+=======
+
+Airflow has a simple plugin manager built-in that can integrate external
+features to its core by simply dropping files in your
+``$AIRFLOW_HOME/plugins`` folder.
+
+The python modules in the ``plugins`` folder get imported,
+and **hooks**, **operators**, **macros**, **executors** and web **views**
+get integrated to Airflow's main collections and become available for use.
+
+What for?
+---------
+
+Airflow offers a generic toolbox for working with data. Different
+organizations have different stacks and different needs. Using Airflow
+plugins can be a way for companies to customize their Airflow installation
+to reflect their ecosystem.
+
+Plugins can be used as an easy way to write, share and activate new sets of
+features.
+
+There's also a need for a set of more complex applications to interact with
+different flavors of data and metadata.
+
+Examples:
+
+* A set of tools to parse Hive logs and expose Hive metadata (CPU /IO / phases/ skew /...)
+* An anomaly detection framework, allowing people to collect metrics, set thresholds and alerts
+* An auditing tool, helping understand who accesses what
+* A config-driven SLA monitoring tool, allowing you to set monitored tables and at what time
+  they should land, alert people, and expose visualizations of outages
+* ...
+
+Why build on top of Airflow?
+----------------------------
+
+Airflow has many components that can be reused when building an application:
+
+* A web server you can use to render your views
+* A metadata database to store your models
+* Access to your databases, and knowledge of how to connect to them
+* An array of workers that your application can push workload to
+* Airflow is deployed, you can just piggy back on it's deployment logistics
+* Basic charting capabilities, underlying libraries and abstractions
+
+
+Interface
+---------
+
+To create a plugin you will need to derive the
+``airflow.plugins_manager.AirflowPlugin`` class and reference the objects
+you want to plug into Airflow. Here's what the class you need to derive
+looks like:
+
+
+.. code:: python
+
+    class AirflowPlugin(object):
+        # The name of your plugin (str)
+        name = None
+        # A list of class(es) derived from BaseOperator
+        operators = []
+        # A list of class(es) derived from BaseHook
+        hooks = []
+        # A list of class(es) derived from BaseExecutor
+        executors = []
+        # A list of references to inject into the macros namespace
+        macros = []
+        # A list of objects created from a class derived
+        # from flask_admin.BaseView
+        admin_views = []
+        # A list of Blueprint object created from flask.Blueprint
+        flask_blueprints = []
+        # A list of menu links (flask_admin.base.MenuLink)
+        menu_links = []
+
+
+Example
+-------
+
+The code below defines a plugin that injects a set of dummy object
+definitions in Airflow.
+
+.. code:: python
+
+    # This is the class you derive to create a plugin
+    from airflow.plugins_manager import AirflowPlugin
+
+    from flask import Blueprint
+    from flask_admin import BaseView, expose
+    from flask_admin.base import MenuLink
+
+    # Importing base classes that we need to derive
+    from airflow.hooks.base_hook import BaseHook
+    from airflow.models import  BaseOperator
+    from airflow.executors.base_executor import BaseExecutor
+
+    # Will show up under airflow.hooks.PluginHook
+    class PluginHook(BaseHook):
+        pass
+
+    # Will show up under airflow.operators.PluginOperator
+    class PluginOperator(BaseOperator):
+        pass
+
+    # Will show up under airflow.executors.PluginExecutor
+    class PluginExecutor(BaseExecutor):
+        pass
+
+    # Creating a flask admin BaseView
+    class TestView(BaseView):
+        @expose('/')
+        def test(self):
+            # in this example, put your test_plugin/test.html template at airflow/plugins/templates/test_plugin/test.html
+            return self.render("test_plugin/test.html", content="Hello galaxy!")
+    v = TestView(category="Test Plugin", name="Test View")
+
+    # Creating a flask blueprint to intergrate the templates and static folder
+    bp = Blueprint(
+        "test_plugin", __name__,
+        template_folder='templates', # registers airflow/plugins/templates as a Jinja template folder 
+        static_folder='static',
+        static_url_path='/static/test_plugin')
+        
+    ml = MenuLink(
+        category='Test Plugin',
+        name='Test Menu Link',
+        url='http://pythonhosted.org/airflow/')
+
+    # Defining the plugin class
+    class AirflowTestPlugin(AirflowPlugin):
+        name = "test_plugin"
+        operators = [PluginOperator]
+        flask_blueprints = [bp]
+        hooks = [PluginHook]
+        executors = [PluginExecutor]
+        admin_views = [v]
+        menu_links = [ml]

http://git-wip-us.apache.org/repos/asf/incubator-airflow-site/blob/9e19165c/_sources/profiling.txt
----------------------------------------------------------------------
diff --git a/_sources/profiling.txt b/_sources/profiling.txt
new file mode 100644
index 0000000..93e6b6b
--- /dev/null
+++ b/_sources/profiling.txt
@@ -0,0 +1,39 @@
+Data Profiling
+==============
+
+Part of being productive with data is having the right weapons to
+profile the data you are working with. Airflow provides a simple query
+interface to write SQL and get results quickly, and a charting application
+letting you visualize data.
+
+Adhoc Queries
+-------------
+The adhoc query UI allows for simple SQL interactions with the database
+connections registered in Airflow.
+
+.. image:: img/adhoc.png
+
+Charts
+------
+A simple UI built on top of flask-admin and highcharts allows building
+data visualizations and charts easily. Fill in a form with a label, SQL,
+chart type, pick a source database from your environment's connectons,
+select a few other options, and save it for later use.
+
+You can even use the same templating and macros available when writing
+airflow pipelines, parameterizing your queries and modifying parameters
+directly in the URL.
+
+These charts are basic, but they're easy to create, modify and share.
+
+Chart Screenshot
+................
+
+.. image:: img/chart.png
+
+-----
+
+Chart Form Screenshot
+.....................
+
+.. image:: img/chart_form.png

http://git-wip-us.apache.org/repos/asf/incubator-airflow-site/blob/9e19165c/_sources/project.txt
----------------------------------------------------------------------
diff --git a/_sources/project.txt b/_sources/project.txt
new file mode 100644
index 0000000..1e6622f
--- /dev/null
+++ b/_sources/project.txt
@@ -0,0 +1,58 @@
+Project
+=======
+
+History
+-------
+
+Airflow was started in the fall of 2014 by Maxime Beauchemin at Airbnb.
+It was open source from the very first commit and officially brought under
+the Airbnb Github and announced in the spring of 2015.
+
+The project joined the Apache Software Foundation's incubation program in the
+winter of 2016.
+
+
+Committers
+----------
+
+- @mistercrunch (Maxime "Max" Beauchemin)
+- @r39132 (Siddharth "Sid" Anand)
+- @criccomini (Chris Riccomini)
+- @bolkedebruin (Bolke de Bruin)
+- @artwr (Arthur Wiedmer)
+- @jlowin (Jeremiah Lowin)
+- @patrickleotardif (Patrick Leo Tardif)
+- @aoen (Dan Davydov)
+
+For the full list of contributors, take a look at `Airflow's Github
+Contributor page:
+<https://github.com/apache/incubator-airflow/graphs/contributors>`_
+
+
+Resources & links
+-----------------
+
+* Mailing list (send emails to
+  ``dev-subscribe@airflow.incubator.apache.org`` and
+  ``commits-subscribe@airflow.incubator.apache.org``
+  to subscribe to each)
+* `Issues <https://issues.apache.org/jira/browse/AIRFLOW>`_
+* `Airbnb Blog Post about Airflow <http://nerds.airbnb.com/airflow/>`_
+* `Airflow Common Pitfalls <https://cwiki.apache.org/confluence/display/AIRFLOW/Common+Pitfalls>`_
+* `Hadoop Summit Airflow Video <https://www.youtube.com/watch?v=oYp49mBwH60>`_
+* `Airflow at Agari Blog Post <http://agari.com/blog/airflow-agari>`_
+* `Talk: Best practices with Airflow (nov 2015) <https://youtu.be/dgaoqOZlvEA>`_
+* `Airflow Lesson 1: TriggerDagRunOperator <https://www.linkedin.com/pulse/airflow-lesson-1-triggerdagrunoperator-siddharth-anand?published=t>`_
+* `Docker Airflow (externally maintained) <https://github.com/puckel/docker-airflow>`_
+* `Airflow: Tips, Tricks, and Pitfalls @ Handy <https://medium.com/handy-tech/airflow-tips-tricks-and-pitfalls-9ba53fba14eb#.o2snqeoz7>`_
+* `Airflow Chef recipe (community contributed) <https://github.com/bahchis/airflow-cookbook>`_ ,
+  `another here <https://supermarket.chef.io/cookbooks/airflow>`_
+* Airflow Puppet Module (community contributed) `puppet-airflow <https://github.com/similarweb/puppet-airflow>`_,
+  `airflow <https://forge.puppetlabs.com/similarweb/airflow>`_
+* `Gitter (chat) Channel <https://gitter.im/airbnb/airflow>`_
+
+
+Roadmap
+-------
+
+Please refer to the Roadmap on `the wiki <https://cwiki.apache.org/confluence/display/AIRFLOW/Airflow+Home>`_

http://git-wip-us.apache.org/repos/asf/incubator-airflow-site/blob/9e19165c/_sources/scheduler.txt
----------------------------------------------------------------------
diff --git a/_sources/scheduler.txt b/_sources/scheduler.txt
new file mode 100644
index 0000000..9c8a618
--- /dev/null
+++ b/_sources/scheduler.txt
@@ -0,0 +1,101 @@
+Scheduling & Triggers
+=====================
+
+The Airflow scheduler monitors all tasks and all DAGs, and triggers the
+task instances whose dependencies have been met. Behind the scenes,
+it monitors and stays in sync with a folder for all DAG objects it may contain,
+and periodically (every minute or so) inspects active tasks to see whether
+they can be triggered.
+
+The Airflow scheduler is designed to run as a persistent service in an
+Airflow production environment. To kick it off, all you need to do is
+execute ``airflow scheduler``. It will use the configuration specified in
+``airflow.cfg``.
+
+Note that if you run a DAG on a ``schedule_interval`` of one day,
+the run stamped ``2016-01-01`` will be trigger soon after ``2016-01-01T23:59``.
+In other words, the job instance is started once the period it covers
+has ended.
+
+The scheduler starts an instance of the executor specified in the your
+``airflow.cfg``. If it happens to be the ``LocalExecutor``, tasks will be
+executed as subprocesses; in the case of ``CeleryExecutor`` and
+``MesosExecutor``, tasks are executed remotely.
+
+To start a scheduler, simply run the command:
+
+.. code:: bash
+
+    airflow scheduler
+
+
+DAG Runs
+''''''''
+
+A DAG Run is an object representing an instantiation of the DAG in time.
+
+Each DAG may or may not have a schedule, which informs how ``DAG Runs`` are
+created. ``schedule_interval`` is defined as a DAG arguments, and receives
+preferably a
+`cron expression <https://en.wikipedia.org/wiki/Cron#CRON_expression>`_ as
+a ``str``, or a ``datetime.timedelta`` object. Alternatively, you can also
+use one of these cron "preset":
+
++--------------+----------------------------------------------------------------+---------------+
+| preset       | Run once a year at midnight of January 1                       | cron          |
++==============+================================================================+===============+
+| ``None``     | Don't schedule, use for exclusively "externally triggered"     |               |
+|              | DAGs                                                           |               |
++--------------+----------------------------------------------------------------+---------------+
+| ``@once``    | Schedule once and only once                                    |               |
++--------------+----------------------------------------------------------------+---------------+
+| ``@hourly``  | Run once an hour at the beginning of the hour                  | ``0 * * * *`` |
++--------------+----------------------------------------------------------------+---------------+
+| ``@daily``   | Run once a day at midnight                                     | ``0 0 * * *`` |
++--------------+----------------------------------------------------------------+---------------+
+| ``@weekly``  | Run once a week at midnight on Sunday morning                  | ``0 0 * * 0`` |
++--------------+----------------------------------------------------------------+---------------+
+| ``@monthly`` | Run once a month at midnight of the first day of the month     | ``0 0 1 * *`` |
++--------------+----------------------------------------------------------------+---------------+
+| ``@yearly``  | Run once a year at midnight of January 1                       | ``0 0 1 1 *`` |
++--------------+----------------------------------------------------------------+---------------+
+
+
+Your DAG will be instantiated
+for each schedule, while creating a ``DAG Run`` entry for each schedule.
+
+DAG runs have a state associated to them (running, failed, success) and
+informs the scheduler on which set of schedules should be evaluated for
+task submissions. Without the metadata at the DAG run level, the Airflow
+scheduler would have much more work to do in order to figure out what tasks
+should be triggered and come to a crawl. It might also create undesired
+processing when changing the shape of your DAG, by say adding in new
+tasks.
+
+External Triggers
+'''''''''''''''''
+
+Note that ``DAG Runs`` can also be created manually through the CLI while
+running an ``airflow trigger_dag`` command, where you can define a
+specific ``run_id``. The ``DAG Runs`` created externally to the
+scheduler get associated to the trigger's timestamp, and will be displayed
+in the UI alongside scheduled ``DAG runs``.
+
+
+To Keep in Mind
+'''''''''''''''
+* The first ``DAG Run`` is created based on the minimum ``start_date`` for the
+  tasks in your DAG.
+* Subsequent ``DAG Runs`` are created by the scheduler process, based on
+  your DAG's ``schedule_interval``, sequentially.
+* When clearing a set of tasks' state in hope of getting them to re-run,
+  it is important to keep in mind the ``DAG Run``'s state too as it defines
+  whether the scheduler should look into triggering tasks for that run.
+
+Here are some of the ways you can **unblock tasks**:
+
+* From the UI, you can **clear** (as in delete the status of) individual task instances from the task instances dialog, while defining whether you want to includes the past/future and the upstream/downstream dependencies. Note that a confirmation window comes next and allows you to see the set you are about to clear.
+* The CLI command ``airflow clear -h`` has lots of options when it comes to clearing task instance states, including specifying date ranges, targeting task_ids by specifying a regular expression, flags for including upstream and downstream relatives, and targeting task instances in specific states (``failed``, or ``success``)
+* Marking task instances as successful can be done through the UI. This is mostly to fix false negatives, or for instance when the fix has been applied outside of Airflow.
+* The ``airflow backfill`` CLI subcommand has a flag to ``--mark_success`` and allows selecting subsections of the DAG as well as specifying date ranges.
+

http://git-wip-us.apache.org/repos/asf/incubator-airflow-site/blob/9e19165c/_sources/security.txt
----------------------------------------------------------------------
diff --git a/_sources/security.txt b/_sources/security.txt
new file mode 100644
index 0000000..1f33548
--- /dev/null
+++ b/_sources/security.txt
@@ -0,0 +1,249 @@
+Security
+========
+
+Web Authentication
+------------------
+
+By default, all gates are opened. An easy way to restrict access
+to the web application is to do it at the network level, or by using
+SSH tunnels.
+
+It is however possible to switch on authentication by either using one of the supplied
+backends or create your own.
+
+Password
+''''''''
+
+One of the simplest mechanisms for authentication is requiring users to specify a password before logging in.
+Password authentication requires the used of the ``password`` subpackage in your requirements file. Password hashing
+uses bcrypt before storing passwords.
+
+.. code-block:: bash
+
+    [webserver]
+    authenticate = True
+    auth_backend = airflow.contrib.auth.backends.password_auth
+
+When password auth is enabled, an initial user credential will need to be created before anyone can login. An initial
+user was not created in the migrations for this authenication backend to prevent default Airflow installations from
+attack. Creating a new user has to be done via a Python REPL on the same machine Airflow is installed.
+
+.. code-block:: bash
+
+    # navigate to the airflow installation directory
+    $ cd ~/airflow
+    $ python
+    Python 2.7.9 (default, Feb 10 2015, 03:28:08)
+    Type "help", "copyright", "credits" or "license" for more information.
+    >>> import airflow
+    >>> from airflow import models, settings
+    >>> from airflow.contrib.auth.backends.password_auth import PasswordUser
+    >>> user = PasswordUser(models.User())
+    >>> user.username = 'new_user_name'
+    >>> user.email = 'new_user_email@example.com'
+    >>> user.password = 'set_the_password'
+    >>> session = settings.Session()
+    >>> session.add(user)
+    >>> session.commit()
+    >>> session.close()
+    >>> exit()
+
+LDAP
+''''
+
+To turn on LDAP authentication configure your ``airflow.cfg`` as follows. Please note that the example uses
+an encrypted connection to the ldap server as you probably do not want passwords be readable on the network level.
+It is however possible to configure without encryption if you really want to.
+
+Additionally, if you are using Active Directory, and are not explicitly specifying an OU that your users are in,
+you will need to change ``search_scope`` to "SUBTREE".
+
+Valid search_scope options can be found in the `ldap3 Documentation <http://ldap3.readthedocs.org/searches.html?highlight=search_scope>`_
+
+.. code-block:: bash
+
+    [webserver]
+    authenticate = True
+    auth_backend = airflow.contrib.auth.backends.ldap_auth
+
+    [ldap]
+    uri = ldaps://<your.ldap.server>:<port>
+    user_filter = objectClass=*
+    user_name_attr = uid # in case of Active Directory you would use sAMAccountName
+    superuser_filter = memberOf=CN=airflow-super-users,OU=Groups,OU=RWC,OU=US,OU=NORAM,DC=example,DC=com
+    data_profiler_filter = memberOf=CN=airflow-data-profilers,OU=Groups,OU=RWC,OU=US,OU=NORAM,DC=example,DC=com
+    bind_user = cn=Manager,dc=example,dc=com
+    bind_password = insecure
+    basedn = dc=example,dc=com
+    cacert = /etc/ca/ldap_ca.crt
+    search_scope = LEVEL # Set this to SUBTREE if using Active Directory, and not specifying an Organizational Unit
+
+The superuser_filter and data_profiler_filter are optional. If defined, these configurations allow you to specify LDAP groups that users must belong to in order to have superuser (admin) and data-profiler permissions. If undefined, all users will be superusers and data profilers.
+
+Roll your own
+'''''''''''''
+
+Airflow uses ``flask_login`` and
+exposes a set of hooks in the ``airflow.default_login`` module. You can
+alter the content and make it part of the ``PYTHONPATH`` and configure it as a backend in ``airflow.cfg```.
+
+.. code-block:: bash
+
+    [webserver]
+    authenticate = True
+    auth_backend = mypackage.auth
+
+Multi-tenancy
+-------------
+
+You can filter the list of dags in webserver by owner name, when authentication
+is turned on, by setting webserver.filter_by_owner as true in your ``airflow.cfg``
+With this, when a user authenticates and logs into webserver, it will see only the dags
+which it is owner of. A super_user, will be able to see all the dags although.
+This makes the web UI a multi-tenant UI, where a user will only be able to see dags
+created by itself.
+
+
+Kerberos
+--------
+Airflow has initial support for Kerberos. This means that airflow can renew kerberos
+tickets for itself and store it in the ticket cache. The hooks and dags can make use of ticket
+to authenticate against kerberized services.
+
+Limitations
+'''''''''''
+
+Please note that at this time not all hooks have been adjusted to make use of this functionality yet.
+Also it does not integrate kerberos into the web interface and you will have to rely on network
+level security for now to make sure your service remains secure.
+
+Celery integration has not been tried and tested yet. However if you generate a key tab for every host
+and launch a ticket renewer next to every worker it will most likely work.
+
+Enabling kerberos
+'''''''''''''''''
+
+#### Airflow
+
+To enable kerberos you will need to generate a (service) key tab.
+
+.. code-block:: bash
+
+    # in the kadmin.local or kadmin shell, create the airflow principal
+    kadmin:  addprinc -randkey airflow/fully.qualified.domain.name@YOUR-REALM.COM
+
+    # Create the airflow keytab file that will contain the airflow principal
+    kadmin:  xst -norandkey -k airflow.keytab airflow/fully.qualified.domain.name
+
+Now store this file in a location where the airflow user can read it (chmod 600). And then add the following to
+your ``airflow.cfg``
+
+.. code-block:: bash
+
+    [core]
+    security = kerberos
+
+    [kerberos]
+    keytab = /etc/airflow/airflow.keytab
+    reinit_frequency = 3600
+    principal = airflow
+
+Launch the ticket renewer by
+
+.. code-block:: bash
+
+    # run ticket renewer
+    airflow kerberos
+
+#### Hadoop
+
+If want to use impersonation this needs to be enabled in ``core-site.xml`` of your hadoop config.
+
+.. code-block:: bash
+
+    <property>
+      <name>hadoop.proxyuser.airflow.groups</name>
+      <value>*</value>
+    </property>
+
+    <property>
+      <name>hadoop.proxyuser.airflow.users</name>
+      <value>*</value>
+    </property>
+
+    <property>
+      <name>hadoop.proxyuser.airflow.hosts</name>
+      <value>*</value>
+    </property>
+
+Of course if you need to tighten your security replace the asterisk with something more appropriate.
+
+Using kerberos authentication
+'''''''''''''''''''''''''''''
+
+The hive hook has been updated to take advantage of kerberos authentication. To allow your DAGs to use it simply
+update the connection details with, for example:
+
+.. code-block:: bash
+
+    { "use_beeline": true, "principal": "hive/_HOST@EXAMPLE.COM"}
+
+Adjust the principal to your settings. The _HOST part will be replaced by the fully qualified domain name of
+the server.
+
+You can specify if you would like to use the dag owner as the user for the connection or the user specified in the login
+section of the connection. For the login user specify the following as extra:
+
+.. code-block:: bash
+
+    { "use_beeline": true, "principal": "hive/_HOST@EXAMPLE.COM", "proxy_user": "login"}
+
+For the DAG owner use:
+
+.. code-block:: bash
+
+    { "use_beeline": true, "principal": "hive/_HOST@EXAMPLE.COM", "proxy_user": "owner"}
+
+and in your DAG, when initializing the HiveOperator, specify
+
+.. code-block:: bash
+
+    run_as_owner=True
+
+GitHub Enterprise (GHE) Authentication
+''''''''''''''''''''''''''''''''''''''
+
+The GitHub Enterprise authentication backend can be used to authenticate users
+against an installation of GitHub Enterprise using OAuth2. You can optionally
+specify a team whitelist (composed of slug cased team names) to restrict login
+to only members of those teams.
+
+*NOTE* If you do not specify a team whitelist, anyone with a valid account on
+your GHE installation will be able to login to Airflow.
+
+.. code-block:: bash
+
+    [webserver]
+    authenticate = True
+    auth_backend = airflow.contrib.auth.backends.github_enterprise_auth
+
+    [github_enterprise]
+    host = github.example.com
+    client_id = oauth_key_from_github_enterprise
+    client_secret = oauth_secret_from_github_enterprise
+    oauth_callback_route = /example/ghe_oauth/callback
+    allowed_teams = example_team_1, example_team_2
+
+Setting up GHE Authentication
+'''''''''''''''''''''''''''''
+
+An application must be setup in GHE before you can use the GHE authentication
+backend. In order to setup an application:
+
+1. Navigate to your GHE profile
+2. Select 'Applications' from the left hand nav
+3. Select the 'Developer Applications' tab
+4. Click 'Register new application'
+5. Fill in the required information (the 'Authorization callback URL' must be fully qualifed e.g. http://airflow.example.com/example/ghe_oauth/callback)
+6. Click 'Register application'
+7. Copy 'Client ID', 'Client Secret', and your callback route to your airflow.cfg according to the above example

http://git-wip-us.apache.org/repos/asf/incubator-airflow-site/blob/9e19165c/_sources/start.txt
----------------------------------------------------------------------
diff --git a/_sources/start.txt b/_sources/start.txt
new file mode 100644
index 0000000..cc41d4b
--- /dev/null
+++ b/_sources/start.txt
@@ -0,0 +1,49 @@
+Quick Start
+-----------
+
+The installation is quick and straightforward.
+
+.. code-block:: bash
+
+    # airflow needs a home, ~/airflow is the default,
+    # but you can lay foundation somewhere else if you prefer
+    # (optional)
+    export AIRFLOW_HOME=~/airflow
+
+    # install from pypi using pip
+    pip install airflow
+
+    # initialize the database
+    airflow initdb
+
+    # start the web server, default port is 8080
+    airflow webserver -p 8080
+
+Upon running these commands, Airflow will create the ``$AIRFLOW_HOME`` folder
+and lay an "airflow.cfg" file with defaults that get you going fast. You can
+inspect the file either in ``$AIRFLOW_HOME/airflow.cfg``, or through the UI in
+the ``Admin->Configuration`` menu. The PID file for the webserver will be stored
+in ``$AIRFLOW_HOME/airflow-webserver.pid`` or in ``/run/airflow/webserver.pid``
+if started by systemd.
+
+Out of the box, Airflow uses a sqlite database, which you should outgrow
+fairly quickly since no parallelization is possible using this database
+backend. It works in conjunction with the ``SequentialExecutor`` which will
+only run task instances sequentially. While this is very limiting, it allows
+you to get up and running quickly and take a tour of the UI and the
+command line utilities.
+
+Here are a few commands that will trigger a few task instances. You should
+be able to see the status of the jobs change in the ``example1`` DAG as you
+run the commands below.
+
+.. code-block:: bash
+
+    # run your first task instance
+    airflow run example_bash_operator runme_0 2015-01-01
+    # run a backfill over 2 days
+    airflow backfill example_bash_operator -s 2015-01-01 -e 2015-01-02
+
+What's Next?
+''''''''''''
+From this point, you can head to the :doc:`tutorial` section for further examples or the :doc:`configuration` section if you're ready to get your hands dirty.

http://git-wip-us.apache.org/repos/asf/incubator-airflow-site/blob/9e19165c/_sources/tutorial.txt
----------------------------------------------------------------------
diff --git a/_sources/tutorial.txt b/_sources/tutorial.txt
new file mode 100644
index 0000000..e9d382b
--- /dev/null
+++ b/_sources/tutorial.txt
@@ -0,0 +1,429 @@
+
+Tutorial
+================
+
+This tutorial walks you through some of the fundamental Airflow concepts,
+objects, and their usage while writing your first pipeline.
+
+Example Pipeline definition
+---------------------------
+
+Here is an example of a basic pipeline definition. Do not worry if this looks
+complicated, a line by line explanation follows below.
+
+.. code:: python
+
+    """
+    Code that goes along with the Airflow tutorial located at:
+    https://github.com/airbnb/airflow/blob/master/airflow/example_dags/tutorial.py
+    """
+    from airflow import DAG
+    from airflow.operators import BashOperator
+    from datetime import datetime, timedelta
+
+
+    default_args = {
+        'owner': 'airflow',
+        'depends_on_past': False,
+        'start_date': datetime(2015, 6, 1),
+        'email': ['airflow@airflow.com'],
+        'email_on_failure': False,
+        'email_on_retry': False,
+        'retries': 1,
+        'retry_delay': timedelta(minutes=5),
+        # 'queue': 'bash_queue',
+        # 'pool': 'backfill',
+        # 'priority_weight': 10,
+        # 'end_date': datetime(2016, 1, 1),
+    }
+
+    dag = DAG('tutorial', default_args=default_args)
+
+    # t1, t2 and t3 are examples of tasks created by instatiating operators
+    t1 = BashOperator(
+        task_id='print_date',
+        bash_command='date',
+        dag=dag)
+
+    t2 = BashOperator(
+        task_id='sleep',
+        bash_command='sleep 5',
+        retries=3,
+        dag=dag)
+
+    templated_command = """
+        {% for i in range(5) %}
+            echo "{{ ds }}"
+            echo "{{ macros.ds_add(ds, 7)}}"
+            echo "{{ params.my_param }}"
+        {% endfor %}
+    """
+
+    t3 = BashOperator(
+        task_id='templated',
+        bash_command=templated_command,
+        params={'my_param': 'Parameter I passed in'},
+        dag=dag)
+
+    t2.set_upstream(t1)
+    t3.set_upstream(t1)
+
+
+It's a DAG definition file
+--------------------------
+
+One thing to wrap your head around (it may not be very intuitive for everyone
+at first) is that this Airflow Python script is really
+just a configuration file specifying the DAG's structure as code.
+The actual tasks defined here will run in a different context from
+the context of this script. Different tasks run on different workers
+at different points in time, which means that this script cannot be used
+to cross communicate between tasks. Note that for this
+purpose we have a more advanced feature called ``XCom``.
+
+People sometimes think of the DAG definition file as a place where they
+can do some actual data processing - that is not the case at all!
+The script's purpose is to define a DAG object. It needs to evaluate
+quickly (seconds, not minutes) since the scheduler will execute it
+periodically to reflect the changes if any.
+
+
+Importing Modules
+-----------------
+
+An Airflow pipeline is just a Python script that happens to define an
+Airflow DAG object. Let's start by importing the libraries we will need.
+
+.. code:: python
+
+    # The DAG object; we'll need this to instantiate a DAG
+    from airflow import DAG
+
+    # Operators; we need this to operate!
+    from airflow.operators import BashOperator
+
+Default Arguments
+-----------------
+We're about to create a DAG and some tasks, and we have the choice to
+explicitly pass a set of arguments to each task's constructor
+(which would become redundant), or (better!) we can define a dictionary
+of default parameters that we can use when creating tasks.
+
+.. code:: python
+
+    from datetime import datetime, timedelta
+
+    default_args = {
+        'owner': 'airflow',
+        'depends_on_past': False,
+        'start_date': datetime(2015, 6, 1),
+        'email': ['airflow@airflow.com'],
+        'email_on_failure': False,
+        'email_on_retry': False,
+        'retries': 1,
+        'retry_delay': timedelta(minutes=5),
+        # 'queue': 'bash_queue',
+        # 'pool': 'backfill',
+        # 'priority_weight': 10,
+        # 'end_date': datetime(2016, 1, 1),
+    }
+
+For more information about the BaseOperator's parameters and what they do,
+refer to the :py:class:``airflow.models.BaseOperator`` documentation.
+
+Also, note that you could easily define different sets of arguments that
+would serve different purposes. An example of that would be to have
+different settings between a production and development environment.
+
+
+Instantiate a DAG
+-----------------
+
+We'll need a DAG object to nest our tasks into. Here we pass a string
+that defines the ``dag_id``, which serves as a unique identifier for your DAG.
+We also pass the default argument dictionary that we just defined and
+define a ``schedule_interval`` of 1 day for the DAG.
+
+.. code:: python
+
+    dag = DAG(
+        'tutorial', default_args=default_args, schedule_interval=timedelta(1))
+
+Tasks
+-----
+Tasks are generated when instantiating operator objects. An object
+instantiated from an operator is called a constructor. The first argument
+``task_id`` acts as a unique identifier for the task.
+
+.. code:: python
+
+    t1 = BashOperator(
+        task_id='print_date',
+        bash_command='date',
+        dag=dag)
+
+    t2 = BashOperator(
+        task_id='sleep',
+        bash_command='sleep 5',
+        retries=3,
+        dag=dag)
+
+Notice how we pass a mix of operator specific arguments (``bash_command``) and
+an argument common to all operators (``retries``) inherited
+from BaseOperator to the operator's constructor. This is simpler than
+passing every argument for every constructor call. Also, notice that in
+the second task we override the ``retries`` parameter with ``3``.
+
+The precedence rules for a task are as follows:
+
+1.  Explicitly passed arguments
+2.  Values that exist in the ``default_args`` dictionary
+3.  The operator's default value, if one exists
+
+A task must include or inherit the arguments ``task_id`` and ``owner``,
+otherwise Airflow will raise an exception.
+
+Templating with Jinja
+---------------------
+Airflow leverages the power of
+`Jinja Templating <http://jinja.pocoo.org/docs/dev/>`_  and provides
+the pipeline author
+with a set of built-in parameters and macros. Airflow also provides
+hooks for the pipeline author to define their own parameters, macros and
+templates.
+
+This tutorial barely scratches the surface of what you can do with
+templating in Airflow, but the goal of this section is to let you know
+this feature exists, get you familiar with double curly brackets, and
+point to the most common template variable: ``{{ ds }}``.
+
+.. code:: python
+
+    templated_command = """
+        {% for i in range(5) %}
+            echo "{{ ds }}"
+            echo "{{ macros.ds_add(ds, 7) }}"
+            echo "{{ params.my_param }}"
+        {% endfor %}
+    """
+
+    t3 = BashOperator(
+        task_id='templated',
+        bash_command=templated_command,
+        params={'my_param': 'Parameter I passed in'},
+        dag=dag)
+
+Notice that the ``templated_command`` contains code logic in ``{% %}`` blocks,
+references parameters like ``{{ ds }}``, calls a function as in
+``{{ macros.ds_add(ds, 7)}}``, and references a user-defined parameter
+in ``{{ params.my_param }}``.
+
+The ``params`` hook in ``BaseOperator`` allows you to pass a dictionary of
+parameters and/or objects to your templates. Please take the time
+to understand how the parameter ``my_param`` makes it through to the template.
+
+Files can also be passed to the ``bash_command`` argument, like
+``bash_command='templated_command.sh'``, where the file location is relative to
+the directory containing the pipeline file (``tutorial.py`` in this case). This
+may be desirable for many reasons, like separating your script's logic and
+pipeline code, allowing for proper code highlighting in files composed in
+different languages, and general flexibility in structuring pipelines. It is
+also possible to define your ``template_searchpath`` as pointing to any folder
+locations in the DAG constructor call.
+
+For more information on the variables and macros that can be referenced
+in templates, make sure to read through the :ref:`macros` section
+
+Setting up Dependencies
+-----------------------
+We have two simple tasks that do not depend on each other. Here's a few ways
+you can define dependencies between them:
+
+.. code:: python
+
+    t2.set_upstream(t1)
+
+    # This means that t2 will depend on t1
+    # running successfully to run
+    # It is equivalent to
+    # t1.set_downstream(t2)
+
+    t3.set_upstream(t1)
+
+    # all of this is equivalent to
+    # dag.set_dependency('print_date', 'sleep')
+    # dag.set_dependency('print_date', 'templated')
+
+Note that when executing your script, Airflow will raise exceptions when
+it finds cycles in your DAG or when a dependency is referenced more
+than once.
+
+Recap
+-----
+Alright, so we have a pretty basic DAG. At this point your code should look
+something like this:
+
+.. code:: python
+
+    """
+    Code that goes along with the Airflow located at:
+    http://airflow.readthedocs.org/en/latest/tutorial.html
+    """
+    from airflow import DAG
+    from airflow.operators import BashOperator
+    from datetime import datetime, timedelta
+
+
+    default_args = {
+        'owner': 'airflow',
+        'depends_on_past': False,
+        'start_date': datetime(2015, 6, 1),
+        'email': ['airflow@airflow.com'],
+        'email_on_failure': False,
+        'email_on_retry': False,
+        'retries': 1,
+        'retry_delay': timedelta(minutes=5),
+        # 'queue': 'bash_queue',
+        # 'pool': 'backfill',
+        # 'priority_weight': 10,
+        # 'end_date': datetime(2016, 1, 1),
+    }
+
+    dag = DAG(
+        'tutorial', default_args=default_args, schedule_interval=timedelta(1))
+
+    # t1, t2 and t3 are examples of tasks created by instatiating operators
+    t1 = BashOperator(
+        task_id='print_date',
+        bash_command='date',
+        dag=dag)
+
+    t2 = BashOperator(
+        task_id='sleep',
+        bash_command='sleep 5',
+        retries=3,
+        dag=dag)
+
+    templated_command = """
+        {% for i in range(5) %}
+            echo "{{ ds }}"
+            echo "{{ macros.ds_add(ds, 7)}}"
+            echo "{{ params.my_param }}"
+        {% endfor %}
+    """
+
+    t3 = BashOperator(
+        task_id='templated',
+        bash_command=templated_command,
+        params={'my_param': 'Parameter I passed in'},
+        dag=dag)
+
+    t2.set_upstream(t1)
+    t3.set_upstream(t1)
+
+Testing
+--------
+
+Running the Script
+''''''''''''''''''
+
+Time to run some tests. First let's make sure that the pipeline
+parses. Let's assume we're saving the code from the previous step in
+``tutorial.py`` in the DAGs folder referenced in your ``airflow.cfg``.
+The default location for your DAGs is ``~/airflow/dags``.
+
+.. code-block:: bash
+
+    python ~/airflow/dags/tutorial.py
+
+If the script does not raise an exception it means that you haven't done
+anything horribly wrong, and that your Airflow environment is somewhat
+sound.
+
+Command Line Metadata Validation
+'''''''''''''''''''''''''''''''''
+Let's run a few commands to validate this script further.
+
+.. code-block:: bash
+
+    # print the list of active DAGs
+    airflow list_dags
+
+    # prints the list of tasks the "tutorial" dag_id
+    airflow list_tasks tutorial
+
+    # prints the hierarchy of tasks in the tutorial DAG
+    airflow list_tasks tutorial --tree
+
+
+Testing
+'''''''
+Let's test by running the actual task instances on a specific date. The
+date specified in this context is an ``execution_date``, which simulates the
+scheduler running your task or dag at a specific date + time:
+
+.. code-block:: bash
+
+    # command layout: command subcommand dag_id task_id date
+
+    # testing print_date
+    airflow test tutorial print_date 2015-06-01
+
+    # testing sleep
+    airflow test tutorial sleep 2015-06-01
+
+Now remember what we did with templating earlier? See how this template
+gets rendered and executed by running this command:
+
+.. code-block:: bash
+
+    # testing templated
+    airflow test tutorial templated 2015-06-01
+
+This should result in displaying a verbose log of events and ultimately
+running your bash command and printing the result.
+
+Note that the ``airflow test`` command runs task instances locally, outputs
+their log to stdout (on screen), doesn't bother with dependencies, and
+doesn't communicate state (running, success, failed, ...) to the database.
+It simply allows testing a single task instance.
+
+Backfill
+''''''''
+Everything looks like it's running fine so let's run a backfill.
+``backfill`` will respect your dependencies, emit logs into files and talk to
+the database to record status. If you do have a webserver up, you'll be able
+to track the progress. ``airflow webserver`` will start a web server if you
+are interested in tracking the progress visually as your backfill progresses.
+
+Note that if you use ``depends_on_past=True``, individual task instances
+will depend on the success of the preceding task instance, except for the
+start_date specified itself, for which this dependency is disregarded.
+
+The date range in this context is a ``start_date`` and optionally an ``end_date``,
+which are used to populate the run schedule with task instances from this dag.
+
+.. code-block:: bash
+
+    # optional, start a web server in debug mode in the background
+    # airflow webserver --debug &
+
+    # start your backfill on a date range
+    airflow backfill tutorial -s 2015-06-01 -e 2015-06-07
+
+What's Next?
+-------------
+That's it, you've written, tested and backfilled your very first Airflow
+pipeline. Merging your code into a code repository that has a master scheduler
+running against it should get it to get triggered and run every day.
+
+Here's a few things you might want to do next:
+
+* Take an in-depth tour of the UI - click all the things!
+* Keep reading the docs! Especially the sections on:
+
+    * Command line interface
+    * Operators
+    * Macros
+
+* Write your first pipeline!

http://git-wip-us.apache.org/repos/asf/incubator-airflow-site/blob/9e19165c/_sources/ui.txt
----------------------------------------------------------------------
diff --git a/_sources/ui.txt b/_sources/ui.txt
new file mode 100644
index 0000000..4b232fa
--- /dev/null
+++ b/_sources/ui.txt
@@ -0,0 +1,102 @@
+UI / Screenshots
+=================
+The Airflow UI make it easy to monitor and troubleshoot your data pipelines.
+Here's a quick overview of some of the features and visualizations you
+can find in the Airflow UI.
+
+
+DAGs View
+.........
+List of the DAGs in your environment, and a set of shortcuts to useful pages.
+You can see exactly how many tasks succeeded, failed, or are currently
+running at a glance.
+
+------------
+
+.. image:: img/dags.png
+
+------------
+
+
+Tree View
+.........
+A tree representation of the DAG that spans across time. If a pipeline is
+late, you can quickly see where the different steps are and identify
+the blocking ones.
+
+------------
+
+.. image:: img/tree.png
+
+------------
+
+Graph View
+..........
+The graph view is perhaps the most comprehensive. Visualize your DAG's
+dependencies and their current status for a specific run.
+
+------------
+
+.. image:: img/graph.png
+
+------------
+
+Variable View
+.............
+The variable view allows you to list, create, edit or delete the key-value pair
+of a variable used during jobs. Value of a variable will be hidden if the key contains
+any words in ('password', 'secret', 'passwd', 'authorization', 'api_key', 'apikey', 'access_token')
+by default, but can be configured to show in clear-text.
+
+------------
+
+.. image:: img/variable_hidden.png
+
+------------
+
+Gantt Chart
+...........
+The Gantt chart lets you analyse task duration and overlap. You can quickly
+identify bottlenecks and where the bulk of the time is spent for specific
+DAG runs.
+
+------------
+
+.. image:: img/gantt.png
+
+------------
+
+Task Duration
+.............
+The duration of your different tasks over the past N runs. This view lets
+you find outliers and quickly understand where the time is spent in your
+DAG over many runs.
+
+
+------------
+
+.. image:: img/duration.png
+
+------------
+
+Code View
+.........
+Transparency is everything. While the code for your pipeline is in source
+control, this is a quick way to get to the code that generates the DAG and
+provide yet more context.
+
+------------
+
+.. image:: img/code.png
+
+------------
+
+Task Instance Context Menu
+..........................
+From the pages seen above (tree view, graph view, gantt, ...), it is always
+possible to click on a task instance, and get to this rich context menu
+that can take you to more detailed metadata, and perform some actions.
+
+------------
+
+.. image:: img/context.png

http://git-wip-us.apache.org/repos/asf/incubator-airflow-site/blob/9e19165c/_static/ajax-loader.gif
----------------------------------------------------------------------
diff --git a/_static/ajax-loader.gif b/_static/ajax-loader.gif
new file mode 100644
index 0000000..61faf8c
Binary files /dev/null and b/_static/ajax-loader.gif differ

http://git-wip-us.apache.org/repos/asf/incubator-airflow-site/blob/9e19165c/_static/apache.jpg
----------------------------------------------------------------------
diff --git a/_static/apache.jpg b/_static/apache.jpg
new file mode 100644
index 0000000..312251f
Binary files /dev/null and b/_static/apache.jpg differ