You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by GitBox <gi...@apache.org> on 2020/12/08 21:13:24 UTC

[GitHub] [airflow] dimberman commented on a change in pull request #12930: Upgrading to Airflow 2.0 doc

dimberman commented on a change in pull request #12930:
URL: https://github.com/apache/airflow/pull/12930#discussion_r538807709



##########
File path: docs/apache-airflow/upgrading-to-2.rst
##########
@@ -0,0 +1,1164 @@
+.. 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.
+
+
+Upgrading to Airflow 2.0+
+-------------------------
+
+.. contents:: :local:
+
+Apache Airflow 2.0 is a major release and the purpose of this document is to assist
+users to migrate from Airflow 1.10.x to Airflow 2.0.
+
+Step 1: Upgrade to Python 3
+'''''''''''''''''''''''''''
+
+Airflow 1.10 will be the last release series to support Python 2. Airflow 2.0.0 will require Python 3.6+.
+
+If you have a specific task that still requires Python 2 then you can use the PythonVirtualenvOperator for this.

Review comment:
       ```suggestion
   If you have a specific task that still requires Python 2 then you can use the PythonVirtualenvOperator or the KubernetesPodOperator for this.
   ```

##########
File path: docs/apache-airflow/upgrading-to-2.rst
##########
@@ -0,0 +1,1164 @@
+.. 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.
+
+
+Upgrading to Airflow 2.0+
+-------------------------
+
+.. contents:: :local:
+
+Apache Airflow 2.0 is a major release and the purpose of this document is to assist
+users to migrate from Airflow 1.10.x to Airflow 2.0.
+
+Step 1: Upgrade to Python 3
+'''''''''''''''''''''''''''
+
+Airflow 1.10 will be the last release series to support Python 2. Airflow 2.0.0 will require Python 3.6+.
+
+If you have a specific task that still requires Python 2 then you can use the PythonVirtualenvOperator for this.
+
+For a list of breaking changes between Python 2 and Python 3, please refer to this
+[handy blog](https://blog.couchbase.com/tips-and-tricks-for-upgrading-from-python-2-to-python-3/)
+from the CouchBaseDB team.
+
+
+Step 2: Upgrade to Airflow 1.10.14 (a.k.a our "bridge" release)
+'''''''''''''''''''''''''''''''''''''''''''''''''''''''''''''''
+
+To minimize friction for users upgrading from Airflow 1.10 to Airflow 2.0 and beyond, a "bridge"
+release and final 1.10 version will be made available. Airflow 1.10.14 includes support for various critical features
+that make it easy for users to test DAGs and make sure they are as compatible with Airflow 2.0, so that the . We strongly recommend that all users upgrading to Airflow 2.0 first
+upgrade to Airflow 1.10.14 is straight forward.
+
+Features in 1.10.14 include:
+
+1. Most breaking DAG and architecture changes of Airflow 2.0 have been backported to Airflow 1.10.14. This backward-compatibility does not mean
+that 1.10.14 will process these DAGs the same way as Airflow 2.0. What this does mean is that most Airflow 2.0

Review comment:
       ```suggestion
   that 1.10.14 will process these DAGs the same way as Airflow 2.0. Instead, this means that most Airflow 2.0
   ```

##########
File path: docs/apache-airflow/upgrading-to-2.rst
##########
@@ -0,0 +1,1164 @@
+.. 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.
+
+
+Upgrading to Airflow 2.0+
+-------------------------
+
+.. contents:: :local:
+
+Apache Airflow 2.0 is a major release and the purpose of this document is to assist
+users to migrate from Airflow 1.10.x to Airflow 2.0.
+
+Step 1: Upgrade to Python 3
+'''''''''''''''''''''''''''
+
+Airflow 1.10 will be the last release series to support Python 2. Airflow 2.0.0 will require Python 3.6+.
+
+If you have a specific task that still requires Python 2 then you can use the PythonVirtualenvOperator for this.
+
+For a list of breaking changes between Python 2 and Python 3, please refer to this
+[handy blog](https://blog.couchbase.com/tips-and-tricks-for-upgrading-from-python-2-to-python-3/)
+from the CouchBaseDB team.
+
+
+Step 2: Upgrade to Airflow 1.10.14 (a.k.a our "bridge" release)
+'''''''''''''''''''''''''''''''''''''''''''''''''''''''''''''''
+
+To minimize friction for users upgrading from Airflow 1.10 to Airflow 2.0 and beyond, a "bridge"
+release and final 1.10 version will be made available. Airflow 1.10.14 includes support for various critical features
+that make it easy for users to test DAGs and make sure they are as compatible with Airflow 2.0, so that the . We strongly recommend that all users upgrading to Airflow 2.0 first
+upgrade to Airflow 1.10.14 is straight forward.
+
+Features in 1.10.14 include:
+
+1. Most breaking DAG and architecture changes of Airflow 2.0 have been backported to Airflow 1.10.14. This backward-compatibility does not mean
+that 1.10.14 will process these DAGs the same way as Airflow 2.0. What this does mean is that most Airflow 2.0
+compatible DAGs will work in Airflow 1.10.14. This backport will give users time to modify their DAGs over time
+without any service disruption.
+
+2. We have also backported the updated Airflow 2.0 CLI commands to Airflow 1.10.4, so that users can modify their scripts
+to be compatible with Airflow 2.0 before the upgrade.
+
+3. For users of the KubernetesExecutor, we have backported the ``pod_template_file`` capability for the KubernetesExecutor
+as well as a script that will generate a ``pod_template_file`` based on your ``airflow.cfg`` settings. To generate this file
+simply run the following command:
+
+.. code-block:: bash
+
+     airflow generate_pod_template -o <output file path>
+
+Once you have performed this step, simply write out the file path to this file in the ``pod_template_file`` section of the ``kubernetes``
+section of your ``airflow.cfg``
+
+Step 3: Install and run the Upgrade check scripts
+'''''''''''''''''''''''''''''''''''''''''''''''''
+
+After upgrading to Airflow 1.10.14, we recommend that you install the "upgrade check" scripts. These scripts will read through your ``airflow.cfg`` and all of your Dags and will give a detailed report of all changes required before upgrading. We are testing this script diligently, and our goal is that any Airflow setup that can pass these tests will be able to upgrade to 2.0 without any issues.
+
+.. code-block:: bash
+
+     pip install apache-airflow-upgrade-check
+
+Once this is installed, please run the upgrade check script.
+
+.. code-block:: bash
+
+     airflow upgrade_check
+
+More details about this process are here  :ref:`Upgrade Check Scripts<upgrade-check:upgrade_check>`
+
+
+Step 4: Set Operators to Backport Providers
+'''''''''''''''''''''''''''''''''''''''''''
+
+Now that you are set up in airflow 1.10.14 with python a 3.6+ environment, you are ready to start porting your DAGs to Airfow 2.0 compliance!
+
+The most important step in this transition is also the easiest step to do in pieces. All Airflow 2.0 operators are backwards compatible with Airflow 1.10
+using the [backport providers](./backport-providers.rst) service. In your own time, you can transition to using these backport-providers
+by pip installing the provider via ``pypi`` and changing the import path.
+
+For example: While historically you might have imported the DockerOperator in this fashion:
+
+.. code-block:: python
+
+    from airflow.operators.docker_operator import DockerOperator
+
+You would now run this command to import the provider:
+
+.. code-block:: bash
+
+    pip install apache-airflow-backport-providers-docker
+
+
+and then import the operator with this path:
+
+.. code-block:: python
+
+    from airflow.providers.docker.operators.docker import DockerOperator
+
+Pleaes note that the backport provider packages are just backports of the provider packages compatible with Airflow 2.0.
+Those provider packages are installed automatically when you install airflow with extras.
+For example:
+
+.. code-block:: bash
+
+    pip install airflow[docker]
+
+automatically installs the ``apache-airflow-providers-docker`` package.
+But you can manage/upgrade remove provider packages separately from the airflow core.
+
+
+Step 5: Upgrade Airflow DAGs
+''''''''''''''''''''''''''''
+
+**Change to undefined variable handling in templates**
+
+Prior to Airflow 2.0 Jinja Templates would permit the use of undefined variables. They would render as an
+empty string, with no indication to the user an undefined variable was used. With this release, any template
+rendering involving undefined variables will fail the task, as well as displaying an error in the UI when
+rendering.
+
+The behavior can be reverted when instantiating a DAG.
+
+.. code-block:: python
+
+    import jinja2
+
+    dag = DAG('simple_dag', template_undefined=jinja2.Undefined)
+
+
+**Changes to the KubernetesPodOperator**
+
+Much like the ``KubernetesExecutor``, the ``KubernetesPodOperator`` will no longer take Airflow custom classes and will
+instead expect either a pod_template yaml file, or ``kubernetes.client.models`` objects.
+
+The one notable exception is that we will continue to support the ``airflow.kubernetes.secret.Secret`` class.
+
+Whereas previously a user would import each individual class to build the pod as so:
+
+.. code-block:: python
+
+    from airflow.kubernetes.pod import Port
+    from airflow.kubernetes.volume import Volume
+    from airflow.kubernetes.secret import Secret
+    from airflow.kubernetes.volume_mount import VolumeMount
+
+
+    volume_config = {
+        'persistentVolumeClaim': {
+            'claimName': 'test-volume'
+        }
+    }
+    volume = Volume(name='test-volume', configs=volume_config)
+    volume_mount = VolumeMount('test-volume',
+                               mount_path='/root/mount_file',
+                               sub_path=None,
+                               read_only=True)
+
+    port = Port('http', 80)
+    secret_file = Secret('volume', '/etc/sql_conn', 'airflow-secrets', 'sql_alchemy_conn')
+    secret_env = Secret('env', 'SQL_CONN', 'airflow-secrets', 'sql_alchemy_conn')
+
+    k = KubernetesPodOperator(
+        namespace='default',
+        image="ubuntu:16.04",
+        cmds=["bash", "-cx"],
+        arguments=["echo", "10"],
+        labels={"foo": "bar"},
+        secrets=[secret_file, secret_env],
+        ports=[port],
+        volumes=[volume],
+        volume_mounts=[volume_mount],
+        name="airflow-test-pod",
+        task_id="task",
+        affinity=affinity,
+        is_delete_operator_pod=True,
+        hostnetwork=False,
+        tolerations=tolerations,
+        configmaps=configmaps,
+        init_containers=[init_container],
+        priority_class_name="medium",
+    )
+
+
+Now the user can use the ``kubernetes.client.models`` class as a single point of entry for creating all k8s objects.
+
+.. code-block:: python
+
+    from kubernetes.client import models as k8s
+    from airflow.kubernetes.secret import Secret
+
+
+    configmaps = ['test-configmap-1', 'test-configmap-2']
+
+    volume = k8s.V1Volume(
+        name='test-volume',
+        persistent_volume_claim=k8s.V1PersistentVolumeClaimVolumeSource(claim_name='test-volume'),
+    )
+
+    port = k8s.V1ContainerPort(name='http', container_port=80)
+    secret_file = Secret('volume', '/etc/sql_conn', 'airflow-secrets', 'sql_alchemy_conn')
+    secret_env = Secret('env', 'SQL_CONN', 'airflow-secrets', 'sql_alchemy_conn')
+    secret_all_keys = Secret('env', None, 'airflow-secrets-2')
+    volume_mount = k8s.V1VolumeMount(
+        name='test-volume', mount_path='/root/mount_file', sub_path=None, read_only=True
+    )
+
+    k = KubernetesPodOperator(
+        namespace='default',
+        image="ubuntu:16.04",
+        cmds=["bash", "-cx"],
+        arguments=["echo", "10"],
+        labels={"foo": "bar"},
+        secrets=[secret_file, secret_env],
+        ports=[port],
+        volumes=[volume],
+        volume_mounts=[volume_mount],
+        name="airflow-test-pod",
+        task_id="task",
+        is_delete_operator_pod=True,
+        hostnetwork=False)
+
+
+We decided to keep the Secret class as users seem to really like that simplifies the complexity of mounting
+Kubernetes secrets into workers.
+
+For a more detailed list of changes to the KubernetesPodOperator API, please read the section in the Appendix titled "Changed Parameters for the KubernetesPodOperator"
+
+
+**Change default value for dag_run_conf_overrides_params**
+
+DagRun configuration dictionary will now by default overwrite params dictionary. If you pass some key-value pairs
+through ``airflow dags backfill -c`` or ``airflow dags trigger -c``, the key-value pairs will
+override the existing ones in params. You can revert this behaviour by setting ``dag_run_conf_overrides_params`` to ``False``
+in your ``airflow.cfg``.
+
+**DAG discovery safe mode is now case insensitive**
+
+When ``DAG_DISCOVERY_SAFE_MODE`` is active, Airflow will now filter all files that contain the string ``airflow`` and ``dag``
+in a case insensitive mode. This is being changed to better support the new ``@dag`` decorator.
+
+**Change to Permissions**
+
+The DAG-level permission actions, ``can_dag_read`` and ``can_dag_edit`` are going away. They are being replaced with ``can_read`` and ``can_edit``. When a role is given DAG-level access, the resource name (or "view menu", in Flask App-Builder parlance) will now be prefixed with ``DAG:``. So the action ``can_dag_read`` on ``example_dag_id``, is now represented as ``can_read`` on ``DAG:example_dag_id``.
+
+*As part of running ``db upgrade``, existing permissions will be migrated for you.*
+
+When DAGs are initialized with the ``access_control`` variable set, any usage of the old permission names will automatically be updated in the database, so this won't be a breaking change. A DeprecationWarning will be raised.
+
+**Drop legacy UI in favor of FAB RBAC UI**
+
+.. warning::
+    Breaking change
+
+    Previously we were using two versions of UI, which were hard to maintain as we need to implement/update the same feature
+    in both versions. With this release we've removed the older UI in favor of Flask App Builder RBAC UI. No need to set the
+    RBAC UI explicitly in the configuration now as this is the only default UI. We did it to avoid
+    the huge maintenance burden of two independent user interfaces
+
+Please note that that custom auth backends will need re-writing to target new FAB based UI.
+
+As part of this change, a few configuration items in ``[webserver]`` section are removed and no longer applicable,
+including ``authenticate``, ``filter_by_owner``, ``owner_mode``, and ``rbac``.
+
+Before upgrading to this release, we recommend activating the new FAB RBAC UI. For that, you should set
+the ``rbac`` options  in ``[webserver]`` in the ``airflow.cfg`` file to ``true``
+
+.. code-block:: bash
+
+    [webserver]
+    rbac = true
+
+In order to login to the interface, you need to create an administrator account.
+
+.. code-block:: bash
+
+    airflow create_user \
+        --role Admin \
+        --username admin \
+        --firstname FIRST_NAME \
+        --lastname LAST_NAME \
+        --email EMAIL@example.org
+
+If you have already installed Airflow 2.0, you can create a user with the command ``airflow users create``.
+You don't need to make changes to the configuration file as the FAB RBAC UI is
+the only supported UI.
+
+.. code-block:: bash
+
+    airflow users create \
+        --role Admin \
+        --username admin \
+        --firstname FIRST_NAME \
+        --lastname LAST_NAME \
+        --email EMAIL@example.org
+
+**Breaking Change in OAuth**
+
+The flask-ouathlib has been replaced with authlib because flask-outhlib has
+been deprecated in favour of authlib.
+The Old and New provider configuration keys that have changed are as follows
+
+==================== ==================
+Old Keys             New keys
+==================== ==================
+consumer_key         client_id
+consumer_secret      client_secret
+base_url             api_base_url
+request_token_params client_kwargs
+==================== ==================
+
+For more information, visit https://flask-appbuilder.readthedocs.io/en/latest/security.html#authentication-oauth
+
+
+Step 6: Upgrade Configuration settings
+'''''''''''''''''''''''''''''''''''''''''''
+
+Airflow 2.0 is stricter with respect to expectations on configuration data and requires explicit
+specifications of configuration values in more cases rather than defaulting to a generic value.
+
+Some of these are detailed in the Upgrade Check guide, but a significant area of change is with
+respect to the Kubernetes Executor. This is called out below for users of the Kubernetes Executor.
+
+**Upgrade KubernetesExecutor settings**
+
+*The KubernetesExecutor Will No Longer Read from the airflow.cfg for Base Pod Configurations.*
+
+In Airflow 2.0, the KubernetesExecutor will require a base pod template written in yaml. This file can exist
+anywhere on the host machine and will be linked using the ``pod_template_file`` configuration in the airflow.cfg.

Review comment:
       ```suggestion
   anywhere on the host machine and will be linked using the ``pod_template_file`` configuration in the airflow.cfg. You can create a ``pod_template_file`` by running the following command:  ``airflow generate_pod_template`` ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org